From db4a8a1c6413b3a769914b011a88287eed4fd8ce Mon Sep 17 00:00:00 2001 From: Radu Berinde Date: Fri, 7 Dec 2018 09:15:08 -0500 Subject: [PATCH] Update protobuf-related deps This change updates the protobuf deps and makes various fixes. Most fixes are related to new `XXX_` fields. In particular, the new fields interfere with JSON marshaling and printing with `%v`. Note that the new fields could be omitted but that is undesirable: `XXX_NoUnkeyedLiteral` is a good idea and there would be a performance cost associated with removing `XXX_sizecache`. Some more info on important protobuf changes here: https://groups.google.com/forum/#!topic/golang-nuts/F5xFHTfwRnY Change lists: https://github.com/gogo/protobuf/compare/1adfc126...936d59a5 https://github.com/google/go-genproto/compare/f676e0f3...a1fde740 Note that gogo/protobuf was forked under cockroachdb, and I made a fix in our fork. This commit should be reviewed: https://github.com/cockroachdb/gogoproto/commit/d26e6ac0df62ac84a2f6c203eb968c1ef40fb97d Informs #30774. Release note: None --- Gopkg.lock | 12 +- Gopkg.toml | 5 + Makefile | 3 +- pkg/acceptance/cluster/testconfig.pb.go | 140 +- pkg/build/info.pb.go | 52 +- pkg/ccl/backupccl/backup.pb.go | 194 +- pkg/ccl/baseccl/encryption_options.pb.go | 82 +- pkg/ccl/changefeedccl/table_history.go | 2 +- .../engineccl/enginepbccl/key_registry.pb.go | 196 +- .../engineccl/enginepbccl/stats.pb.go | 51 +- pkg/ccl/utilccl/licenseccl/license.pb.go | 64 +- pkg/ccl/utilccl/licenseccl/license_test.go | 8 +- pkg/cli/debug.go | 2 +- pkg/cli/flags_test.go | 16 +- pkg/cli/systembench/systembenchpb/ping.pb.go | 110 +- pkg/config/system.pb.go | 64 +- pkg/config/zone.pb.go | 287 +- pkg/config/zone_yaml.go | 10 + pkg/gossip/gossip.pb.go | 568 +- pkg/internal/client/lease.pb.go | 54 +- pkg/jobs/jobspb/jobs.pb.go | 982 ++- pkg/kv/dist_sender_server_test.go | 2 +- pkg/roachpb/api.pb.go | 5342 +++++++++++++---- pkg/roachpb/app_stats.pb.go | 289 +- pkg/roachpb/batch_test.go | 71 +- pkg/roachpb/data.go | 2 +- pkg/roachpb/data.pb.go | 743 ++- pkg/roachpb/data_test.go | 43 +- pkg/roachpb/errors.go | 15 +- pkg/roachpb/errors.pb.go | 1442 ++++- pkg/roachpb/internal.pb.go | 192 +- pkg/roachpb/internal_raft.pb.go | 166 +- pkg/roachpb/io-formats.pb.go | 223 +- pkg/roachpb/metadata.pb.go | 614 +- pkg/roachpb/version_test.go | 32 +- pkg/rpc/heartbeat.pb.go | 164 +- pkg/server/admin_test.go | 2 +- pkg/server/diagnosticspb/diagnostics.pb.go | 402 +- pkg/server/node.go | 2 +- pkg/server/serverpb/admin.pb.go | 2835 +++++++-- pkg/server/serverpb/authentication.pb.go | 227 +- pkg/server/serverpb/init.pb.go | 100 +- pkg/server/serverpb/status.pb.go | 3053 ++++++++-- pkg/server/status/statuspb/status.pb.go | 273 +- pkg/settings/cluster/cluster_version.pb.go | 56 +- pkg/sql/distsqlpb/api.pb.go | 444 +- pkg/sql/distsqlpb/data.pb.go | 847 ++- pkg/sql/distsqlpb/flow_diagram_test.go | 6 +- pkg/sql/distsqlpb/processors.pb.go | 1834 +++++- pkg/sql/distsqlrun/cluster_test.go | 2 +- pkg/sql/distsqlrun/stats.pb.go | 492 +- pkg/sql/pgwire/pgerror/errors.pb.go | 99 +- pkg/sql/sqlbase/encoded_datum.pb.go | 34 +- pkg/sql/sqlbase/join_type.pb.go | 16 +- pkg/sql/sqlbase/privilege.pb.go | 90 +- pkg/sql/sqlbase/structured.pb.go | 962 ++- pkg/sql/sqlbase/table_test.go | 2 +- pkg/sql/stats/histogram.pb.go | 97 +- pkg/storage/allocator.go | 15 +- pkg/storage/allocator_test.go | 15 +- pkg/storage/api.pb.go | 337 +- .../batcheval/cmd_refresh_range_test.go | 2 +- pkg/storage/closedts/ctpb/entry.pb.go | 128 +- pkg/storage/consistency_queue_test.go | 4 +- .../engine/enginepb/file_registry.pb.go | 122 +- pkg/storage/engine/enginepb/mvcc.pb.go | 185 +- pkg/storage/engine/enginepb/mvcc3.pb.go | 520 +- pkg/storage/engine/enginepb/rocksdb.pb.go | 92 +- pkg/storage/engine/mvcc_test.go | 2 +- pkg/storage/raft.pb.go | 422 +- pkg/storage/rangefeed/resolved_timestamp.go | 2 +- pkg/storage/replica_consistency.go | 8 +- pkg/storage/storagepb/lease_status.pb.go | 85 +- pkg/storage/storagepb/liveness.pb.go | 63 +- pkg/storage/storagepb/log.pb.go | 136 +- pkg/storage/storagepb/proposer_kv.pb.go | 487 +- pkg/storage/storagepb/state.pb.go | 165 +- pkg/storage/store.go | 4 +- pkg/testutils/lint/lint_test.go | 1 + pkg/testutils/zerofields/no_zero_field.go | 7 +- pkg/ts/tspb/timeseries.pb.go | 305 +- pkg/util/hlc/legacy_timestamp.pb.go | 54 +- pkg/util/hlc/timestamp.pb.go | 47 +- pkg/util/log/log.pb.go | 158 +- pkg/util/metric/metric.pb.go | 122 +- pkg/util/protoutil/clone.pb.go | 54 +- pkg/util/tracing/recorded_span.pb.go | 168 +- pkg/util/unresolved_addr.go | 2 +- pkg/util/unresolved_addr.pb.go | 54 +- vendor | 2 +- 90 files changed, 22585 insertions(+), 5300 deletions(-) diff --git a/Gopkg.lock b/Gopkg.lock index 13290ee0b871..4d93d0b15ab5 100644 --- a/Gopkg.lock +++ b/Gopkg.lock @@ -572,7 +572,8 @@ revision = "60d456a402782453be397030407e34decaf04d73" [[projects]] - digest = "1:aba848b77ff362b03db51ac06379c36a120bc05b4cc6c5c74f6e6fa310454fa5" + branch = "v1.2.0-with-clone-fix" + digest = "1:c6e7d6ec253d51e58ec35b760f33b344946fe87d62dcbce6274b3100e4b08573" name = "github.com/gogo/protobuf" packages = [ "gogoproto", @@ -596,6 +597,7 @@ "proto", "protoc-gen-gogo/descriptor", "protoc-gen-gogo/generator", + "protoc-gen-gogo/generator/internal/remap", "protoc-gen-gogo/grpc", "protoc-gen-gogo/plugin", "sortkeys", @@ -604,8 +606,8 @@ "vanity/command", ] pruneopts = "T" - revision = "1adfc126b41513cc696b209667c8656ea7aac67c" - version = "v1.0.0" + revision = "ca10b809dba0aa7ed3e43b8202027f20e3d40608" + source = "https://github.com/cockroachdb/gogoproto" [[projects]] branch = "master" @@ -1579,7 +1581,7 @@ [[projects]] branch = "master" - digest = "1:532c984ea59063abc959da8bbdb0548d9426c30ac0cd241aff484f9bceca5727" + digest = "1:a7d48ca460ca1b4f6ccd8c95502443afa05df88aee84de7dbeb667a8754e8fa6" name = "google.golang.org/genproto" packages = [ "googleapis/api/annotations", @@ -1588,7 +1590,7 @@ "googleapis/rpc/status", ] pruneopts = "UT" - revision = "f676e0f3ac6395ff1a529ae59a6670878a8371a6" + revision = "a1fde740824669f161e97af4237fb80b8734444d" [[projects]] digest = "1:3a98314fd2e43bbd905b33125dad80b10111ba6e5e541db8ed2a953fe01fbb31" diff --git a/Gopkg.toml b/Gopkg.toml index 2b4915626008..61e7b5e0fbfe 100644 --- a/Gopkg.toml +++ b/Gopkg.toml @@ -86,6 +86,11 @@ ignored = [ name = "golang.org/x/crypto" branch = "master" +[[constraint]] + name = "github.com/gogo/protobuf" + source = "https://github.com/cockroachdb/gogoproto" + branch = "v1.2.0-with-clone-fix" + # github.com/docker/docker depends on a few functions not included in the # latest release: reference.{FamiliarName,ParseNormalizedNamed,TagNameOnly}. # diff --git a/Makefile b/Makefile index 021f75498659..a09654585adb 100644 --- a/Makefile +++ b/Makefile @@ -1075,10 +1075,9 @@ bin/.go_protobuf_sources: $(PROTOC) $(GO_PROTOS) $(GOGOPROTO_PROTO) bin/.bootstr build/werror.sh $(PROTOC) -Ipkg:./vendor/github.com:$(GOGO_PROTOBUF_PATH):$(PROTOBUF_PATH):$(COREOS_PATH):$(GRPC_GATEWAY_GOOGLEAPIS_PATH) --gogoroach_out=$(PROTO_MAPPINGS),plugins=grpc,import_prefix=github.com/cockroachdb/cockroach/pkg/:./pkg $$dir/*.proto; \ done $(SED_INPLACE) -E \ - -e '/import _/d' \ + -e '/import _ /d' \ -e 's!import (fmt|math) "github.com/cockroachdb/cockroach/pkg/(fmt|math)"! !g' \ -e 's!github\.com/cockroachdb/cockroach/pkg/(etcd)!go.etcd.io/\1!g' \ - -e 's!cockroachdb/cockroach/pkg/(prometheus/client_model)!\1/go!g' \ -e 's!github.com/cockroachdb/cockroach/pkg/(bytes|encoding/binary|errors|fmt|io|math|github\.com|(google\.)?golang\.org)!\1!g' \ -e 's!golang.org/x/net/context!context!g' \ $(GO_SOURCES) diff --git a/pkg/acceptance/cluster/testconfig.pb.go b/pkg/acceptance/cluster/testconfig.pb.go index 4753ce2ef89d..61373b881bb2 100644 --- a/pkg/acceptance/cluster/testconfig.pb.go +++ b/pkg/acceptance/cluster/testconfig.pb.go @@ -1,17 +1,6 @@ // Code generated by protoc-gen-gogo. DO NOT EDIT. // source: acceptance/cluster/testconfig.proto -/* - Package cluster is a generated protocol buffer package. - - It is generated from these files: - acceptance/cluster/testconfig.proto - - It has these top-level messages: - StoreConfig - NodeConfig - TestConfig -*/ package cluster import proto "github.com/gogo/protobuf/proto" @@ -76,28 +65,82 @@ func (x *InitMode) UnmarshalJSON(data []byte) error { *x = InitMode(value) return nil } -func (InitMode) EnumDescriptor() ([]byte, []int) { return fileDescriptorTestconfig, []int{0} } +func (InitMode) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_testconfig_6780bb4cf4b056d7, []int{0} +} // StoreConfig holds the configuration of a collection of similar stores. type StoreConfig struct { - MaxRanges int32 `protobuf:"varint,2,opt,name=max_ranges,json=maxRanges" json:"max_ranges"` + MaxRanges int32 `protobuf:"varint,2,opt,name=max_ranges,json=maxRanges" json:"max_ranges"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *StoreConfig) Reset() { *m = StoreConfig{} } -func (m *StoreConfig) String() string { return proto.CompactTextString(m) } -func (*StoreConfig) ProtoMessage() {} -func (*StoreConfig) Descriptor() ([]byte, []int) { return fileDescriptorTestconfig, []int{0} } +func (m *StoreConfig) Reset() { *m = StoreConfig{} } +func (m *StoreConfig) String() string { return proto.CompactTextString(m) } +func (*StoreConfig) ProtoMessage() {} +func (*StoreConfig) Descriptor() ([]byte, []int) { + return fileDescriptor_testconfig_6780bb4cf4b056d7, []int{0} +} +func (m *StoreConfig) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *StoreConfig) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *StoreConfig) XXX_Merge(src proto.Message) { + xxx_messageInfo_StoreConfig.Merge(dst, src) +} +func (m *StoreConfig) XXX_Size() int { + return m.Size() +} +func (m *StoreConfig) XXX_DiscardUnknown() { + xxx_messageInfo_StoreConfig.DiscardUnknown(m) +} + +var xxx_messageInfo_StoreConfig proto.InternalMessageInfo // NodeConfig holds the configuration of a collection of similar nodes. type NodeConfig struct { - Version string `protobuf:"bytes,1,opt,name=version" json:"version"` - Stores []StoreConfig `protobuf:"bytes,2,rep,name=stores" json:"stores"` + Version string `protobuf:"bytes,1,opt,name=version" json:"version"` + Stores []StoreConfig `protobuf:"bytes,2,rep,name=stores" json:"stores"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *NodeConfig) Reset() { *m = NodeConfig{} } +func (m *NodeConfig) String() string { return proto.CompactTextString(m) } +func (*NodeConfig) ProtoMessage() {} +func (*NodeConfig) Descriptor() ([]byte, []int) { + return fileDescriptor_testconfig_6780bb4cf4b056d7, []int{1} +} +func (m *NodeConfig) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *NodeConfig) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *NodeConfig) XXX_Merge(src proto.Message) { + xxx_messageInfo_NodeConfig.Merge(dst, src) +} +func (m *NodeConfig) XXX_Size() int { + return m.Size() +} +func (m *NodeConfig) XXX_DiscardUnknown() { + xxx_messageInfo_NodeConfig.DiscardUnknown(m) } -func (m *NodeConfig) Reset() { *m = NodeConfig{} } -func (m *NodeConfig) String() string { return proto.CompactTextString(m) } -func (*NodeConfig) ProtoMessage() {} -func (*NodeConfig) Descriptor() ([]byte, []int) { return fileDescriptorTestconfig, []int{1} } +var xxx_messageInfo_NodeConfig proto.InternalMessageInfo type TestConfig struct { Name string `protobuf:"bytes,1,opt,name=name" json:"name"` @@ -108,13 +151,39 @@ type TestConfig struct { InitMode InitMode `protobuf:"varint,4,opt,name=init_mode,json=initMode,enum=cockroach.acceptance.cluster.InitMode" json:"init_mode"` // When set, the cluster is started as quickly as possible, without waiting // for ranges to replicate, or even ports to be opened. - NoWait bool `protobuf:"varint,5,opt,name=no_wait,json=noWait" json:"no_wait"` + NoWait bool `protobuf:"varint,5,opt,name=no_wait,json=noWait" json:"no_wait"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *TestConfig) Reset() { *m = TestConfig{} } -func (m *TestConfig) String() string { return proto.CompactTextString(m) } -func (*TestConfig) ProtoMessage() {} -func (*TestConfig) Descriptor() ([]byte, []int) { return fileDescriptorTestconfig, []int{2} } +func (m *TestConfig) Reset() { *m = TestConfig{} } +func (m *TestConfig) String() string { return proto.CompactTextString(m) } +func (*TestConfig) ProtoMessage() {} +func (*TestConfig) Descriptor() ([]byte, []int) { + return fileDescriptor_testconfig_6780bb4cf4b056d7, []int{2} +} +func (m *TestConfig) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *TestConfig) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *TestConfig) XXX_Merge(src proto.Message) { + xxx_messageInfo_TestConfig.Merge(dst, src) +} +func (m *TestConfig) XXX_Size() int { + return m.Size() +} +func (m *TestConfig) XXX_DiscardUnknown() { + xxx_messageInfo_TestConfig.DiscardUnknown(m) +} + +var xxx_messageInfo_TestConfig proto.InternalMessageInfo func init() { proto.RegisterType((*StoreConfig)(nil), "cockroach.acceptance.cluster.StoreConfig") @@ -235,6 +304,9 @@ func encodeVarintTestconfig(dAtA []byte, offset int, v uint64) int { return offset + 1 } func (m *StoreConfig) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovTestconfig(uint64(m.MaxRanges)) @@ -242,6 +314,9 @@ func (m *StoreConfig) Size() (n int) { } func (m *NodeConfig) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Version) @@ -256,6 +331,9 @@ func (m *NodeConfig) Size() (n int) { } func (m *TestConfig) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Name) @@ -737,9 +815,11 @@ var ( ErrIntOverflowTestconfig = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("acceptance/cluster/testconfig.proto", fileDescriptorTestconfig) } +func init() { + proto.RegisterFile("acceptance/cluster/testconfig.proto", fileDescriptor_testconfig_6780bb4cf4b056d7) +} -var fileDescriptorTestconfig = []byte{ +var fileDescriptor_testconfig_6780bb4cf4b056d7 = []byte{ // 415 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x7c, 0x90, 0xcd, 0x6e, 0xd3, 0x40, 0x14, 0x85, 0x3d, 0xf9, 0x21, 0xc9, 0x2d, 0x45, 0xd1, 0x08, 0x24, 0xab, 0x2a, 0x53, 0x93, 0x4a, diff --git a/pkg/build/info.pb.go b/pkg/build/info.pb.go index 73bf3fb6ea56..284657bfe82a 100644 --- a/pkg/build/info.pb.go +++ b/pkg/build/info.pb.go @@ -1,15 +1,6 @@ // Code generated by protoc-gen-gogo. DO NOT EDIT. // source: build/info.proto -/* - Package build is a generated protocol buffer package. - - It is generated from these files: - build/info.proto - - It has these top-level messages: - Info -*/ package build import proto "github.com/gogo/protobuf/proto" @@ -48,13 +39,39 @@ type Info struct { // alternatively, we could set jsonpb.Unmarshaler.AllowUnknownFields // to true in httputil.doJSONRequest, but that comes at the expense // of run-time type checking, which is nice to have. - Dependencies *string `protobuf:"bytes,10000,opt,name=dependencies" json:"dependencies,omitempty"` + Dependencies *string `protobuf:"bytes,10000,opt,name=dependencies" json:"dependencies,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *Info) Reset() { *m = Info{} } -func (m *Info) String() string { return proto.CompactTextString(m) } -func (*Info) ProtoMessage() {} -func (*Info) Descriptor() ([]byte, []int) { return fileDescriptorInfo, []int{0} } +func (m *Info) Reset() { *m = Info{} } +func (m *Info) String() string { return proto.CompactTextString(m) } +func (*Info) ProtoMessage() {} +func (*Info) Descriptor() ([]byte, []int) { + return fileDescriptor_info_6ca0bae8d1d234a0, []int{0} +} +func (m *Info) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Info) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *Info) XXX_Merge(src proto.Message) { + xxx_messageInfo_Info.Merge(dst, src) +} +func (m *Info) XXX_Size() int { + return m.Size() +} +func (m *Info) XXX_DiscardUnknown() { + xxx_messageInfo_Info.DiscardUnknown(m) +} + +var xxx_messageInfo_Info proto.InternalMessageInfo func init() { proto.RegisterType((*Info)(nil), "cockroach.build.Info") @@ -137,6 +154,9 @@ func encodeVarintInfo(dAtA []byte, offset int, v uint64) int { return offset + 1 } func (m *Info) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.GoVersion) @@ -654,9 +674,9 @@ var ( ErrIntOverflowInfo = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("build/info.proto", fileDescriptorInfo) } +func init() { proto.RegisterFile("build/info.proto", fileDescriptor_info_6ca0bae8d1d234a0) } -var fileDescriptorInfo = []byte{ +var fileDescriptor_info_6ca0bae8d1d234a0 = []byte{ // 315 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x54, 0x91, 0xb1, 0x4e, 0x32, 0x41, 0x14, 0x85, 0xd9, 0x9f, 0xe5, 0x07, 0xae, 0x24, 0xe8, 0xc4, 0x98, 0x89, 0xc5, 0x42, 0xa4, 0x90, diff --git a/pkg/ccl/backupccl/backup.pb.go b/pkg/ccl/backupccl/backup.pb.go index 35a795e8d74d..1322bd9877b0 100644 --- a/pkg/ccl/backupccl/backup.pb.go +++ b/pkg/ccl/backupccl/backup.pb.go @@ -1,25 +1,15 @@ // Code generated by protoc-gen-gogo. DO NOT EDIT. // source: ccl/backupccl/backup.proto -/* - Package backupccl is a generated protocol buffer package. - - It is generated from these files: - ccl/backupccl/backup.proto - - It has these top-level messages: - BackupDescriptor -*/ package backupccl import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" -import cockroach_build "github.com/cockroachdb/cockroach/pkg/build" -import cockroach_roachpb3 "github.com/cockroachdb/cockroach/pkg/roachpb" -import cockroach_roachpb1 "github.com/cockroachdb/cockroach/pkg/roachpb" -import cockroach_sql_sqlbase1 "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" -import cockroach_util_hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" +import build "github.com/cockroachdb/cockroach/pkg/build" +import roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" +import sqlbase "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" +import hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" import github_com_cockroachdb_cockroach_pkg_sql_sqlbase "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" import github_com_cockroachdb_cockroach_pkg_util_uuid "github.com/cockroachdb/cockroach/pkg/util/uuid" @@ -57,7 +47,9 @@ var MVCCFilter_value = map[string]int32{ func (x MVCCFilter) String() string { return proto.EnumName(MVCCFilter_name, int32(x)) } -func (MVCCFilter) EnumDescriptor() ([]byte, []int) { return fileDescriptorBackup, []int{0} } +func (MVCCFilter) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_backup_60eb759f8ef88651, []int{0} +} // BackupDescriptor represents a consistent snapshot of ranges. // @@ -66,73 +58,149 @@ func (MVCCFilter) EnumDescriptor() ([]byte, []int) { return fileDescriptorBackup // ranges in a backup is the same, but the start may vary (to allow individual // tables to be backed up on different schedules). type BackupDescriptor struct { - StartTime cockroach_util_hlc.Timestamp `protobuf:"bytes,1,opt,name=start_time,json=startTime" json:"start_time"` - EndTime cockroach_util_hlc.Timestamp `protobuf:"bytes,2,opt,name=end_time,json=endTime" json:"end_time"` - MVCCFilter MVCCFilter `protobuf:"varint,13,opt,name=mvcc_filter,json=mvccFilter,proto3,enum=cockroach.ccl.backupccl.MVCCFilter" json:"mvcc_filter,omitempty"` + StartTime hlc.Timestamp `protobuf:"bytes,1,opt,name=start_time,json=startTime,proto3" json:"start_time"` + EndTime hlc.Timestamp `protobuf:"bytes,2,opt,name=end_time,json=endTime,proto3" json:"end_time"` + MVCCFilter MVCCFilter `protobuf:"varint,13,opt,name=mvcc_filter,json=mvccFilter,proto3,enum=cockroach.ccl.backupccl.MVCCFilter" json:"mvcc_filter,omitempty"` // Even if StartTime is zero, we only get revisions since gc threshold, so // do not allow AS OF SYSTEM TIME before revision_start_time. - RevisionStartTime cockroach_util_hlc.Timestamp `protobuf:"bytes,17,opt,name=revision_start_time,json=revisionStartTime" json:"revision_start_time"` + RevisionStartTime hlc.Timestamp `protobuf:"bytes,17,opt,name=revision_start_time,json=revisionStartTime,proto3" json:"revision_start_time"` // Spans contains the spans requested for backup. The keyranges covered by // `files` may be a subset of this if there were ranges with no changes since // the last backup. For all tables in the backup descriptor, these spans must // completely cover each table's span. For example, if a table with ID 51 were // being backed up, then the span `/Table/5{1-2}` must be completely covered. - Spans []cockroach_roachpb1.Span `protobuf:"bytes,3,rep,name=spans" json:"spans"` + Spans []roachpb.Span `protobuf:"bytes,3,rep,name=spans,proto3" json:"spans"` // IntroducedSpans are a subset of spans, set only when creating incremental // backups that cover spans not included in a previous backup. Spans contained // here are covered in the interval (0, startTime], which, in conjunction with // the coverage from (startTime, endTime] implied for all spans in Spans, // results in coverage from [0, endTime] for these spans. - IntroducedSpans []cockroach_roachpb1.Span `protobuf:"bytes,15,rep,name=introduced_spans,json=introducedSpans" json:"introduced_spans"` - DescriptorChanges []BackupDescriptor_DescriptorRevision `protobuf:"bytes,16,rep,name=descriptor_changes,json=descriptorChanges" json:"descriptor_changes"` - Files []BackupDescriptor_File `protobuf:"bytes,4,rep,name=files" json:"files"` - Descriptors []cockroach_sql_sqlbase1.Descriptor `protobuf:"bytes,5,rep,name=descriptors" json:"descriptors"` + IntroducedSpans []roachpb.Span `protobuf:"bytes,15,rep,name=introduced_spans,json=introducedSpans,proto3" json:"introduced_spans"` + DescriptorChanges []BackupDescriptor_DescriptorRevision `protobuf:"bytes,16,rep,name=descriptor_changes,json=descriptorChanges,proto3" json:"descriptor_changes"` + Files []BackupDescriptor_File `protobuf:"bytes,4,rep,name=files,proto3" json:"files"` + Descriptors []sqlbase.Descriptor `protobuf:"bytes,5,rep,name=descriptors,proto3" json:"descriptors"` // databases in descriptors that have all tables also in descriptors. - CompleteDbs []github_com_cockroachdb_cockroach_pkg_sql_sqlbase.ID `protobuf:"varint,14,rep,packed,name=complete_dbs,json=completeDbs,casttype=github.com/cockroachdb/cockroach/pkg/sql/sqlbase.ID" json:"complete_dbs,omitempty"` - EntryCounts cockroach_roachpb3.BulkOpSummary `protobuf:"bytes,12,opt,name=entry_counts,json=entryCounts" json:"entry_counts"` - Dir cockroach_roachpb3.ExportStorage `protobuf:"bytes,7,opt,name=dir" json:"dir"` + CompleteDbs []github_com_cockroachdb_cockroach_pkg_sql_sqlbase.ID `protobuf:"varint,14,rep,packed,name=complete_dbs,json=completeDbs,proto3,casttype=github.com/cockroachdb/cockroach/pkg/sql/sqlbase.ID" json:"complete_dbs,omitempty"` + EntryCounts roachpb.BulkOpSummary `protobuf:"bytes,12,opt,name=entry_counts,json=entryCounts,proto3" json:"entry_counts"` + Dir roachpb.ExportStorage `protobuf:"bytes,7,opt,name=dir,proto3" json:"dir"` FormatVersion uint32 `protobuf:"varint,8,opt,name=format_version,json=formatVersion,proto3" json:"format_version,omitempty"` ClusterID github_com_cockroachdb_cockroach_pkg_util_uuid.UUID `protobuf:"bytes,9,opt,name=cluster_id,json=clusterId,proto3,customtype=github.com/cockroachdb/cockroach/pkg/util/uuid.UUID" json:"cluster_id"` // node_id and build_info of the gateway node (which writes the descriptor). - NodeID github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,10,opt,name=node_id,json=nodeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"node_id,omitempty"` - BuildInfo cockroach_build.Info `protobuf:"bytes,11,opt,name=build_info,json=buildInfo" json:"build_info"` + NodeID github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,10,opt,name=node_id,json=nodeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"node_id,omitempty"` + BuildInfo build.Info `protobuf:"bytes,11,opt,name=build_info,json=buildInfo,proto3" json:"build_info"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *BackupDescriptor) Reset() { *m = BackupDescriptor{} } -func (m *BackupDescriptor) String() string { return proto.CompactTextString(m) } -func (*BackupDescriptor) ProtoMessage() {} -func (*BackupDescriptor) Descriptor() ([]byte, []int) { return fileDescriptorBackup, []int{0} } +func (m *BackupDescriptor) Reset() { *m = BackupDescriptor{} } +func (m *BackupDescriptor) String() string { return proto.CompactTextString(m) } +func (*BackupDescriptor) ProtoMessage() {} +func (*BackupDescriptor) Descriptor() ([]byte, []int) { + return fileDescriptor_backup_60eb759f8ef88651, []int{0} +} +func (m *BackupDescriptor) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *BackupDescriptor) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *BackupDescriptor) XXX_Merge(src proto.Message) { + xxx_messageInfo_BackupDescriptor.Merge(dst, src) +} +func (m *BackupDescriptor) XXX_Size() int { + return m.Size() +} +func (m *BackupDescriptor) XXX_DiscardUnknown() { + xxx_messageInfo_BackupDescriptor.DiscardUnknown(m) +} + +var xxx_messageInfo_BackupDescriptor proto.InternalMessageInfo // BackupDescriptor_File represents a file that contains the diff for a key // range between two timestamps. type BackupDescriptor_File struct { - Span cockroach_roachpb1.Span `protobuf:"bytes,1,opt,name=span" json:"span"` - Path string `protobuf:"bytes,2,opt,name=path,proto3" json:"path,omitempty"` - Sha512 []byte `protobuf:"bytes,4,opt,name=sha512,proto3" json:"sha512,omitempty"` - EntryCounts cockroach_roachpb3.BulkOpSummary `protobuf:"bytes,6,opt,name=entry_counts,json=entryCounts" json:"entry_counts"` + Span roachpb.Span `protobuf:"bytes,1,opt,name=span,proto3" json:"span"` + Path string `protobuf:"bytes,2,opt,name=path,proto3" json:"path,omitempty"` + Sha512 []byte `protobuf:"bytes,4,opt,name=sha512,proto3" json:"sha512,omitempty"` + EntryCounts roachpb.BulkOpSummary `protobuf:"bytes,6,opt,name=entry_counts,json=entryCounts,proto3" json:"entry_counts"` // StartTime 0 is sometimes legitimately used, so it is only meaningful if // EndTime is non-zero, otherwise both just inherit from containing backup. - StartTime cockroach_util_hlc.Timestamp `protobuf:"bytes,7,opt,name=start_time,json=startTime" json:"start_time"` - EndTime cockroach_util_hlc.Timestamp `protobuf:"bytes,8,opt,name=end_time,json=endTime" json:"end_time"` + StartTime hlc.Timestamp `protobuf:"bytes,7,opt,name=start_time,json=startTime,proto3" json:"start_time"` + EndTime hlc.Timestamp `protobuf:"bytes,8,opt,name=end_time,json=endTime,proto3" json:"end_time"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *BackupDescriptor_File) Reset() { *m = BackupDescriptor_File{} } +func (m *BackupDescriptor_File) String() string { return proto.CompactTextString(m) } +func (*BackupDescriptor_File) ProtoMessage() {} +func (*BackupDescriptor_File) Descriptor() ([]byte, []int) { + return fileDescriptor_backup_60eb759f8ef88651, []int{0, 0} +} +func (m *BackupDescriptor_File) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *BackupDescriptor_File) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *BackupDescriptor_File) XXX_Merge(src proto.Message) { + xxx_messageInfo_BackupDescriptor_File.Merge(dst, src) +} +func (m *BackupDescriptor_File) XXX_Size() int { + return m.Size() +} +func (m *BackupDescriptor_File) XXX_DiscardUnknown() { + xxx_messageInfo_BackupDescriptor_File.DiscardUnknown(m) } -func (m *BackupDescriptor_File) Reset() { *m = BackupDescriptor_File{} } -func (m *BackupDescriptor_File) String() string { return proto.CompactTextString(m) } -func (*BackupDescriptor_File) ProtoMessage() {} -func (*BackupDescriptor_File) Descriptor() ([]byte, []int) { return fileDescriptorBackup, []int{0, 0} } +var xxx_messageInfo_BackupDescriptor_File proto.InternalMessageInfo type BackupDescriptor_DescriptorRevision struct { - Time cockroach_util_hlc.Timestamp `protobuf:"bytes,1,opt,name=time" json:"time"` - ID github_com_cockroachdb_cockroach_pkg_sql_sqlbase.ID `protobuf:"varint,2,opt,name=ID,proto3,casttype=github.com/cockroachdb/cockroach/pkg/sql/sqlbase.ID" json:"ID,omitempty"` - Desc *cockroach_sql_sqlbase1.Descriptor `protobuf:"bytes,3,opt,name=desc" json:"desc,omitempty"` + Time hlc.Timestamp `protobuf:"bytes,1,opt,name=time,proto3" json:"time"` + ID github_com_cockroachdb_cockroach_pkg_sql_sqlbase.ID `protobuf:"varint,2,opt,name=ID,proto3,casttype=github.com/cockroachdb/cockroach/pkg/sql/sqlbase.ID" json:"ID,omitempty"` + Desc *sqlbase.Descriptor `protobuf:"bytes,3,opt,name=desc,proto3" json:"desc,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *BackupDescriptor_DescriptorRevision) Reset() { *m = BackupDescriptor_DescriptorRevision{} } func (m *BackupDescriptor_DescriptorRevision) String() string { return proto.CompactTextString(m) } func (*BackupDescriptor_DescriptorRevision) ProtoMessage() {} func (*BackupDescriptor_DescriptorRevision) Descriptor() ([]byte, []int) { - return fileDescriptorBackup, []int{0, 1} + return fileDescriptor_backup_60eb759f8ef88651, []int{0, 1} +} +func (m *BackupDescriptor_DescriptorRevision) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *BackupDescriptor_DescriptorRevision) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *BackupDescriptor_DescriptorRevision) XXX_Merge(src proto.Message) { + xxx_messageInfo_BackupDescriptor_DescriptorRevision.Merge(dst, src) } +func (m *BackupDescriptor_DescriptorRevision) XXX_Size() int { + return m.Size() +} +func (m *BackupDescriptor_DescriptorRevision) XXX_DiscardUnknown() { + xxx_messageInfo_BackupDescriptor_DescriptorRevision.DiscardUnknown(m) +} + +var xxx_messageInfo_BackupDescriptor_DescriptorRevision proto.InternalMessageInfo func init() { proto.RegisterType((*BackupDescriptor)(nil), "cockroach.ccl.backupccl.BackupDescriptor") @@ -423,6 +491,9 @@ func encodeVarintBackup(dAtA []byte, offset int, v uint64) int { return offset + 1 } func (m *BackupDescriptor) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.StartTime.Size() @@ -489,6 +560,9 @@ func (m *BackupDescriptor) Size() (n int) { } func (m *BackupDescriptor_File) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.Span.Size() @@ -511,6 +585,9 @@ func (m *BackupDescriptor_File) Size() (n int) { } func (m *BackupDescriptor_DescriptorRevision) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.Time.Size() @@ -653,7 +730,7 @@ func (m *BackupDescriptor) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Spans = append(m.Spans, cockroach_roachpb1.Span{}) + m.Spans = append(m.Spans, roachpb.Span{}) if err := m.Spans[len(m.Spans)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -715,7 +792,7 @@ func (m *BackupDescriptor) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Descriptors = append(m.Descriptors, cockroach_sql_sqlbase1.Descriptor{}) + m.Descriptors = append(m.Descriptors, sqlbase.Descriptor{}) if err := m.Descriptors[len(m.Descriptors)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -938,6 +1015,17 @@ func (m *BackupDescriptor) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + var count int + for _, integer := range dAtA { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.CompleteDbs) == 0 { + m.CompleteDbs = make([]github_com_cockroachdb_cockroach_pkg_sql_sqlbase.ID, 0, elementCount) + } for iNdEx < postIndex { var v github_com_cockroachdb_cockroach_pkg_sql_sqlbase.ID for shift := uint(0); ; shift += 7 { @@ -985,7 +1073,7 @@ func (m *BackupDescriptor) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.IntroducedSpans = append(m.IntroducedSpans, cockroach_roachpb1.Span{}) + m.IntroducedSpans = append(m.IntroducedSpans, roachpb.Span{}) if err := m.IntroducedSpans[len(m.IntroducedSpans)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -1407,7 +1495,7 @@ func (m *BackupDescriptor_DescriptorRevision) Unmarshal(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Desc == nil { - m.Desc = &cockroach_sql_sqlbase1.Descriptor{} + m.Desc = &sqlbase.Descriptor{} } if err := m.Desc.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err @@ -1539,9 +1627,9 @@ var ( ErrIntOverflowBackup = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("ccl/backupccl/backup.proto", fileDescriptorBackup) } +func init() { proto.RegisterFile("ccl/backupccl/backup.proto", fileDescriptor_backup_60eb759f8ef88651) } -var fileDescriptorBackup = []byte{ +var fileDescriptor_backup_60eb759f8ef88651 = []byte{ // 869 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x55, 0x4d, 0x6f, 0xdb, 0x46, 0x10, 0x35, 0x2d, 0x5a, 0x1f, 0x23, 0xcb, 0xa1, 0xb7, 0x1f, 0x21, 0x84, 0x56, 0x52, 0x1a, 0x14, diff --git a/pkg/ccl/baseccl/encryption_options.pb.go b/pkg/ccl/baseccl/encryption_options.pb.go index 0ffb484103b7..7f278f22fc85 100644 --- a/pkg/ccl/baseccl/encryption_options.pb.go +++ b/pkg/ccl/baseccl/encryption_options.pb.go @@ -1,16 +1,6 @@ // Code generated by protoc-gen-gogo. DO NOT EDIT. // source: ccl/baseccl/encryption_options.proto -/* - Package baseccl is a generated protocol buffer package. - - It is generated from these files: - ccl/baseccl/encryption_options.proto - - It has these top-level messages: - EncryptionKeyFiles - EncryptionOptions -*/ package baseccl import proto "github.com/gogo/protobuf/proto" @@ -48,38 +38,86 @@ func (x EncryptionKeySource) String() string { return proto.EnumName(EncryptionKeySource_name, int32(x)) } func (EncryptionKeySource) EnumDescriptor() ([]byte, []int) { - return fileDescriptorEncryptionOptions, []int{0} + return fileDescriptor_encryption_options_da37372b1cf064db, []int{0} } // EncryptionKeyFiles is used when plain key files are passed. type EncryptionKeyFiles struct { - CurrentKey string `protobuf:"bytes,1,opt,name=current_key,json=currentKey,proto3" json:"current_key,omitempty"` - OldKey string `protobuf:"bytes,2,opt,name=old_key,json=oldKey,proto3" json:"old_key,omitempty"` + CurrentKey string `protobuf:"bytes,1,opt,name=current_key,json=currentKey,proto3" json:"current_key,omitempty"` + OldKey string `protobuf:"bytes,2,opt,name=old_key,json=oldKey,proto3" json:"old_key,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *EncryptionKeyFiles) Reset() { *m = EncryptionKeyFiles{} } func (m *EncryptionKeyFiles) String() string { return proto.CompactTextString(m) } func (*EncryptionKeyFiles) ProtoMessage() {} func (*EncryptionKeyFiles) Descriptor() ([]byte, []int) { - return fileDescriptorEncryptionOptions, []int{0} + return fileDescriptor_encryption_options_da37372b1cf064db, []int{0} +} +func (m *EncryptionKeyFiles) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) } +func (m *EncryptionKeyFiles) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *EncryptionKeyFiles) XXX_Merge(src proto.Message) { + xxx_messageInfo_EncryptionKeyFiles.Merge(dst, src) +} +func (m *EncryptionKeyFiles) XXX_Size() int { + return m.Size() +} +func (m *EncryptionKeyFiles) XXX_DiscardUnknown() { + xxx_messageInfo_EncryptionKeyFiles.DiscardUnknown(m) +} + +var xxx_messageInfo_EncryptionKeyFiles proto.InternalMessageInfo // EncryptionOptions defines the per-store encryption options. type EncryptionOptions struct { // The store key source. Defines which fields are useful. KeySource EncryptionKeySource `protobuf:"varint,1,opt,name=key_source,json=keySource,proto3,enum=cockroach.ccl.baseccl.EncryptionKeySource" json:"key_source,omitempty"` // Set if key_source == KeyFiles. - KeyFiles *EncryptionKeyFiles `protobuf:"bytes,2,opt,name=key_files,json=keyFiles" json:"key_files,omitempty"` + KeyFiles *EncryptionKeyFiles `protobuf:"bytes,2,opt,name=key_files,json=keyFiles,proto3" json:"key_files,omitempty"` // Default data key rotation in seconds. - DataKeyRotationPeriod int64 `protobuf:"varint,3,opt,name=data_key_rotation_period,json=dataKeyRotationPeriod,proto3" json:"data_key_rotation_period,omitempty"` + DataKeyRotationPeriod int64 `protobuf:"varint,3,opt,name=data_key_rotation_period,json=dataKeyRotationPeriod,proto3" json:"data_key_rotation_period,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *EncryptionOptions) Reset() { *m = EncryptionOptions{} } func (m *EncryptionOptions) String() string { return proto.CompactTextString(m) } func (*EncryptionOptions) ProtoMessage() {} func (*EncryptionOptions) Descriptor() ([]byte, []int) { - return fileDescriptorEncryptionOptions, []int{1} + return fileDescriptor_encryption_options_da37372b1cf064db, []int{1} } +func (m *EncryptionOptions) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *EncryptionOptions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *EncryptionOptions) XXX_Merge(src proto.Message) { + xxx_messageInfo_EncryptionOptions.Merge(dst, src) +} +func (m *EncryptionOptions) XXX_Size() int { + return m.Size() +} +func (m *EncryptionOptions) XXX_DiscardUnknown() { + xxx_messageInfo_EncryptionOptions.DiscardUnknown(m) +} + +var xxx_messageInfo_EncryptionOptions proto.InternalMessageInfo func init() { proto.RegisterType((*EncryptionKeyFiles)(nil), "cockroach.ccl.baseccl.EncryptionKeyFiles") @@ -164,6 +202,9 @@ func encodeVarintEncryptionOptions(dAtA []byte, offset int, v uint64) int { return offset + 1 } func (m *EncryptionKeyFiles) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.CurrentKey) @@ -178,6 +219,9 @@ func (m *EncryptionKeyFiles) Size() (n int) { } func (m *EncryptionOptions) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.KeySource != 0 { @@ -541,10 +585,10 @@ var ( ) func init() { - proto.RegisterFile("ccl/baseccl/encryption_options.proto", fileDescriptorEncryptionOptions) + proto.RegisterFile("ccl/baseccl/encryption_options.proto", fileDescriptor_encryption_options_da37372b1cf064db) } -var fileDescriptorEncryptionOptions = []byte{ +var fileDescriptor_encryption_options_da37372b1cf064db = []byte{ // 309 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x91, 0xcf, 0x4a, 0xc3, 0x40, 0x10, 0xc6, 0xbb, 0x16, 0xfa, 0x67, 0x2a, 0xa2, 0xab, 0xc5, 0xe2, 0x61, 0xad, 0xd5, 0x43, 0xed, diff --git a/pkg/ccl/changefeedccl/table_history.go b/pkg/ccl/changefeedccl/table_history.go index fb26b65024c8..c344b5884f8c 100644 --- a/pkg/ccl/changefeedccl/table_history.go +++ b/pkg/ccl/changefeedccl/table_history.go @@ -278,7 +278,7 @@ func fetchTableDescriptorVersions( } unsafeValue := it.UnsafeValue() if unsafeValue == nil { - return errors.Errorf(`"%s" was dropped or truncated`, origName) + return errors.Errorf(`"%v" was dropped or truncated`, origName) } value := roachpb.Value{RawBytes: unsafeValue} var desc sqlbase.Descriptor diff --git a/pkg/ccl/storageccl/engineccl/enginepbccl/key_registry.pb.go b/pkg/ccl/storageccl/engineccl/enginepbccl/key_registry.pb.go index 35f79a9864db..956fd19c1e9b 100644 --- a/pkg/ccl/storageccl/engineccl/enginepbccl/key_registry.pb.go +++ b/pkg/ccl/storageccl/engineccl/enginepbccl/key_registry.pb.go @@ -1,27 +1,13 @@ // Code generated by protoc-gen-gogo. DO NOT EDIT. // source: ccl/storageccl/engineccl/enginepbccl/key_registry.proto -/* - Package enginepbccl is a generated protocol buffer package. - - It is generated from these files: - ccl/storageccl/engineccl/enginepbccl/key_registry.proto - ccl/storageccl/engineccl/enginepbccl/stats.proto - - It has these top-level messages: - DataKeysRegistry - KeyInfo - SecretKey - EncryptionSettings - EncryptionStatus -*/ package enginepbccl import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" -import sortkeys "github.com/gogo/protobuf/sortkeys" +import github_com_gogo_protobuf_sortkeys "github.com/gogo/protobuf/sortkeys" import io "io" @@ -63,25 +49,53 @@ var EncryptionType_value = map[string]int32{ func (x EncryptionType) String() string { return proto.EnumName(EncryptionType_name, int32(x)) } -func (EncryptionType) EnumDescriptor() ([]byte, []int) { return fileDescriptorKeyRegistry, []int{0} } +func (EncryptionType) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_key_registry_e0594f6f5f853d74, []int{0} +} // DataKeysRegistry contains all data keys (including the raw key) as well // as store key information (excluding raw key). // This is written to disk. type DataKeysRegistry struct { // Map of key_id to KeyInfo (raw key is not included). - StoreKeys map[string]*KeyInfo `protobuf:"bytes,1,rep,name=store_keys,json=storeKeys" json:"store_keys,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value"` + StoreKeys map[string]*KeyInfo `protobuf:"bytes,1,rep,name=store_keys,json=storeKeys,proto3" json:"store_keys,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` // Map of key_id to SecretKey (raw key is included). - DataKeys map[string]*SecretKey `protobuf:"bytes,2,rep,name=data_keys,json=dataKeys" json:"data_keys,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value"` + DataKeys map[string]*SecretKey `protobuf:"bytes,2,rep,name=data_keys,json=dataKeys,proto3" json:"data_keys,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` // Active key IDs. Empty means no keys loaded yet. - ActiveStoreKeyId string `protobuf:"bytes,3,opt,name=active_store_key_id,json=activeStoreKeyId,proto3" json:"active_store_key_id,omitempty"` - ActiveDataKeyId string `protobuf:"bytes,4,opt,name=active_data_key_id,json=activeDataKeyId,proto3" json:"active_data_key_id,omitempty"` + ActiveStoreKeyId string `protobuf:"bytes,3,opt,name=active_store_key_id,json=activeStoreKeyId,proto3" json:"active_store_key_id,omitempty"` + ActiveDataKeyId string `protobuf:"bytes,4,opt,name=active_data_key_id,json=activeDataKeyId,proto3" json:"active_data_key_id,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *DataKeysRegistry) Reset() { *m = DataKeysRegistry{} } +func (m *DataKeysRegistry) String() string { return proto.CompactTextString(m) } +func (*DataKeysRegistry) ProtoMessage() {} +func (*DataKeysRegistry) Descriptor() ([]byte, []int) { + return fileDescriptor_key_registry_e0594f6f5f853d74, []int{0} +} +func (m *DataKeysRegistry) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *DataKeysRegistry) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *DataKeysRegistry) XXX_Merge(src proto.Message) { + xxx_messageInfo_DataKeysRegistry.Merge(dst, src) +} +func (m *DataKeysRegistry) XXX_Size() int { + return m.Size() +} +func (m *DataKeysRegistry) XXX_DiscardUnknown() { + xxx_messageInfo_DataKeysRegistry.DiscardUnknown(m) } -func (m *DataKeysRegistry) Reset() { *m = DataKeysRegistry{} } -func (m *DataKeysRegistry) String() string { return proto.CompactTextString(m) } -func (*DataKeysRegistry) ProtoMessage() {} -func (*DataKeysRegistry) Descriptor() ([]byte, []int) { return fileDescriptorKeyRegistry, []int{0} } +var xxx_messageInfo_DataKeysRegistry proto.InternalMessageInfo // KeyInfo contains information about the key, but not the key itself. // This is safe to pass around, log, and store. @@ -99,27 +113,79 @@ type KeyInfo struct { // This does not apply to store keys, only data keys. WasExposed bool `protobuf:"varint,5,opt,name=was_exposed,json=wasExposed,proto3" json:"was_exposed,omitempty"` // ID of the key that caused this key to be created. - ParentKeyId string `protobuf:"bytes,6,opt,name=parent_key_id,json=parentKeyId,proto3" json:"parent_key_id,omitempty"` + ParentKeyId string `protobuf:"bytes,6,opt,name=parent_key_id,json=parentKeyId,proto3" json:"parent_key_id,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *KeyInfo) Reset() { *m = KeyInfo{} } +func (m *KeyInfo) String() string { return proto.CompactTextString(m) } +func (*KeyInfo) ProtoMessage() {} +func (*KeyInfo) Descriptor() ([]byte, []int) { + return fileDescriptor_key_registry_e0594f6f5f853d74, []int{1} +} +func (m *KeyInfo) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *KeyInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *KeyInfo) XXX_Merge(src proto.Message) { + xxx_messageInfo_KeyInfo.Merge(dst, src) +} +func (m *KeyInfo) XXX_Size() int { + return m.Size() +} +func (m *KeyInfo) XXX_DiscardUnknown() { + xxx_messageInfo_KeyInfo.DiscardUnknown(m) } -func (m *KeyInfo) Reset() { *m = KeyInfo{} } -func (m *KeyInfo) String() string { return proto.CompactTextString(m) } -func (*KeyInfo) ProtoMessage() {} -func (*KeyInfo) Descriptor() ([]byte, []int) { return fileDescriptorKeyRegistry, []int{1} } +var xxx_messageInfo_KeyInfo proto.InternalMessageInfo // SecretKey contains the information about the key AND the raw key itself. // This should never be logged, displayed, or stored outside of the key registry. // The name is intended to make users of the key wary of the usage. type SecretKey struct { - Info *KeyInfo `protobuf:"bytes,1,opt,name=info" json:"info,omitempty"` + Info *KeyInfo `protobuf:"bytes,1,opt,name=info,proto3" json:"info,omitempty"` // The raw key. - Key []byte `protobuf:"bytes,2,opt,name=key,proto3" json:"key,omitempty"` + Key []byte `protobuf:"bytes,2,opt,name=key,proto3" json:"key,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *SecretKey) Reset() { *m = SecretKey{} } -func (m *SecretKey) String() string { return proto.CompactTextString(m) } -func (*SecretKey) ProtoMessage() {} -func (*SecretKey) Descriptor() ([]byte, []int) { return fileDescriptorKeyRegistry, []int{2} } +func (m *SecretKey) Reset() { *m = SecretKey{} } +func (m *SecretKey) String() string { return proto.CompactTextString(m) } +func (*SecretKey) ProtoMessage() {} +func (*SecretKey) Descriptor() ([]byte, []int) { + return fileDescriptor_key_registry_e0594f6f5f853d74, []int{2} +} +func (m *SecretKey) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SecretKey) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *SecretKey) XXX_Merge(src proto.Message) { + xxx_messageInfo_SecretKey.Merge(dst, src) +} +func (m *SecretKey) XXX_Size() int { + return m.Size() +} +func (m *SecretKey) XXX_DiscardUnknown() { + xxx_messageInfo_SecretKey.DiscardUnknown(m) +} + +var xxx_messageInfo_SecretKey proto.InternalMessageInfo // EncryptionSettings describes the encryption settings for a file. // This is stored as a protobuf.Any inside the FileEntry as described in: @@ -129,17 +195,45 @@ type EncryptionSettings struct { // Fields for AES-CTR. Empty when encryption_type = Plaintext. KeyId string `protobuf:"bytes,2,opt,name=key_id,json=keyId,proto3" json:"key_id,omitempty"` // len(nonce) + sizeof(counter) should add up to AES_Blocksize (128 bits). - Nonce []byte `protobuf:"bytes,3,opt,name=nonce,proto3" json:"nonce,omitempty"` - Counter uint32 `protobuf:"varint,4,opt,name=counter,proto3" json:"counter,omitempty"` + Nonce []byte `protobuf:"bytes,3,opt,name=nonce,proto3" json:"nonce,omitempty"` + Counter uint32 `protobuf:"varint,4,opt,name=counter,proto3" json:"counter,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *EncryptionSettings) Reset() { *m = EncryptionSettings{} } -func (m *EncryptionSettings) String() string { return proto.CompactTextString(m) } -func (*EncryptionSettings) ProtoMessage() {} -func (*EncryptionSettings) Descriptor() ([]byte, []int) { return fileDescriptorKeyRegistry, []int{3} } +func (m *EncryptionSettings) Reset() { *m = EncryptionSettings{} } +func (m *EncryptionSettings) String() string { return proto.CompactTextString(m) } +func (*EncryptionSettings) ProtoMessage() {} +func (*EncryptionSettings) Descriptor() ([]byte, []int) { + return fileDescriptor_key_registry_e0594f6f5f853d74, []int{3} +} +func (m *EncryptionSettings) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *EncryptionSettings) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *EncryptionSettings) XXX_Merge(src proto.Message) { + xxx_messageInfo_EncryptionSettings.Merge(dst, src) +} +func (m *EncryptionSettings) XXX_Size() int { + return m.Size() +} +func (m *EncryptionSettings) XXX_DiscardUnknown() { + xxx_messageInfo_EncryptionSettings.DiscardUnknown(m) +} + +var xxx_messageInfo_EncryptionSettings proto.InternalMessageInfo func init() { proto.RegisterType((*DataKeysRegistry)(nil), "cockroach.ccl.storageccl.engineccl.enginepbccl.DataKeysRegistry") + proto.RegisterMapType((map[string]*SecretKey)(nil), "cockroach.ccl.storageccl.engineccl.enginepbccl.DataKeysRegistry.DataKeysEntry") + proto.RegisterMapType((map[string]*KeyInfo)(nil), "cockroach.ccl.storageccl.engineccl.enginepbccl.DataKeysRegistry.StoreKeysEntry") proto.RegisterType((*KeyInfo)(nil), "cockroach.ccl.storageccl.engineccl.enginepbccl.KeyInfo") proto.RegisterType((*SecretKey)(nil), "cockroach.ccl.storageccl.engineccl.enginepbccl.SecretKey") proto.RegisterType((*EncryptionSettings)(nil), "cockroach.ccl.storageccl.engineccl.enginepbccl.EncryptionSettings") @@ -165,7 +259,7 @@ func (m *DataKeysRegistry) MarshalTo(dAtA []byte) (int, error) { for k := range m.StoreKeys { keysForStoreKeys = append(keysForStoreKeys, string(k)) } - sortkeys.Strings(keysForStoreKeys) + github_com_gogo_protobuf_sortkeys.Strings(keysForStoreKeys) for _, k := range keysForStoreKeys { dAtA[i] = 0xa i++ @@ -198,7 +292,7 @@ func (m *DataKeysRegistry) MarshalTo(dAtA []byte) (int, error) { for k := range m.DataKeys { keysForDataKeys = append(keysForDataKeys, string(k)) } - sortkeys.Strings(keysForDataKeys) + github_com_gogo_protobuf_sortkeys.Strings(keysForDataKeys) for _, k := range keysForDataKeys { dAtA[i] = 0x12 i++ @@ -381,6 +475,9 @@ func encodeVarintKeyRegistry(dAtA []byte, offset int, v uint64) int { return offset + 1 } func (m *DataKeysRegistry) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.StoreKeys) > 0 { @@ -421,6 +518,9 @@ func (m *DataKeysRegistry) Size() (n int) { } func (m *KeyInfo) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.EncryptionType != 0 { @@ -448,6 +548,9 @@ func (m *KeyInfo) Size() (n int) { } func (m *SecretKey) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Info != nil { @@ -462,6 +565,9 @@ func (m *SecretKey) Size() (n int) { } func (m *EncryptionSettings) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.EncryptionType != 0 { @@ -1411,10 +1517,10 @@ var ( ) func init() { - proto.RegisterFile("ccl/storageccl/engineccl/enginepbccl/key_registry.proto", fileDescriptorKeyRegistry) + proto.RegisterFile("ccl/storageccl/engineccl/enginepbccl/key_registry.proto", fileDescriptor_key_registry_e0594f6f5f853d74) } -var fileDescriptorKeyRegistry = []byte{ +var fileDescriptor_key_registry_e0594f6f5f853d74 = []byte{ // 585 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x94, 0xcf, 0x6e, 0xd3, 0x40, 0x10, 0xc6, 0xbb, 0x49, 0x93, 0x36, 0x93, 0x26, 0x8d, 0x96, 0x82, 0xa2, 0x1e, 0x4c, 0x14, 0x84, diff --git a/pkg/ccl/storageccl/engineccl/enginepbccl/stats.pb.go b/pkg/ccl/storageccl/engineccl/enginepbccl/stats.pb.go index 0ee99e3002a4..a15af6039c88 100644 --- a/pkg/ccl/storageccl/engineccl/enginepbccl/stats.pb.go +++ b/pkg/ccl/storageccl/engineccl/enginepbccl/stats.pb.go @@ -14,18 +14,50 @@ var _ = proto.Marshal var _ = fmt.Errorf var _ = math.Inf +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package + // EncryptionStatus contains encryption-related information. type EncryptionStatus struct { // Information about the active store key, if any. - ActiveStoreKey *KeyInfo `protobuf:"bytes,1,opt,name=active_store_key,json=activeStoreKey" json:"active_store_key,omitempty"` + ActiveStoreKey *KeyInfo `protobuf:"bytes,1,opt,name=active_store_key,json=activeStoreKey,proto3" json:"active_store_key,omitempty"` // Information about the active data key, if any. - ActiveDataKey *KeyInfo `protobuf:"bytes,2,opt,name=active_data_key,json=activeDataKey" json:"active_data_key,omitempty"` + ActiveDataKey *KeyInfo `protobuf:"bytes,2,opt,name=active_data_key,json=activeDataKey,proto3" json:"active_data_key,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *EncryptionStatus) Reset() { *m = EncryptionStatus{} } +func (m *EncryptionStatus) String() string { return proto.CompactTextString(m) } +func (*EncryptionStatus) ProtoMessage() {} +func (*EncryptionStatus) Descriptor() ([]byte, []int) { + return fileDescriptor_stats_c630a5af5cbf0575, []int{0} +} +func (m *EncryptionStatus) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *EncryptionStatus) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *EncryptionStatus) XXX_Merge(src proto.Message) { + xxx_messageInfo_EncryptionStatus.Merge(dst, src) +} +func (m *EncryptionStatus) XXX_Size() int { + return m.Size() +} +func (m *EncryptionStatus) XXX_DiscardUnknown() { + xxx_messageInfo_EncryptionStatus.DiscardUnknown(m) } -func (m *EncryptionStatus) Reset() { *m = EncryptionStatus{} } -func (m *EncryptionStatus) String() string { return proto.CompactTextString(m) } -func (*EncryptionStatus) ProtoMessage() {} -func (*EncryptionStatus) Descriptor() ([]byte, []int) { return fileDescriptorStats, []int{0} } +var xxx_messageInfo_EncryptionStatus proto.InternalMessageInfo func init() { proto.RegisterType((*EncryptionStatus)(nil), "cockroach.ccl.storageccl.engineccl.enginepbccl.EncryptionStatus") @@ -78,6 +110,9 @@ func encodeVarintStats(dAtA []byte, offset int, v uint64) int { return offset + 1 } func (m *EncryptionStatus) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.ActiveStoreKey != nil { @@ -326,10 +361,10 @@ var ( ) func init() { - proto.RegisterFile("ccl/storageccl/engineccl/enginepbccl/stats.proto", fileDescriptorStats) + proto.RegisterFile("ccl/storageccl/engineccl/enginepbccl/stats.proto", fileDescriptor_stats_c630a5af5cbf0575) } -var fileDescriptorStats = []byte{ +var fileDescriptor_stats_c630a5af5cbf0575 = []byte{ // 245 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x32, 0x48, 0x4e, 0xce, 0xd1, 0x2f, 0x2e, 0xc9, 0x2f, 0x4a, 0x4c, 0x4f, 0x05, 0x31, 0x53, 0xf3, 0xd2, 0x33, 0xf3, 0x90, 0x58, diff --git a/pkg/ccl/utilccl/licenseccl/license.pb.go b/pkg/ccl/utilccl/licenseccl/license.pb.go index 58b1f3e690bb..604d52ca96ba 100644 --- a/pkg/ccl/utilccl/licenseccl/license.pb.go +++ b/pkg/ccl/utilccl/licenseccl/license.pb.go @@ -1,15 +1,6 @@ // Code generated by protoc-gen-gogo. DO NOT EDIT. // source: ccl/utilccl/licenseccl/license.proto -/* - Package licenseccl is a generated protocol buffer package. - - It is generated from these files: - ccl/utilccl/licenseccl/license.proto - - It has these top-level messages: - License -*/ package licenseccl import proto "github.com/gogo/protobuf/proto" @@ -53,19 +44,47 @@ var License_Type_value = map[string]int32{ func (x License_Type) String() string { return proto.EnumName(License_Type_name, int32(x)) } -func (License_Type) EnumDescriptor() ([]byte, []int) { return fileDescriptorLicense, []int{0, 0} } +func (License_Type) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_license_20831399b5b16de7, []int{0, 0} +} type License struct { - ClusterID []github_com_cockroachdb_cockroach_pkg_util_uuid.UUID `protobuf:"bytes,1,rep,name=cluster_id,json=clusterId,customtype=github.com/cockroachdb/cockroach/pkg/util/uuid.UUID" json:"cluster_id"` - ValidUntilUnixSec int64 `protobuf:"varint,2,opt,name=valid_until_unix_sec,json=validUntilUnixSec,proto3" json:"valid_until_unix_sec,omitempty"` - Type License_Type `protobuf:"varint,3,opt,name=type,proto3,enum=cockroach.ccl.utilccl.licenseccl.License_Type" json:"type,omitempty"` - OrganizationName string `protobuf:"bytes,4,opt,name=organization_name,json=organizationName,proto3" json:"organization_name,omitempty"` + ClusterID []github_com_cockroachdb_cockroach_pkg_util_uuid.UUID `protobuf:"bytes,1,rep,name=cluster_id,json=clusterId,proto3,customtype=github.com/cockroachdb/cockroach/pkg/util/uuid.UUID" json:"cluster_id"` + ValidUntilUnixSec int64 `protobuf:"varint,2,opt,name=valid_until_unix_sec,json=validUntilUnixSec,proto3" json:"valid_until_unix_sec,omitempty"` + Type License_Type `protobuf:"varint,3,opt,name=type,proto3,enum=cockroach.ccl.utilccl.licenseccl.License_Type" json:"type,omitempty"` + OrganizationName string `protobuf:"bytes,4,opt,name=organization_name,json=organizationName,proto3" json:"organization_name,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *License) Reset() { *m = License{} } +func (m *License) String() string { return proto.CompactTextString(m) } +func (*License) ProtoMessage() {} +func (*License) Descriptor() ([]byte, []int) { + return fileDescriptor_license_20831399b5b16de7, []int{0} +} +func (m *License) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *License) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *License) XXX_Merge(src proto.Message) { + xxx_messageInfo_License.Merge(dst, src) +} +func (m *License) XXX_Size() int { + return m.Size() +} +func (m *License) XXX_DiscardUnknown() { + xxx_messageInfo_License.DiscardUnknown(m) } -func (m *License) Reset() { *m = License{} } -func (m *License) String() string { return proto.CompactTextString(m) } -func (*License) ProtoMessage() {} -func (*License) Descriptor() ([]byte, []int) { return fileDescriptorLicense, []int{0} } +var xxx_messageInfo_License proto.InternalMessageInfo func init() { proto.RegisterType((*License)(nil), "cockroach.ccl.utilccl.licenseccl.License") @@ -127,6 +146,9 @@ func encodeVarintLicense(dAtA []byte, offset int, v uint64) int { return offset + 1 } func (m *License) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.ClusterID) > 0 { @@ -415,9 +437,11 @@ var ( ErrIntOverflowLicense = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("ccl/utilccl/licenseccl/license.proto", fileDescriptorLicense) } +func init() { + proto.RegisterFile("ccl/utilccl/licenseccl/license.proto", fileDescriptor_license_20831399b5b16de7) +} -var fileDescriptorLicense = []byte{ +var fileDescriptor_license_20831399b5b16de7 = []byte{ // 354 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x84, 0x90, 0x41, 0x6b, 0xf2, 0x30, 0x1c, 0xc6, 0x1b, 0x95, 0xf7, 0xc5, 0xf0, 0xbe, 0x52, 0x8b, 0x87, 0xb2, 0x43, 0x2d, 0xe2, 0xa1, diff --git a/pkg/ccl/utilccl/licenseccl/license_test.go b/pkg/ccl/utilccl/licenseccl/license_test.go index a54236122881..fb7c17c00b4b 100644 --- a/pkg/ccl/utilccl/licenseccl/license_test.go +++ b/pkg/ccl/utilccl/licenseccl/license_test.go @@ -67,10 +67,10 @@ func TestLicense(t *testing.T) { var lic *License if tc.licType != -1 { s, err := License{ - tc.grantedTo, - tc.expiration.Unix(), - tc.licType, - fmt.Sprintf("tc-%d", i), + ClusterID: tc.grantedTo, + ValidUntilUnixSec: tc.expiration.Unix(), + Type: tc.licType, + OrganizationName: fmt.Sprintf("tc-%d", i), }.Encode() if err != nil { t.Fatal(err) diff --git a/pkg/cli/debug.go b/pkg/cli/debug.go index f8656829e95f..9a6982c77289 100644 --- a/pkg/cli/debug.go +++ b/pkg/cli/debug.go @@ -1158,7 +1158,7 @@ func removeDeadReplicas( if err != nil { return nil, err } - fmt.Printf("Scanning replicas on store %s for dead peers %v\n", storeIdent, + fmt.Printf("Scanning replicas on store %s for dead peers %v\n", storeIdent.String(), removeDeadReplicasOpts.deadStoreIDs) if _, ok := deadStoreIDs[storeIdent.StoreID]; ok { diff --git a/pkg/cli/flags_test.go b/pkg/cli/flags_test.go index 55dde9af7dd1..82f4bfee3218 100644 --- a/pkg/cli/flags_test.go +++ b/pkg/cli/flags_test.go @@ -427,9 +427,21 @@ func TestServerConnSettings(t *testing.T) { t.Errorf("%d. serverCfg.AdvertiseAddr expected '%s', but got '%s'. td.args was '%#v'.", i, td.expectedAdvertiseAddr, serverCfg.AdvertiseAddr, td.args) } - if td.expLocalityAdvertiseAddr != fmt.Sprintf("%s", serverCfg.LocalityAddresses) { + var locAddrStr strings.Builder + locAddrStr.WriteString("[") + for i, a := range serverCfg.LocalityAddresses { + if i > 0 { + locAddrStr.WriteString(" ") + } + fmt.Fprintf( + &locAddrStr, "{{%s %s} %s}", + a.Address.NetworkField, a.Address.AddressField, a.LocalityTier.String(), + ) + } + locAddrStr.WriteString("]") + if td.expLocalityAdvertiseAddr != locAddrStr.String() { t.Errorf("%d. serverCfg.expLocalityAdvertiseAddr expected '%s', but got '%s'. td.args was '%#v'.", - i, td.expLocalityAdvertiseAddr, serverCfg.LocalityAddresses, td.args) + i, td.expLocalityAdvertiseAddr, locAddrStr.String(), td.args) } }) } diff --git a/pkg/cli/systembench/systembenchpb/ping.pb.go b/pkg/cli/systembench/systembenchpb/ping.pb.go index fe6116e5a835..7df76f3be043 100644 --- a/pkg/cli/systembench/systembenchpb/ping.pb.go +++ b/pkg/cli/systembench/systembenchpb/ping.pb.go @@ -1,24 +1,16 @@ // Code generated by protoc-gen-gogo. DO NOT EDIT. // source: cli/systembench/systembenchpb/ping.proto -/* -Package systembench is a generated protocol buffer package. - -It is generated from these files: - cli/systembench/systembenchpb/ping.proto - -It has these top-level messages: - PingRequest - PingResponse -*/ package systembench import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" -import context "context" -import grpc "google.golang.org/grpc" +import ( + context "context" + grpc "google.golang.org/grpc" +) import io "io" @@ -34,22 +26,74 @@ var _ = math.Inf const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package type PingRequest struct { - Payload []byte `protobuf:"bytes,1,opt,name=payload" json:"payload,omitempty"` + Payload []byte `protobuf:"bytes,1,opt,name=payload" json:"payload,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *PingRequest) Reset() { *m = PingRequest{} } -func (m *PingRequest) String() string { return proto.CompactTextString(m) } -func (*PingRequest) ProtoMessage() {} -func (*PingRequest) Descriptor() ([]byte, []int) { return fileDescriptorPing, []int{0} } +func (m *PingRequest) Reset() { *m = PingRequest{} } +func (m *PingRequest) String() string { return proto.CompactTextString(m) } +func (*PingRequest) ProtoMessage() {} +func (*PingRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_ping_abd52f6b3a68c1dd, []int{0} +} +func (m *PingRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *PingRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *PingRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_PingRequest.Merge(dst, src) +} +func (m *PingRequest) XXX_Size() int { + return m.Size() +} +func (m *PingRequest) XXX_DiscardUnknown() { + xxx_messageInfo_PingRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_PingRequest proto.InternalMessageInfo type PingResponse struct { - Payload []byte `protobuf:"bytes,1,opt,name=payload" json:"payload,omitempty"` + Payload []byte `protobuf:"bytes,1,opt,name=payload" json:"payload,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *PingResponse) Reset() { *m = PingResponse{} } -func (m *PingResponse) String() string { return proto.CompactTextString(m) } -func (*PingResponse) ProtoMessage() {} -func (*PingResponse) Descriptor() ([]byte, []int) { return fileDescriptorPing, []int{1} } +func (m *PingResponse) Reset() { *m = PingResponse{} } +func (m *PingResponse) String() string { return proto.CompactTextString(m) } +func (*PingResponse) ProtoMessage() {} +func (*PingResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_ping_abd52f6b3a68c1dd, []int{1} +} +func (m *PingResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *PingResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *PingResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_PingResponse.Merge(dst, src) +} +func (m *PingResponse) XXX_Size() int { + return m.Size() +} +func (m *PingResponse) XXX_DiscardUnknown() { + xxx_messageInfo_PingResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_PingResponse proto.InternalMessageInfo func init() { proto.RegisterType((*PingRequest)(nil), "systembench.PingRequest") @@ -64,8 +108,9 @@ var _ grpc.ClientConn // is compatible with the grpc package it is being compiled against. const _ = grpc.SupportPackageIsVersion4 -// Client API for Pinger service - +// PingerClient is the client API for Pinger service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream. type PingerClient interface { Ping(ctx context.Context, in *PingRequest, opts ...grpc.CallOption) (*PingResponse, error) } @@ -80,15 +125,14 @@ func NewPingerClient(cc *grpc.ClientConn) PingerClient { func (c *pingerClient) Ping(ctx context.Context, in *PingRequest, opts ...grpc.CallOption) (*PingResponse, error) { out := new(PingResponse) - err := grpc.Invoke(ctx, "/systembench.Pinger/Ping", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/systembench.Pinger/Ping", in, out, opts...) if err != nil { return nil, err } return out, nil } -// Server API for Pinger service - +// PingerServer is the server API for Pinger service. type PingerServer interface { Ping(context.Context, *PingRequest) (*PingResponse, error) } @@ -186,6 +230,9 @@ func encodeVarintPing(dAtA []byte, offset int, v uint64) int { return offset + 1 } func (m *PingRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Payload != nil { @@ -196,6 +243,9 @@ func (m *PingRequest) Size() (n int) { } func (m *PingResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Payload != nil { @@ -485,9 +535,11 @@ var ( ErrIntOverflowPing = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("cli/systembench/systembenchpb/ping.proto", fileDescriptorPing) } +func init() { + proto.RegisterFile("cli/systembench/systembenchpb/ping.proto", fileDescriptor_ping_abd52f6b3a68c1dd) +} -var fileDescriptorPing = []byte{ +var fileDescriptor_ping_abd52f6b3a68c1dd = []byte{ // 167 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0xd2, 0x48, 0xce, 0xc9, 0xd4, 0x2f, 0xae, 0x2c, 0x2e, 0x49, 0xcd, 0x4d, 0x4a, 0xcd, 0x4b, 0xce, 0x40, 0x66, 0x17, 0x24, 0xe9, diff --git a/pkg/config/system.pb.go b/pkg/config/system.pb.go index 2e71bb89853d..2c39f10760a9 100644 --- a/pkg/config/system.pb.go +++ b/pkg/config/system.pb.go @@ -1,29 +1,12 @@ // Code generated by protoc-gen-gogo. DO NOT EDIT. // source: config/system.proto -/* - Package config is a generated protocol buffer package. - - It is generated from these files: - config/system.proto - config/zone.proto - - It has these top-level messages: - SystemConfigEntries - GCPolicy - Constraint - Constraints - LeasePreference - ZoneConfig - Subzone - SubzoneSpan -*/ package config import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" -import cockroach_roachpb1 "github.com/cockroachdb/cockroach/pkg/roachpb" +import roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" import io "io" @@ -39,13 +22,39 @@ var _ = math.Inf const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package type SystemConfigEntries struct { - Values []cockroach_roachpb1.KeyValue `protobuf:"bytes,1,rep,name=values" json:"values"` + Values []roachpb.KeyValue `protobuf:"bytes,1,rep,name=values" json:"values"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *SystemConfigEntries) Reset() { *m = SystemConfigEntries{} } -func (m *SystemConfigEntries) String() string { return proto.CompactTextString(m) } -func (*SystemConfigEntries) ProtoMessage() {} -func (*SystemConfigEntries) Descriptor() ([]byte, []int) { return fileDescriptorSystem, []int{0} } +func (m *SystemConfigEntries) Reset() { *m = SystemConfigEntries{} } +func (m *SystemConfigEntries) String() string { return proto.CompactTextString(m) } +func (*SystemConfigEntries) ProtoMessage() {} +func (*SystemConfigEntries) Descriptor() ([]byte, []int) { + return fileDescriptor_system_1131a0f264e73217, []int{0} +} +func (m *SystemConfigEntries) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SystemConfigEntries) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *SystemConfigEntries) XXX_Merge(src proto.Message) { + xxx_messageInfo_SystemConfigEntries.Merge(dst, src) +} +func (m *SystemConfigEntries) XXX_Size() int { + return m.Size() +} +func (m *SystemConfigEntries) XXX_DiscardUnknown() { + xxx_messageInfo_SystemConfigEntries.DiscardUnknown(m) +} + +var xxx_messageInfo_SystemConfigEntries proto.InternalMessageInfo func init() { proto.RegisterType((*SystemConfigEntries)(nil), "cockroach.config.SystemConfigEntries") @@ -90,6 +99,9 @@ func encodeVarintSystem(dAtA []byte, offset int, v uint64) int { return offset + 1 } func (m *SystemConfigEntries) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.Values) > 0 { @@ -169,7 +181,7 @@ func (m *SystemConfigEntries) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Values = append(m.Values, cockroach_roachpb1.KeyValue{}) + m.Values = append(m.Values, roachpb.KeyValue{}) if err := m.Values[len(m.Values)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -300,9 +312,9 @@ var ( ErrIntOverflowSystem = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("config/system.proto", fileDescriptorSystem) } +func init() { proto.RegisterFile("config/system.proto", fileDescriptor_system_1131a0f264e73217) } -var fileDescriptorSystem = []byte{ +var fileDescriptor_system_1131a0f264e73217 = []byte{ // 173 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x12, 0x4e, 0xce, 0xcf, 0x4b, 0xcb, 0x4c, 0xd7, 0x2f, 0xae, 0x2c, 0x2e, 0x49, 0xcd, 0xd5, 0x2b, 0x28, 0xca, 0x2f, 0xc9, 0x17, diff --git a/pkg/config/zone.pb.go b/pkg/config/zone.pb.go index e3da5f56a3f7..0b8223c5b966 100644 --- a/pkg/config/zone.pb.go +++ b/pkg/config/zone.pb.go @@ -18,6 +18,12 @@ var _ = proto.Marshal var _ = fmt.Errorf var _ = math.Inf +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package + type Constraint_Type int32 const ( @@ -57,7 +63,9 @@ func (x *Constraint_Type) UnmarshalJSON(data []byte) error { *x = Constraint_Type(value) return nil } -func (Constraint_Type) EnumDescriptor() ([]byte, []int) { return fileDescriptorZone, []int{1, 0} } +func (Constraint_Type) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_zone_09a0ac64b8ce7ce1, []int{1, 0} +} // GCPolicy defines garbage collection policies which apply to MVCC // values within a zone. @@ -69,13 +77,39 @@ type GCPolicy struct { // TTLSeconds specifies the maximum age of a value before it's // garbage collected. Only older versions of values are garbage // collected. Specifying <=0 mean older versions are never GC'd. - TTLSeconds int32 `protobuf:"varint,1,opt,name=ttl_seconds,json=ttlSeconds" json:"ttl_seconds"` + TTLSeconds int32 `protobuf:"varint,1,opt,name=ttl_seconds,json=ttlSeconds" json:"ttl_seconds"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *GCPolicy) Reset() { *m = GCPolicy{} } +func (m *GCPolicy) String() string { return proto.CompactTextString(m) } +func (*GCPolicy) ProtoMessage() {} +func (*GCPolicy) Descriptor() ([]byte, []int) { + return fileDescriptor_zone_09a0ac64b8ce7ce1, []int{0} +} +func (m *GCPolicy) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *GCPolicy) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *GCPolicy) XXX_Merge(src proto.Message) { + xxx_messageInfo_GCPolicy.Merge(dst, src) +} +func (m *GCPolicy) XXX_Size() int { + return m.Size() +} +func (m *GCPolicy) XXX_DiscardUnknown() { + xxx_messageInfo_GCPolicy.DiscardUnknown(m) } -func (m *GCPolicy) Reset() { *m = GCPolicy{} } -func (m *GCPolicy) String() string { return proto.CompactTextString(m) } -func (*GCPolicy) ProtoMessage() {} -func (*GCPolicy) Descriptor() ([]byte, []int) { return fileDescriptorZone, []int{0} } +var xxx_messageInfo_GCPolicy proto.InternalMessageInfo // Constraint constrains the stores a replica can be stored on. type Constraint struct { @@ -83,12 +117,38 @@ type Constraint struct { // Key is only set if this is a constraint on locality. Key string `protobuf:"bytes,2,opt,name=key" json:"key"` // Value to constrain to. - Value string `protobuf:"bytes,3,opt,name=value" json:"value"` + Value string `protobuf:"bytes,3,opt,name=value" json:"value"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Constraint) Reset() { *m = Constraint{} } +func (*Constraint) ProtoMessage() {} +func (*Constraint) Descriptor() ([]byte, []int) { + return fileDescriptor_zone_09a0ac64b8ce7ce1, []int{1} +} +func (m *Constraint) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Constraint) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *Constraint) XXX_Merge(src proto.Message) { + xxx_messageInfo_Constraint.Merge(dst, src) +} +func (m *Constraint) XXX_Size() int { + return m.Size() +} +func (m *Constraint) XXX_DiscardUnknown() { + xxx_messageInfo_Constraint.DiscardUnknown(m) } -func (m *Constraint) Reset() { *m = Constraint{} } -func (*Constraint) ProtoMessage() {} -func (*Constraint) Descriptor() ([]byte, []int) { return fileDescriptorZone, []int{1} } +var xxx_messageInfo_Constraint proto.InternalMessageInfo // Constraints is a collection of constraints. type Constraints struct { @@ -97,25 +157,77 @@ type Constraints struct { // to all replicas of the range. // As of v2.0, only REQUIRED constraints are allowed when num_replicas is // set to a non-zero value. - NumReplicas int32 `protobuf:"varint,7,opt,name=num_replicas,json=numReplicas" json:"num_replicas"` - Constraints []Constraint `protobuf:"bytes,6,rep,name=constraints" json:"constraints"` + NumReplicas int32 `protobuf:"varint,7,opt,name=num_replicas,json=numReplicas" json:"num_replicas"` + Constraints []Constraint `protobuf:"bytes,6,rep,name=constraints" json:"constraints"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Constraints) Reset() { *m = Constraints{} } +func (m *Constraints) String() string { return proto.CompactTextString(m) } +func (*Constraints) ProtoMessage() {} +func (*Constraints) Descriptor() ([]byte, []int) { + return fileDescriptor_zone_09a0ac64b8ce7ce1, []int{2} +} +func (m *Constraints) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Constraints) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *Constraints) XXX_Merge(src proto.Message) { + xxx_messageInfo_Constraints.Merge(dst, src) +} +func (m *Constraints) XXX_Size() int { + return m.Size() +} +func (m *Constraints) XXX_DiscardUnknown() { + xxx_messageInfo_Constraints.DiscardUnknown(m) } -func (m *Constraints) Reset() { *m = Constraints{} } -func (m *Constraints) String() string { return proto.CompactTextString(m) } -func (*Constraints) ProtoMessage() {} -func (*Constraints) Descriptor() ([]byte, []int) { return fileDescriptorZone, []int{2} } +var xxx_messageInfo_Constraints proto.InternalMessageInfo // LeasePreference specifies a preference about where range leases should be // located. type LeasePreference struct { - Constraints []Constraint `protobuf:"bytes,1,rep,name=constraints" json:"constraints" yaml:"constraints,flow"` + Constraints []Constraint `protobuf:"bytes,1,rep,name=constraints" json:"constraints" yaml:"constraints,flow"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *LeasePreference) Reset() { *m = LeasePreference{} } -func (m *LeasePreference) String() string { return proto.CompactTextString(m) } -func (*LeasePreference) ProtoMessage() {} -func (*LeasePreference) Descriptor() ([]byte, []int) { return fileDescriptorZone, []int{3} } +func (m *LeasePreference) Reset() { *m = LeasePreference{} } +func (m *LeasePreference) String() string { return proto.CompactTextString(m) } +func (*LeasePreference) ProtoMessage() {} +func (*LeasePreference) Descriptor() ([]byte, []int) { + return fileDescriptor_zone_09a0ac64b8ce7ce1, []int{3} +} +func (m *LeasePreference) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *LeasePreference) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *LeasePreference) XXX_Merge(src proto.Message) { + xxx_messageInfo_LeasePreference.Merge(dst, src) +} +func (m *LeasePreference) XXX_Size() int { + return m.Size() +} +func (m *LeasePreference) XXX_DiscardUnknown() { + xxx_messageInfo_LeasePreference.DiscardUnknown(m) +} + +var xxx_messageInfo_LeasePreference proto.InternalMessageInfo // ZoneConfig holds configuration that applies to one or more ranges. // @@ -163,13 +275,39 @@ type ZoneConfig struct { // allow binary searching. SubzoneSpans can be easily derived from a // TableDescriptor, but are denormalized here to make GetZoneConfigForKey // lookups efficient. - SubzoneSpans []SubzoneSpan `protobuf:"bytes,7,rep,name=subzone_spans,json=subzoneSpans" json:"subzone_spans" yaml:"-"` + SubzoneSpans []SubzoneSpan `protobuf:"bytes,7,rep,name=subzone_spans,json=subzoneSpans" json:"subzone_spans" yaml:"-"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ZoneConfig) Reset() { *m = ZoneConfig{} } +func (m *ZoneConfig) String() string { return proto.CompactTextString(m) } +func (*ZoneConfig) ProtoMessage() {} +func (*ZoneConfig) Descriptor() ([]byte, []int) { + return fileDescriptor_zone_09a0ac64b8ce7ce1, []int{4} +} +func (m *ZoneConfig) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ZoneConfig) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ZoneConfig) XXX_Merge(src proto.Message) { + xxx_messageInfo_ZoneConfig.Merge(dst, src) +} +func (m *ZoneConfig) XXX_Size() int { + return m.Size() +} +func (m *ZoneConfig) XXX_DiscardUnknown() { + xxx_messageInfo_ZoneConfig.DiscardUnknown(m) } -func (m *ZoneConfig) Reset() { *m = ZoneConfig{} } -func (m *ZoneConfig) String() string { return proto.CompactTextString(m) } -func (*ZoneConfig) ProtoMessage() {} -func (*ZoneConfig) Descriptor() ([]byte, []int) { return fileDescriptorZone, []int{4} } +var xxx_messageInfo_ZoneConfig proto.InternalMessageInfo type Subzone struct { // IndexID is the ID of the SQL table index that the subzone represents. It @@ -181,13 +319,39 @@ type Subzone struct { PartitionName string `protobuf:"bytes,2,opt,name=partition_name,json=partitionName" json:"partition_name"` // Config stores the ZoneConfig that applies to this Subzone. It never // contains nested subzones. - Config ZoneConfig `protobuf:"bytes,3,opt,name=config" json:"config"` + Config ZoneConfig `protobuf:"bytes,3,opt,name=config" json:"config"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *Subzone) Reset() { *m = Subzone{} } -func (m *Subzone) String() string { return proto.CompactTextString(m) } -func (*Subzone) ProtoMessage() {} -func (*Subzone) Descriptor() ([]byte, []int) { return fileDescriptorZone, []int{5} } +func (m *Subzone) Reset() { *m = Subzone{} } +func (m *Subzone) String() string { return proto.CompactTextString(m) } +func (*Subzone) ProtoMessage() {} +func (*Subzone) Descriptor() ([]byte, []int) { + return fileDescriptor_zone_09a0ac64b8ce7ce1, []int{5} +} +func (m *Subzone) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Subzone) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *Subzone) XXX_Merge(src proto.Message) { + xxx_messageInfo_Subzone.Merge(dst, src) +} +func (m *Subzone) XXX_Size() int { + return m.Size() +} +func (m *Subzone) XXX_DiscardUnknown() { + xxx_messageInfo_Subzone.DiscardUnknown(m) +} + +var xxx_messageInfo_Subzone proto.InternalMessageInfo type SubzoneSpan struct { // Key stores a key suffix that represents the inclusive lower bound for this @@ -203,13 +367,39 @@ type SubzoneSpan struct { EndKey github_com_cockroachdb_cockroach_pkg_roachpb.Key `protobuf:"bytes,2,opt,name=end_key,json=endKey,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.Key" json:"end_key,omitempty"` // SubzoneIndex is the slice index of the Subzone this span belongs to in the // parent ZoneConfig's Subzones field. - SubzoneIndex int32 `protobuf:"varint,3,opt,name=subzone_index,json=subzoneIndex" json:"subzone_index"` + SubzoneIndex int32 `protobuf:"varint,3,opt,name=subzone_index,json=subzoneIndex" json:"subzone_index"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *SubzoneSpan) Reset() { *m = SubzoneSpan{} } -func (m *SubzoneSpan) String() string { return proto.CompactTextString(m) } -func (*SubzoneSpan) ProtoMessage() {} -func (*SubzoneSpan) Descriptor() ([]byte, []int) { return fileDescriptorZone, []int{6} } +func (m *SubzoneSpan) Reset() { *m = SubzoneSpan{} } +func (m *SubzoneSpan) String() string { return proto.CompactTextString(m) } +func (*SubzoneSpan) ProtoMessage() {} +func (*SubzoneSpan) Descriptor() ([]byte, []int) { + return fileDescriptor_zone_09a0ac64b8ce7ce1, []int{6} +} +func (m *SubzoneSpan) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SubzoneSpan) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *SubzoneSpan) XXX_Merge(src proto.Message) { + xxx_messageInfo_SubzoneSpan.Merge(dst, src) +} +func (m *SubzoneSpan) XXX_Size() int { + return m.Size() +} +func (m *SubzoneSpan) XXX_DiscardUnknown() { + xxx_messageInfo_SubzoneSpan.DiscardUnknown(m) +} + +var xxx_messageInfo_SubzoneSpan proto.InternalMessageInfo func init() { proto.RegisterType((*GCPolicy)(nil), "cockroach.config.GCPolicy") @@ -1009,6 +1199,9 @@ func encodeVarintPopulateZone(dAtA []byte, v uint64) []byte { return dAtA } func (m *GCPolicy) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovZone(uint64(m.TTLSeconds)) @@ -1016,6 +1209,9 @@ func (m *GCPolicy) Size() (n int) { } func (m *Constraint) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovZone(uint64(m.Type)) @@ -1027,6 +1223,9 @@ func (m *Constraint) Size() (n int) { } func (m *Constraints) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.Constraints) > 0 { @@ -1040,6 +1239,9 @@ func (m *Constraints) Size() (n int) { } func (m *LeasePreference) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.Constraints) > 0 { @@ -1052,6 +1254,9 @@ func (m *LeasePreference) Size() (n int) { } func (m *ZoneConfig) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.RangeMinBytes != nil { @@ -1097,6 +1302,9 @@ func (m *ZoneConfig) Size() (n int) { } func (m *Subzone) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovZone(uint64(m.IndexID)) @@ -1108,6 +1316,9 @@ func (m *Subzone) Size() (n int) { } func (m *SubzoneSpan) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Key != nil { @@ -2183,9 +2394,9 @@ var ( ErrIntOverflowZone = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("config/zone.proto", fileDescriptorZone) } +func init() { proto.RegisterFile("config/zone.proto", fileDescriptor_zone_09a0ac64b8ce7ce1) } -var fileDescriptorZone = []byte{ +var fileDescriptor_zone_09a0ac64b8ce7ce1 = []byte{ // 843 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x54, 0xbf, 0x73, 0xe3, 0x44, 0x14, 0xf6, 0xda, 0x8e, 0xad, 0x7b, 0x76, 0x12, 0xdf, 0xde, 0x71, 0xf1, 0x05, 0xce, 0x32, 0x9a, diff --git a/pkg/config/zone_yaml.go b/pkg/config/zone_yaml.go index 8932ee5c6f30..85491ab8f9dc 100644 --- a/pkg/config/zone_yaml.go +++ b/pkg/config/zone_yaml.go @@ -298,3 +298,13 @@ func (c *ZoneConfig) UnmarshalYAML(unmarshal func(interface{}) error) error { *c = zoneConfigFromMarshalable(aux, *c) return nil } + +var _ yaml.Marshaler = GCPolicy{} + +// MarshalYAML implements yaml.Marshaler. +func (p GCPolicy) MarshalYAML() (interface{}, error) { + type marshalableGCPolicy struct { + TTLSeconds int32 `yaml:"ttlseconds"` + } + return marshalableGCPolicy{TTLSeconds: p.TTLSeconds}, nil +} diff --git a/pkg/gossip/gossip.pb.go b/pkg/gossip/gossip.pb.go index cbffd7c222ff..e192ae2eea5f 100644 --- a/pkg/gossip/gossip.pb.go +++ b/pkg/gossip/gossip.pb.go @@ -1,41 +1,24 @@ // Code generated by protoc-gen-gogo. DO NOT EDIT. // source: gossip/gossip.proto -/* - Package gossip is a generated protocol buffer package. - - It is generated from these files: - gossip/gossip.proto - - It has these top-level messages: - BootstrapInfo - Request - Response - ConnStatus - MetricSnap - OutgoingConnStatus - ClientStatus - ServerStatus - Connectivity - InfoStatus - Info -*/ package gossip import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" -import cockroach_roachpb1 "github.com/cockroachdb/cockroach/pkg/roachpb" -import cockroach_util_hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" -import cockroach_util "github.com/cockroachdb/cockroach/pkg/util" +import roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" +import util "github.com/cockroachdb/cockroach/pkg/util" +import hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" import github_com_cockroachdb_cockroach_pkg_roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" import github_com_cockroachdb_cockroach_pkg_util_uuid "github.com/cockroachdb/cockroach/pkg/util/uuid" -import context "context" -import grpc "google.golang.org/grpc" +import ( + context "context" + grpc "google.golang.org/grpc" +) -import sortkeys "github.com/gogo/protobuf/sortkeys" +import github_com_gogo_protobuf_sortkeys "github.com/gogo/protobuf/sortkeys" import io "io" @@ -54,35 +37,87 @@ const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package // gossip network from a cold start. type BootstrapInfo struct { // Addresses of other nodes in the cluster. - Addresses []cockroach_util.UnresolvedAddr `protobuf:"bytes,1,rep,name=addresses" json:"addresses"` + Addresses []util.UnresolvedAddr `protobuf:"bytes,1,rep,name=addresses,proto3" json:"addresses"` // Timestamp at which the bootstrap info was written. - Timestamp cockroach_util_hlc.Timestamp `protobuf:"bytes,2,opt,name=timestamp" json:"timestamp"` + Timestamp hlc.Timestamp `protobuf:"bytes,2,opt,name=timestamp,proto3" json:"timestamp"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *BootstrapInfo) Reset() { *m = BootstrapInfo{} } +func (m *BootstrapInfo) String() string { return proto.CompactTextString(m) } +func (*BootstrapInfo) ProtoMessage() {} +func (*BootstrapInfo) Descriptor() ([]byte, []int) { + return fileDescriptor_gossip_04b416ff741efab3, []int{0} +} +func (m *BootstrapInfo) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *BootstrapInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *BootstrapInfo) XXX_Merge(src proto.Message) { + xxx_messageInfo_BootstrapInfo.Merge(dst, src) +} +func (m *BootstrapInfo) XXX_Size() int { + return m.Size() +} +func (m *BootstrapInfo) XXX_DiscardUnknown() { + xxx_messageInfo_BootstrapInfo.DiscardUnknown(m) } -func (m *BootstrapInfo) Reset() { *m = BootstrapInfo{} } -func (m *BootstrapInfo) String() string { return proto.CompactTextString(m) } -func (*BootstrapInfo) ProtoMessage() {} -func (*BootstrapInfo) Descriptor() ([]byte, []int) { return fileDescriptorGossip, []int{0} } +var xxx_messageInfo_BootstrapInfo proto.InternalMessageInfo // Request is the request struct passed with the Gossip RPC. type Request struct { // Requesting node's ID. NodeID github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,1,opt,name=node_id,json=nodeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"node_id,omitempty"` // Address of the requesting client. - Addr cockroach_util.UnresolvedAddr `protobuf:"bytes,2,opt,name=addr" json:"addr"` + Addr util.UnresolvedAddr `protobuf:"bytes,2,opt,name=addr,proto3" json:"addr"` // Map of high water timestamps from infos originating at other // nodes, as seen by the requester. - HighWaterStamps map[github_com_cockroachdb_cockroach_pkg_roachpb.NodeID]int64 `protobuf:"bytes,3,rep,name=high_water_stamps,json=highWaterStamps,castkey=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"high_water_stamps" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` + HighWaterStamps map[github_com_cockroachdb_cockroach_pkg_roachpb.NodeID]int64 `protobuf:"bytes,3,rep,name=high_water_stamps,json=highWaterStamps,proto3,castkey=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"high_water_stamps" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` // Delta of Infos originating at sender. - Delta map[string]*Info `protobuf:"bytes,4,rep,name=delta" json:"delta,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value"` + Delta map[string]*Info `protobuf:"bytes,4,rep,name=delta,proto3" json:"delta,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` // Cluster ID to prevent illegal connections. - ClusterID github_com_cockroachdb_cockroach_pkg_util_uuid.UUID `protobuf:"bytes,5,opt,name=cluster_id,json=clusterId,proto3,customtype=github.com/cockroachdb/cockroach/pkg/util/uuid.UUID" json:"cluster_id"` + ClusterID github_com_cockroachdb_cockroach_pkg_util_uuid.UUID `protobuf:"bytes,5,opt,name=cluster_id,json=clusterId,proto3,customtype=github.com/cockroachdb/cockroach/pkg/util/uuid.UUID" json:"cluster_id"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Request) Reset() { *m = Request{} } +func (m *Request) String() string { return proto.CompactTextString(m) } +func (*Request) ProtoMessage() {} +func (*Request) Descriptor() ([]byte, []int) { + return fileDescriptor_gossip_04b416ff741efab3, []int{1} +} +func (m *Request) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Request) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *Request) XXX_Merge(src proto.Message) { + xxx_messageInfo_Request.Merge(dst, src) +} +func (m *Request) XXX_Size() int { + return m.Size() +} +func (m *Request) XXX_DiscardUnknown() { + xxx_messageInfo_Request.DiscardUnknown(m) } -func (m *Request) Reset() { *m = Request{} } -func (m *Request) String() string { return proto.CompactTextString(m) } -func (*Request) ProtoMessage() {} -func (*Request) Descriptor() ([]byte, []int) { return fileDescriptorGossip, []int{1} } +var xxx_messageInfo_Request proto.InternalMessageInfo // Response is returned from the Gossip.Gossip RPC. // Delta will be nil in the event that Alternate is set. @@ -90,110 +125,344 @@ type Response struct { // Responding Node's ID. NodeID github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,1,opt,name=node_id,json=nodeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"node_id,omitempty"` // Address of the responding client. - Addr cockroach_util.UnresolvedAddr `protobuf:"bytes,2,opt,name=addr" json:"addr"` + Addr util.UnresolvedAddr `protobuf:"bytes,2,opt,name=addr,proto3" json:"addr"` // Non-nil means client should retry with this address. - AlternateAddr *cockroach_util.UnresolvedAddr `protobuf:"bytes,3,opt,name=alternate_addr,json=alternateAddr" json:"alternate_addr,omitempty"` + AlternateAddr *util.UnresolvedAddr `protobuf:"bytes,3,opt,name=alternate_addr,json=alternateAddr,proto3" json:"alternate_addr,omitempty"` // Node ID of the alternate address, if alternate_addr is not nil. AlternateNodeID github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,4,opt,name=alternate_node_id,json=alternateNodeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"alternate_node_id,omitempty"` // Delta of Infos which are fresh according to the map of Node info messages // passed with the request. - Delta map[string]*Info `protobuf:"bytes,5,rep,name=delta" json:"delta,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value"` + Delta map[string]*Info `protobuf:"bytes,5,rep,name=delta,proto3" json:"delta,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` // Map of high water timestamps from infos originating at other // nodes, as seen by the responder. - HighWaterStamps map[github_com_cockroachdb_cockroach_pkg_roachpb.NodeID]int64 `protobuf:"bytes,6,rep,name=high_water_stamps,json=highWaterStamps,castkey=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"high_water_stamps" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` + HighWaterStamps map[github_com_cockroachdb_cockroach_pkg_roachpb.NodeID]int64 `protobuf:"bytes,6,rep,name=high_water_stamps,json=highWaterStamps,proto3,castkey=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"high_water_stamps" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Response) Reset() { *m = Response{} } +func (m *Response) String() string { return proto.CompactTextString(m) } +func (*Response) ProtoMessage() {} +func (*Response) Descriptor() ([]byte, []int) { + return fileDescriptor_gossip_04b416ff741efab3, []int{2} +} +func (m *Response) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Response) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *Response) XXX_Merge(src proto.Message) { + xxx_messageInfo_Response.Merge(dst, src) +} +func (m *Response) XXX_Size() int { + return m.Size() +} +func (m *Response) XXX_DiscardUnknown() { + xxx_messageInfo_Response.DiscardUnknown(m) } -func (m *Response) Reset() { *m = Response{} } -func (m *Response) String() string { return proto.CompactTextString(m) } -func (*Response) ProtoMessage() {} -func (*Response) Descriptor() ([]byte, []int) { return fileDescriptorGossip, []int{2} } +var xxx_messageInfo_Response proto.InternalMessageInfo type ConnStatus struct { - NodeID github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,1,opt,name=node_id,json=nodeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"node_id,omitempty"` - Address string `protobuf:"bytes,2,opt,name=address,proto3" json:"address,omitempty"` - AgeNanos int64 `protobuf:"varint,3,opt,name=age_nanos,json=ageNanos,proto3" json:"age_nanos,omitempty"` + NodeID github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,1,opt,name=node_id,json=nodeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"node_id,omitempty"` + Address string `protobuf:"bytes,2,opt,name=address,proto3" json:"address,omitempty"` + AgeNanos int64 `protobuf:"varint,3,opt,name=age_nanos,json=ageNanos,proto3" json:"age_nanos,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ConnStatus) Reset() { *m = ConnStatus{} } +func (*ConnStatus) ProtoMessage() {} +func (*ConnStatus) Descriptor() ([]byte, []int) { + return fileDescriptor_gossip_04b416ff741efab3, []int{3} +} +func (m *ConnStatus) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ConnStatus) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ConnStatus) XXX_Merge(src proto.Message) { + xxx_messageInfo_ConnStatus.Merge(dst, src) +} +func (m *ConnStatus) XXX_Size() int { + return m.Size() +} +func (m *ConnStatus) XXX_DiscardUnknown() { + xxx_messageInfo_ConnStatus.DiscardUnknown(m) } -func (m *ConnStatus) Reset() { *m = ConnStatus{} } -func (*ConnStatus) ProtoMessage() {} -func (*ConnStatus) Descriptor() ([]byte, []int) { return fileDescriptorGossip, []int{3} } +var xxx_messageInfo_ConnStatus proto.InternalMessageInfo type MetricSnap struct { - BytesReceived int64 `protobuf:"varint,2,opt,name=bytes_received,json=bytesReceived,proto3" json:"bytes_received,omitempty"` - BytesSent int64 `protobuf:"varint,3,opt,name=bytes_sent,json=bytesSent,proto3" json:"bytes_sent,omitempty"` - InfosReceived int64 `protobuf:"varint,4,opt,name=infos_received,json=infosReceived,proto3" json:"infos_received,omitempty"` - InfosSent int64 `protobuf:"varint,5,opt,name=infos_sent,json=infosSent,proto3" json:"infos_sent,omitempty"` - ConnsRefused int64 `protobuf:"varint,6,opt,name=conns_refused,json=connsRefused,proto3" json:"conns_refused,omitempty"` + BytesReceived int64 `protobuf:"varint,2,opt,name=bytes_received,json=bytesReceived,proto3" json:"bytes_received,omitempty"` + BytesSent int64 `protobuf:"varint,3,opt,name=bytes_sent,json=bytesSent,proto3" json:"bytes_sent,omitempty"` + InfosReceived int64 `protobuf:"varint,4,opt,name=infos_received,json=infosReceived,proto3" json:"infos_received,omitempty"` + InfosSent int64 `protobuf:"varint,5,opt,name=infos_sent,json=infosSent,proto3" json:"infos_sent,omitempty"` + ConnsRefused int64 `protobuf:"varint,6,opt,name=conns_refused,json=connsRefused,proto3" json:"conns_refused,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *MetricSnap) Reset() { *m = MetricSnap{} } +func (*MetricSnap) ProtoMessage() {} +func (*MetricSnap) Descriptor() ([]byte, []int) { + return fileDescriptor_gossip_04b416ff741efab3, []int{4} +} +func (m *MetricSnap) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *MetricSnap) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *MetricSnap) XXX_Merge(src proto.Message) { + xxx_messageInfo_MetricSnap.Merge(dst, src) +} +func (m *MetricSnap) XXX_Size() int { + return m.Size() +} +func (m *MetricSnap) XXX_DiscardUnknown() { + xxx_messageInfo_MetricSnap.DiscardUnknown(m) } -func (m *MetricSnap) Reset() { *m = MetricSnap{} } -func (*MetricSnap) ProtoMessage() {} -func (*MetricSnap) Descriptor() ([]byte, []int) { return fileDescriptorGossip, []int{4} } +var xxx_messageInfo_MetricSnap proto.InternalMessageInfo type OutgoingConnStatus struct { - ConnStatus `protobuf:"bytes,1,opt,name=conn_status,json=connStatus,embedded=conn_status" json:"conn_status"` - MetricSnap `protobuf:"bytes,2,opt,name=metrics,embedded=metrics" json:"metrics"` + ConnStatus `protobuf:"bytes,1,opt,name=conn_status,json=connStatus,proto3,embedded=conn_status" json:"conn_status"` + MetricSnap `protobuf:"bytes,2,opt,name=metrics,proto3,embedded=metrics" json:"metrics"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *OutgoingConnStatus) Reset() { *m = OutgoingConnStatus{} } +func (*OutgoingConnStatus) ProtoMessage() {} +func (*OutgoingConnStatus) Descriptor() ([]byte, []int) { + return fileDescriptor_gossip_04b416ff741efab3, []int{5} +} +func (m *OutgoingConnStatus) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *OutgoingConnStatus) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *OutgoingConnStatus) XXX_Merge(src proto.Message) { + xxx_messageInfo_OutgoingConnStatus.Merge(dst, src) +} +func (m *OutgoingConnStatus) XXX_Size() int { + return m.Size() +} +func (m *OutgoingConnStatus) XXX_DiscardUnknown() { + xxx_messageInfo_OutgoingConnStatus.DiscardUnknown(m) } -func (m *OutgoingConnStatus) Reset() { *m = OutgoingConnStatus{} } -func (*OutgoingConnStatus) ProtoMessage() {} -func (*OutgoingConnStatus) Descriptor() ([]byte, []int) { return fileDescriptorGossip, []int{5} } +var xxx_messageInfo_OutgoingConnStatus proto.InternalMessageInfo type ClientStatus struct { - ConnStatus []OutgoingConnStatus `protobuf:"bytes,1,rep,name=conn_status,json=connStatus" json:"conn_status"` - MaxConns int32 `protobuf:"varint,2,opt,name=max_conns,json=maxConns,proto3" json:"max_conns,omitempty"` + ConnStatus []OutgoingConnStatus `protobuf:"bytes,1,rep,name=conn_status,json=connStatus,proto3" json:"conn_status"` + MaxConns int32 `protobuf:"varint,2,opt,name=max_conns,json=maxConns,proto3" json:"max_conns,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *ClientStatus) Reset() { *m = ClientStatus{} } -func (*ClientStatus) ProtoMessage() {} -func (*ClientStatus) Descriptor() ([]byte, []int) { return fileDescriptorGossip, []int{6} } +func (m *ClientStatus) Reset() { *m = ClientStatus{} } +func (*ClientStatus) ProtoMessage() {} +func (*ClientStatus) Descriptor() ([]byte, []int) { + return fileDescriptor_gossip_04b416ff741efab3, []int{6} +} +func (m *ClientStatus) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ClientStatus) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ClientStatus) XXX_Merge(src proto.Message) { + xxx_messageInfo_ClientStatus.Merge(dst, src) +} +func (m *ClientStatus) XXX_Size() int { + return m.Size() +} +func (m *ClientStatus) XXX_DiscardUnknown() { + xxx_messageInfo_ClientStatus.DiscardUnknown(m) +} + +var xxx_messageInfo_ClientStatus proto.InternalMessageInfo type ServerStatus struct { - ConnStatus []ConnStatus `protobuf:"bytes,1,rep,name=conn_status,json=connStatus" json:"conn_status"` - MaxConns int32 `protobuf:"varint,2,opt,name=max_conns,json=maxConns,proto3" json:"max_conns,omitempty"` - MetricSnap `protobuf:"bytes,3,opt,name=metrics,embedded=metrics" json:"metrics"` + ConnStatus []ConnStatus `protobuf:"bytes,1,rep,name=conn_status,json=connStatus,proto3" json:"conn_status"` + MaxConns int32 `protobuf:"varint,2,opt,name=max_conns,json=maxConns,proto3" json:"max_conns,omitempty"` + MetricSnap `protobuf:"bytes,3,opt,name=metrics,proto3,embedded=metrics" json:"metrics"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ServerStatus) Reset() { *m = ServerStatus{} } +func (*ServerStatus) ProtoMessage() {} +func (*ServerStatus) Descriptor() ([]byte, []int) { + return fileDescriptor_gossip_04b416ff741efab3, []int{7} +} +func (m *ServerStatus) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ServerStatus) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ServerStatus) XXX_Merge(src proto.Message) { + xxx_messageInfo_ServerStatus.Merge(dst, src) +} +func (m *ServerStatus) XXX_Size() int { + return m.Size() +} +func (m *ServerStatus) XXX_DiscardUnknown() { + xxx_messageInfo_ServerStatus.DiscardUnknown(m) } -func (m *ServerStatus) Reset() { *m = ServerStatus{} } -func (*ServerStatus) ProtoMessage() {} -func (*ServerStatus) Descriptor() ([]byte, []int) { return fileDescriptorGossip, []int{7} } +var xxx_messageInfo_ServerStatus proto.InternalMessageInfo type Connectivity struct { - SentinelNodeID github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,1,opt,name=sentinel_node_id,json=sentinelNodeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"sentinel_node_id,omitempty"` - ClientConns []Connectivity_Conn `protobuf:"bytes,2,rep,name=client_conns,json=clientConns" json:"client_conns"` + SentinelNodeID github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,1,opt,name=sentinel_node_id,json=sentinelNodeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"sentinel_node_id,omitempty"` + ClientConns []Connectivity_Conn `protobuf:"bytes,2,rep,name=client_conns,json=clientConns,proto3" json:"client_conns"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Connectivity) Reset() { *m = Connectivity{} } +func (*Connectivity) ProtoMessage() {} +func (*Connectivity) Descriptor() ([]byte, []int) { + return fileDescriptor_gossip_04b416ff741efab3, []int{8} +} +func (m *Connectivity) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Connectivity) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *Connectivity) XXX_Merge(src proto.Message) { + xxx_messageInfo_Connectivity.Merge(dst, src) +} +func (m *Connectivity) XXX_Size() int { + return m.Size() +} +func (m *Connectivity) XXX_DiscardUnknown() { + xxx_messageInfo_Connectivity.DiscardUnknown(m) } -func (m *Connectivity) Reset() { *m = Connectivity{} } -func (*Connectivity) ProtoMessage() {} -func (*Connectivity) Descriptor() ([]byte, []int) { return fileDescriptorGossip, []int{8} } +var xxx_messageInfo_Connectivity proto.InternalMessageInfo type Connectivity_Conn struct { - SourceID github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,1,opt,name=source_id,json=sourceId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"source_id,omitempty"` - TargetID github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,2,opt,name=target_id,json=targetId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"target_id,omitempty"` + SourceID github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,1,opt,name=source_id,json=sourceId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"source_id,omitempty"` + TargetID github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,2,opt,name=target_id,json=targetId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"target_id,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *Connectivity_Conn) Reset() { *m = Connectivity_Conn{} } -func (m *Connectivity_Conn) String() string { return proto.CompactTextString(m) } -func (*Connectivity_Conn) ProtoMessage() {} -func (*Connectivity_Conn) Descriptor() ([]byte, []int) { return fileDescriptorGossip, []int{8, 0} } +func (m *Connectivity_Conn) Reset() { *m = Connectivity_Conn{} } +func (m *Connectivity_Conn) String() string { return proto.CompactTextString(m) } +func (*Connectivity_Conn) ProtoMessage() {} +func (*Connectivity_Conn) Descriptor() ([]byte, []int) { + return fileDescriptor_gossip_04b416ff741efab3, []int{8, 0} +} +func (m *Connectivity_Conn) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Connectivity_Conn) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *Connectivity_Conn) XXX_Merge(src proto.Message) { + xxx_messageInfo_Connectivity_Conn.Merge(dst, src) +} +func (m *Connectivity_Conn) XXX_Size() int { + return m.Size() +} +func (m *Connectivity_Conn) XXX_DiscardUnknown() { + xxx_messageInfo_Connectivity_Conn.DiscardUnknown(m) +} + +var xxx_messageInfo_Connectivity_Conn proto.InternalMessageInfo // InfoStatus contains information about the current status of the infoStore. type InfoStatus struct { - Infos map[string]Info `protobuf:"bytes,1,rep,name=infos" json:"infos" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value"` - Client ClientStatus `protobuf:"bytes,2,opt,name=client" json:"client"` - Server ServerStatus `protobuf:"bytes,3,opt,name=server" json:"server"` - Connectivity Connectivity `protobuf:"bytes,4,opt,name=connectivity" json:"connectivity"` + Infos map[string]Info `protobuf:"bytes,1,rep,name=infos,proto3" json:"infos" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Client ClientStatus `protobuf:"bytes,2,opt,name=client,proto3" json:"client"` + Server ServerStatus `protobuf:"bytes,3,opt,name=server,proto3" json:"server"` + Connectivity Connectivity `protobuf:"bytes,4,opt,name=connectivity,proto3" json:"connectivity"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *InfoStatus) Reset() { *m = InfoStatus{} } +func (m *InfoStatus) String() string { return proto.CompactTextString(m) } +func (*InfoStatus) ProtoMessage() {} +func (*InfoStatus) Descriptor() ([]byte, []int) { + return fileDescriptor_gossip_04b416ff741efab3, []int{9} +} +func (m *InfoStatus) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *InfoStatus) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *InfoStatus) XXX_Merge(src proto.Message) { + xxx_messageInfo_InfoStatus.Merge(dst, src) +} +func (m *InfoStatus) XXX_Size() int { + return m.Size() +} +func (m *InfoStatus) XXX_DiscardUnknown() { + xxx_messageInfo_InfoStatus.DiscardUnknown(m) } -func (m *InfoStatus) Reset() { *m = InfoStatus{} } -func (m *InfoStatus) String() string { return proto.CompactTextString(m) } -func (*InfoStatus) ProtoMessage() {} -func (*InfoStatus) Descriptor() ([]byte, []int) { return fileDescriptorGossip, []int{9} } +var xxx_messageInfo_InfoStatus proto.InternalMessageInfo // Info is the basic unit of information traded over the // gossip network. type Info struct { - Value cockroach_roachpb1.Value `protobuf:"bytes,1,opt,name=value" json:"value"` + Value roachpb.Value `protobuf:"bytes,1,opt,name=value,proto3" json:"value"` // Wall time of info when generated by originating node (Unix-nanos). OrigStamp int64 `protobuf:"varint,2,opt,name=orig_stamp,json=origStamp,proto3" json:"orig_stamp,omitempty"` // Wall time when info is to be discarded (Unix-nanos). @@ -203,18 +472,48 @@ type Info struct { // Originating node's ID. NodeID github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,5,opt,name=node_id,json=nodeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"node_id,omitempty"` // Peer node ID which passed this info. - PeerID github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,6,opt,name=peer_id,json=peerId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"peer_id,omitempty"` + PeerID github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,6,opt,name=peer_id,json=peerId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"peer_id,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *Info) Reset() { *m = Info{} } -func (m *Info) String() string { return proto.CompactTextString(m) } -func (*Info) ProtoMessage() {} -func (*Info) Descriptor() ([]byte, []int) { return fileDescriptorGossip, []int{10} } +func (m *Info) Reset() { *m = Info{} } +func (m *Info) String() string { return proto.CompactTextString(m) } +func (*Info) ProtoMessage() {} +func (*Info) Descriptor() ([]byte, []int) { + return fileDescriptor_gossip_04b416ff741efab3, []int{10} +} +func (m *Info) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Info) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *Info) XXX_Merge(src proto.Message) { + xxx_messageInfo_Info.Merge(dst, src) +} +func (m *Info) XXX_Size() int { + return m.Size() +} +func (m *Info) XXX_DiscardUnknown() { + xxx_messageInfo_Info.DiscardUnknown(m) +} + +var xxx_messageInfo_Info proto.InternalMessageInfo func init() { proto.RegisterType((*BootstrapInfo)(nil), "cockroach.gossip.BootstrapInfo") proto.RegisterType((*Request)(nil), "cockroach.gossip.Request") + proto.RegisterMapType((map[string]*Info)(nil), "cockroach.gossip.Request.DeltaEntry") + proto.RegisterMapType((map[github_com_cockroachdb_cockroach_pkg_roachpb.NodeID]int64)(nil), "cockroach.gossip.Request.HighWaterStampsEntry") proto.RegisterType((*Response)(nil), "cockroach.gossip.Response") + proto.RegisterMapType((map[string]*Info)(nil), "cockroach.gossip.Response.DeltaEntry") + proto.RegisterMapType((map[github_com_cockroachdb_cockroach_pkg_roachpb.NodeID]int64)(nil), "cockroach.gossip.Response.HighWaterStampsEntry") proto.RegisterType((*ConnStatus)(nil), "cockroach.gossip.ConnStatus") proto.RegisterType((*MetricSnap)(nil), "cockroach.gossip.MetricSnap") proto.RegisterType((*OutgoingConnStatus)(nil), "cockroach.gossip.OutgoingConnStatus") @@ -223,6 +522,7 @@ func init() { proto.RegisterType((*Connectivity)(nil), "cockroach.gossip.Connectivity") proto.RegisterType((*Connectivity_Conn)(nil), "cockroach.gossip.Connectivity.Conn") proto.RegisterType((*InfoStatus)(nil), "cockroach.gossip.InfoStatus") + proto.RegisterMapType((map[string]Info)(nil), "cockroach.gossip.InfoStatus.InfosEntry") proto.RegisterType((*Info)(nil), "cockroach.gossip.Info") } @@ -234,8 +534,9 @@ var _ grpc.ClientConn // is compatible with the grpc package it is being compiled against. const _ = grpc.SupportPackageIsVersion4 -// Client API for Gossip service - +// GossipClient is the client API for Gossip service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream. type GossipClient interface { Gossip(ctx context.Context, opts ...grpc.CallOption) (Gossip_GossipClient, error) } @@ -249,7 +550,7 @@ func NewGossipClient(cc *grpc.ClientConn) GossipClient { } func (c *gossipClient) Gossip(ctx context.Context, opts ...grpc.CallOption) (Gossip_GossipClient, error) { - stream, err := grpc.NewClientStream(ctx, &_Gossip_serviceDesc.Streams[0], c.cc, "/cockroach.gossip.Gossip/Gossip", opts...) + stream, err := c.cc.NewStream(ctx, &_Gossip_serviceDesc.Streams[0], "/cockroach.gossip.Gossip/Gossip", opts...) if err != nil { return nil, err } @@ -279,8 +580,7 @@ func (x *gossipGossipClient) Recv() (*Response, error) { return m, nil } -// Server API for Gossip service - +// GossipServer is the server API for Gossip service. type GossipServer interface { Gossip(Gossip_GossipServer) error } @@ -401,7 +701,7 @@ func (m *Request) MarshalTo(dAtA []byte) (int, error) { for k := range m.HighWaterStamps { keysForHighWaterStamps = append(keysForHighWaterStamps, int32(k)) } - sortkeys.Int32s(keysForHighWaterStamps) + github_com_gogo_protobuf_sortkeys.Int32s(keysForHighWaterStamps) for _, k := range keysForHighWaterStamps { dAtA[i] = 0x1a i++ @@ -421,7 +721,7 @@ func (m *Request) MarshalTo(dAtA []byte) (int, error) { for k := range m.Delta { keysForDelta = append(keysForDelta, string(k)) } - sortkeys.Strings(keysForDelta) + github_com_gogo_protobuf_sortkeys.Strings(keysForDelta) for _, k := range keysForDelta { dAtA[i] = 0x22 i++ @@ -508,7 +808,7 @@ func (m *Response) MarshalTo(dAtA []byte) (int, error) { for k := range m.Delta { keysForDelta = append(keysForDelta, string(k)) } - sortkeys.Strings(keysForDelta) + github_com_gogo_protobuf_sortkeys.Strings(keysForDelta) for _, k := range keysForDelta { dAtA[i] = 0x2a i++ @@ -541,7 +841,7 @@ func (m *Response) MarshalTo(dAtA []byte) (int, error) { for k := range m.HighWaterStamps { keysForHighWaterStamps = append(keysForHighWaterStamps, int32(k)) } - sortkeys.Int32s(keysForHighWaterStamps) + github_com_gogo_protobuf_sortkeys.Int32s(keysForHighWaterStamps) for _, k := range keysForHighWaterStamps { dAtA[i] = 0x32 i++ @@ -831,7 +1131,7 @@ func (m *InfoStatus) MarshalTo(dAtA []byte) (int, error) { for k := range m.Infos { keysForInfos = append(keysForInfos, string(k)) } - sortkeys.Strings(keysForInfos) + github_com_gogo_protobuf_sortkeys.Strings(keysForInfos) for _, k := range keysForInfos { dAtA[i] = 0xa i++ @@ -945,6 +1245,9 @@ func encodeVarintGossip(dAtA []byte, offset int, v uint64) int { return offset + 1 } func (m *BootstrapInfo) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.Addresses) > 0 { @@ -959,6 +1262,9 @@ func (m *BootstrapInfo) Size() (n int) { } func (m *Request) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.NodeID != 0 { @@ -993,6 +1299,9 @@ func (m *Request) Size() (n int) { } func (m *Response) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.NodeID != 0 { @@ -1032,6 +1341,9 @@ func (m *Response) Size() (n int) { } func (m *ConnStatus) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.NodeID != 0 { @@ -1048,6 +1360,9 @@ func (m *ConnStatus) Size() (n int) { } func (m *MetricSnap) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.BytesReceived != 0 { @@ -1069,6 +1384,9 @@ func (m *MetricSnap) Size() (n int) { } func (m *OutgoingConnStatus) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.ConnStatus.Size() @@ -1079,6 +1397,9 @@ func (m *OutgoingConnStatus) Size() (n int) { } func (m *ClientStatus) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.ConnStatus) > 0 { @@ -1094,6 +1415,9 @@ func (m *ClientStatus) Size() (n int) { } func (m *ServerStatus) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.ConnStatus) > 0 { @@ -1111,6 +1435,9 @@ func (m *ServerStatus) Size() (n int) { } func (m *Connectivity) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.SentinelNodeID != 0 { @@ -1126,6 +1453,9 @@ func (m *Connectivity) Size() (n int) { } func (m *Connectivity_Conn) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.SourceID != 0 { @@ -1138,6 +1468,9 @@ func (m *Connectivity_Conn) Size() (n int) { } func (m *InfoStatus) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.Infos) > 0 { @@ -1159,6 +1492,9 @@ func (m *InfoStatus) Size() (n int) { } func (m *Info) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.Value.Size() @@ -1249,7 +1585,7 @@ func (m *BootstrapInfo) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Addresses = append(m.Addresses, cockroach_util.UnresolvedAddr{}) + m.Addresses = append(m.Addresses, util.UnresolvedAddr{}) if err := m.Addresses[len(m.Addresses)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -1758,7 +2094,7 @@ func (m *Response) Unmarshal(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.AlternateAddr == nil { - m.AlternateAddr = &cockroach_util.UnresolvedAddr{} + m.AlternateAddr = &util.UnresolvedAddr{} } if err := m.AlternateAddr.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err @@ -3356,9 +3692,9 @@ var ( ErrIntOverflowGossip = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("gossip/gossip.proto", fileDescriptorGossip) } +func init() { proto.RegisterFile("gossip/gossip.proto", fileDescriptor_gossip_04b416ff741efab3) } -var fileDescriptorGossip = []byte{ +var fileDescriptor_gossip_04b416ff741efab3 = []byte{ // 1211 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xcc, 0x57, 0xcf, 0x6f, 0x1b, 0x45, 0x14, 0xce, 0xc6, 0x3f, 0xe2, 0x7d, 0x76, 0xd2, 0x76, 0xa8, 0x90, 0x71, 0xa9, 0x1d, 0xb9, 0x54, diff --git a/pkg/internal/client/lease.pb.go b/pkg/internal/client/lease.pb.go index 39c56d1d521f..e0c2ed18d689 100644 --- a/pkg/internal/client/lease.pb.go +++ b/pkg/internal/client/lease.pb.go @@ -1,21 +1,12 @@ // Code generated by protoc-gen-gogo. DO NOT EDIT. // source: internal/client/lease.proto -/* - Package client is a generated protocol buffer package. - - It is generated from these files: - internal/client/lease.proto - - It has these top-level messages: - LeaseVal -*/ package client import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" -import cockroach_util_hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" +import hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" import io "io" @@ -35,13 +26,39 @@ type LeaseVal struct { // owns the lease. Owner string `protobuf:"bytes,1,opt,name=owner" json:"owner"` // The expiration time of the lease. - Expiration cockroach_util_hlc.Timestamp `protobuf:"bytes,2,opt,name=expiration" json:"expiration"` + Expiration hlc.Timestamp `protobuf:"bytes,2,opt,name=expiration" json:"expiration"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *LeaseVal) Reset() { *m = LeaseVal{} } -func (m *LeaseVal) String() string { return proto.CompactTextString(m) } -func (*LeaseVal) ProtoMessage() {} -func (*LeaseVal) Descriptor() ([]byte, []int) { return fileDescriptorLease, []int{0} } +func (m *LeaseVal) Reset() { *m = LeaseVal{} } +func (m *LeaseVal) String() string { return proto.CompactTextString(m) } +func (*LeaseVal) ProtoMessage() {} +func (*LeaseVal) Descriptor() ([]byte, []int) { + return fileDescriptor_lease_aad6f82586d4116a, []int{0} +} +func (m *LeaseVal) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *LeaseVal) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *LeaseVal) XXX_Merge(src proto.Message) { + xxx_messageInfo_LeaseVal.Merge(dst, src) +} +func (m *LeaseVal) XXX_Size() int { + return m.Size() +} +func (m *LeaseVal) XXX_DiscardUnknown() { + xxx_messageInfo_LeaseVal.DiscardUnknown(m) +} + +var xxx_messageInfo_LeaseVal proto.InternalMessageInfo func init() { proto.RegisterType((*LeaseVal)(nil), "cockroach.client.LeaseVal") @@ -86,6 +103,9 @@ func encodeVarintLease(dAtA []byte, offset int, v uint64) int { return offset + 1 } func (m *LeaseVal) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Owner) @@ -322,9 +342,9 @@ var ( ErrIntOverflowLease = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("internal/client/lease.proto", fileDescriptorLease) } +func init() { proto.RegisterFile("internal/client/lease.proto", fileDescriptor_lease_aad6f82586d4116a) } -var fileDescriptorLease = []byte{ +var fileDescriptor_lease_aad6f82586d4116a = []byte{ // 207 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x92, 0xce, 0xcc, 0x2b, 0x49, 0x2d, 0xca, 0x4b, 0xcc, 0xd1, 0x4f, 0xce, 0xc9, 0x4c, 0xcd, 0x2b, 0xd1, 0xcf, 0x49, 0x4d, 0x2c, diff --git a/pkg/jobs/jobspb/jobs.pb.go b/pkg/jobs/jobspb/jobs.pb.go index d12c5b78800c..e8e03b72bfe9 100644 --- a/pkg/jobs/jobspb/jobs.pb.go +++ b/pkg/jobs/jobspb/jobs.pb.go @@ -1,46 +1,20 @@ // Code generated by protoc-gen-gogo. DO NOT EDIT. // source: jobs/jobspb/jobs.proto -/* - Package jobspb is a generated protocol buffer package. - - It is generated from these files: - jobs/jobspb/jobs.proto - - It has these top-level messages: - Lease - BackupDetails - BackupProgress - RestoreDetails - RestoreProgress - ImportDetails - ImportProgress - ResumeSpanList - DroppedTableDetails - SchemaChangeDetails - SchemaChangeProgress - ChangefeedTarget - ChangefeedDetails - ResolvedSpan - ChangefeedProgress - Payload - Progress -*/ package jobspb import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" -import cockroach_roachpb1 "github.com/cockroachdb/cockroach/pkg/roachpb" -import cockroach_roachpb2 "github.com/cockroachdb/cockroach/pkg/roachpb" -import cockroach_sql_sqlbase1 "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" -import cockroach_util_hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" +import roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" +import sqlbase "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" +import hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" import github_com_cockroachdb_cockroach_pkg_roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" import github_com_cockroachdb_cockroach_pkg_sql_sqlbase "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" -import sortkeys "github.com/gogo/protobuf/sortkeys" -import binary "encoding/binary" +import github_com_gogo_protobuf_sortkeys "github.com/gogo/protobuf/sortkeys" +import encoding_binary "encoding/binary" import io "io" @@ -80,7 +54,9 @@ var Status_value = map[string]int32{ func (x Status) String() string { return proto.EnumName(Status_name, int32(x)) } -func (Status) EnumDescriptor() ([]byte, []int) { return fileDescriptorJobs, []int{0} } +func (Status) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_jobs_7f594aaf3aa20aa4, []int{0} +} type Type int32 @@ -110,78 +86,234 @@ var Type_value = map[string]int32{ "CHANGEFEED": 5, } -func (Type) EnumDescriptor() ([]byte, []int) { return fileDescriptorJobs, []int{1} } +func (Type) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_jobs_7f594aaf3aa20aa4, []int{1} +} type Lease struct { // The ID of the node that holds the lease. NodeID github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,1,opt,name=node_id,json=nodeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"node_id,omitempty"` // The epoch of the lease holder's node liveness entry. - Epoch int64 `protobuf:"varint,2,opt,name=epoch,proto3" json:"epoch,omitempty"` + Epoch int64 `protobuf:"varint,2,opt,name=epoch,proto3" json:"epoch,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Lease) Reset() { *m = Lease{} } +func (m *Lease) String() string { return proto.CompactTextString(m) } +func (*Lease) ProtoMessage() {} +func (*Lease) Descriptor() ([]byte, []int) { + return fileDescriptor_jobs_7f594aaf3aa20aa4, []int{0} +} +func (m *Lease) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Lease) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *Lease) XXX_Merge(src proto.Message) { + xxx_messageInfo_Lease.Merge(dst, src) +} +func (m *Lease) XXX_Size() int { + return m.Size() +} +func (m *Lease) XXX_DiscardUnknown() { + xxx_messageInfo_Lease.DiscardUnknown(m) } -func (m *Lease) Reset() { *m = Lease{} } -func (m *Lease) String() string { return proto.CompactTextString(m) } -func (*Lease) ProtoMessage() {} -func (*Lease) Descriptor() ([]byte, []int) { return fileDescriptorJobs, []int{0} } +var xxx_messageInfo_Lease proto.InternalMessageInfo type BackupDetails struct { - StartTime cockroach_util_hlc.Timestamp `protobuf:"bytes,1,opt,name=start_time,json=startTime" json:"start_time"` - EndTime cockroach_util_hlc.Timestamp `protobuf:"bytes,2,opt,name=end_time,json=endTime" json:"end_time"` - URI string `protobuf:"bytes,3,opt,name=uri,proto3" json:"uri,omitempty"` - BackupDescriptor []byte `protobuf:"bytes,4,opt,name=backup_descriptor,json=backupDescriptor,proto3" json:"backup_descriptor,omitempty"` + StartTime hlc.Timestamp `protobuf:"bytes,1,opt,name=start_time,json=startTime,proto3" json:"start_time"` + EndTime hlc.Timestamp `protobuf:"bytes,2,opt,name=end_time,json=endTime,proto3" json:"end_time"` + URI string `protobuf:"bytes,3,opt,name=uri,proto3" json:"uri,omitempty"` + BackupDescriptor []byte `protobuf:"bytes,4,opt,name=backup_descriptor,json=backupDescriptor,proto3" json:"backup_descriptor,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *BackupDetails) Reset() { *m = BackupDetails{} } +func (m *BackupDetails) String() string { return proto.CompactTextString(m) } +func (*BackupDetails) ProtoMessage() {} +func (*BackupDetails) Descriptor() ([]byte, []int) { + return fileDescriptor_jobs_7f594aaf3aa20aa4, []int{1} +} +func (m *BackupDetails) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *BackupDetails) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *BackupDetails) XXX_Merge(src proto.Message) { + xxx_messageInfo_BackupDetails.Merge(dst, src) +} +func (m *BackupDetails) XXX_Size() int { + return m.Size() +} +func (m *BackupDetails) XXX_DiscardUnknown() { + xxx_messageInfo_BackupDetails.DiscardUnknown(m) } -func (m *BackupDetails) Reset() { *m = BackupDetails{} } -func (m *BackupDetails) String() string { return proto.CompactTextString(m) } -func (*BackupDetails) ProtoMessage() {} -func (*BackupDetails) Descriptor() ([]byte, []int) { return fileDescriptorJobs, []int{1} } +var xxx_messageInfo_BackupDetails proto.InternalMessageInfo type BackupProgress struct { + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *BackupProgress) Reset() { *m = BackupProgress{} } +func (m *BackupProgress) String() string { return proto.CompactTextString(m) } +func (*BackupProgress) ProtoMessage() {} +func (*BackupProgress) Descriptor() ([]byte, []int) { + return fileDescriptor_jobs_7f594aaf3aa20aa4, []int{2} +} +func (m *BackupProgress) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *BackupProgress) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *BackupProgress) XXX_Merge(src proto.Message) { + xxx_messageInfo_BackupProgress.Merge(dst, src) +} +func (m *BackupProgress) XXX_Size() int { + return m.Size() +} +func (m *BackupProgress) XXX_DiscardUnknown() { + xxx_messageInfo_BackupProgress.DiscardUnknown(m) } -func (m *BackupProgress) Reset() { *m = BackupProgress{} } -func (m *BackupProgress) String() string { return proto.CompactTextString(m) } -func (*BackupProgress) ProtoMessage() {} -func (*BackupProgress) Descriptor() ([]byte, []int) { return fileDescriptorJobs, []int{2} } +var xxx_messageInfo_BackupProgress proto.InternalMessageInfo type RestoreDetails struct { - EndTime cockroach_util_hlc.Timestamp `protobuf:"bytes,4,opt,name=end_time,json=endTime" json:"end_time"` - TableRewrites map[github_com_cockroachdb_cockroach_pkg_sql_sqlbase.ID]*RestoreDetails_TableRewrite `protobuf:"bytes,2,rep,name=table_rewrites,json=tableRewrites,castkey=github.com/cockroachdb/cockroach/pkg/sql/sqlbase.ID" json:"table_rewrites,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value"` - URIs []string `protobuf:"bytes,3,rep,name=uris" json:"uris,omitempty"` - TableDescs []*cockroach_sql_sqlbase1.TableDescriptor `protobuf:"bytes,5,rep,name=table_descs,json=tableDescs" json:"table_descs,omitempty"` - OverrideDB string `protobuf:"bytes,6,opt,name=override_db,json=overrideDb,proto3" json:"override_db,omitempty"` + EndTime hlc.Timestamp `protobuf:"bytes,4,opt,name=end_time,json=endTime,proto3" json:"end_time"` + TableRewrites map[github_com_cockroachdb_cockroach_pkg_sql_sqlbase.ID]*RestoreDetails_TableRewrite `protobuf:"bytes,2,rep,name=table_rewrites,json=tableRewrites,proto3,castkey=github.com/cockroachdb/cockroach/pkg/sql/sqlbase.ID" json:"table_rewrites,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + URIs []string `protobuf:"bytes,3,rep,name=uris,proto3" json:"uris,omitempty"` + TableDescs []*sqlbase.TableDescriptor `protobuf:"bytes,5,rep,name=table_descs,json=tableDescs,proto3" json:"table_descs,omitempty"` + OverrideDB string `protobuf:"bytes,6,opt,name=override_db,json=overrideDb,proto3" json:"override_db,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *RestoreDetails) Reset() { *m = RestoreDetails{} } +func (m *RestoreDetails) String() string { return proto.CompactTextString(m) } +func (*RestoreDetails) ProtoMessage() {} +func (*RestoreDetails) Descriptor() ([]byte, []int) { + return fileDescriptor_jobs_7f594aaf3aa20aa4, []int{3} +} +func (m *RestoreDetails) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RestoreDetails) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RestoreDetails) XXX_Merge(src proto.Message) { + xxx_messageInfo_RestoreDetails.Merge(dst, src) +} +func (m *RestoreDetails) XXX_Size() int { + return m.Size() +} +func (m *RestoreDetails) XXX_DiscardUnknown() { + xxx_messageInfo_RestoreDetails.DiscardUnknown(m) } -func (m *RestoreDetails) Reset() { *m = RestoreDetails{} } -func (m *RestoreDetails) String() string { return proto.CompactTextString(m) } -func (*RestoreDetails) ProtoMessage() {} -func (*RestoreDetails) Descriptor() ([]byte, []int) { return fileDescriptorJobs, []int{3} } +var xxx_messageInfo_RestoreDetails proto.InternalMessageInfo type RestoreDetails_TableRewrite struct { - TableID github_com_cockroachdb_cockroach_pkg_sql_sqlbase.ID `protobuf:"varint,1,opt,name=table_id,json=tableId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/sql/sqlbase.ID" json:"table_id,omitempty"` - ParentID github_com_cockroachdb_cockroach_pkg_sql_sqlbase.ID `protobuf:"varint,2,opt,name=parent_id,json=parentId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/sql/sqlbase.ID" json:"parent_id,omitempty"` + TableID github_com_cockroachdb_cockroach_pkg_sql_sqlbase.ID `protobuf:"varint,1,opt,name=table_id,json=tableId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/sql/sqlbase.ID" json:"table_id,omitempty"` + ParentID github_com_cockroachdb_cockroach_pkg_sql_sqlbase.ID `protobuf:"varint,2,opt,name=parent_id,json=parentId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/sql/sqlbase.ID" json:"parent_id,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *RestoreDetails_TableRewrite) Reset() { *m = RestoreDetails_TableRewrite{} } func (m *RestoreDetails_TableRewrite) String() string { return proto.CompactTextString(m) } func (*RestoreDetails_TableRewrite) ProtoMessage() {} func (*RestoreDetails_TableRewrite) Descriptor() ([]byte, []int) { - return fileDescriptorJobs, []int{3, 0} + return fileDescriptor_jobs_7f594aaf3aa20aa4, []int{3, 0} +} +func (m *RestoreDetails_TableRewrite) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RestoreDetails_TableRewrite) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RestoreDetails_TableRewrite) XXX_Merge(src proto.Message) { + xxx_messageInfo_RestoreDetails_TableRewrite.Merge(dst, src) +} +func (m *RestoreDetails_TableRewrite) XXX_Size() int { + return m.Size() } +func (m *RestoreDetails_TableRewrite) XXX_DiscardUnknown() { + xxx_messageInfo_RestoreDetails_TableRewrite.DiscardUnknown(m) +} + +var xxx_messageInfo_RestoreDetails_TableRewrite proto.InternalMessageInfo type RestoreProgress struct { - HighWater []byte `protobuf:"bytes,1,opt,name=high_water,json=highWater,proto3" json:"high_water,omitempty"` + HighWater []byte `protobuf:"bytes,1,opt,name=high_water,json=highWater,proto3" json:"high_water,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *RestoreProgress) Reset() { *m = RestoreProgress{} } -func (m *RestoreProgress) String() string { return proto.CompactTextString(m) } -func (*RestoreProgress) ProtoMessage() {} -func (*RestoreProgress) Descriptor() ([]byte, []int) { return fileDescriptorJobs, []int{4} } +func (m *RestoreProgress) Reset() { *m = RestoreProgress{} } +func (m *RestoreProgress) String() string { return proto.CompactTextString(m) } +func (*RestoreProgress) ProtoMessage() {} +func (*RestoreProgress) Descriptor() ([]byte, []int) { + return fileDescriptor_jobs_7f594aaf3aa20aa4, []int{4} +} +func (m *RestoreProgress) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RestoreProgress) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RestoreProgress) XXX_Merge(src proto.Message) { + xxx_messageInfo_RestoreProgress.Merge(dst, src) +} +func (m *RestoreProgress) XXX_Size() int { + return m.Size() +} +func (m *RestoreProgress) XXX_DiscardUnknown() { + xxx_messageInfo_RestoreProgress.DiscardUnknown(m) +} + +var xxx_messageInfo_RestoreProgress proto.InternalMessageInfo type ImportDetails struct { - Tables []ImportDetails_Table `protobuf:"bytes,1,rep,name=tables" json:"tables"` - URIs []string `protobuf:"bytes,2,rep,name=uris" json:"uris,omitempty"` - Format cockroach_roachpb2.IOFileFormat `protobuf:"bytes,3,opt,name=format" json:"format"` + Tables []ImportDetails_Table `protobuf:"bytes,1,rep,name=tables,proto3" json:"tables"` + URIs []string `protobuf:"bytes,2,rep,name=uris,proto3" json:"uris,omitempty"` + Format roachpb.IOFileFormat `protobuf:"bytes,3,opt,name=format,proto3" json:"format"` SSTSize int64 `protobuf:"varint,4,opt,name=sst_size,json=sstSize,proto3" json:"sst_size,omitempty"` Oversample int64 `protobuf:"varint,9,opt,name=oversample,proto3" json:"oversample,omitempty"` SkipFKs bool `protobuf:"varint,10,opt,name=skip_fks,json=skipFks,proto3" json:"skip_fks,omitempty"` @@ -192,59 +324,189 @@ type ImportDetails struct { // It is populated with the sampling phase's results. These must be // used if a job is resumed to guarantee that AddSSTable will not attempt // to add ranges with an old split point within them. - Samples [][]byte `protobuf:"bytes,8,rep,name=samples" json:"samples,omitempty"` + Samples [][]byte `protobuf:"bytes,8,rep,name=samples,proto3" json:"samples,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ImportDetails) Reset() { *m = ImportDetails{} } +func (m *ImportDetails) String() string { return proto.CompactTextString(m) } +func (*ImportDetails) ProtoMessage() {} +func (*ImportDetails) Descriptor() ([]byte, []int) { + return fileDescriptor_jobs_7f594aaf3aa20aa4, []int{5} +} +func (m *ImportDetails) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ImportDetails) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ImportDetails) XXX_Merge(src proto.Message) { + xxx_messageInfo_ImportDetails.Merge(dst, src) +} +func (m *ImportDetails) XXX_Size() int { + return m.Size() +} +func (m *ImportDetails) XXX_DiscardUnknown() { + xxx_messageInfo_ImportDetails.DiscardUnknown(m) } -func (m *ImportDetails) Reset() { *m = ImportDetails{} } -func (m *ImportDetails) String() string { return proto.CompactTextString(m) } -func (*ImportDetails) ProtoMessage() {} -func (*ImportDetails) Descriptor() ([]byte, []int) { return fileDescriptorJobs, []int{5} } +var xxx_messageInfo_ImportDetails proto.InternalMessageInfo type ImportDetails_Table struct { - Desc *cockroach_sql_sqlbase1.TableDescriptor `protobuf:"bytes,1,opt,name=desc" json:"desc,omitempty"` - Name string `protobuf:"bytes,18,opt,name=name,proto3" json:"name,omitempty"` - SeqVal int64 `protobuf:"varint,19,opt,name=seq_val,json=seqVal,proto3" json:"seq_val,omitempty"` + Desc *sqlbase.TableDescriptor `protobuf:"bytes,1,opt,name=desc,proto3" json:"desc,omitempty"` + Name string `protobuf:"bytes,18,opt,name=name,proto3" json:"name,omitempty"` + SeqVal int64 `protobuf:"varint,19,opt,name=seq_val,json=seqVal,proto3" json:"seq_val,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *ImportDetails_Table) Reset() { *m = ImportDetails_Table{} } -func (m *ImportDetails_Table) String() string { return proto.CompactTextString(m) } -func (*ImportDetails_Table) ProtoMessage() {} -func (*ImportDetails_Table) Descriptor() ([]byte, []int) { return fileDescriptorJobs, []int{5, 0} } +func (m *ImportDetails_Table) Reset() { *m = ImportDetails_Table{} } +func (m *ImportDetails_Table) String() string { return proto.CompactTextString(m) } +func (*ImportDetails_Table) ProtoMessage() {} +func (*ImportDetails_Table) Descriptor() ([]byte, []int) { + return fileDescriptor_jobs_7f594aaf3aa20aa4, []int{5, 0} +} +func (m *ImportDetails_Table) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ImportDetails_Table) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ImportDetails_Table) XXX_Merge(src proto.Message) { + xxx_messageInfo_ImportDetails_Table.Merge(dst, src) +} +func (m *ImportDetails_Table) XXX_Size() int { + return m.Size() +} +func (m *ImportDetails_Table) XXX_DiscardUnknown() { + xxx_messageInfo_ImportDetails_Table.DiscardUnknown(m) +} + +var xxx_messageInfo_ImportDetails_Table proto.InternalMessageInfo type ImportProgress struct { - SamplingProgress []float32 `protobuf:"fixed32,1,rep,packed,name=sampling_progress,json=samplingProgress" json:"sampling_progress,omitempty"` - ReadProgress []float32 `protobuf:"fixed32,2,rep,packed,name=read_progress,json=readProgress" json:"read_progress,omitempty"` - WriteProgress []float32 `protobuf:"fixed32,3,rep,packed,name=write_progress,json=writeProgress" json:"write_progress,omitempty"` + SamplingProgress []float32 `protobuf:"fixed32,1,rep,packed,name=sampling_progress,json=samplingProgress,proto3" json:"sampling_progress,omitempty"` + ReadProgress []float32 `protobuf:"fixed32,2,rep,packed,name=read_progress,json=readProgress,proto3" json:"read_progress,omitempty"` + WriteProgress []float32 `protobuf:"fixed32,3,rep,packed,name=write_progress,json=writeProgress,proto3" json:"write_progress,omitempty"` // The spans of split keys which have had their SSTable's generated. // This allows us to skip the shuffle stage for already-completed // spans when resuming an import job. - SpanProgress []cockroach_roachpb1.Span `protobuf:"bytes,4,rep,name=span_progress,json=spanProgress" json:"span_progress"` + SpanProgress []roachpb.Span `protobuf:"bytes,4,rep,name=span_progress,json=spanProgress,proto3" json:"span_progress"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ImportProgress) Reset() { *m = ImportProgress{} } +func (m *ImportProgress) String() string { return proto.CompactTextString(m) } +func (*ImportProgress) ProtoMessage() {} +func (*ImportProgress) Descriptor() ([]byte, []int) { + return fileDescriptor_jobs_7f594aaf3aa20aa4, []int{6} +} +func (m *ImportProgress) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ImportProgress) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ImportProgress) XXX_Merge(src proto.Message) { + xxx_messageInfo_ImportProgress.Merge(dst, src) +} +func (m *ImportProgress) XXX_Size() int { + return m.Size() +} +func (m *ImportProgress) XXX_DiscardUnknown() { + xxx_messageInfo_ImportProgress.DiscardUnknown(m) } -func (m *ImportProgress) Reset() { *m = ImportProgress{} } -func (m *ImportProgress) String() string { return proto.CompactTextString(m) } -func (*ImportProgress) ProtoMessage() {} -func (*ImportProgress) Descriptor() ([]byte, []int) { return fileDescriptorJobs, []int{6} } +var xxx_messageInfo_ImportProgress proto.InternalMessageInfo type ResumeSpanList struct { - ResumeSpans []cockroach_roachpb1.Span `protobuf:"bytes,1,rep,name=resume_spans,json=resumeSpans" json:"resume_spans"` + ResumeSpans []roachpb.Span `protobuf:"bytes,1,rep,name=resume_spans,json=resumeSpans,proto3" json:"resume_spans"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ResumeSpanList) Reset() { *m = ResumeSpanList{} } +func (m *ResumeSpanList) String() string { return proto.CompactTextString(m) } +func (*ResumeSpanList) ProtoMessage() {} +func (*ResumeSpanList) Descriptor() ([]byte, []int) { + return fileDescriptor_jobs_7f594aaf3aa20aa4, []int{7} +} +func (m *ResumeSpanList) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ResumeSpanList) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ResumeSpanList) XXX_Merge(src proto.Message) { + xxx_messageInfo_ResumeSpanList.Merge(dst, src) +} +func (m *ResumeSpanList) XXX_Size() int { + return m.Size() +} +func (m *ResumeSpanList) XXX_DiscardUnknown() { + xxx_messageInfo_ResumeSpanList.DiscardUnknown(m) } -func (m *ResumeSpanList) Reset() { *m = ResumeSpanList{} } -func (m *ResumeSpanList) String() string { return proto.CompactTextString(m) } -func (*ResumeSpanList) ProtoMessage() {} -func (*ResumeSpanList) Descriptor() ([]byte, []int) { return fileDescriptorJobs, []int{7} } +var xxx_messageInfo_ResumeSpanList proto.InternalMessageInfo type DroppedTableDetails struct { - Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` - ID github_com_cockroachdb_cockroach_pkg_sql_sqlbase.ID `protobuf:"varint,2,opt,name=ID,proto3,casttype=github.com/cockroachdb/cockroach/pkg/sql/sqlbase.ID" json:"ID,omitempty"` - Status Status `protobuf:"varint,3,opt,name=status,proto3,enum=cockroach.sql.jobs.jobspb.Status" json:"status,omitempty"` + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + ID github_com_cockroachdb_cockroach_pkg_sql_sqlbase.ID `protobuf:"varint,2,opt,name=ID,proto3,casttype=github.com/cockroachdb/cockroach/pkg/sql/sqlbase.ID" json:"ID,omitempty"` + Status Status `protobuf:"varint,3,opt,name=status,proto3,enum=cockroach.sql.jobs.jobspb.Status" json:"status,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *DroppedTableDetails) Reset() { *m = DroppedTableDetails{} } -func (m *DroppedTableDetails) String() string { return proto.CompactTextString(m) } -func (*DroppedTableDetails) ProtoMessage() {} -func (*DroppedTableDetails) Descriptor() ([]byte, []int) { return fileDescriptorJobs, []int{8} } +func (m *DroppedTableDetails) Reset() { *m = DroppedTableDetails{} } +func (m *DroppedTableDetails) String() string { return proto.CompactTextString(m) } +func (*DroppedTableDetails) ProtoMessage() {} +func (*DroppedTableDetails) Descriptor() ([]byte, []int) { + return fileDescriptor_jobs_7f594aaf3aa20aa4, []int{8} +} +func (m *DroppedTableDetails) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *DroppedTableDetails) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *DroppedTableDetails) XXX_Merge(src proto.Message) { + xxx_messageInfo_DroppedTableDetails.Merge(dst, src) +} +func (m *DroppedTableDetails) XXX_Size() int { + return m.Size() +} +func (m *DroppedTableDetails) XXX_DiscardUnknown() { + xxx_messageInfo_DroppedTableDetails.DiscardUnknown(m) +} + +var xxx_messageInfo_DroppedTableDetails proto.InternalMessageInfo type SchemaChangeDetails struct { // A schema change can involve running multiple processors backfilling @@ -253,33 +515,111 @@ type SchemaChangeDetails struct { // non-overlapping contiguous areas of the KV space that still need to // be processed. The index represents the index of a mutation in a // mutation list containing mutations for the same mutationID. - ResumeSpanList []ResumeSpanList `protobuf:"bytes,2,rep,name=resume_span_list,json=resumeSpanList" json:"resume_span_list"` - DroppedTables []DroppedTableDetails `protobuf:"bytes,3,rep,name=dropped_tables,json=droppedTables" json:"dropped_tables"` + ResumeSpanList []ResumeSpanList `protobuf:"bytes,2,rep,name=resume_span_list,json=resumeSpanList,proto3" json:"resume_span_list"` + DroppedTables []DroppedTableDetails `protobuf:"bytes,3,rep,name=dropped_tables,json=droppedTables,proto3" json:"dropped_tables"` // The descriptor ID of the dropped database which created this job. - DroppedDatabaseID github_com_cockroachdb_cockroach_pkg_sql_sqlbase.ID `protobuf:"varint,4,opt,name=dropped_database_id,json=droppedDatabaseId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/sql/sqlbase.ID" json:"dropped_database_id,omitempty"` + DroppedDatabaseID github_com_cockroachdb_cockroach_pkg_sql_sqlbase.ID `protobuf:"varint,4,opt,name=dropped_database_id,json=droppedDatabaseId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/sql/sqlbase.ID" json:"dropped_database_id,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *SchemaChangeDetails) Reset() { *m = SchemaChangeDetails{} } +func (m *SchemaChangeDetails) String() string { return proto.CompactTextString(m) } +func (*SchemaChangeDetails) ProtoMessage() {} +func (*SchemaChangeDetails) Descriptor() ([]byte, []int) { + return fileDescriptor_jobs_7f594aaf3aa20aa4, []int{9} +} +func (m *SchemaChangeDetails) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SchemaChangeDetails) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *SchemaChangeDetails) XXX_Merge(src proto.Message) { + xxx_messageInfo_SchemaChangeDetails.Merge(dst, src) +} +func (m *SchemaChangeDetails) XXX_Size() int { + return m.Size() +} +func (m *SchemaChangeDetails) XXX_DiscardUnknown() { + xxx_messageInfo_SchemaChangeDetails.DiscardUnknown(m) } -func (m *SchemaChangeDetails) Reset() { *m = SchemaChangeDetails{} } -func (m *SchemaChangeDetails) String() string { return proto.CompactTextString(m) } -func (*SchemaChangeDetails) ProtoMessage() {} -func (*SchemaChangeDetails) Descriptor() ([]byte, []int) { return fileDescriptorJobs, []int{9} } +var xxx_messageInfo_SchemaChangeDetails proto.InternalMessageInfo type SchemaChangeProgress struct { + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *SchemaChangeProgress) Reset() { *m = SchemaChangeProgress{} } -func (m *SchemaChangeProgress) String() string { return proto.CompactTextString(m) } -func (*SchemaChangeProgress) ProtoMessage() {} -func (*SchemaChangeProgress) Descriptor() ([]byte, []int) { return fileDescriptorJobs, []int{10} } +func (m *SchemaChangeProgress) Reset() { *m = SchemaChangeProgress{} } +func (m *SchemaChangeProgress) String() string { return proto.CompactTextString(m) } +func (*SchemaChangeProgress) ProtoMessage() {} +func (*SchemaChangeProgress) Descriptor() ([]byte, []int) { + return fileDescriptor_jobs_7f594aaf3aa20aa4, []int{10} +} +func (m *SchemaChangeProgress) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SchemaChangeProgress) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *SchemaChangeProgress) XXX_Merge(src proto.Message) { + xxx_messageInfo_SchemaChangeProgress.Merge(dst, src) +} +func (m *SchemaChangeProgress) XXX_Size() int { + return m.Size() +} +func (m *SchemaChangeProgress) XXX_DiscardUnknown() { + xxx_messageInfo_SchemaChangeProgress.DiscardUnknown(m) +} + +var xxx_messageInfo_SchemaChangeProgress proto.InternalMessageInfo type ChangefeedTarget struct { - StatementTimeName string `protobuf:"bytes,1,opt,name=statement_time_name,json=statementTimeName,proto3" json:"statement_time_name,omitempty"` + StatementTimeName string `protobuf:"bytes,1,opt,name=statement_time_name,json=statementTimeName,proto3" json:"statement_time_name,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *ChangefeedTarget) Reset() { *m = ChangefeedTarget{} } -func (m *ChangefeedTarget) String() string { return proto.CompactTextString(m) } -func (*ChangefeedTarget) ProtoMessage() {} -func (*ChangefeedTarget) Descriptor() ([]byte, []int) { return fileDescriptorJobs, []int{11} } +func (m *ChangefeedTarget) Reset() { *m = ChangefeedTarget{} } +func (m *ChangefeedTarget) String() string { return proto.CompactTextString(m) } +func (*ChangefeedTarget) ProtoMessage() {} +func (*ChangefeedTarget) Descriptor() ([]byte, []int) { + return fileDescriptor_jobs_7f594aaf3aa20aa4, []int{11} +} +func (m *ChangefeedTarget) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ChangefeedTarget) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ChangefeedTarget) XXX_Merge(src proto.Message) { + xxx_messageInfo_ChangefeedTarget.Merge(dst, src) +} +func (m *ChangefeedTarget) XXX_Size() int { + return m.Size() +} +func (m *ChangefeedTarget) XXX_DiscardUnknown() { + xxx_messageInfo_ChangefeedTarget.DiscardUnknown(m) +} + +var xxx_messageInfo_ChangefeedTarget proto.InternalMessageInfo type ChangefeedDetails struct { // Targets contains the user-specified tables and databases to watch, mapping @@ -299,35 +639,113 @@ type ChangefeedDetails struct { // renames can be detected. They are also used to construct an error message // if the descriptor id no longer exists when the jobs is unpaused (which can // happen if it was dropped or truncated). - Targets ChangefeedTargets `protobuf:"bytes,6,rep,name=targets,casttype=ChangefeedTargets,castkey=github.com/cockroachdb/cockroach/pkg/sql/sqlbase.ID" json:"targets" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value"` - SinkURI string `protobuf:"bytes,3,opt,name=sink_uri,json=sinkUri,proto3" json:"sink_uri,omitempty"` - Opts map[string]string `protobuf:"bytes,4,rep,name=opts" json:"opts,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` - StatementTime cockroach_util_hlc.Timestamp `protobuf:"bytes,7,opt,name=statement_time,json=statementTime" json:"statement_time"` + Targets ChangefeedTargets `protobuf:"bytes,6,rep,name=targets,proto3,casttype=ChangefeedTargets,castkey=github.com/cockroachdb/cockroach/pkg/sql/sqlbase.ID" json:"targets" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + SinkURI string `protobuf:"bytes,3,opt,name=sink_uri,json=sinkUri,proto3" json:"sink_uri,omitempty"` + Opts map[string]string `protobuf:"bytes,4,rep,name=opts,proto3" json:"opts,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + StatementTime hlc.Timestamp `protobuf:"bytes,7,opt,name=statement_time,json=statementTime,proto3" json:"statement_time"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ChangefeedDetails) Reset() { *m = ChangefeedDetails{} } +func (m *ChangefeedDetails) String() string { return proto.CompactTextString(m) } +func (*ChangefeedDetails) ProtoMessage() {} +func (*ChangefeedDetails) Descriptor() ([]byte, []int) { + return fileDescriptor_jobs_7f594aaf3aa20aa4, []int{12} +} +func (m *ChangefeedDetails) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ChangefeedDetails) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ChangefeedDetails) XXX_Merge(src proto.Message) { + xxx_messageInfo_ChangefeedDetails.Merge(dst, src) +} +func (m *ChangefeedDetails) XXX_Size() int { + return m.Size() +} +func (m *ChangefeedDetails) XXX_DiscardUnknown() { + xxx_messageInfo_ChangefeedDetails.DiscardUnknown(m) } -func (m *ChangefeedDetails) Reset() { *m = ChangefeedDetails{} } -func (m *ChangefeedDetails) String() string { return proto.CompactTextString(m) } -func (*ChangefeedDetails) ProtoMessage() {} -func (*ChangefeedDetails) Descriptor() ([]byte, []int) { return fileDescriptorJobs, []int{12} } +var xxx_messageInfo_ChangefeedDetails proto.InternalMessageInfo type ResolvedSpan struct { - Span cockroach_roachpb1.Span `protobuf:"bytes,1,opt,name=span" json:"span"` - Timestamp cockroach_util_hlc.Timestamp `protobuf:"bytes,2,opt,name=timestamp" json:"timestamp"` + Span roachpb.Span `protobuf:"bytes,1,opt,name=span,proto3" json:"span"` + Timestamp hlc.Timestamp `protobuf:"bytes,2,opt,name=timestamp,proto3" json:"timestamp"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ResolvedSpan) Reset() { *m = ResolvedSpan{} } +func (m *ResolvedSpan) String() string { return proto.CompactTextString(m) } +func (*ResolvedSpan) ProtoMessage() {} +func (*ResolvedSpan) Descriptor() ([]byte, []int) { + return fileDescriptor_jobs_7f594aaf3aa20aa4, []int{13} +} +func (m *ResolvedSpan) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ResolvedSpan) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ResolvedSpan) XXX_Merge(src proto.Message) { + xxx_messageInfo_ResolvedSpan.Merge(dst, src) +} +func (m *ResolvedSpan) XXX_Size() int { + return m.Size() +} +func (m *ResolvedSpan) XXX_DiscardUnknown() { + xxx_messageInfo_ResolvedSpan.DiscardUnknown(m) } -func (m *ResolvedSpan) Reset() { *m = ResolvedSpan{} } -func (m *ResolvedSpan) String() string { return proto.CompactTextString(m) } -func (*ResolvedSpan) ProtoMessage() {} -func (*ResolvedSpan) Descriptor() ([]byte, []int) { return fileDescriptorJobs, []int{13} } +var xxx_messageInfo_ResolvedSpan proto.InternalMessageInfo type ChangefeedProgress struct { - ResolvedSpans []ResolvedSpan `protobuf:"bytes,2,rep,name=resolved_spans,json=resolvedSpans" json:"resolved_spans"` + ResolvedSpans []ResolvedSpan `protobuf:"bytes,2,rep,name=resolved_spans,json=resolvedSpans,proto3" json:"resolved_spans"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *ChangefeedProgress) Reset() { *m = ChangefeedProgress{} } -func (m *ChangefeedProgress) String() string { return proto.CompactTextString(m) } -func (*ChangefeedProgress) ProtoMessage() {} -func (*ChangefeedProgress) Descriptor() ([]byte, []int) { return fileDescriptorJobs, []int{14} } +func (m *ChangefeedProgress) Reset() { *m = ChangefeedProgress{} } +func (m *ChangefeedProgress) String() string { return proto.CompactTextString(m) } +func (*ChangefeedProgress) ProtoMessage() {} +func (*ChangefeedProgress) Descriptor() ([]byte, []int) { + return fileDescriptor_jobs_7f594aaf3aa20aa4, []int{14} +} +func (m *ChangefeedProgress) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ChangefeedProgress) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ChangefeedProgress) XXX_Merge(src proto.Message) { + xxx_messageInfo_ChangefeedProgress.Merge(dst, src) +} +func (m *ChangefeedProgress) XXX_Size() int { + return m.Size() +} +func (m *ChangefeedProgress) XXX_DiscardUnknown() { + xxx_messageInfo_ChangefeedProgress.DiscardUnknown(m) +} + +var xxx_messageInfo_ChangefeedProgress proto.InternalMessageInfo type Payload struct { Description string `protobuf:"bytes,1,opt,name=description,proto3" json:"description,omitempty"` @@ -337,23 +755,49 @@ type Payload struct { // precision, and use microsecond integers directly. StartedMicros int64 `protobuf:"varint,3,opt,name=started_micros,json=startedMicros,proto3" json:"started_micros,omitempty"` FinishedMicros int64 `protobuf:"varint,4,opt,name=finished_micros,json=finishedMicros,proto3" json:"finished_micros,omitempty"` - DescriptorIDs []github_com_cockroachdb_cockroach_pkg_sql_sqlbase.ID `protobuf:"varint,6,rep,packed,name=descriptor_ids,json=descriptorIds,casttype=github.com/cockroachdb/cockroach/pkg/sql/sqlbase.ID" json:"descriptor_ids,omitempty"` + DescriptorIDs []github_com_cockroachdb_cockroach_pkg_sql_sqlbase.ID `protobuf:"varint,6,rep,packed,name=descriptor_ids,json=descriptorIds,proto3,casttype=github.com/cockroachdb/cockroach/pkg/sql/sqlbase.ID" json:"descriptor_ids,omitempty"` Error string `protobuf:"bytes,8,opt,name=error,proto3" json:"error,omitempty"` // ID 9 is intentionally reserved for lease information. - Lease *Lease `protobuf:"bytes,9,opt,name=lease" json:"lease,omitempty"` + Lease *Lease `protobuf:"bytes,9,opt,name=lease,proto3" json:"lease,omitempty"` // Types that are valid to be assigned to Details: // *Payload_Backup // *Payload_Restore // *Payload_SchemaChange // *Payload_Import // *Payload_Changefeed - Details isPayload_Details `protobuf_oneof:"details"` + Details isPayload_Details `protobuf_oneof:"details"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Payload) Reset() { *m = Payload{} } +func (m *Payload) String() string { return proto.CompactTextString(m) } +func (*Payload) ProtoMessage() {} +func (*Payload) Descriptor() ([]byte, []int) { + return fileDescriptor_jobs_7f594aaf3aa20aa4, []int{15} +} +func (m *Payload) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Payload) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *Payload) XXX_Merge(src proto.Message) { + xxx_messageInfo_Payload.Merge(dst, src) +} +func (m *Payload) XXX_Size() int { + return m.Size() +} +func (m *Payload) XXX_DiscardUnknown() { + xxx_messageInfo_Payload.DiscardUnknown(m) } -func (m *Payload) Reset() { *m = Payload{} } -func (m *Payload) String() string { return proto.CompactTextString(m) } -func (*Payload) ProtoMessage() {} -func (*Payload) Descriptor() ([]byte, []int) { return fileDescriptorJobs, []int{15} } +var xxx_messageInfo_Payload proto.InternalMessageInfo type isPayload_Details interface { isPayload_Details() @@ -362,19 +806,19 @@ type isPayload_Details interface { } type Payload_Backup struct { - Backup *BackupDetails `protobuf:"bytes,10,opt,name=backup,oneof"` + Backup *BackupDetails `protobuf:"bytes,10,opt,name=backup,proto3,oneof"` } type Payload_Restore struct { - Restore *RestoreDetails `protobuf:"bytes,11,opt,name=restore,oneof"` + Restore *RestoreDetails `protobuf:"bytes,11,opt,name=restore,proto3,oneof"` } type Payload_SchemaChange struct { - SchemaChange *SchemaChangeDetails `protobuf:"bytes,12,opt,name=schemaChange,oneof"` + SchemaChange *SchemaChangeDetails `protobuf:"bytes,12,opt,name=schemaChange,proto3,oneof"` } type Payload_Import struct { - Import *ImportDetails `protobuf:"bytes,13,opt,name=import,oneof"` + Import *ImportDetails `protobuf:"bytes,13,opt,name=import,proto3,oneof"` } type Payload_Changefeed struct { - Changefeed *ChangefeedDetails `protobuf:"bytes,14,opt,name=changefeed,oneof"` + Changefeed *ChangefeedDetails `protobuf:"bytes,14,opt,name=changefeed,proto3,oneof"` } func (*Payload_Backup) isPayload_Details() {} @@ -526,27 +970,27 @@ func _Payload_OneofSizer(msg proto.Message) (n int) { switch x := m.Details.(type) { case *Payload_Backup: s := proto.Size(x.Backup) - n += proto.SizeVarint(10<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *Payload_Restore: s := proto.Size(x.Restore) - n += proto.SizeVarint(11<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *Payload_SchemaChange: s := proto.Size(x.SchemaChange) - n += proto.SizeVarint(12<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *Payload_Import: s := proto.Size(x.Import) - n += proto.SizeVarint(13<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *Payload_Changefeed: s := proto.Size(x.Changefeed) - n += proto.SizeVarint(14<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case nil: @@ -569,13 +1013,39 @@ type Progress struct { // *Progress_SchemaChange // *Progress_Import // *Progress_Changefeed - Details isProgress_Details `protobuf_oneof:"details"` + Details isProgress_Details `protobuf_oneof:"details"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *Progress) Reset() { *m = Progress{} } -func (m *Progress) String() string { return proto.CompactTextString(m) } -func (*Progress) ProtoMessage() {} -func (*Progress) Descriptor() ([]byte, []int) { return fileDescriptorJobs, []int{16} } +func (m *Progress) Reset() { *m = Progress{} } +func (m *Progress) String() string { return proto.CompactTextString(m) } +func (*Progress) ProtoMessage() {} +func (*Progress) Descriptor() ([]byte, []int) { + return fileDescriptor_jobs_7f594aaf3aa20aa4, []int{16} +} +func (m *Progress) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Progress) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *Progress) XXX_Merge(src proto.Message) { + xxx_messageInfo_Progress.Merge(dst, src) +} +func (m *Progress) XXX_Size() int { + return m.Size() +} +func (m *Progress) XXX_DiscardUnknown() { + xxx_messageInfo_Progress.DiscardUnknown(m) +} + +var xxx_messageInfo_Progress proto.InternalMessageInfo type isProgress_Progress interface { isProgress_Progress() @@ -592,22 +1062,22 @@ type Progress_FractionCompleted struct { FractionCompleted float32 `protobuf:"fixed32,1,opt,name=fraction_completed,json=fractionCompleted,proto3,oneof"` } type Progress_HighWater struct { - HighWater *cockroach_util_hlc.Timestamp `protobuf:"bytes,3,opt,name=high_water,json=highWater,oneof"` + HighWater *hlc.Timestamp `protobuf:"bytes,3,opt,name=high_water,json=highWater,proto3,oneof"` } type Progress_Backup struct { - Backup *BackupProgress `protobuf:"bytes,10,opt,name=backup,oneof"` + Backup *BackupProgress `protobuf:"bytes,10,opt,name=backup,proto3,oneof"` } type Progress_Restore struct { - Restore *RestoreProgress `protobuf:"bytes,11,opt,name=restore,oneof"` + Restore *RestoreProgress `protobuf:"bytes,11,opt,name=restore,proto3,oneof"` } type Progress_SchemaChange struct { - SchemaChange *SchemaChangeProgress `protobuf:"bytes,12,opt,name=schemaChange,oneof"` + SchemaChange *SchemaChangeProgress `protobuf:"bytes,12,opt,name=schemaChange,proto3,oneof"` } type Progress_Import struct { - Import *ImportProgress `protobuf:"bytes,13,opt,name=import,oneof"` + Import *ImportProgress `protobuf:"bytes,13,opt,name=import,proto3,oneof"` } type Progress_Changefeed struct { - Changefeed *ChangefeedProgress `protobuf:"bytes,14,opt,name=changefeed,oneof"` + Changefeed *ChangefeedProgress `protobuf:"bytes,14,opt,name=changefeed,proto3,oneof"` } func (*Progress_FractionCompleted) isProgress_Progress() {} @@ -638,7 +1108,7 @@ func (m *Progress) GetFractionCompleted() float32 { return 0 } -func (m *Progress) GetHighWater() *cockroach_util_hlc.Timestamp { +func (m *Progress) GetHighWater() *hlc.Timestamp { if x, ok := m.GetProgress().(*Progress_HighWater); ok { return x.HighWater } @@ -757,7 +1227,7 @@ func _Progress_OneofUnmarshaler(msg proto.Message, tag, wire int, b *proto.Buffe if wire != proto.WireBytes { return true, proto.ErrInternalBadWireType } - msg := new(cockroach_util_hlc.Timestamp) + msg := new(hlc.Timestamp) err := b.DecodeMessage(msg) m.Progress = &Progress_HighWater{msg} return true, err @@ -811,11 +1281,11 @@ func _Progress_OneofSizer(msg proto.Message) (n int) { // progress switch x := m.Progress.(type) { case *Progress_FractionCompleted: - n += proto.SizeVarint(1<<3 | proto.WireFixed32) + n += 1 // tag and wire n += 4 case *Progress_HighWater: s := proto.Size(x.HighWater) - n += proto.SizeVarint(3<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case nil: @@ -826,27 +1296,27 @@ func _Progress_OneofSizer(msg proto.Message) (n int) { switch x := m.Details.(type) { case *Progress_Backup: s := proto.Size(x.Backup) - n += proto.SizeVarint(10<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *Progress_Restore: s := proto.Size(x.Restore) - n += proto.SizeVarint(11<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *Progress_SchemaChange: s := proto.Size(x.SchemaChange) - n += proto.SizeVarint(12<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *Progress_Import: s := proto.Size(x.Import) - n += proto.SizeVarint(13<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *Progress_Changefeed: s := proto.Size(x.Changefeed) - n += proto.SizeVarint(14<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case nil: @@ -861,6 +1331,7 @@ func init() { proto.RegisterType((*BackupDetails)(nil), "cockroach.sql.jobs.jobspb.BackupDetails") proto.RegisterType((*BackupProgress)(nil), "cockroach.sql.jobs.jobspb.BackupProgress") proto.RegisterType((*RestoreDetails)(nil), "cockroach.sql.jobs.jobspb.RestoreDetails") + proto.RegisterMapType((map[github_com_cockroachdb_cockroach_pkg_sql_sqlbase.ID]*RestoreDetails_TableRewrite)(nil), "cockroach.sql.jobs.jobspb.RestoreDetails.TableRewritesEntry") proto.RegisterType((*RestoreDetails_TableRewrite)(nil), "cockroach.sql.jobs.jobspb.RestoreDetails.TableRewrite") proto.RegisterType((*RestoreProgress)(nil), "cockroach.sql.jobs.jobspb.RestoreProgress") proto.RegisterType((*ImportDetails)(nil), "cockroach.sql.jobs.jobspb.ImportDetails") @@ -872,6 +1343,8 @@ func init() { proto.RegisterType((*SchemaChangeProgress)(nil), "cockroach.sql.jobs.jobspb.SchemaChangeProgress") proto.RegisterType((*ChangefeedTarget)(nil), "cockroach.sql.jobs.jobspb.ChangefeedTarget") proto.RegisterType((*ChangefeedDetails)(nil), "cockroach.sql.jobs.jobspb.ChangefeedDetails") + proto.RegisterMapType((map[string]string)(nil), "cockroach.sql.jobs.jobspb.ChangefeedDetails.OptsEntry") + proto.RegisterMapType((ChangefeedTargets)(nil), "cockroach.sql.jobs.jobspb.ChangefeedDetails.TargetsEntry") proto.RegisterType((*ResolvedSpan)(nil), "cockroach.sql.jobs.jobspb.ResolvedSpan") proto.RegisterType((*ChangefeedProgress)(nil), "cockroach.sql.jobs.jobspb.ChangefeedProgress") proto.RegisterType((*Payload)(nil), "cockroach.sql.jobs.jobspb.Payload") @@ -1018,7 +1491,7 @@ func (m *RestoreDetails) MarshalTo(dAtA []byte) (int, error) { for k := range m.TableRewrites { keysForTableRewrites = append(keysForTableRewrites, uint32(k)) } - sortkeys.Uint32s(keysForTableRewrites) + github_com_gogo_protobuf_sortkeys.Uint32s(keysForTableRewrites) for _, k := range keysForTableRewrites { dAtA[i] = 0x12 i++ @@ -1302,7 +1775,7 @@ func (m *ImportProgress) MarshalTo(dAtA []byte) (int, error) { i = encodeVarintJobs(dAtA, i, uint64(len(m.SamplingProgress)*4)) for _, num := range m.SamplingProgress { f7 := math.Float32bits(float32(num)) - binary.LittleEndian.PutUint32(dAtA[i:], uint32(f7)) + encoding_binary.LittleEndian.PutUint32(dAtA[i:], uint32(f7)) i += 4 } } @@ -1312,7 +1785,7 @@ func (m *ImportProgress) MarshalTo(dAtA []byte) (int, error) { i = encodeVarintJobs(dAtA, i, uint64(len(m.ReadProgress)*4)) for _, num := range m.ReadProgress { f8 := math.Float32bits(float32(num)) - binary.LittleEndian.PutUint32(dAtA[i:], uint32(f8)) + encoding_binary.LittleEndian.PutUint32(dAtA[i:], uint32(f8)) i += 4 } } @@ -1322,7 +1795,7 @@ func (m *ImportProgress) MarshalTo(dAtA []byte) (int, error) { i = encodeVarintJobs(dAtA, i, uint64(len(m.WriteProgress)*4)) for _, num := range m.WriteProgress { f9 := math.Float32bits(float32(num)) - binary.LittleEndian.PutUint32(dAtA[i:], uint32(f9)) + encoding_binary.LittleEndian.PutUint32(dAtA[i:], uint32(f9)) i += 4 } } @@ -1520,7 +1993,7 @@ func (m *ChangefeedDetails) MarshalTo(dAtA []byte) (int, error) { for k := range m.Opts { keysForOpts = append(keysForOpts, string(k)) } - sortkeys.Strings(keysForOpts) + github_com_gogo_protobuf_sortkeys.Strings(keysForOpts) for _, k := range keysForOpts { dAtA[i] = 0x22 i++ @@ -1542,7 +2015,7 @@ func (m *ChangefeedDetails) MarshalTo(dAtA []byte) (int, error) { for k := range m.Targets { keysForTargets = append(keysForTargets, uint32(k)) } - sortkeys.Uint32s(keysForTargets) + github_com_gogo_protobuf_sortkeys.Uint32s(keysForTargets) for _, k := range keysForTargets { dAtA[i] = 0x32 i++ @@ -1839,7 +2312,7 @@ func (m *Progress_FractionCompleted) MarshalTo(dAtA []byte) (int, error) { i := 0 dAtA[i] = 0xd i++ - binary.LittleEndian.PutUint32(dAtA[i:], uint32(math.Float32bits(float32(m.FractionCompleted)))) + encoding_binary.LittleEndian.PutUint32(dAtA[i:], uint32(math.Float32bits(float32(m.FractionCompleted)))) i += 4 return i, nil } @@ -1937,6 +2410,9 @@ func encodeVarintJobs(dAtA []byte, offset int, v uint64) int { return offset + 1 } func (m *Lease) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.NodeID != 0 { @@ -1949,6 +2425,9 @@ func (m *Lease) Size() (n int) { } func (m *BackupDetails) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.StartTime.Size() @@ -1967,12 +2446,18 @@ func (m *BackupDetails) Size() (n int) { } func (m *BackupProgress) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l return n } func (m *RestoreDetails) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.TableRewrites) > 0 { @@ -2010,6 +2495,9 @@ func (m *RestoreDetails) Size() (n int) { } func (m *RestoreDetails_TableRewrite) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.TableID != 0 { @@ -2022,6 +2510,9 @@ func (m *RestoreDetails_TableRewrite) Size() (n int) { } func (m *RestoreProgress) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.HighWater) @@ -2032,6 +2523,9 @@ func (m *RestoreProgress) Size() (n int) { } func (m *ImportDetails) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.Tables) > 0 { @@ -2077,6 +2571,9 @@ func (m *ImportDetails) Size() (n int) { } func (m *ImportDetails_Table) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Desc != nil { @@ -2094,6 +2591,9 @@ func (m *ImportDetails_Table) Size() (n int) { } func (m *ImportProgress) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.SamplingProgress) > 0 { @@ -2115,6 +2615,9 @@ func (m *ImportProgress) Size() (n int) { } func (m *ResumeSpanList) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.ResumeSpans) > 0 { @@ -2127,6 +2630,9 @@ func (m *ResumeSpanList) Size() (n int) { } func (m *DroppedTableDetails) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Name) @@ -2143,6 +2649,9 @@ func (m *DroppedTableDetails) Size() (n int) { } func (m *SchemaChangeDetails) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.ResumeSpanList) > 0 { @@ -2164,12 +2673,18 @@ func (m *SchemaChangeDetails) Size() (n int) { } func (m *SchemaChangeProgress) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l return n } func (m *ChangefeedTarget) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.StatementTimeName) @@ -2180,6 +2695,9 @@ func (m *ChangefeedTarget) Size() (n int) { } func (m *ChangefeedDetails) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.SinkURI) @@ -2209,6 +2727,9 @@ func (m *ChangefeedDetails) Size() (n int) { } func (m *ResolvedSpan) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.Span.Size() @@ -2219,6 +2740,9 @@ func (m *ResolvedSpan) Size() (n int) { } func (m *ChangefeedProgress) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.ResolvedSpans) > 0 { @@ -2231,6 +2755,9 @@ func (m *ChangefeedProgress) Size() (n int) { } func (m *Payload) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Description) @@ -2269,6 +2796,9 @@ func (m *Payload) Size() (n int) { } func (m *Payload_Backup) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Backup != nil { @@ -2278,6 +2808,9 @@ func (m *Payload_Backup) Size() (n int) { return n } func (m *Payload_Restore) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Restore != nil { @@ -2287,6 +2820,9 @@ func (m *Payload_Restore) Size() (n int) { return n } func (m *Payload_SchemaChange) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.SchemaChange != nil { @@ -2296,6 +2832,9 @@ func (m *Payload_SchemaChange) Size() (n int) { return n } func (m *Payload_Import) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Import != nil { @@ -2305,6 +2844,9 @@ func (m *Payload_Import) Size() (n int) { return n } func (m *Payload_Changefeed) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Changefeed != nil { @@ -2314,6 +2856,9 @@ func (m *Payload_Changefeed) Size() (n int) { return n } func (m *Progress) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Progress != nil { @@ -2333,12 +2878,18 @@ func (m *Progress) Size() (n int) { } func (m *Progress_FractionCompleted) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 5 return n } func (m *Progress_HighWater) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.HighWater != nil { @@ -2348,6 +2899,9 @@ func (m *Progress_HighWater) Size() (n int) { return n } func (m *Progress_Backup) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Backup != nil { @@ -2357,6 +2911,9 @@ func (m *Progress_Backup) Size() (n int) { return n } func (m *Progress_Restore) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Restore != nil { @@ -2366,6 +2923,9 @@ func (m *Progress_Restore) Size() (n int) { return n } func (m *Progress_SchemaChange) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.SchemaChange != nil { @@ -2375,6 +2935,9 @@ func (m *Progress_SchemaChange) Size() (n int) { return n } func (m *Progress_Import) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Import != nil { @@ -2384,6 +2947,9 @@ func (m *Progress_Import) Size() (n int) { return n } func (m *Progress_Changefeed) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Changefeed != nil { @@ -2940,7 +3506,7 @@ func (m *RestoreDetails) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.TableDescs = append(m.TableDescs, &cockroach_sql_sqlbase1.TableDescriptor{}) + m.TableDescs = append(m.TableDescs, &sqlbase.TableDescriptor{}) if err := m.TableDescs[len(m.TableDescs)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -3514,7 +4080,7 @@ func (m *ImportDetails_Table) Unmarshal(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Desc == nil { - m.Desc = &cockroach_sql_sqlbase1.TableDescriptor{} + m.Desc = &sqlbase.TableDescriptor{} } if err := m.Desc.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err @@ -3624,7 +4190,7 @@ func (m *ImportProgress) Unmarshal(dAtA []byte) error { if (iNdEx + 4) > l { return io.ErrUnexpectedEOF } - v = uint32(binary.LittleEndian.Uint32(dAtA[iNdEx:])) + v = uint32(encoding_binary.LittleEndian.Uint32(dAtA[iNdEx:])) iNdEx += 4 v2 := float32(math.Float32frombits(v)) m.SamplingProgress = append(m.SamplingProgress, v2) @@ -3651,12 +4217,17 @@ func (m *ImportProgress) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + elementCount = packedLen / 4 + if elementCount != 0 && len(m.SamplingProgress) == 0 { + m.SamplingProgress = make([]float32, 0, elementCount) + } for iNdEx < postIndex { var v uint32 if (iNdEx + 4) > l { return io.ErrUnexpectedEOF } - v = uint32(binary.LittleEndian.Uint32(dAtA[iNdEx:])) + v = uint32(encoding_binary.LittleEndian.Uint32(dAtA[iNdEx:])) iNdEx += 4 v2 := float32(math.Float32frombits(v)) m.SamplingProgress = append(m.SamplingProgress, v2) @@ -3670,7 +4241,7 @@ func (m *ImportProgress) Unmarshal(dAtA []byte) error { if (iNdEx + 4) > l { return io.ErrUnexpectedEOF } - v = uint32(binary.LittleEndian.Uint32(dAtA[iNdEx:])) + v = uint32(encoding_binary.LittleEndian.Uint32(dAtA[iNdEx:])) iNdEx += 4 v2 := float32(math.Float32frombits(v)) m.ReadProgress = append(m.ReadProgress, v2) @@ -3697,12 +4268,17 @@ func (m *ImportProgress) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + elementCount = packedLen / 4 + if elementCount != 0 && len(m.ReadProgress) == 0 { + m.ReadProgress = make([]float32, 0, elementCount) + } for iNdEx < postIndex { var v uint32 if (iNdEx + 4) > l { return io.ErrUnexpectedEOF } - v = uint32(binary.LittleEndian.Uint32(dAtA[iNdEx:])) + v = uint32(encoding_binary.LittleEndian.Uint32(dAtA[iNdEx:])) iNdEx += 4 v2 := float32(math.Float32frombits(v)) m.ReadProgress = append(m.ReadProgress, v2) @@ -3716,7 +4292,7 @@ func (m *ImportProgress) Unmarshal(dAtA []byte) error { if (iNdEx + 4) > l { return io.ErrUnexpectedEOF } - v = uint32(binary.LittleEndian.Uint32(dAtA[iNdEx:])) + v = uint32(encoding_binary.LittleEndian.Uint32(dAtA[iNdEx:])) iNdEx += 4 v2 := float32(math.Float32frombits(v)) m.WriteProgress = append(m.WriteProgress, v2) @@ -3743,12 +4319,17 @@ func (m *ImportProgress) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + elementCount = packedLen / 4 + if elementCount != 0 && len(m.WriteProgress) == 0 { + m.WriteProgress = make([]float32, 0, elementCount) + } for iNdEx < postIndex { var v uint32 if (iNdEx + 4) > l { return io.ErrUnexpectedEOF } - v = uint32(binary.LittleEndian.Uint32(dAtA[iNdEx:])) + v = uint32(encoding_binary.LittleEndian.Uint32(dAtA[iNdEx:])) iNdEx += 4 v2 := float32(math.Float32frombits(v)) m.WriteProgress = append(m.WriteProgress, v2) @@ -3782,7 +4363,7 @@ func (m *ImportProgress) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.SpanProgress = append(m.SpanProgress, cockroach_roachpb1.Span{}) + m.SpanProgress = append(m.SpanProgress, roachpb.Span{}) if err := m.SpanProgress[len(m.SpanProgress)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -3863,7 +4444,7 @@ func (m *ResumeSpanList) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.ResumeSpans = append(m.ResumeSpans, cockroach_roachpb1.Span{}) + m.ResumeSpans = append(m.ResumeSpans, roachpb.Span{}) if err := m.ResumeSpans[len(m.ResumeSpans)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -4962,6 +5543,17 @@ func (m *Payload) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + var count int + for _, integer := range dAtA { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.DescriptorIDs) == 0 { + m.DescriptorIDs = make([]github_com_cockroachdb_cockroach_pkg_sql_sqlbase.ID, 0, elementCount) + } for iNdEx < postIndex { var v github_com_cockroachdb_cockroach_pkg_sql_sqlbase.ID for shift := uint(0); ; shift += 7 { @@ -5263,7 +5855,7 @@ func (m *Progress) Unmarshal(dAtA []byte) error { if (iNdEx + 4) > l { return io.ErrUnexpectedEOF } - v = uint32(binary.LittleEndian.Uint32(dAtA[iNdEx:])) + v = uint32(encoding_binary.LittleEndian.Uint32(dAtA[iNdEx:])) iNdEx += 4 m.Progress = &Progress_FractionCompleted{float32(math.Float32frombits(v))} case 2: @@ -5311,7 +5903,7 @@ func (m *Progress) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - v := &cockroach_util_hlc.Timestamp{} + v := &hlc.Timestamp{} if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -5632,9 +6224,9 @@ var ( ErrIntOverflowJobs = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("jobs/jobspb/jobs.proto", fileDescriptorJobs) } +func init() { proto.RegisterFile("jobs/jobspb/jobs.proto", fileDescriptor_jobs_7f594aaf3aa20aa4) } -var fileDescriptorJobs = []byte{ +var fileDescriptor_jobs_7f594aaf3aa20aa4 = []byte{ // 2144 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x58, 0x5f, 0x6f, 0xe3, 0x58, 0x15, 0x8f, 0x13, 0x27, 0x71, 0x4e, 0xfe, 0xd4, 0xbd, 0xad, 0x66, 0xbd, 0xd1, 0x6c, 0x13, 0x02, diff --git a/pkg/kv/dist_sender_server_test.go b/pkg/kv/dist_sender_server_test.go index 6cdaa19b8c97..fd90db1b551c 100644 --- a/pkg/kv/dist_sender_server_test.go +++ b/pkg/kv/dist_sender_server_test.go @@ -1025,7 +1025,7 @@ func TestMultiRangeScanReverseScanInconsistent(t *testing.T) { } if l := len(rows); l != 1 { - t.Fatalf("%d: expected 1 row; got %d\n%s", i, l, rows) + t.Fatalf("%d: expected 1 row; got %d\n%v", i, l, rows) } if key := string(rows[0].Key); keys[0] != key { t.Errorf("expected key %q; got %q", keys[0], key) diff --git a/pkg/roachpb/api.pb.go b/pkg/roachpb/api.pb.go index aae2c7536291..09f4bf75c78f 100644 --- a/pkg/roachpb/api.pb.go +++ b/pkg/roachpb/api.pb.go @@ -1,215 +1,25 @@ // Code generated by protoc-gen-gogo. DO NOT EDIT. // source: roachpb/api.proto -/* - Package roachpb is a generated protocol buffer package. - - It is generated from these files: - roachpb/api.proto - roachpb/app_stats.proto - roachpb/data.proto - roachpb/errors.proto - roachpb/internal.proto - roachpb/internal_raft.proto - roachpb/io-formats.proto - roachpb/metadata.proto - - It has these top-level messages: - RangeInfo - RequestHeader - ResponseHeader - GetRequest - GetResponse - PutRequest - PutResponse - ConditionalPutRequest - ConditionalPutResponse - InitPutRequest - InitPutResponse - IncrementRequest - IncrementResponse - DeleteRequest - DeleteResponse - DeleteRangeRequest - DeleteRangeResponse - ClearRangeRequest - ClearRangeResponse - ScanOptions - ScanRequest - ScanResponse - ReverseScanRequest - ReverseScanResponse - CheckConsistencyRequest - CheckConsistencyResponse - RecomputeStatsRequest - RecomputeStatsResponse - BeginTransactionRequest - BeginTransactionResponse - EndTransactionRequest - EndTransactionResponse - AdminSplitRequest - AdminSplitResponse - AdminMergeRequest - AdminMergeResponse - AdminTransferLeaseRequest - AdminTransferLeaseResponse - AdminChangeReplicasRequest - AdminChangeReplicasResponse - AdminRelocateRangeRequest - AdminRelocateRangeResponse - HeartbeatTxnRequest - HeartbeatTxnResponse - GCRequest - GCResponse - PushTxnRequest - PushTxnResponse - QueryTxnRequest - QueryTxnResponse - QueryIntentRequest - QueryIntentResponse - ResolveIntentRequest - ResolveIntentResponse - ResolveIntentRangeRequest - ResolveIntentRangeResponse - MergeRequest - MergeResponse - TruncateLogRequest - TruncateLogResponse - RequestLeaseRequest - TransferLeaseRequest - LeaseInfoRequest - LeaseInfoResponse - RequestLeaseResponse - ComputeChecksumRequest - ComputeChecksumResponse - ExportStorage - WriteBatchRequest - WriteBatchResponse - ExportRequest - BulkOpSummary - ExportResponse - ImportRequest - ImportResponse - AdminScatterRequest - AdminScatterResponse - AddSSTableRequest - AddSSTableResponse - RefreshRequest - RefreshResponse - RefreshRangeRequest - RefreshRangeResponse - SubsumeRequest - SubsumeResponse - RangeStatsRequest - RangeStatsResponse - RequestUnion - ResponseUnion - Header - BatchRequest - BatchResponse - RangeFeedRequest - RangeFeedValue - RangeFeedCheckpoint - RangeFeedError - RangeFeedEvent - StatementStatistics - SensitiveInfo - NumericStat - StatementStatisticsKey - CollectedStatementStatistics - ExplainTreePlanNode - Span - Value - KeyValue - StoreIdent - SplitTrigger - MergeTrigger - ChangeReplicasTrigger - ModifiedSpanTrigger - InternalCommitTrigger - ObservedTimestamp - Transaction - Intent - SequencedWrite - Lease - AbortSpanEntry - TxnCoordMeta - NotLeaseHolderError - NodeUnavailableError - UnsupportedRequestError - RangeNotFoundError - RangeKeyMismatchError - ReadWithinUncertaintyIntervalError - TransactionAbortedError - TransactionPushError - TransactionRetryError - TransactionStatusError - WriteIntentError - WriteTooOldError - OpRequiresTxnError - ConditionFailedError - LeaseRejectedError - SendError - AmbiguousResultError - RaftGroupDeletedError - ReplicaCorruptionError - ReplicaTooOldError - StoreNotFoundError - UnhandledRetryableError - HandledRetryableTxnError - TxnAlreadyEncounteredErrorError - IntegerOverflowError - MixedSuccessError - BatchTimestampBeforeGCError - IntentMissingError - MergeInProgressError - RangeFeedRetryError - ErrorDetail - ErrPosition - Error - InternalTimeSeriesData - InternalTimeSeriesSample - RaftTruncatedState - RaftTombstone - RaftSnapshotData - IOFileFormat - CSVOptions - MySQLOutfileOptions - PgCopyOptions - PgDumpOptions - Attributes - ReplicationTarget - ReplicaDescriptor - ReplicaIdent - RangeDescriptor - Percentiles - StoreCapacity - NodeDescriptor - LocalityAddress - StoreDescriptor - StoreDeadReplicas - Locality - Tier - Version -*/ package roachpb import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" -import cockroach_storage_engine_enginepb1 "github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb" -import cockroach_storage_engine_enginepb "github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb" -import cockroach_util_hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" -import cockroach_util_tracing "github.com/cockroachdb/cockroach/pkg/util/tracing" +import enginepb "github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb" +import hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" +import tracing "github.com/cockroachdb/cockroach/pkg/util/tracing" import github_com_cockroachdb_cockroach_pkg_util_uuid "github.com/cockroachdb/cockroach/pkg/util/uuid" import bytes "bytes" -import context "context" -import grpc "google.golang.org/grpc" +import ( + context "context" + grpc "google.golang.org/grpc" +) -import binary "encoding/binary" +import encoding_binary "encoding/binary" import io "io" @@ -258,7 +68,9 @@ var ReadConsistencyType_value = map[string]int32{ func (x ReadConsistencyType) String() string { return proto.EnumName(ReadConsistencyType_name, int32(x)) } -func (ReadConsistencyType) EnumDescriptor() ([]byte, []int) { return fileDescriptorApi, []int{0} } +func (ReadConsistencyType) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_api_a51b49ba855d15ec, []int{0} +} // ScanFormat is an enumeration of the available response formats for MVCCScan // operations. @@ -284,7 +96,9 @@ var ScanFormat_value = map[string]int32{ func (x ScanFormat) String() string { return proto.EnumName(ScanFormat_name, int32(x)) } -func (ScanFormat) EnumDescriptor() ([]byte, []int) { return fileDescriptorApi, []int{1} } +func (ScanFormat) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_api_a51b49ba855d15ec, []int{1} +} // TxnPushType determines what action to take when pushing a transaction. type PushTxnType int32 @@ -317,7 +131,9 @@ var PushTxnType_value = map[string]int32{ func (x PushTxnType) String() string { return proto.EnumName(PushTxnType_name, int32(x)) } -func (PushTxnType) EnumDescriptor() ([]byte, []int) { return fileDescriptorApi, []int{2} } +func (PushTxnType) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_api_a51b49ba855d15ec, []int{2} +} type ExportStorageProvider int32 @@ -353,7 +169,9 @@ var ExportStorageProvider_value = map[string]int32{ func (x ExportStorageProvider) String() string { return proto.EnumName(ExportStorageProvider_name, int32(x)) } -func (ExportStorageProvider) EnumDescriptor() ([]byte, []int) { return fileDescriptorApi, []int{3} } +func (ExportStorageProvider) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_api_a51b49ba855d15ec, []int{3} +} type MVCCFilter int32 @@ -374,7 +192,9 @@ var MVCCFilter_value = map[string]int32{ func (x MVCCFilter) String() string { return proto.EnumName(MVCCFilter_name, int32(x)) } -func (MVCCFilter) EnumDescriptor() ([]byte, []int) { return fileDescriptorApi, []int{4} } +func (MVCCFilter) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_api_a51b49ba855d15ec, []int{4} +} type ResponseHeader_ResumeReason int32 @@ -405,7 +225,7 @@ func (x ResponseHeader_ResumeReason) String() string { return proto.EnumName(ResponseHeader_ResumeReason_name, int32(x)) } func (ResponseHeader_ResumeReason) EnumDescriptor() ([]byte, []int) { - return fileDescriptorApi, []int{2, 0} + return fileDescriptor_api_a51b49ba855d15ec, []int{2, 0} } type QueryIntentRequest_IfMissingBehavior int32 @@ -441,20 +261,46 @@ func (x QueryIntentRequest_IfMissingBehavior) String() string { return proto.EnumName(QueryIntentRequest_IfMissingBehavior_name, int32(x)) } func (QueryIntentRequest_IfMissingBehavior) EnumDescriptor() ([]byte, []int) { - return fileDescriptorApi, []int{50, 0} + return fileDescriptor_api_a51b49ba855d15ec, []int{50, 0} } // RangeInfo describes a range which executed a request. It contains // the range descriptor and lease information at the time of execution. type RangeInfo struct { - Desc RangeDescriptor `protobuf:"bytes,1,opt,name=desc" json:"desc"` - Lease Lease `protobuf:"bytes,2,opt,name=lease" json:"lease"` + Desc RangeDescriptor `protobuf:"bytes,1,opt,name=desc,proto3" json:"desc"` + Lease Lease `protobuf:"bytes,2,opt,name=lease,proto3" json:"lease"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{0} +} +func (m *RangeInfo) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RangeInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RangeInfo) XXX_Merge(src proto.Message) { + xxx_messageInfo_RangeInfo.Merge(dst, src) +} +func (m *RangeInfo) XXX_Size() int { + return m.Size() +} +func (m *RangeInfo) XXX_DiscardUnknown() { + xxx_messageInfo_RangeInfo.DiscardUnknown(m) } -func (m *RangeInfo) Reset() { *m = RangeInfo{} } -func (m *RangeInfo) String() string { return proto.CompactTextString(m) } -func (*RangeInfo) ProtoMessage() {} -func (*RangeInfo) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{0} } +var xxx_messageInfo_RangeInfo proto.InternalMessageInfo // RequestHeader is supplied with every storage node request. type RequestHeader struct { @@ -467,20 +313,46 @@ type RequestHeader struct { // including Key and excluding EndKey. EndKey Key `protobuf:"bytes,4,opt,name=end_key,json=endKey,proto3,casttype=Key" json:"end_key,omitempty"` // A one-indexed transactional sequence number. - Sequence int32 `protobuf:"varint,5,opt,name=sequence,proto3" json:"sequence,omitempty"` + Sequence int32 `protobuf:"varint,5,opt,name=sequence,proto3" json:"sequence,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *RequestHeader) Reset() { *m = RequestHeader{} } -func (m *RequestHeader) String() string { return proto.CompactTextString(m) } -func (*RequestHeader) ProtoMessage() {} -func (*RequestHeader) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{1} } +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_a51b49ba855d15ec, []int{1} +} +func (m *RequestHeader) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RequestHeader) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RequestHeader) XXX_Merge(src proto.Message) { + xxx_messageInfo_RequestHeader.Merge(dst, src) +} +func (m *RequestHeader) XXX_Size() int { + return m.Size() +} +func (m *RequestHeader) XXX_DiscardUnknown() { + xxx_messageInfo_RequestHeader.DiscardUnknown(m) +} + +var xxx_messageInfo_RequestHeader proto.InternalMessageInfo // ResponseHeader is returned with every storage node response. type ResponseHeader struct { // txn is non-nil if the request specified a non-nil transaction. // The transaction timestamp and/or priority may have been updated, // depending on the outcome of the request. - Txn *Transaction `protobuf:"bytes,3,opt,name=txn" json:"txn,omitempty"` + Txn *Transaction `protobuf:"bytes,3,opt,name=txn,proto3" json:"txn,omitempty"` // The next span to resume from when the response doesn't cover the full span // requested. This can happen when a bound on the keys is set through // max_span_request_keys in the batch header or when a scan has been stopped @@ -490,7 +362,7 @@ type ResponseHeader struct { // operated on. The span is set to the original span if the request // was ignored because max_span_request_keys was hit due to another // request in the batch. For a reverse scan the end_key is updated. - ResumeSpan *Span `protobuf:"bytes,4,opt,name=resume_span,json=resumeSpan" json:"resume_span,omitempty"` + ResumeSpan *Span `protobuf:"bytes,4,opt,name=resume_span,json=resumeSpan,proto3" json:"resume_span,omitempty"` // When resume_span is populated, this specifies the reason why the operation // wasn't completed and needs to be resumed. // This field appeared in v2.0. Responses from storage coming from older @@ -501,42 +373,120 @@ type ResponseHeader struct { NumKeys int64 `protobuf:"varint,5,opt,name=num_keys,json=numKeys,proto3" json:"num_keys,omitempty"` // Range or list of ranges used to execute the request. Multiple // ranges may be returned for Scan, ReverseScan or DeleteRange. - RangeInfos []RangeInfo `protobuf:"bytes,6,rep,name=range_infos,json=rangeInfos" json:"range_infos"` + RangeInfos []RangeInfo `protobuf:"bytes,6,rep,name=range_infos,json=rangeInfos,proto3" json:"range_infos"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{2} +} +func (m *ResponseHeader) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ResponseHeader) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ResponseHeader) XXX_Merge(src proto.Message) { + xxx_messageInfo_ResponseHeader.Merge(dst, src) +} +func (m *ResponseHeader) XXX_Size() int { + return m.Size() +} +func (m *ResponseHeader) XXX_DiscardUnknown() { + xxx_messageInfo_ResponseHeader.DiscardUnknown(m) } -func (m *ResponseHeader) Reset() { *m = ResponseHeader{} } -func (m *ResponseHeader) String() string { return proto.CompactTextString(m) } -func (*ResponseHeader) ProtoMessage() {} -func (*ResponseHeader) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{2} } +var xxx_messageInfo_ResponseHeader proto.InternalMessageInfo // A GetRequest is the argument for the Get() method. type GetRequest struct { - RequestHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{3} +} +func (m *GetRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *GetRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *GetRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_GetRequest.Merge(dst, src) +} +func (m *GetRequest) XXX_Size() int { + return m.Size() +} +func (m *GetRequest) XXX_DiscardUnknown() { + xxx_messageInfo_GetRequest.DiscardUnknown(m) } -func (m *GetRequest) Reset() { *m = GetRequest{} } -func (m *GetRequest) String() string { return proto.CompactTextString(m) } -func (*GetRequest) ProtoMessage() {} -func (*GetRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{3} } +var xxx_messageInfo_GetRequest proto.InternalMessageInfo // A GetResponse is the return value from the Get() method. // If the key doesn't exist, Value will be nil. type GetResponse struct { - ResponseHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` - Value *Value `protobuf:"bytes,2,opt,name=value" json:"value,omitempty"` + 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. - IntentValue *Value `protobuf:"bytes,3,opt,name=intent_value,json=intentValue" json:"intent_value,omitempty"` + IntentValue *Value `protobuf:"bytes,3,opt,name=intent_value,json=intentValue,proto3" json:"intent_value,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{4} +} +func (m *GetResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *GetResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *GetResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_GetResponse.Merge(dst, src) +} +func (m *GetResponse) XXX_Size() int { + return m.Size() +} +func (m *GetResponse) XXX_DiscardUnknown() { + xxx_messageInfo_GetResponse.DiscardUnknown(m) } -func (m *GetResponse) Reset() { *m = GetResponse{} } -func (m *GetResponse) String() string { return proto.CompactTextString(m) } -func (*GetResponse) ProtoMessage() {} -func (*GetResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{4} } +var xxx_messageInfo_GetResponse proto.InternalMessageInfo // A PutRequest is the argument to the Put() method. type PutRequest struct { - RequestHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` - Value Value `protobuf:"bytes,2,opt,name=value" json:"value"` + RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + Value Value `protobuf:"bytes,2,opt,name=value,proto3" json:"value"` // Specify as true to put the value without a corresponding // timestamp. This option should be used with care as it precludes // the use of this value with transactions. @@ -544,23 +494,75 @@ type PutRequest struct { // NOTE: For internal use only! Set to indicate that the put is // writing to virgin keyspace and no reads are necessary to // rationalize MVCC. - Blind bool `protobuf:"varint,4,opt,name=blind,proto3" json:"blind,omitempty"` + Blind bool `protobuf:"varint,4,opt,name=blind,proto3" json:"blind,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{5} +} +func (m *PutRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *PutRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *PutRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_PutRequest.Merge(dst, src) +} +func (m *PutRequest) XXX_Size() int { + return m.Size() +} +func (m *PutRequest) XXX_DiscardUnknown() { + xxx_messageInfo_PutRequest.DiscardUnknown(m) } -func (m *PutRequest) Reset() { *m = PutRequest{} } -func (m *PutRequest) String() string { return proto.CompactTextString(m) } -func (*PutRequest) ProtoMessage() {} -func (*PutRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{5} } +var xxx_messageInfo_PutRequest proto.InternalMessageInfo // A PutResponse is the return value from the Put() method. type PutResponse struct { - ResponseHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{6} +} +func (m *PutResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *PutResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *PutResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_PutResponse.Merge(dst, src) +} +func (m *PutResponse) XXX_Size() int { + return m.Size() +} +func (m *PutResponse) XXX_DiscardUnknown() { + xxx_messageInfo_PutResponse.DiscardUnknown(m) } -func (m *PutResponse) Reset() { *m = PutResponse{} } -func (m *PutResponse) String() string { return proto.CompactTextString(m) } -func (*PutResponse) ProtoMessage() {} -func (*PutResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{6} } +var xxx_messageInfo_PutResponse proto.InternalMessageInfo // A ConditionalPutRequest is the argument to the ConditionalPut() method. // @@ -569,34 +571,86 @@ func (*PutResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []i // - If key exists, but value is empty and exp_value is not nil but empty, sets value. // - Otherwise, returns an error containing the actual value of the key. type ConditionalPutRequest struct { - RequestHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` // The value to put. - Value Value `protobuf:"bytes,2,opt,name=value" json:"value"` + Value Value `protobuf:"bytes,2,opt,name=value,proto3" json:"value"` // Set exp_value.bytes empty to test for non-existence. Specify as nil // to indicate there should be no existing entry. This is different // from the expectation that the value exists but is empty. - ExpValue *Value `protobuf:"bytes,3,opt,name=exp_value,json=expValue" json:"exp_value,omitempty"` + ExpValue *Value `protobuf:"bytes,3,opt,name=exp_value,json=expValue,proto3" json:"exp_value,omitempty"` // NOTE: For internal use only! Set to indicate that the put is // writing to virgin keyspace and no reads are necessary to // rationalize MVCC. - Blind bool `protobuf:"varint,4,opt,name=blind,proto3" json:"blind,omitempty"` + Blind bool `protobuf:"varint,4,opt,name=blind,proto3" json:"blind,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{7} +} +func (m *ConditionalPutRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ConditionalPutRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ConditionalPutRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_ConditionalPutRequest.Merge(dst, src) +} +func (m *ConditionalPutRequest) XXX_Size() int { + return m.Size() +} +func (m *ConditionalPutRequest) XXX_DiscardUnknown() { + xxx_messageInfo_ConditionalPutRequest.DiscardUnknown(m) } -func (m *ConditionalPutRequest) Reset() { *m = ConditionalPutRequest{} } -func (m *ConditionalPutRequest) String() string { return proto.CompactTextString(m) } -func (*ConditionalPutRequest) ProtoMessage() {} -func (*ConditionalPutRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{7} } +var xxx_messageInfo_ConditionalPutRequest proto.InternalMessageInfo // A ConditionalPutResponse is the return value from the // ConditionalPut() method. type ConditionalPutResponse struct { - ResponseHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{8} +} +func (m *ConditionalPutResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ConditionalPutResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ConditionalPutResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_ConditionalPutResponse.Merge(dst, src) +} +func (m *ConditionalPutResponse) XXX_Size() int { + return m.Size() +} +func (m *ConditionalPutResponse) XXX_DiscardUnknown() { + xxx_messageInfo_ConditionalPutResponse.DiscardUnknown(m) } -func (m *ConditionalPutResponse) Reset() { *m = ConditionalPutResponse{} } -func (m *ConditionalPutResponse) String() string { return proto.CompactTextString(m) } -func (*ConditionalPutResponse) ProtoMessage() {} -func (*ConditionalPutResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{8} } +var xxx_messageInfo_ConditionalPutResponse proto.InternalMessageInfo // An InitPutRequest is the argument to the InitPut() method. // @@ -605,30 +659,82 @@ func (*ConditionalPutResponse) Descriptor() ([]byte, []int) { return fileDescrip // If failOnTombstones is set to true, tombstone values count as mismatched // values and will cause a ConditionFailedError. type InitPutRequest struct { - RequestHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` - Value Value `protobuf:"bytes,2,opt,name=value" json:"value"` + RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + Value Value `protobuf:"bytes,2,opt,name=value,proto3" json:"value"` // NOTE: For internal use only! Set to indicate that the put is // writing to virgin keyspace and no reads are necessary to // rationalize MVCC. Blind bool `protobuf:"varint,3,opt,name=blind,proto3" json:"blind,omitempty"` // If true, tombstones cause ConditionFailedErrors. - FailOnTombstones bool `protobuf:"varint,4,opt,name=failOnTombstones,proto3" json:"failOnTombstones,omitempty"` + FailOnTombstones bool `protobuf:"varint,4,opt,name=failOnTombstones,proto3" json:"failOnTombstones,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *InitPutRequest) Reset() { *m = InitPutRequest{} } -func (m *InitPutRequest) String() string { return proto.CompactTextString(m) } -func (*InitPutRequest) ProtoMessage() {} -func (*InitPutRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{9} } +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_a51b49ba855d15ec, []int{9} +} +func (m *InitPutRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *InitPutRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *InitPutRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_InitPutRequest.Merge(dst, src) +} +func (m *InitPutRequest) XXX_Size() int { + return m.Size() +} +func (m *InitPutRequest) XXX_DiscardUnknown() { + xxx_messageInfo_InitPutRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_InitPutRequest proto.InternalMessageInfo // A InitPutResponse is the return value from the InitPut() method. type InitPutResponse struct { - ResponseHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{10} +} +func (m *InitPutResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *InitPutResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *InitPutResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_InitPutResponse.Merge(dst, src) +} +func (m *InitPutResponse) XXX_Size() int { + return m.Size() +} +func (m *InitPutResponse) XXX_DiscardUnknown() { + xxx_messageInfo_InitPutResponse.DiscardUnknown(m) } -func (m *InitPutResponse) Reset() { *m = InitPutResponse{} } -func (m *InitPutResponse) String() string { return proto.CompactTextString(m) } -func (*InitPutResponse) ProtoMessage() {} -func (*InitPutResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{10} } +var xxx_messageInfo_InitPutResponse proto.InternalMessageInfo // An IncrementRequest is the argument to the Increment() method. It // increments the value for key, and returns the new value. If no @@ -637,52 +743,156 @@ func (*InitPutResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, // by Put() or ConditionalPut(). Similarly, Put() and ConditionalPut() // cannot be invoked on an incremented key. type IncrementRequest struct { - RequestHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` - Increment int64 `protobuf:"varint,2,opt,name=increment,proto3" json:"increment,omitempty"` + RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + Increment int64 `protobuf:"varint,2,opt,name=increment,proto3" json:"increment,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{11} +} +func (m *IncrementRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *IncrementRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *IncrementRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_IncrementRequest.Merge(dst, src) +} +func (m *IncrementRequest) XXX_Size() int { + return m.Size() +} +func (m *IncrementRequest) XXX_DiscardUnknown() { + xxx_messageInfo_IncrementRequest.DiscardUnknown(m) } -func (m *IncrementRequest) Reset() { *m = IncrementRequest{} } -func (m *IncrementRequest) String() string { return proto.CompactTextString(m) } -func (*IncrementRequest) ProtoMessage() {} -func (*IncrementRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{11} } +var xxx_messageInfo_IncrementRequest proto.InternalMessageInfo // An IncrementResponse is the return value from the Increment // method. The new value after increment is specified in NewValue. If // the value could not be decoded as specified, Error will be set. type IncrementResponse struct { - ResponseHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` - NewValue int64 `protobuf:"varint,2,opt,name=new_value,json=newValue,proto3" json:"new_value,omitempty"` + ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + NewValue int64 `protobuf:"varint,2,opt,name=new_value,json=newValue,proto3" json:"new_value,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{12} +} +func (m *IncrementResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *IncrementResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *IncrementResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_IncrementResponse.Merge(dst, src) +} +func (m *IncrementResponse) XXX_Size() int { + return m.Size() +} +func (m *IncrementResponse) XXX_DiscardUnknown() { + xxx_messageInfo_IncrementResponse.DiscardUnknown(m) } -func (m *IncrementResponse) Reset() { *m = IncrementResponse{} } -func (m *IncrementResponse) String() string { return proto.CompactTextString(m) } -func (*IncrementResponse) ProtoMessage() {} -func (*IncrementResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{12} } +var xxx_messageInfo_IncrementResponse proto.InternalMessageInfo // A DeleteRequest is the argument to the Delete() method. type DeleteRequest struct { - RequestHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{13} +} +func (m *DeleteRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *DeleteRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *DeleteRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_DeleteRequest.Merge(dst, src) +} +func (m *DeleteRequest) XXX_Size() int { + return m.Size() +} +func (m *DeleteRequest) XXX_DiscardUnknown() { + xxx_messageInfo_DeleteRequest.DiscardUnknown(m) } -func (m *DeleteRequest) Reset() { *m = DeleteRequest{} } -func (m *DeleteRequest) String() string { return proto.CompactTextString(m) } -func (*DeleteRequest) ProtoMessage() {} -func (*DeleteRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{13} } +var xxx_messageInfo_DeleteRequest proto.InternalMessageInfo // A DeleteResponse is the return value from the Delete() method. type DeleteResponse struct { - ResponseHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{14} +} +func (m *DeleteResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *DeleteResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *DeleteResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_DeleteResponse.Merge(dst, src) +} +func (m *DeleteResponse) XXX_Size() int { + return m.Size() +} +func (m *DeleteResponse) XXX_DiscardUnknown() { + xxx_messageInfo_DeleteResponse.DiscardUnknown(m) } -func (m *DeleteResponse) Reset() { *m = DeleteResponse{} } -func (m *DeleteResponse) String() string { return proto.CompactTextString(m) } -func (*DeleteResponse) ProtoMessage() {} -func (*DeleteResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{14} } +var xxx_messageInfo_DeleteResponse proto.InternalMessageInfo // A DeleteRangeRequest is the argument to the DeleteRange() method. It // specifies the range of keys to delete. type DeleteRangeRequest struct { - RequestHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` // return the keys that are deleted in the response. ReturnKeys bool `protobuf:"varint,3,opt,name=return_keys,json=returnKeys,proto3" json:"return_keys,omitempty"` // delete "inline" keys which are stored without MVCC timestamps. Note that @@ -696,26 +906,78 @@ type DeleteRangeRequest struct { // // Inline values cannot be deleted transactionally; a DeleteRange with // "inline" set to true will fail if it is executed within a transaction. - Inline bool `protobuf:"varint,4,opt,name=inline,proto3" json:"inline,omitempty"` + Inline bool `protobuf:"varint,4,opt,name=inline,proto3" json:"inline,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{15} +} +func (m *DeleteRangeRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *DeleteRangeRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *DeleteRangeRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_DeleteRangeRequest.Merge(dst, src) +} +func (m *DeleteRangeRequest) XXX_Size() int { + return m.Size() +} +func (m *DeleteRangeRequest) XXX_DiscardUnknown() { + xxx_messageInfo_DeleteRangeRequest.DiscardUnknown(m) } -func (m *DeleteRangeRequest) Reset() { *m = DeleteRangeRequest{} } -func (m *DeleteRangeRequest) String() string { return proto.CompactTextString(m) } -func (*DeleteRangeRequest) ProtoMessage() {} -func (*DeleteRangeRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{15} } +var xxx_messageInfo_DeleteRangeRequest proto.InternalMessageInfo // A DeleteRangeResponse is the return value from the DeleteRange() // method. type DeleteRangeResponse struct { - ResponseHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` // All the deleted keys if return_keys is set. - Keys []Key `protobuf:"bytes,2,rep,name=keys,casttype=Key" json:"keys,omitempty"` + Keys []Key `protobuf:"bytes,2,rep,name=keys,proto3,casttype=Key" json:"keys,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{16} +} +func (m *DeleteRangeResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *DeleteRangeResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *DeleteRangeResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_DeleteRangeResponse.Merge(dst, src) +} +func (m *DeleteRangeResponse) XXX_Size() int { + return m.Size() +} +func (m *DeleteRangeResponse) XXX_DiscardUnknown() { + xxx_messageInfo_DeleteRangeResponse.DiscardUnknown(m) } -func (m *DeleteRangeResponse) Reset() { *m = DeleteRangeResponse{} } -func (m *DeleteRangeResponse) String() string { return proto.CompactTextString(m) } -func (*DeleteRangeResponse) ProtoMessage() {} -func (*DeleteRangeResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{16} } +var xxx_messageInfo_DeleteRangeResponse proto.InternalMessageInfo // A ClearRangeRequest is the argument to the ClearRange() method. It // specifies a range of keys to clear from the underlying engine. Note @@ -732,23 +994,75 @@ func (*DeleteRangeResponse) Descriptor() ([]byte, []int) { return fileDescriptor // range which is guaranteed to be both inactive and not see future // writes. Ignoring this warning may result in data loss. type ClearRangeRequest struct { - RequestHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *ClearRangeRequest) Reset() { *m = ClearRangeRequest{} } -func (m *ClearRangeRequest) String() string { return proto.CompactTextString(m) } -func (*ClearRangeRequest) ProtoMessage() {} -func (*ClearRangeRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{17} } +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_a51b49ba855d15ec, []int{17} +} +func (m *ClearRangeRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ClearRangeRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ClearRangeRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_ClearRangeRequest.Merge(dst, src) +} +func (m *ClearRangeRequest) XXX_Size() int { + return m.Size() +} +func (m *ClearRangeRequest) XXX_DiscardUnknown() { + xxx_messageInfo_ClearRangeRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_ClearRangeRequest proto.InternalMessageInfo // A ClearRangeResponse is the return value from the ClearRange() method. type ClearRangeResponse struct { - ResponseHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{18} +} +func (m *ClearRangeResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ClearRangeResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ClearRangeResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_ClearRangeResponse.Merge(dst, src) +} +func (m *ClearRangeResponse) XXX_Size() int { + return m.Size() +} +func (m *ClearRangeResponse) XXX_DiscardUnknown() { + xxx_messageInfo_ClearRangeResponse.DiscardUnknown(m) } -func (m *ClearRangeResponse) Reset() { *m = ClearRangeResponse{} } -func (m *ClearRangeResponse) String() string { return proto.CompactTextString(m) } -func (*ClearRangeResponse) ProtoMessage() {} -func (*ClearRangeResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{18} } +var xxx_messageInfo_ClearRangeResponse proto.InternalMessageInfo // ScanOptions is a collection of options for a batch of scans. The options // apply to all the scans in the batch. @@ -787,117 +1101,299 @@ type ScanOptions struct { // This can only be set if stop_at_range_boundary is set. If // header.max_span_request_keys is set, min_results needs to be <= // header.max_span_request_keys. - MinResults int64 `protobuf:"varint,2,opt,name=min_results,json=minResults,proto3" json:"min_results,omitempty"` + MinResults int64 `protobuf:"varint,2,opt,name=min_results,json=minResults,proto3" json:"min_results,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{19} +} +func (m *ScanOptions) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ScanOptions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ScanOptions) XXX_Merge(src proto.Message) { + xxx_messageInfo_ScanOptions.Merge(dst, src) +} +func (m *ScanOptions) XXX_Size() int { + return m.Size() +} +func (m *ScanOptions) XXX_DiscardUnknown() { + xxx_messageInfo_ScanOptions.DiscardUnknown(m) } -func (m *ScanOptions) Reset() { *m = ScanOptions{} } -func (m *ScanOptions) String() string { return proto.CompactTextString(m) } -func (*ScanOptions) ProtoMessage() {} -func (*ScanOptions) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{19} } +var xxx_messageInfo_ScanOptions proto.InternalMessageInfo // A ScanRequest is the argument to the Scan() method. It specifies the // start and end keys for an ascending scan of [start,end) and the maximum // number of results (unbounded if zero). type ScanRequest struct { - RequestHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` // The desired format for the response. If set to BATCH_RESPONSE, the server // will set the batch_responses field in the ScanResponse instead of the rows // field. - ScanFormat ScanFormat `protobuf:"varint,4,opt,name=scan_format,json=scanFormat,proto3,enum=cockroach.roachpb.ScanFormat" json:"scan_format,omitempty"` + ScanFormat ScanFormat `protobuf:"varint,4,opt,name=scan_format,json=scanFormat,proto3,enum=cockroach.roachpb.ScanFormat" json:"scan_format,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{20} +} +func (m *ScanRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ScanRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ScanRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_ScanRequest.Merge(dst, src) +} +func (m *ScanRequest) XXX_Size() int { + return m.Size() +} +func (m *ScanRequest) XXX_DiscardUnknown() { + xxx_messageInfo_ScanRequest.DiscardUnknown(m) } -func (m *ScanRequest) Reset() { *m = ScanRequest{} } -func (m *ScanRequest) String() string { return proto.CompactTextString(m) } -func (*ScanRequest) ProtoMessage() {} -func (*ScanRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{20} } +var xxx_messageInfo_ScanRequest proto.InternalMessageInfo // A ScanResponse is the return value from the Scan() method. type ScanResponse struct { - ResponseHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` // Empty if no rows were scanned. - Rows []KeyValue `protobuf:"bytes,2,rep,name=rows" json:"rows"` + Rows []KeyValue `protobuf:"bytes,2,rep,name=rows,proto3" json:"rows"` // The intent rows seen when performing a scan at the READ_UNCOMMITTED // consistency level. These rows do not count against the MaxSpanRequestKeys // count. - IntentRows []KeyValue `protobuf:"bytes,3,rep,name=intent_rows,json=intentRows" json:"intent_rows"` + 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 // slices, alternating from key to value. Each entry in this field is // complete - there are no key/value pairs that are split across more than one // entry. There are num_keys total pairs across all entries, as defined by the // ResponseHeader. If set, rows will not be set and vice versa. - BatchResponses [][]byte `protobuf:"bytes,4,rep,name=batch_responses,json=batchResponses" json:"batch_responses,omitempty"` + BatchResponses [][]byte `protobuf:"bytes,4,rep,name=batch_responses,json=batchResponses,proto3" json:"batch_responses,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{21} +} +func (m *ScanResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ScanResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ScanResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_ScanResponse.Merge(dst, src) +} +func (m *ScanResponse) XXX_Size() int { + return m.Size() +} +func (m *ScanResponse) XXX_DiscardUnknown() { + xxx_messageInfo_ScanResponse.DiscardUnknown(m) } -func (m *ScanResponse) Reset() { *m = ScanResponse{} } -func (m *ScanResponse) String() string { return proto.CompactTextString(m) } -func (*ScanResponse) ProtoMessage() {} -func (*ScanResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{21} } +var xxx_messageInfo_ScanResponse proto.InternalMessageInfo // A ReverseScanRequest is the argument to the ReverseScan() method. It specifies the // start and end keys for a descending scan of [start,end) and the maximum // number of results (unbounded if zero). type ReverseScanRequest struct { - RequestHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` // The desired format for the response. If set to BATCH_RESPONSE, the server // will set the batch_responses field in the ScanResponse instead of the rows // field. - ScanFormat ScanFormat `protobuf:"varint,4,opt,name=scan_format,json=scanFormat,proto3,enum=cockroach.roachpb.ScanFormat" json:"scan_format,omitempty"` + ScanFormat ScanFormat `protobuf:"varint,4,opt,name=scan_format,json=scanFormat,proto3,enum=cockroach.roachpb.ScanFormat" json:"scan_format,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{22} +} +func (m *ReverseScanRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ReverseScanRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ReverseScanRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_ReverseScanRequest.Merge(dst, src) +} +func (m *ReverseScanRequest) XXX_Size() int { + return m.Size() +} +func (m *ReverseScanRequest) XXX_DiscardUnknown() { + xxx_messageInfo_ReverseScanRequest.DiscardUnknown(m) } -func (m *ReverseScanRequest) Reset() { *m = ReverseScanRequest{} } -func (m *ReverseScanRequest) String() string { return proto.CompactTextString(m) } -func (*ReverseScanRequest) ProtoMessage() {} -func (*ReverseScanRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{22} } +var xxx_messageInfo_ReverseScanRequest proto.InternalMessageInfo // A ReverseScanResponse is the return value from the ReverseScan() method. type ReverseScanResponse struct { - ResponseHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` // Empty if no rows were scanned. - Rows []KeyValue `protobuf:"bytes,2,rep,name=rows" json:"rows"` + Rows []KeyValue `protobuf:"bytes,2,rep,name=rows,proto3" json:"rows"` // The intent rows seen when performing a scan at the READ_UNCOMMITTED // consistency level. These rows do not count against the MaxSpanRequestKeys // count. - IntentRows []KeyValue `protobuf:"bytes,3,rep,name=intent_rows,json=intentRows" json:"intent_rows"` + 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 // slices, alternating from key to value. Each entry in this field is // complete - there are no key/value pairs that are split across more than one // entry. There are num_keys total pairs across all entries, as defined by the // ResponseHeader. If set, rows will not be set and vice versa. - BatchResponses [][]byte `protobuf:"bytes,4,rep,name=batch_responses,json=batchResponses" json:"batch_responses,omitempty"` + BatchResponses [][]byte `protobuf:"bytes,4,rep,name=batch_responses,json=batchResponses,proto3" json:"batch_responses,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{23} +} +func (m *ReverseScanResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ReverseScanResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ReverseScanResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_ReverseScanResponse.Merge(dst, src) +} +func (m *ReverseScanResponse) XXX_Size() int { + return m.Size() +} +func (m *ReverseScanResponse) XXX_DiscardUnknown() { + xxx_messageInfo_ReverseScanResponse.DiscardUnknown(m) } -func (m *ReverseScanResponse) Reset() { *m = ReverseScanResponse{} } -func (m *ReverseScanResponse) String() string { return proto.CompactTextString(m) } -func (*ReverseScanResponse) ProtoMessage() {} -func (*ReverseScanResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{23} } +var xxx_messageInfo_ReverseScanResponse proto.InternalMessageInfo // A CheckConsistencyRequest is the argument to the CheckConsistency() method. // It specifies the start and end keys for a span of ranges to which a // consistency check should be applied. A consistency check on a range involves // running a ComputeChecksum on the range followed by a storage.CollectChecksum. type CheckConsistencyRequest struct { - RequestHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` // log a diff of inconsistencies if such inconsistencies are found. - WithDiff bool `protobuf:"varint,2,opt,name=with_diff,json=withDiff,proto3" json:"with_diff,omitempty"` + WithDiff bool `protobuf:"varint,2,opt,name=with_diff,json=withDiff,proto3" json:"with_diff,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{24} +} +func (m *CheckConsistencyRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *CheckConsistencyRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *CheckConsistencyRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_CheckConsistencyRequest.Merge(dst, src) +} +func (m *CheckConsistencyRequest) XXX_Size() int { + return m.Size() +} +func (m *CheckConsistencyRequest) XXX_DiscardUnknown() { + xxx_messageInfo_CheckConsistencyRequest.DiscardUnknown(m) } -func (m *CheckConsistencyRequest) Reset() { *m = CheckConsistencyRequest{} } -func (m *CheckConsistencyRequest) String() string { return proto.CompactTextString(m) } -func (*CheckConsistencyRequest) ProtoMessage() {} -func (*CheckConsistencyRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{24} } +var xxx_messageInfo_CheckConsistencyRequest proto.InternalMessageInfo // A CheckConsistencyResponse is the return value from the CheckConsistency() method. // If a replica finds itself to be inconsistent with its lease holder it will panic. type CheckConsistencyResponse struct { - ResponseHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *CheckConsistencyResponse) Reset() { *m = CheckConsistencyResponse{} } -func (m *CheckConsistencyResponse) String() string { return proto.CompactTextString(m) } -func (*CheckConsistencyResponse) ProtoMessage() {} -func (*CheckConsistencyResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{25} } +func (m *CheckConsistencyResponse) Reset() { *m = CheckConsistencyResponse{} } +func (m *CheckConsistencyResponse) String() string { return proto.CompactTextString(m) } +func (*CheckConsistencyResponse) ProtoMessage() {} +func (*CheckConsistencyResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_api_a51b49ba855d15ec, []int{25} +} +func (m *CheckConsistencyResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *CheckConsistencyResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *CheckConsistencyResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_CheckConsistencyResponse.Merge(dst, src) +} +func (m *CheckConsistencyResponse) XXX_Size() int { + return m.Size() +} +func (m *CheckConsistencyResponse) XXX_DiscardUnknown() { + xxx_messageInfo_CheckConsistencyResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_CheckConsistencyResponse proto.InternalMessageInfo // An RecomputeStatsRequest triggers a stats recomputation on the Range addressed by // the request. @@ -911,54 +1407,158 @@ func (*CheckConsistencyResponse) Descriptor() ([]byte, []int) { return fileDescr // it is safe to invoke at any time, including repeatedly, though it should be // used conservatively due to performing a full scan of the Range. type RecomputeStatsRequest struct { - RequestHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` // When dry_run is true, the stats delta is computed, but no stats adjustment // is performed. This isn't useful outside of testing since RecomputeStats is // safe and idempotent. - DryRun bool `protobuf:"varint,2,opt,name=dry_run,json=dryRun,proto3" json:"dry_run,omitempty"` + DryRun bool `protobuf:"varint,2,opt,name=dry_run,json=dryRun,proto3" json:"dry_run,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{26} +} +func (m *RecomputeStatsRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RecomputeStatsRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RecomputeStatsRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_RecomputeStatsRequest.Merge(dst, src) +} +func (m *RecomputeStatsRequest) XXX_Size() int { + return m.Size() +} +func (m *RecomputeStatsRequest) XXX_DiscardUnknown() { + xxx_messageInfo_RecomputeStatsRequest.DiscardUnknown(m) } -func (m *RecomputeStatsRequest) Reset() { *m = RecomputeStatsRequest{} } -func (m *RecomputeStatsRequest) String() string { return proto.CompactTextString(m) } -func (*RecomputeStatsRequest) ProtoMessage() {} -func (*RecomputeStatsRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{26} } +var xxx_messageInfo_RecomputeStatsRequest proto.InternalMessageInfo // An RecomputeStatsResponse is the response to an RecomputeStatsRequest. type RecomputeStatsResponse struct { - ResponseHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` // added_delta is the adjustment made to the range's stats, i.e. `new_stats = old_stats + added_delta`. - AddedDelta cockroach_storage_engine_enginepb.MVCCStatsDelta `protobuf:"bytes,2,opt,name=added_delta,json=addedDelta" json:"added_delta"` + AddedDelta enginepb.MVCCStatsDelta `protobuf:"bytes,2,opt,name=added_delta,json=addedDelta,proto3" json:"added_delta"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{27} +} +func (m *RecomputeStatsResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RecomputeStatsResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RecomputeStatsResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_RecomputeStatsResponse.Merge(dst, src) +} +func (m *RecomputeStatsResponse) XXX_Size() int { + return m.Size() +} +func (m *RecomputeStatsResponse) XXX_DiscardUnknown() { + xxx_messageInfo_RecomputeStatsResponse.DiscardUnknown(m) } -func (m *RecomputeStatsResponse) Reset() { *m = RecomputeStatsResponse{} } -func (m *RecomputeStatsResponse) String() string { return proto.CompactTextString(m) } -func (*RecomputeStatsResponse) ProtoMessage() {} -func (*RecomputeStatsResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{27} } +var xxx_messageInfo_RecomputeStatsResponse proto.InternalMessageInfo // A BeginTransactionRequest is the argument to the BeginTransaction() method. type BeginTransactionRequest struct { - RequestHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{28} +} +func (m *BeginTransactionRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *BeginTransactionRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *BeginTransactionRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_BeginTransactionRequest.Merge(dst, src) +} +func (m *BeginTransactionRequest) XXX_Size() int { + return m.Size() +} +func (m *BeginTransactionRequest) XXX_DiscardUnknown() { + xxx_messageInfo_BeginTransactionRequest.DiscardUnknown(m) } -func (m *BeginTransactionRequest) Reset() { *m = BeginTransactionRequest{} } -func (m *BeginTransactionRequest) String() string { return proto.CompactTextString(m) } -func (*BeginTransactionRequest) ProtoMessage() {} -func (*BeginTransactionRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{28} } +var xxx_messageInfo_BeginTransactionRequest proto.InternalMessageInfo // A BeginTransactionResponse is the return value from the BeginTransaction() method. type BeginTransactionResponse struct { - ResponseHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *BeginTransactionResponse) Reset() { *m = BeginTransactionResponse{} } +func (m *BeginTransactionResponse) String() string { return proto.CompactTextString(m) } +func (*BeginTransactionResponse) ProtoMessage() {} +func (*BeginTransactionResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_api_a51b49ba855d15ec, []int{29} +} +func (m *BeginTransactionResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *BeginTransactionResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *BeginTransactionResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_BeginTransactionResponse.Merge(dst, src) +} +func (m *BeginTransactionResponse) XXX_Size() int { + return m.Size() +} +func (m *BeginTransactionResponse) XXX_DiscardUnknown() { + xxx_messageInfo_BeginTransactionResponse.DiscardUnknown(m) } -func (m *BeginTransactionResponse) Reset() { *m = BeginTransactionResponse{} } -func (m *BeginTransactionResponse) String() string { return proto.CompactTextString(m) } -func (*BeginTransactionResponse) ProtoMessage() {} -func (*BeginTransactionResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{29} } +var xxx_messageInfo_BeginTransactionResponse proto.InternalMessageInfo // An EndTransactionRequest is the argument to the EndTransaction() method. It // specifies whether to commit or roll back an extant transaction. type EndTransactionRequest struct { - RequestHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` // False to abort and rollback. Commit bool `protobuf:"varint,2,opt,name=commit,proto3" json:"commit,omitempty"` // If set, deadline represents the maximum timestamp at which the transaction @@ -971,17 +1571,17 @@ type EndTransactionRequest struct { // trouble with the deadline check. A Serializable txn that has had its // timestamp pushed has already lost before the deadline check: it will be // forced to restart. - Deadline *cockroach_util_hlc.Timestamp `protobuf:"bytes,3,opt,name=deadline" json:"deadline,omitempty"` + Deadline *hlc.Timestamp `protobuf:"bytes,3,opt,name=deadline,proto3" json:"deadline,omitempty"` // commit triggers. Note that commit triggers are for // internal use only and will cause an error if requested through the // external-facing KV API. - InternalCommitTrigger *InternalCommitTrigger `protobuf:"bytes,4,opt,name=internal_commit_trigger,json=internalCommitTrigger" json:"internal_commit_trigger,omitempty"` + InternalCommitTrigger *InternalCommitTrigger `protobuf:"bytes,4,opt,name=internal_commit_trigger,json=internalCommitTrigger,proto3" json:"internal_commit_trigger,omitempty"` // List of intents written by the transaction. These are spans where // write intents have been written which must be resolved on txn // completion. Note that these spans may be condensed to cover // aggregate spans if the keys written by the transaction exceeded // a size threshold. - IntentSpans []Span `protobuf:"bytes,5,rep,name=intent_spans,json=intentSpans" json:"intent_spans"` + IntentSpans []Span `protobuf:"bytes,5,rep,name=intent_spans,json=intentSpans,proto3" json:"intent_spans"` // Requires that the transaction completes as a 1 phase commit. This // guarantees that all writes are to the same range and that no // intents are left in the event of an error. @@ -995,13 +1595,39 @@ type EndTransactionRequest struct { // independently of the main thread of client operation, as in the // case of an asynchronous abort from the TxnCoordSender on a failed // heartbeat. - Poison bool `protobuf:"varint,9,opt,name=poison,proto3" json:"poison,omitempty"` + Poison bool `protobuf:"varint,9,opt,name=poison,proto3" json:"poison,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{30} +} +func (m *EndTransactionRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *EndTransactionRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *EndTransactionRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_EndTransactionRequest.Merge(dst, src) +} +func (m *EndTransactionRequest) XXX_Size() int { + return m.Size() +} +func (m *EndTransactionRequest) XXX_DiscardUnknown() { + xxx_messageInfo_EndTransactionRequest.DiscardUnknown(m) } -func (m *EndTransactionRequest) Reset() { *m = EndTransactionRequest{} } -func (m *EndTransactionRequest) String() string { return proto.CompactTextString(m) } -func (*EndTransactionRequest) ProtoMessage() {} -func (*EndTransactionRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{30} } +var xxx_messageInfo_EndTransactionRequest proto.InternalMessageInfo // An EndTransactionResponse is the return value from the // EndTransaction() method. The final transaction record is returned @@ -1011,17 +1637,43 @@ func (*EndTransactionRequest) Descriptor() ([]byte, []int) { return fileDescript // final txn commit timestamp in order to preserve causal ordering // between subsequent transactions. type EndTransactionResponse struct { - ResponseHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` // True if the transaction committed on the one phase commit path. // This means that all writes which were part of the transaction // were written as a single, atomic write batch to just one range. - OnePhaseCommit bool `protobuf:"varint,4,opt,name=one_phase_commit,json=onePhaseCommit,proto3" json:"one_phase_commit,omitempty"` + OnePhaseCommit bool `protobuf:"varint,4,opt,name=one_phase_commit,json=onePhaseCommit,proto3" json:"one_phase_commit,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *EndTransactionResponse) Reset() { *m = EndTransactionResponse{} } -func (m *EndTransactionResponse) String() string { return proto.CompactTextString(m) } -func (*EndTransactionResponse) ProtoMessage() {} -func (*EndTransactionResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{31} } +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_a51b49ba855d15ec, []int{31} +} +func (m *EndTransactionResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *EndTransactionResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *EndTransactionResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_EndTransactionResponse.Merge(dst, src) +} +func (m *EndTransactionResponse) XXX_Size() int { + return m.Size() +} +func (m *EndTransactionResponse) XXX_DiscardUnknown() { + xxx_messageInfo_EndTransactionResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_EndTransactionResponse proto.InternalMessageInfo // An AdminSplitRequest is the argument to the AdminSplit() method. The // existing range which contains header.key is split by @@ -1044,25 +1696,77 @@ func (*EndTransactionResponse) Descriptor() ([]byte, []int) { return fileDescrip // metadata (e.g. abort span and range stats must be copied or // recomputed). type AdminSplitRequest struct { - RequestHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` - SplitKey Key `protobuf:"bytes,2,opt,name=split_key,json=splitKey,proto3,casttype=Key" json:"split_key,omitempty"` + RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + SplitKey Key `protobuf:"bytes,2,opt,name=split_key,json=splitKey,proto3,casttype=Key" json:"split_key,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{32} +} +func (m *AdminSplitRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *AdminSplitRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *AdminSplitRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_AdminSplitRequest.Merge(dst, src) +} +func (m *AdminSplitRequest) XXX_Size() int { + return m.Size() +} +func (m *AdminSplitRequest) XXX_DiscardUnknown() { + xxx_messageInfo_AdminSplitRequest.DiscardUnknown(m) } -func (m *AdminSplitRequest) Reset() { *m = AdminSplitRequest{} } -func (m *AdminSplitRequest) String() string { return proto.CompactTextString(m) } -func (*AdminSplitRequest) ProtoMessage() {} -func (*AdminSplitRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{32} } +var xxx_messageInfo_AdminSplitRequest proto.InternalMessageInfo // An AdminSplitResponse is the return value from the AdminSplit() // method. type AdminSplitResponse struct { - ResponseHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *AdminSplitResponse) Reset() { *m = AdminSplitResponse{} } -func (m *AdminSplitResponse) String() string { return proto.CompactTextString(m) } -func (*AdminSplitResponse) ProtoMessage() {} -func (*AdminSplitResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{33} } +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_a51b49ba855d15ec, []int{33} +} +func (m *AdminSplitResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *AdminSplitResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *AdminSplitResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_AdminSplitResponse.Merge(dst, src) +} +func (m *AdminSplitResponse) XXX_Size() int { + return m.Size() +} +func (m *AdminSplitResponse) XXX_DiscardUnknown() { + xxx_messageInfo_AdminSplitResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_AdminSplitResponse proto.InternalMessageInfo // An AdminMergeRequest is the argument to the AdminMerge() method. A // merge is performed by calling AdminMerge on the left-hand range of @@ -1074,92 +1778,300 @@ func (*AdminSplitResponse) Descriptor() ([]byte, []int) { return fileDescriptorA // of the subsumed range. If AdminMerge is called on the final range // in the key space, it is a noop. type AdminMergeRequest struct { - RequestHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *AdminMergeRequest) Reset() { *m = AdminMergeRequest{} } -func (m *AdminMergeRequest) String() string { return proto.CompactTextString(m) } -func (*AdminMergeRequest) ProtoMessage() {} -func (*AdminMergeRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{34} } +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_a51b49ba855d15ec, []int{34} +} +func (m *AdminMergeRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *AdminMergeRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *AdminMergeRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_AdminMergeRequest.Merge(dst, src) +} +func (m *AdminMergeRequest) XXX_Size() int { + return m.Size() +} +func (m *AdminMergeRequest) XXX_DiscardUnknown() { + xxx_messageInfo_AdminMergeRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_AdminMergeRequest proto.InternalMessageInfo // An AdminMergeResponse is the return value from the AdminMerge() // method. type AdminMergeResponse struct { - ResponseHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{35} +} +func (m *AdminMergeResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *AdminMergeResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *AdminMergeResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_AdminMergeResponse.Merge(dst, src) +} +func (m *AdminMergeResponse) XXX_Size() int { + return m.Size() +} +func (m *AdminMergeResponse) XXX_DiscardUnknown() { + xxx_messageInfo_AdminMergeResponse.DiscardUnknown(m) } -func (m *AdminMergeResponse) Reset() { *m = AdminMergeResponse{} } -func (m *AdminMergeResponse) String() string { return proto.CompactTextString(m) } -func (*AdminMergeResponse) ProtoMessage() {} -func (*AdminMergeResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{35} } +var xxx_messageInfo_AdminMergeResponse proto.InternalMessageInfo // An AdminTransferLeaseRequest is the argument to the AdminTransferLease() // method. A lease transfer allows an external entity to control the lease // holder for a range. The target of the lease transfer needs to be a valid // replica of the range. type AdminTransferLeaseRequest struct { - RequestHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` - Target StoreID `protobuf:"varint,2,opt,name=target,proto3,casttype=StoreID" json:"target,omitempty"` + RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + Target StoreID `protobuf:"varint,2,opt,name=target,proto3,casttype=StoreID" json:"target,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *AdminTransferLeaseRequest) Reset() { *m = AdminTransferLeaseRequest{} } -func (m *AdminTransferLeaseRequest) String() string { return proto.CompactTextString(m) } -func (*AdminTransferLeaseRequest) ProtoMessage() {} -func (*AdminTransferLeaseRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{36} } +func (m *AdminTransferLeaseRequest) Reset() { *m = AdminTransferLeaseRequest{} } +func (m *AdminTransferLeaseRequest) String() string { return proto.CompactTextString(m) } +func (*AdminTransferLeaseRequest) ProtoMessage() {} +func (*AdminTransferLeaseRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_api_a51b49ba855d15ec, []int{36} +} +func (m *AdminTransferLeaseRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *AdminTransferLeaseRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *AdminTransferLeaseRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_AdminTransferLeaseRequest.Merge(dst, src) +} +func (m *AdminTransferLeaseRequest) XXX_Size() int { + return m.Size() +} +func (m *AdminTransferLeaseRequest) XXX_DiscardUnknown() { + xxx_messageInfo_AdminTransferLeaseRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_AdminTransferLeaseRequest proto.InternalMessageInfo type AdminTransferLeaseResponse struct { - ResponseHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *AdminTransferLeaseResponse) Reset() { *m = AdminTransferLeaseResponse{} } -func (m *AdminTransferLeaseResponse) String() string { return proto.CompactTextString(m) } -func (*AdminTransferLeaseResponse) ProtoMessage() {} -func (*AdminTransferLeaseResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{37} } +func (m *AdminTransferLeaseResponse) Reset() { *m = AdminTransferLeaseResponse{} } +func (m *AdminTransferLeaseResponse) String() string { return proto.CompactTextString(m) } +func (*AdminTransferLeaseResponse) ProtoMessage() {} +func (*AdminTransferLeaseResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_api_a51b49ba855d15ec, []int{37} +} +func (m *AdminTransferLeaseResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *AdminTransferLeaseResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *AdminTransferLeaseResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_AdminTransferLeaseResponse.Merge(dst, src) +} +func (m *AdminTransferLeaseResponse) XXX_Size() int { + return m.Size() +} +func (m *AdminTransferLeaseResponse) XXX_DiscardUnknown() { + xxx_messageInfo_AdminTransferLeaseResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_AdminTransferLeaseResponse proto.InternalMessageInfo // An AdminChangeReplicasRequest is the argument to the AdminChangeReplicas() // method. A change replicas operation allows adding or removing a set of // replicas for a range. type AdminChangeReplicasRequest struct { - RequestHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` - ChangeType ReplicaChangeType `protobuf:"varint,2,opt,name=change_type,json=changeType,proto3,enum=cockroach.roachpb.ReplicaChangeType" json:"change_type,omitempty"` - Targets []ReplicationTarget `protobuf:"bytes,3,rep,name=targets" json:"targets"` + RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + ChangeType ReplicaChangeType `protobuf:"varint,2,opt,name=change_type,json=changeType,proto3,enum=cockroach.roachpb.ReplicaChangeType" json:"change_type,omitempty"` + Targets []ReplicationTarget `protobuf:"bytes,3,rep,name=targets,proto3" json:"targets"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *AdminChangeReplicasRequest) Reset() { *m = AdminChangeReplicasRequest{} } +func (m *AdminChangeReplicasRequest) String() string { return proto.CompactTextString(m) } +func (*AdminChangeReplicasRequest) ProtoMessage() {} +func (*AdminChangeReplicasRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_api_a51b49ba855d15ec, []int{38} +} +func (m *AdminChangeReplicasRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *AdminChangeReplicasRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *AdminChangeReplicasRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_AdminChangeReplicasRequest.Merge(dst, src) +} +func (m *AdminChangeReplicasRequest) XXX_Size() int { + return m.Size() +} +func (m *AdminChangeReplicasRequest) XXX_DiscardUnknown() { + xxx_messageInfo_AdminChangeReplicasRequest.DiscardUnknown(m) } -func (m *AdminChangeReplicasRequest) Reset() { *m = AdminChangeReplicasRequest{} } -func (m *AdminChangeReplicasRequest) String() string { return proto.CompactTextString(m) } -func (*AdminChangeReplicasRequest) ProtoMessage() {} -func (*AdminChangeReplicasRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{38} } +var xxx_messageInfo_AdminChangeReplicasRequest proto.InternalMessageInfo type AdminChangeReplicasResponse struct { - ResponseHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *AdminChangeReplicasResponse) Reset() { *m = AdminChangeReplicasResponse{} } +func (m *AdminChangeReplicasResponse) String() string { return proto.CompactTextString(m) } +func (*AdminChangeReplicasResponse) ProtoMessage() {} +func (*AdminChangeReplicasResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_api_a51b49ba855d15ec, []int{39} +} +func (m *AdminChangeReplicasResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *AdminChangeReplicasResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *AdminChangeReplicasResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_AdminChangeReplicasResponse.Merge(dst, src) +} +func (m *AdminChangeReplicasResponse) XXX_Size() int { + return m.Size() +} +func (m *AdminChangeReplicasResponse) XXX_DiscardUnknown() { + xxx_messageInfo_AdminChangeReplicasResponse.DiscardUnknown(m) } -func (m *AdminChangeReplicasResponse) Reset() { *m = AdminChangeReplicasResponse{} } -func (m *AdminChangeReplicasResponse) String() string { return proto.CompactTextString(m) } -func (*AdminChangeReplicasResponse) ProtoMessage() {} -func (*AdminChangeReplicasResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{39} } +var xxx_messageInfo_AdminChangeReplicasResponse proto.InternalMessageInfo // An AdminRelocateRangeRequest is the argument to the AdminRelocateRange() // method. Relocates the replicas for a range to the specified target stores. // The first store in the list of targets becomes the new leaseholder. type AdminRelocateRangeRequest struct { - RequestHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` - Targets []ReplicationTarget `protobuf:"bytes,2,rep,name=targets" json:"targets"` + RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + Targets []ReplicationTarget `protobuf:"bytes,2,rep,name=targets,proto3" json:"targets"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *AdminRelocateRangeRequest) Reset() { *m = AdminRelocateRangeRequest{} } -func (m *AdminRelocateRangeRequest) String() string { return proto.CompactTextString(m) } -func (*AdminRelocateRangeRequest) ProtoMessage() {} -func (*AdminRelocateRangeRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{40} } +func (m *AdminRelocateRangeRequest) Reset() { *m = AdminRelocateRangeRequest{} } +func (m *AdminRelocateRangeRequest) String() string { return proto.CompactTextString(m) } +func (*AdminRelocateRangeRequest) ProtoMessage() {} +func (*AdminRelocateRangeRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_api_a51b49ba855d15ec, []int{40} +} +func (m *AdminRelocateRangeRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *AdminRelocateRangeRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *AdminRelocateRangeRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_AdminRelocateRangeRequest.Merge(dst, src) +} +func (m *AdminRelocateRangeRequest) XXX_Size() int { + return m.Size() +} +func (m *AdminRelocateRangeRequest) XXX_DiscardUnknown() { + xxx_messageInfo_AdminRelocateRangeRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_AdminRelocateRangeRequest proto.InternalMessageInfo type AdminRelocateRangeResponse struct { - ResponseHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *AdminRelocateRangeResponse) Reset() { *m = AdminRelocateRangeResponse{} } +func (m *AdminRelocateRangeResponse) String() string { return proto.CompactTextString(m) } +func (*AdminRelocateRangeResponse) ProtoMessage() {} +func (*AdminRelocateRangeResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_api_a51b49ba855d15ec, []int{41} +} +func (m *AdminRelocateRangeResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *AdminRelocateRangeResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *AdminRelocateRangeResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_AdminRelocateRangeResponse.Merge(dst, src) +} +func (m *AdminRelocateRangeResponse) XXX_Size() int { + return m.Size() +} +func (m *AdminRelocateRangeResponse) XXX_DiscardUnknown() { + xxx_messageInfo_AdminRelocateRangeResponse.DiscardUnknown(m) } -func (m *AdminRelocateRangeResponse) Reset() { *m = AdminRelocateRangeResponse{} } -func (m *AdminRelocateRangeResponse) String() string { return proto.CompactTextString(m) } -func (*AdminRelocateRangeResponse) ProtoMessage() {} -func (*AdminRelocateRangeResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{41} } +var xxx_messageInfo_AdminRelocateRangeResponse proto.InternalMessageInfo // A HeartbeatTxnRequest is arguments to the HeartbeatTxn() // method. It's sent by transaction coordinators to let the system @@ -1167,64 +2079,194 @@ func (*AdminRelocateRangeResponse) Descriptor() ([]byte, []int) { return fileDes // heartbeat message is different from the heartbeat message in the // gossip protocol. type HeartbeatTxnRequest struct { - RequestHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` - Now cockroach_util_hlc.Timestamp `protobuf:"bytes,2,opt,name=now" json:"now"` + RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + Now hlc.Timestamp `protobuf:"bytes,2,opt,name=now,proto3" json:"now"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *HeartbeatTxnRequest) Reset() { *m = HeartbeatTxnRequest{} } -func (m *HeartbeatTxnRequest) String() string { return proto.CompactTextString(m) } -func (*HeartbeatTxnRequest) ProtoMessage() {} -func (*HeartbeatTxnRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{42} } +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_a51b49ba855d15ec, []int{42} +} +func (m *HeartbeatTxnRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *HeartbeatTxnRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *HeartbeatTxnRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_HeartbeatTxnRequest.Merge(dst, src) +} +func (m *HeartbeatTxnRequest) XXX_Size() int { + return m.Size() +} +func (m *HeartbeatTxnRequest) XXX_DiscardUnknown() { + xxx_messageInfo_HeartbeatTxnRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_HeartbeatTxnRequest proto.InternalMessageInfo // A HeartbeatTxnResponse is the return value from the HeartbeatTxn() // method. It returns the transaction info in the response header. The // returned transaction lets the coordinator know the disposition of // the transaction (i.e. aborted, committed, or pending). type HeartbeatTxnResponse struct { - ResponseHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{43} +} +func (m *HeartbeatTxnResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *HeartbeatTxnResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *HeartbeatTxnResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_HeartbeatTxnResponse.Merge(dst, src) +} +func (m *HeartbeatTxnResponse) XXX_Size() int { + return m.Size() +} +func (m *HeartbeatTxnResponse) XXX_DiscardUnknown() { + xxx_messageInfo_HeartbeatTxnResponse.DiscardUnknown(m) } -func (m *HeartbeatTxnResponse) Reset() { *m = HeartbeatTxnResponse{} } -func (m *HeartbeatTxnResponse) String() string { return proto.CompactTextString(m) } -func (*HeartbeatTxnResponse) ProtoMessage() {} -func (*HeartbeatTxnResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{43} } +var xxx_messageInfo_HeartbeatTxnResponse proto.InternalMessageInfo // A GCRequest is arguments to the GC() method. It's sent by range // lease holders after scanning range data to find expired MVCC values. type GCRequest struct { - RequestHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` - Keys []GCRequest_GCKey `protobuf:"bytes,3,rep,name=keys" json:"keys"` + RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + Keys []GCRequest_GCKey `protobuf:"bytes,3,rep,name=keys,proto3" json:"keys"` // Threshold is the expiration timestamp. - Threshold cockroach_util_hlc.Timestamp `protobuf:"bytes,4,opt,name=threshold" json:"threshold"` + Threshold hlc.Timestamp `protobuf:"bytes,4,opt,name=threshold,proto3" json:"threshold"` // TxnSpanGCThreshold is the timestamp below which inactive transactions were // considered for GC (and thus might have been removed). - TxnSpanGCThreshold cockroach_util_hlc.Timestamp `protobuf:"bytes,5,opt,name=txn_span_gc_threshold,json=txnSpanGcThreshold" json:"txn_span_gc_threshold"` + TxnSpanGCThreshold hlc.Timestamp `protobuf:"bytes,5,opt,name=txn_span_gc_threshold,json=txnSpanGcThreshold,proto3" json:"txn_span_gc_threshold"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *GCRequest) Reset() { *m = GCRequest{} } -func (m *GCRequest) String() string { return proto.CompactTextString(m) } -func (*GCRequest) ProtoMessage() {} -func (*GCRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{44} } +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_a51b49ba855d15ec, []int{44} +} +func (m *GCRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *GCRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *GCRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_GCRequest.Merge(dst, src) +} +func (m *GCRequest) XXX_Size() int { + return m.Size() +} +func (m *GCRequest) XXX_DiscardUnknown() { + xxx_messageInfo_GCRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_GCRequest proto.InternalMessageInfo type GCRequest_GCKey struct { - Key Key `protobuf:"bytes,1,opt,name=key,proto3,casttype=Key" json:"key,omitempty"` - Timestamp cockroach_util_hlc.Timestamp `protobuf:"bytes,2,opt,name=timestamp" json:"timestamp"` + Key Key `protobuf:"bytes,1,opt,name=key,proto3,casttype=Key" json:"key,omitempty"` + Timestamp hlc.Timestamp `protobuf:"bytes,2,opt,name=timestamp,proto3" json:"timestamp"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{44, 0} +} +func (m *GCRequest_GCKey) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *GCRequest_GCKey) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *GCRequest_GCKey) XXX_Merge(src proto.Message) { + xxx_messageInfo_GCRequest_GCKey.Merge(dst, src) +} +func (m *GCRequest_GCKey) XXX_Size() int { + return m.Size() +} +func (m *GCRequest_GCKey) XXX_DiscardUnknown() { + xxx_messageInfo_GCRequest_GCKey.DiscardUnknown(m) } -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 fileDescriptorApi, []int{44, 0} } +var xxx_messageInfo_GCRequest_GCKey proto.InternalMessageInfo // A GCResponse is the return value from the GC() method. type GCResponse struct { - ResponseHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{45} +} +func (m *GCResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *GCResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *GCResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_GCResponse.Merge(dst, src) +} +func (m *GCResponse) XXX_Size() int { + return m.Size() +} +func (m *GCResponse) XXX_DiscardUnknown() { + xxx_messageInfo_GCResponse.DiscardUnknown(m) } -func (m *GCResponse) Reset() { *m = GCResponse{} } -func (m *GCResponse) String() string { return proto.CompactTextString(m) } -func (*GCResponse) ProtoMessage() {} -func (*GCResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{45} } +var xxx_messageInfo_GCResponse proto.InternalMessageInfo // A PushTxnRequest is arguments to the PushTxn() method. It's sent by // readers or writers which have encountered an "intent" laid down by @@ -1240,26 +2282,26 @@ func (*GCResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []in // course of action is determined by the specified push type, and by // the owning txn's status and priority. type PushTxnRequest struct { - RequestHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` // Transaction which encountered the intent, if applicable. For a // non-transactional pusher, pusher_txn will only have the priority set (in // particular, ID won't be set). Used to compare priorities and timestamps if // priorities are equal. - PusherTxn Transaction `protobuf:"bytes,2,opt,name=pusher_txn,json=pusherTxn" json:"pusher_txn"` + PusherTxn Transaction `protobuf:"bytes,2,opt,name=pusher_txn,json=pusherTxn,proto3" json:"pusher_txn"` // Transaction to be pushed, as specified at the intent which led to // the push transaction request. Note that this may not be the most // up-to-date value of the transaction record, but will be set or // merged as appropriate. - PusheeTxn cockroach_storage_engine_enginepb.TxnMeta `protobuf:"bytes,3,opt,name=pushee_txn,json=pusheeTxn" json:"pushee_txn"` + PusheeTxn enginepb.TxnMeta `protobuf:"bytes,3,opt,name=pushee_txn,json=pusheeTxn,proto3" json:"pushee_txn"` // PushTo is the timestamp just after which PusheeTxn is attempted to be // pushed. During conflict resolution, it should be set to the timestamp // of the its conflicting write. - PushTo cockroach_util_hlc.Timestamp `protobuf:"bytes,4,opt,name=push_to,json=pushTo" json:"push_to"` + PushTo hlc.Timestamp `protobuf:"bytes,4,opt,name=push_to,json=pushTo,proto3" json:"push_to"` // Now holds the timestamp used to compare the last heartbeat of the pushee // against. This is necessary since the request header's timestamp does not // necessarily advance with the node clock across retries and hence cannot // detect abandoned transactions. - Now cockroach_util_hlc.Timestamp `protobuf:"bytes,5,opt,name=now" json:"now"` + Now hlc.Timestamp `protobuf:"bytes,5,opt,name=now,proto3" json:"now"` // Readers set this to PUSH_TIMESTAMP to move pushee_txn's provisional // commit timestamp forward. Writers set this to PUSH_ABORT to request // that pushee_txn be aborted if possible. Inconsistent readers set @@ -1268,13 +2310,39 @@ type PushTxnRequest struct { PushType PushTxnType `protobuf:"varint,6,opt,name=push_type,json=pushType,proto3,enum=cockroach.roachpb.PushTxnType" json:"push_type,omitempty"` // Forces the push by overriding the normal checks in PushTxn to // either abort or push the timestamp. - Force bool `protobuf:"varint,7,opt,name=force,proto3" json:"force,omitempty"` + Force bool `protobuf:"varint,7,opt,name=force,proto3" json:"force,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *PushTxnRequest) Reset() { *m = PushTxnRequest{} } -func (m *PushTxnRequest) String() string { return proto.CompactTextString(m) } -func (*PushTxnRequest) ProtoMessage() {} -func (*PushTxnRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{46} } +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_a51b49ba855d15ec, []int{46} +} +func (m *PushTxnRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *PushTxnRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *PushTxnRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_PushTxnRequest.Merge(dst, src) +} +func (m *PushTxnRequest) XXX_Size() int { + return m.Size() +} +func (m *PushTxnRequest) XXX_DiscardUnknown() { + xxx_messageInfo_PushTxnRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_PushTxnRequest proto.InternalMessageInfo // A PushTxnResponse is the return value from the PushTxn() method. It // returns success and the resulting state of PusheeTxn if the @@ -1282,59 +2350,137 @@ func (*PushTxnRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, // subsequently invoke ResolveIntent() on the conflicted key. It // returns an error otherwise. type PushTxnResponse struct { - ResponseHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` // pushee_txn is non-nil if the transaction was pushed and contains // the current value of the transaction. // TODO(tschottdorf): Maybe this can be a TxnMeta instead; probably requires // factoring out the new Priority. - PusheeTxn Transaction `protobuf:"bytes,2,opt,name=pushee_txn,json=pusheeTxn" json:"pushee_txn"` + PusheeTxn Transaction `protobuf:"bytes,2,opt,name=pushee_txn,json=pusheeTxn,proto3" json:"pushee_txn"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *PushTxnResponse) Reset() { *m = PushTxnResponse{} } -func (m *PushTxnResponse) String() string { return proto.CompactTextString(m) } -func (*PushTxnResponse) ProtoMessage() {} -func (*PushTxnResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{47} } +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_a51b49ba855d15ec, []int{47} +} +func (m *PushTxnResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *PushTxnResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *PushTxnResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_PushTxnResponse.Merge(dst, src) +} +func (m *PushTxnResponse) XXX_Size() int { + return m.Size() +} +func (m *PushTxnResponse) XXX_DiscardUnknown() { + xxx_messageInfo_PushTxnResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_PushTxnResponse proto.InternalMessageInfo // A QueryTxnResponse is arguments to the QueryTxn() method. It's sent // by transactions which are waiting to push another transaction because // of conflicting write intents to fetch updates to either the pusher's // or the pushee's transaction records. type QueryTxnRequest struct { - RequestHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` // Transaction record to query. - Txn cockroach_storage_engine_enginepb.TxnMeta `protobuf:"bytes,2,opt,name=txn" json:"txn"` + Txn enginepb.TxnMeta `protobuf:"bytes,2,opt,name=txn,proto3" json:"txn"` // If true, the query will not return until there are changes to either the // transaction status or priority -OR- to the set of dependent transactions. WaitForUpdate bool `protobuf:"varint,3,opt,name=wait_for_update,json=waitForUpdate,proto3" json:"wait_for_update,omitempty"` // Set of known dependent transactions. - KnownWaitingTxns []github_com_cockroachdb_cockroach_pkg_util_uuid.UUID `protobuf:"bytes,4,rep,name=known_waiting_txns,json=knownWaitingTxns,customtype=github.com/cockroachdb/cockroach/pkg/util/uuid.UUID" json:"known_waiting_txns,omitempty"` + KnownWaitingTxns []github_com_cockroachdb_cockroach_pkg_util_uuid.UUID `protobuf:"bytes,4,rep,name=known_waiting_txns,json=knownWaitingTxns,proto3,customtype=github.com/cockroachdb/cockroach/pkg/util/uuid.UUID" json:"known_waiting_txns,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *QueryTxnRequest) Reset() { *m = QueryTxnRequest{} } -func (m *QueryTxnRequest) String() string { return proto.CompactTextString(m) } -func (*QueryTxnRequest) ProtoMessage() {} -func (*QueryTxnRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{48} } +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_a51b49ba855d15ec, []int{48} +} +func (m *QueryTxnRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *QueryTxnRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *QueryTxnRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_QueryTxnRequest.Merge(dst, src) +} +func (m *QueryTxnRequest) XXX_Size() int { + return m.Size() +} +func (m *QueryTxnRequest) XXX_DiscardUnknown() { + xxx_messageInfo_QueryTxnRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_QueryTxnRequest proto.InternalMessageInfo // A QueryTxnResponse is the return value from the QueryTxn() method. type QueryTxnResponse struct { - ResponseHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` // Contains the current state of the queried transaction. If the queried // transaction record does not exist, this will be empty. - QueriedTxn Transaction `protobuf:"bytes,2,opt,name=queried_txn,json=queriedTxn" json:"queried_txn"` + QueriedTxn Transaction `protobuf:"bytes,2,opt,name=queried_txn,json=queriedTxn,proto3" json:"queried_txn"` // Specifies a list of transaction IDs which are waiting on the txn. - WaitingTxns []github_com_cockroachdb_cockroach_pkg_util_uuid.UUID `protobuf:"bytes,3,rep,name=waiting_txns,json=waitingTxns,customtype=github.com/cockroachdb/cockroach/pkg/util/uuid.UUID" json:"waiting_txns,omitempty"` + WaitingTxns []github_com_cockroachdb_cockroach_pkg_util_uuid.UUID `protobuf:"bytes,3,rep,name=waiting_txns,json=waitingTxns,proto3,customtype=github.com/cockroachdb/cockroach/pkg/util/uuid.UUID" json:"waiting_txns,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *QueryTxnResponse) Reset() { *m = QueryTxnResponse{} } -func (m *QueryTxnResponse) String() string { return proto.CompactTextString(m) } -func (*QueryTxnResponse) ProtoMessage() {} -func (*QueryTxnResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{49} } +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_a51b49ba855d15ec, []int{49} +} +func (m *QueryTxnResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *QueryTxnResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *QueryTxnResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_QueryTxnResponse.Merge(dst, src) +} +func (m *QueryTxnResponse) XXX_Size() int { + return m.Size() +} +func (m *QueryTxnResponse) XXX_DiscardUnknown() { + xxx_messageInfo_QueryTxnResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_QueryTxnResponse proto.InternalMessageInfo // A QueryIntentRequest is arguments to the QueryIntent() method. It visits // the specified key and checks whether an intent is present for the given // transaction. type QueryIntentRequest struct { - RequestHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` // The TxnMeta that the intent is expected to have. Specifically, whether an // intent is a match or not is defined as whether an intent exists that could // be committed by the provided transaction. If an intent is found at the @@ -1355,68 +2501,172 @@ type QueryIntentRequest struct { // successfully written an intent at a larger sequence number then it must // have succeeeded in writing an intent at the smaller sequence number as // well. - Txn cockroach_storage_engine_enginepb.TxnMeta `protobuf:"bytes,2,opt,name=txn" json:"txn"` + Txn enginepb.TxnMeta `protobuf:"bytes,2,opt,name=txn,proto3" json:"txn"` // The behavior of the request if the expected intent is found to be not // present. - IfMissing QueryIntentRequest_IfMissingBehavior `protobuf:"varint,3,opt,name=if_missing,json=ifMissing,proto3,enum=cockroach.roachpb.QueryIntentRequest_IfMissingBehavior" json:"if_missing,omitempty"` + IfMissing QueryIntentRequest_IfMissingBehavior `protobuf:"varint,3,opt,name=if_missing,json=ifMissing,proto3,enum=cockroach.roachpb.QueryIntentRequest_IfMissingBehavior" json:"if_missing,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{50} +} +func (m *QueryIntentRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *QueryIntentRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *QueryIntentRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_QueryIntentRequest.Merge(dst, src) +} +func (m *QueryIntentRequest) XXX_Size() int { + return m.Size() +} +func (m *QueryIntentRequest) XXX_DiscardUnknown() { + xxx_messageInfo_QueryIntentRequest.DiscardUnknown(m) } -func (m *QueryIntentRequest) Reset() { *m = QueryIntentRequest{} } -func (m *QueryIntentRequest) String() string { return proto.CompactTextString(m) } -func (*QueryIntentRequest) ProtoMessage() {} -func (*QueryIntentRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{50} } +var xxx_messageInfo_QueryIntentRequest proto.InternalMessageInfo // A QueryIntentResponse is the return value from the QueryIntent() method. type QueryIntentResponse struct { - ResponseHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` // Whether an intent matching the expected transaction was found at the key. - FoundIntent bool `protobuf:"varint,2,opt,name=found_intent,json=foundIntent,proto3" json:"found_intent,omitempty"` + FoundIntent bool `protobuf:"varint,2,opt,name=found_intent,json=foundIntent,proto3" json:"found_intent,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{51} +} +func (m *QueryIntentResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *QueryIntentResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *QueryIntentResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_QueryIntentResponse.Merge(dst, src) +} +func (m *QueryIntentResponse) XXX_Size() int { + return m.Size() +} +func (m *QueryIntentResponse) XXX_DiscardUnknown() { + xxx_messageInfo_QueryIntentResponse.DiscardUnknown(m) } -func (m *QueryIntentResponse) Reset() { *m = QueryIntentResponse{} } -func (m *QueryIntentResponse) String() string { return proto.CompactTextString(m) } -func (*QueryIntentResponse) ProtoMessage() {} -func (*QueryIntentResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{51} } +var xxx_messageInfo_QueryIntentResponse proto.InternalMessageInfo // A ResolveIntentRequest is arguments to the ResolveIntent() // method. It is sent by transaction coordinators after success // calling PushTxn to clean up write intents: either to remove, commit // or move them forward in time. type ResolveIntentRequest struct { - RequestHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` // The transaction whose intent is being resolved. - IntentTxn cockroach_storage_engine_enginepb.TxnMeta `protobuf:"bytes,2,opt,name=intent_txn,json=intentTxn" json:"intent_txn"` + IntentTxn enginepb.TxnMeta `protobuf:"bytes,2,opt,name=intent_txn,json=intentTxn,proto3" json:"intent_txn"` // The status of the transaction. Status TransactionStatus `protobuf:"varint,3,opt,name=status,proto3,enum=cockroach.roachpb.TransactionStatus" json:"status,omitempty"` // Optionally poison the abort span for the transaction the intent's // range. - Poison bool `protobuf:"varint,4,opt,name=poison,proto3" json:"poison,omitempty"` + Poison bool `protobuf:"varint,4,opt,name=poison,proto3" json:"poison,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *ResolveIntentRequest) Reset() { *m = ResolveIntentRequest{} } -func (m *ResolveIntentRequest) String() string { return proto.CompactTextString(m) } -func (*ResolveIntentRequest) ProtoMessage() {} -func (*ResolveIntentRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{52} } +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_a51b49ba855d15ec, []int{52} +} +func (m *ResolveIntentRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ResolveIntentRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ResolveIntentRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_ResolveIntentRequest.Merge(dst, src) +} +func (m *ResolveIntentRequest) XXX_Size() int { + return m.Size() +} +func (m *ResolveIntentRequest) XXX_DiscardUnknown() { + xxx_messageInfo_ResolveIntentRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_ResolveIntentRequest proto.InternalMessageInfo // A ResolveIntentResponse is the return value from the // ResolveIntent() method. type ResolveIntentResponse struct { - ResponseHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{53} +} +func (m *ResolveIntentResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ResolveIntentResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ResolveIntentResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_ResolveIntentResponse.Merge(dst, src) +} +func (m *ResolveIntentResponse) XXX_Size() int { + return m.Size() +} +func (m *ResolveIntentResponse) XXX_DiscardUnknown() { + xxx_messageInfo_ResolveIntentResponse.DiscardUnknown(m) } -func (m *ResolveIntentResponse) Reset() { *m = ResolveIntentResponse{} } -func (m *ResolveIntentResponse) String() string { return proto.CompactTextString(m) } -func (*ResolveIntentResponse) ProtoMessage() {} -func (*ResolveIntentResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{53} } +var xxx_messageInfo_ResolveIntentResponse proto.InternalMessageInfo // A ResolveIntentRangeRequest is arguments to the ResolveIntentRange() method. // It is sent by transaction coordinators after success calling PushTxn to // clean up write intents: either to remove, commit or move them forward in // time. type ResolveIntentRangeRequest struct { - RequestHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` // The transaction whose intents are being resolved. - IntentTxn cockroach_storage_engine_enginepb.TxnMeta `protobuf:"bytes,2,opt,name=intent_txn,json=intentTxn" json:"intent_txn"` + IntentTxn enginepb.TxnMeta `protobuf:"bytes,2,opt,name=intent_txn,json=intentTxn,proto3" json:"intent_txn"` // The status of the transaction. Status TransactionStatus `protobuf:"varint,3,opt,name=status,proto3,enum=cockroach.roachpb.TransactionStatus" json:"status,omitempty"` // Optionally poison the abort span for the transaction on all ranges @@ -1425,47 +2675,151 @@ type ResolveIntentRangeRequest struct { // The minimum timestamp for any intents written by this // transaction. If present, this value can be used to optimize the // iteration over the span to find intents to resolve. - MinTimestamp cockroach_util_hlc.Timestamp `protobuf:"bytes,5,opt,name=min_timestamp,json=minTimestamp" json:"min_timestamp"` + MinTimestamp hlc.Timestamp `protobuf:"bytes,5,opt,name=min_timestamp,json=minTimestamp,proto3" json:"min_timestamp"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *ResolveIntentRangeRequest) Reset() { *m = ResolveIntentRangeRequest{} } -func (m *ResolveIntentRangeRequest) String() string { return proto.CompactTextString(m) } -func (*ResolveIntentRangeRequest) ProtoMessage() {} -func (*ResolveIntentRangeRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{54} } +func (m *ResolveIntentRangeRequest) Reset() { *m = ResolveIntentRangeRequest{} } +func (m *ResolveIntentRangeRequest) String() string { return proto.CompactTextString(m) } +func (*ResolveIntentRangeRequest) ProtoMessage() {} +func (*ResolveIntentRangeRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_api_a51b49ba855d15ec, []int{54} +} +func (m *ResolveIntentRangeRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ResolveIntentRangeRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ResolveIntentRangeRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_ResolveIntentRangeRequest.Merge(dst, src) +} +func (m *ResolveIntentRangeRequest) XXX_Size() int { + return m.Size() +} +func (m *ResolveIntentRangeRequest) XXX_DiscardUnknown() { + xxx_messageInfo_ResolveIntentRangeRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_ResolveIntentRangeRequest proto.InternalMessageInfo // A ResolveIntentRangeResponse is the return value from the // ResolveIntent() method. type ResolveIntentRangeResponse struct { - ResponseHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ResolveIntentRangeResponse) Reset() { *m = ResolveIntentRangeResponse{} } +func (m *ResolveIntentRangeResponse) String() string { return proto.CompactTextString(m) } +func (*ResolveIntentRangeResponse) ProtoMessage() {} +func (*ResolveIntentRangeResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_api_a51b49ba855d15ec, []int{55} +} +func (m *ResolveIntentRangeResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ResolveIntentRangeResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ResolveIntentRangeResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_ResolveIntentRangeResponse.Merge(dst, src) +} +func (m *ResolveIntentRangeResponse) XXX_Size() int { + return m.Size() +} +func (m *ResolveIntentRangeResponse) XXX_DiscardUnknown() { + xxx_messageInfo_ResolveIntentRangeResponse.DiscardUnknown(m) } -func (m *ResolveIntentRangeResponse) Reset() { *m = ResolveIntentRangeResponse{} } -func (m *ResolveIntentRangeResponse) String() string { return proto.CompactTextString(m) } -func (*ResolveIntentRangeResponse) ProtoMessage() {} -func (*ResolveIntentRangeResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{55} } +var xxx_messageInfo_ResolveIntentRangeResponse proto.InternalMessageInfo // A MergeRequest contains arguments to the Merge() method. It // specifies a key and a value which should be merged into the // existing value at that key. type MergeRequest struct { - RequestHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` - Value Value `protobuf:"bytes,2,opt,name=value" json:"value"` + RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + Value Value `protobuf:"bytes,2,opt,name=value,proto3" json:"value"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{56} +} +func (m *MergeRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *MergeRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *MergeRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_MergeRequest.Merge(dst, src) +} +func (m *MergeRequest) XXX_Size() int { + return m.Size() +} +func (m *MergeRequest) XXX_DiscardUnknown() { + xxx_messageInfo_MergeRequest.DiscardUnknown(m) } -func (m *MergeRequest) Reset() { *m = MergeRequest{} } -func (m *MergeRequest) String() string { return proto.CompactTextString(m) } -func (*MergeRequest) ProtoMessage() {} -func (*MergeRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{56} } +var xxx_messageInfo_MergeRequest proto.InternalMessageInfo // MergeResponse is the response to a Merge() operation. type MergeResponse struct { - ResponseHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{57} +} +func (m *MergeResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *MergeResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *MergeResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_MergeResponse.Merge(dst, src) +} +func (m *MergeResponse) XXX_Size() int { + return m.Size() +} +func (m *MergeResponse) XXX_DiscardUnknown() { + xxx_messageInfo_MergeResponse.DiscardUnknown(m) } -func (m *MergeResponse) Reset() { *m = MergeResponse{} } -func (m *MergeResponse) String() string { return proto.CompactTextString(m) } -func (*MergeResponse) ProtoMessage() {} -func (*MergeResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{57} } +var xxx_messageInfo_MergeResponse proto.InternalMessageInfo // TruncateLogRequest is used to remove a prefix of the raft log. While there // is no requirement for correctness that the raft log truncation be synchronized across @@ -1473,50 +2827,128 @@ func (*MergeResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, [ // to identical as possible. The raft leader can also inform decisions about the cutoff point // with its knowledge of the replicas' acknowledgment status. type TruncateLogRequest struct { - RequestHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` // Log entries < this index are to be discarded. Index uint64 `protobuf:"varint,2,opt,name=index,proto3" json:"index,omitempty"` // RangeID is used to double check that the correct range is being truncated. // The header specifies a span, start and end keys, but not the range id // itself. The range may have changed from the one specified in the header // in the case of a merge. - RangeID RangeID `protobuf:"varint,3,opt,name=range_id,json=rangeId,proto3,casttype=RangeID" json:"range_id,omitempty"` + RangeID RangeID `protobuf:"varint,3,opt,name=range_id,json=rangeId,proto3,casttype=RangeID" json:"range_id,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *TruncateLogRequest) Reset() { *m = TruncateLogRequest{} } -func (m *TruncateLogRequest) String() string { return proto.CompactTextString(m) } -func (*TruncateLogRequest) ProtoMessage() {} -func (*TruncateLogRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{58} } +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_a51b49ba855d15ec, []int{58} +} +func (m *TruncateLogRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *TruncateLogRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *TruncateLogRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_TruncateLogRequest.Merge(dst, src) +} +func (m *TruncateLogRequest) XXX_Size() int { + return m.Size() +} +func (m *TruncateLogRequest) XXX_DiscardUnknown() { + xxx_messageInfo_TruncateLogRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_TruncateLogRequest proto.InternalMessageInfo // TruncateLogResponse is the response to a TruncateLog() operation. type TruncateLogResponse struct { - ResponseHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{59} +} +func (m *TruncateLogResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *TruncateLogResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *TruncateLogResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_TruncateLogResponse.Merge(dst, src) +} +func (m *TruncateLogResponse) XXX_Size() int { + return m.Size() +} +func (m *TruncateLogResponse) XXX_DiscardUnknown() { + xxx_messageInfo_TruncateLogResponse.DiscardUnknown(m) } -func (m *TruncateLogResponse) Reset() { *m = TruncateLogResponse{} } -func (m *TruncateLogResponse) String() string { return proto.CompactTextString(m) } -func (*TruncateLogResponse) ProtoMessage() {} -func (*TruncateLogResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{59} } +var xxx_messageInfo_TruncateLogResponse proto.InternalMessageInfo // A RequestLeaseRequest is arguments to the RequestLease() // method. It is sent by the store on behalf of one of its ranges upon receipt // of a command requiring a lease when none is found. type RequestLeaseRequest struct { - RequestHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` - Lease Lease `protobuf:"bytes,2,opt,name=lease" json:"lease"` + RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + Lease Lease `protobuf:"bytes,2,opt,name=lease,proto3" json:"lease"` // The previous lease is specified by the caller to verify // it has not changed when executing this command. - PrevLease Lease `protobuf:"bytes,3,opt,name=prev_lease,json=prevLease" json:"prev_lease"` + PrevLease Lease `protobuf:"bytes,3,opt,name=prev_lease,json=prevLease,proto3" json:"prev_lease"` // The MinLeaseProposedTS of the proposing replica to make sure that leases // issued after a node restart receive a new sequence number (instead of // counting as a lease extension). See #23204. - MinProposedTS *cockroach_util_hlc.Timestamp `protobuf:"bytes,4,opt,name=min_proposed_ts,json=minProposedTs" json:"min_proposed_ts,omitempty"` + MinProposedTS *hlc.Timestamp `protobuf:"bytes,4,opt,name=min_proposed_ts,json=minProposedTs,proto3" json:"min_proposed_ts,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *RequestLeaseRequest) Reset() { *m = RequestLeaseRequest{} } -func (m *RequestLeaseRequest) String() string { return proto.CompactTextString(m) } -func (*RequestLeaseRequest) ProtoMessage() {} -func (*RequestLeaseRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{60} } +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_a51b49ba855d15ec, []int{60} +} +func (m *RequestLeaseRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RequestLeaseRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RequestLeaseRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_RequestLeaseRequest.Merge(dst, src) +} +func (m *RequestLeaseRequest) XXX_Size() int { + return m.Size() +} +func (m *RequestLeaseRequest) XXX_DiscardUnknown() { + xxx_messageInfo_RequestLeaseRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_RequestLeaseRequest proto.InternalMessageInfo // A TransferLeaseRequest represents the arguments to the TransferLease() // method. It is sent by a replica that currently holds the range lease and @@ -1531,136 +2963,394 @@ func (*RequestLeaseRequest) Descriptor() ([]byte, []int) { return fileDescriptor // special in this respect (for example, the proposer of this command is // checked to have been holding the lease when the proposal was made). type TransferLeaseRequest struct { - RequestHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` - Lease Lease `protobuf:"bytes,2,opt,name=lease" json:"lease"` + RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + Lease Lease `protobuf:"bytes,2,opt,name=lease,proto3" json:"lease"` // The previous lease is specified by the caller to verify // it has not changed when executing this command. - PrevLease Lease `protobuf:"bytes,3,opt,name=prev_lease,json=prevLease" json:"prev_lease"` + PrevLease Lease `protobuf:"bytes,3,opt,name=prev_lease,json=prevLease,proto3" json:"prev_lease"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{61} +} +func (m *TransferLeaseRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *TransferLeaseRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *TransferLeaseRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_TransferLeaseRequest.Merge(dst, src) +} +func (m *TransferLeaseRequest) XXX_Size() int { + return m.Size() +} +func (m *TransferLeaseRequest) XXX_DiscardUnknown() { + xxx_messageInfo_TransferLeaseRequest.DiscardUnknown(m) } -func (m *TransferLeaseRequest) Reset() { *m = TransferLeaseRequest{} } -func (m *TransferLeaseRequest) String() string { return proto.CompactTextString(m) } -func (*TransferLeaseRequest) ProtoMessage() {} -func (*TransferLeaseRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{61} } +var xxx_messageInfo_TransferLeaseRequest proto.InternalMessageInfo // LeaseInfoRequest is the argument to the LeaseInfo() method, for getting // information about a range's lease. // It's a point request, so it addresses one single range, and returns the lease // currently in effect for that range. type LeaseInfoRequest struct { - RequestHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{62} +} +func (m *LeaseInfoRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *LeaseInfoRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *LeaseInfoRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_LeaseInfoRequest.Merge(dst, src) +} +func (m *LeaseInfoRequest) XXX_Size() int { + return m.Size() +} +func (m *LeaseInfoRequest) XXX_DiscardUnknown() { + xxx_messageInfo_LeaseInfoRequest.DiscardUnknown(m) } -func (m *LeaseInfoRequest) Reset() { *m = LeaseInfoRequest{} } -func (m *LeaseInfoRequest) String() string { return proto.CompactTextString(m) } -func (*LeaseInfoRequest) ProtoMessage() {} -func (*LeaseInfoRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{62} } +var xxx_messageInfo_LeaseInfoRequest proto.InternalMessageInfo // LeaseInfoResponse is the response to a LeaseInfo() operation. type LeaseInfoResponse struct { - ResponseHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` // The last lease known by the replica serving the request. It can also be the // tentative future lease, if a lease transfer is in progress. - Lease Lease `protobuf:"bytes,2,opt,name=lease" json:"lease"` + Lease Lease `protobuf:"bytes,2,opt,name=lease,proto3" json:"lease"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{63} +} +func (m *LeaseInfoResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *LeaseInfoResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *LeaseInfoResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_LeaseInfoResponse.Merge(dst, src) +} +func (m *LeaseInfoResponse) XXX_Size() int { + return m.Size() +} +func (m *LeaseInfoResponse) XXX_DiscardUnknown() { + xxx_messageInfo_LeaseInfoResponse.DiscardUnknown(m) } -func (m *LeaseInfoResponse) Reset() { *m = LeaseInfoResponse{} } -func (m *LeaseInfoResponse) String() string { return proto.CompactTextString(m) } -func (*LeaseInfoResponse) ProtoMessage() {} -func (*LeaseInfoResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{63} } +var xxx_messageInfo_LeaseInfoResponse proto.InternalMessageInfo // A RequestLeaseResponse is the response to a RequestLease() or TransferLease() // operation. type RequestLeaseResponse struct { - ResponseHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *RequestLeaseResponse) Reset() { *m = RequestLeaseResponse{} } -func (m *RequestLeaseResponse) String() string { return proto.CompactTextString(m) } -func (*RequestLeaseResponse) ProtoMessage() {} -func (*RequestLeaseResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{64} } +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_a51b49ba855d15ec, []int{64} +} +func (m *RequestLeaseResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RequestLeaseResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RequestLeaseResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_RequestLeaseResponse.Merge(dst, src) +} +func (m *RequestLeaseResponse) XXX_Size() int { + return m.Size() +} +func (m *RequestLeaseResponse) XXX_DiscardUnknown() { + xxx_messageInfo_RequestLeaseResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_RequestLeaseResponse proto.InternalMessageInfo // A ComputeChecksumRequest is arguments to the ComputeChecksum() method, to // start computing the checksum for the specified range at the snapshot for this // request command. A response is returned without the checksum. The computed // checksum is retrieved via a storage.CollectChecksumRequest. type ComputeChecksumRequest struct { - RequestHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` // The version used to pick the checksum method. It allows us to use a // consistent checksumming method across replicas. Version uint32 `protobuf:"varint,2,opt,name=version,proto3" json:"version,omitempty"` // Compute a checksum along with a snapshot of the entire range, that will be // used in logging a diff during checksum verification. - Snapshot bool `protobuf:"varint,4,opt,name=snapshot,proto3" json:"snapshot,omitempty"` + Snapshot bool `protobuf:"varint,4,opt,name=snapshot,proto3" json:"snapshot,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{65} +} +func (m *ComputeChecksumRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ComputeChecksumRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ComputeChecksumRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_ComputeChecksumRequest.Merge(dst, src) +} +func (m *ComputeChecksumRequest) XXX_Size() int { + return m.Size() +} +func (m *ComputeChecksumRequest) XXX_DiscardUnknown() { + xxx_messageInfo_ComputeChecksumRequest.DiscardUnknown(m) } -func (m *ComputeChecksumRequest) Reset() { *m = ComputeChecksumRequest{} } -func (m *ComputeChecksumRequest) String() string { return proto.CompactTextString(m) } -func (*ComputeChecksumRequest) ProtoMessage() {} -func (*ComputeChecksumRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{65} } +var xxx_messageInfo_ComputeChecksumRequest proto.InternalMessageInfo // A ComputeChecksumResponse is the response to a ComputeChecksum() operation. type ComputeChecksumResponse struct { - ResponseHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` // ChecksumID is the unique identifier that can be used to get the computed // checksum in a future storage.CollectChecksumRequest. - ChecksumID github_com_cockroachdb_cockroach_pkg_util_uuid.UUID `protobuf:"bytes,2,opt,name=checksum_id,json=checksumId,proto3,customtype=github.com/cockroachdb/cockroach/pkg/util/uuid.UUID" json:"checksum_id"` + ChecksumID github_com_cockroachdb_cockroach_pkg_util_uuid.UUID `protobuf:"bytes,2,opt,name=checksum_id,json=checksumId,proto3,customtype=github.com/cockroachdb/cockroach/pkg/util/uuid.UUID" json:"checksum_id"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *ComputeChecksumResponse) Reset() { *m = ComputeChecksumResponse{} } -func (m *ComputeChecksumResponse) String() string { return proto.CompactTextString(m) } -func (*ComputeChecksumResponse) ProtoMessage() {} -func (*ComputeChecksumResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{66} } +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_a51b49ba855d15ec, []int{66} +} +func (m *ComputeChecksumResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ComputeChecksumResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ComputeChecksumResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_ComputeChecksumResponse.Merge(dst, src) +} +func (m *ComputeChecksumResponse) XXX_Size() int { + return m.Size() +} +func (m *ComputeChecksumResponse) XXX_DiscardUnknown() { + xxx_messageInfo_ComputeChecksumResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_ComputeChecksumResponse proto.InternalMessageInfo type ExportStorage struct { - Provider ExportStorageProvider `protobuf:"varint,1,opt,name=provider,proto3,enum=cockroach.roachpb.ExportStorageProvider" json:"provider,omitempty"` - LocalFile ExportStorage_LocalFilePath `protobuf:"bytes,2,opt,name=LocalFile" json:"LocalFile"` - HttpPath ExportStorage_Http `protobuf:"bytes,3,opt,name=HttpPath" json:"HttpPath"` - GoogleCloudConfig *ExportStorage_GCS `protobuf:"bytes,4,opt,name=GoogleCloudConfig" json:"GoogleCloudConfig,omitempty"` - S3Config *ExportStorage_S3 `protobuf:"bytes,5,opt,name=S3Config" json:"S3Config,omitempty"` - AzureConfig *ExportStorage_Azure `protobuf:"bytes,6,opt,name=AzureConfig" json:"AzureConfig,omitempty"` - WorkloadConfig *ExportStorage_Workload `protobuf:"bytes,7,opt,name=WorkloadConfig" json:"WorkloadConfig,omitempty"` + Provider ExportStorageProvider `protobuf:"varint,1,opt,name=provider,proto3,enum=cockroach.roachpb.ExportStorageProvider" json:"provider,omitempty"` + LocalFile ExportStorage_LocalFilePath `protobuf:"bytes,2,opt,name=LocalFile,proto3" json:"LocalFile"` + HttpPath ExportStorage_Http `protobuf:"bytes,3,opt,name=HttpPath,proto3" json:"HttpPath"` + GoogleCloudConfig *ExportStorage_GCS `protobuf:"bytes,4,opt,name=GoogleCloudConfig,proto3" json:"GoogleCloudConfig,omitempty"` + S3Config *ExportStorage_S3 `protobuf:"bytes,5,opt,name=S3Config,proto3" json:"S3Config,omitempty"` + AzureConfig *ExportStorage_Azure `protobuf:"bytes,6,opt,name=AzureConfig,proto3" json:"AzureConfig,omitempty"` + WorkloadConfig *ExportStorage_Workload `protobuf:"bytes,7,opt,name=WorkloadConfig,proto3" json:"WorkloadConfig,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{67} +} +func (m *ExportStorage) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ExportStorage) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ExportStorage) XXX_Merge(src proto.Message) { + xxx_messageInfo_ExportStorage.Merge(dst, src) +} +func (m *ExportStorage) XXX_Size() int { + return m.Size() +} +func (m *ExportStorage) XXX_DiscardUnknown() { + xxx_messageInfo_ExportStorage.DiscardUnknown(m) } -func (m *ExportStorage) Reset() { *m = ExportStorage{} } -func (m *ExportStorage) String() string { return proto.CompactTextString(m) } -func (*ExportStorage) ProtoMessage() {} -func (*ExportStorage) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{67} } +var xxx_messageInfo_ExportStorage proto.InternalMessageInfo type ExportStorage_LocalFilePath struct { - Path string `protobuf:"bytes,1,opt,name=path,proto3" json:"path,omitempty"` + Path string `protobuf:"bytes,1,opt,name=path,proto3" json:"path,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *ExportStorage_LocalFilePath) Reset() { *m = ExportStorage_LocalFilePath{} } func (m *ExportStorage_LocalFilePath) String() string { return proto.CompactTextString(m) } func (*ExportStorage_LocalFilePath) ProtoMessage() {} func (*ExportStorage_LocalFilePath) Descriptor() ([]byte, []int) { - return fileDescriptorApi, []int{67, 0} + return fileDescriptor_api_a51b49ba855d15ec, []int{67, 0} +} +func (m *ExportStorage_LocalFilePath) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ExportStorage_LocalFilePath) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ExportStorage_LocalFilePath) XXX_Merge(src proto.Message) { + xxx_messageInfo_ExportStorage_LocalFilePath.Merge(dst, src) +} +func (m *ExportStorage_LocalFilePath) XXX_Size() int { + return m.Size() +} +func (m *ExportStorage_LocalFilePath) XXX_DiscardUnknown() { + xxx_messageInfo_ExportStorage_LocalFilePath.DiscardUnknown(m) } +var xxx_messageInfo_ExportStorage_LocalFilePath proto.InternalMessageInfo + type ExportStorage_Http struct { - BaseUri string `protobuf:"bytes,1,opt,name=baseUri,proto3" json:"baseUri,omitempty"` + BaseUri string `protobuf:"bytes,1,opt,name=baseUri,proto3" json:"baseUri,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{67, 1} +} +func (m *ExportStorage_Http) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ExportStorage_Http) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ExportStorage_Http) XXX_Merge(src proto.Message) { + xxx_messageInfo_ExportStorage_Http.Merge(dst, src) +} +func (m *ExportStorage_Http) XXX_Size() int { + return m.Size() +} +func (m *ExportStorage_Http) XXX_DiscardUnknown() { + xxx_messageInfo_ExportStorage_Http.DiscardUnknown(m) } -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 fileDescriptorApi, []int{67, 1} } +var xxx_messageInfo_ExportStorage_Http proto.InternalMessageInfo type ExportStorage_S3 struct { - Bucket string `protobuf:"bytes,1,opt,name=bucket,proto3" json:"bucket,omitempty"` - Prefix string `protobuf:"bytes,2,opt,name=prefix,proto3" json:"prefix,omitempty"` - AccessKey string `protobuf:"bytes,3,opt,name=access_key,json=accessKey,proto3" json:"access_key,omitempty"` - Secret string `protobuf:"bytes,4,opt,name=secret,proto3" json:"secret,omitempty"` - TempToken string `protobuf:"bytes,5,opt,name=temp_token,json=tempToken,proto3" json:"temp_token,omitempty"` - Endpoint string `protobuf:"bytes,6,opt,name=endpoint,proto3" json:"endpoint,omitempty"` - Region string `protobuf:"bytes,7,opt,name=region,proto3" json:"region,omitempty"` + Bucket string `protobuf:"bytes,1,opt,name=bucket,proto3" json:"bucket,omitempty"` + Prefix string `protobuf:"bytes,2,opt,name=prefix,proto3" json:"prefix,omitempty"` + AccessKey string `protobuf:"bytes,3,opt,name=access_key,json=accessKey,proto3" json:"access_key,omitempty"` + Secret string `protobuf:"bytes,4,opt,name=secret,proto3" json:"secret,omitempty"` + TempToken string `protobuf:"bytes,5,opt,name=temp_token,json=tempToken,proto3" json:"temp_token,omitempty"` + Endpoint string `protobuf:"bytes,6,opt,name=endpoint,proto3" json:"endpoint,omitempty"` + Region string `protobuf:"bytes,7,opt,name=region,proto3" json:"region,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{67, 2} +} +func (m *ExportStorage_S3) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ExportStorage_S3) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ExportStorage_S3) XXX_Merge(src proto.Message) { + xxx_messageInfo_ExportStorage_S3.Merge(dst, src) +} +func (m *ExportStorage_S3) XXX_Size() int { + return m.Size() +} +func (m *ExportStorage_S3) XXX_DiscardUnknown() { + xxx_messageInfo_ExportStorage_S3.DiscardUnknown(m) } -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 fileDescriptorApi, []int{67, 2} } +var xxx_messageInfo_ExportStorage_S3 proto.InternalMessageInfo type ExportStorage_GCS struct { Bucket string `protobuf:"bytes,1,opt,name=bucket,proto3" json:"bucket,omitempty"` @@ -1668,245 +3358,711 @@ type ExportStorage_GCS struct { Auth string `protobuf:"bytes,3,opt,name=auth,proto3" json:"auth,omitempty"` // BillingProject if non-empty, is the Google Cloud project to bill for all storage requests. // This is required to be set if using a "requestor pays" bucket. - BillingProject string `protobuf:"bytes,4,opt,name=billing_project,json=billingProject,proto3" json:"billing_project,omitempty"` - Credentials string `protobuf:"bytes,5,opt,name=credentials,proto3" json:"credentials,omitempty"` + BillingProject string `protobuf:"bytes,4,opt,name=billing_project,json=billingProject,proto3" json:"billing_project,omitempty"` + Credentials string `protobuf:"bytes,5,opt,name=credentials,proto3" json:"credentials,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{67, 3} +} +func (m *ExportStorage_GCS) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ExportStorage_GCS) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ExportStorage_GCS) XXX_Merge(src proto.Message) { + xxx_messageInfo_ExportStorage_GCS.Merge(dst, src) +} +func (m *ExportStorage_GCS) XXX_Size() int { + return m.Size() +} +func (m *ExportStorage_GCS) XXX_DiscardUnknown() { + xxx_messageInfo_ExportStorage_GCS.DiscardUnknown(m) } -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 fileDescriptorApi, []int{67, 3} } +var xxx_messageInfo_ExportStorage_GCS proto.InternalMessageInfo type ExportStorage_Azure struct { - Container string `protobuf:"bytes,1,opt,name=container,proto3" json:"container,omitempty"` - Prefix string `protobuf:"bytes,2,opt,name=prefix,proto3" json:"prefix,omitempty"` - AccountName string `protobuf:"bytes,3,opt,name=account_name,json=accountName,proto3" json:"account_name,omitempty"` - AccountKey string `protobuf:"bytes,4,opt,name=account_key,json=accountKey,proto3" json:"account_key,omitempty"` + Container string `protobuf:"bytes,1,opt,name=container,proto3" json:"container,omitempty"` + Prefix string `protobuf:"bytes,2,opt,name=prefix,proto3" json:"prefix,omitempty"` + AccountName string `protobuf:"bytes,3,opt,name=account_name,json=accountName,proto3" json:"account_name,omitempty"` + AccountKey string `protobuf:"bytes,4,opt,name=account_key,json=accountKey,proto3" json:"account_key,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{67, 4} +} +func (m *ExportStorage_Azure) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ExportStorage_Azure) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ExportStorage_Azure) XXX_Merge(src proto.Message) { + xxx_messageInfo_ExportStorage_Azure.Merge(dst, src) +} +func (m *ExportStorage_Azure) XXX_Size() int { + return m.Size() +} +func (m *ExportStorage_Azure) XXX_DiscardUnknown() { + xxx_messageInfo_ExportStorage_Azure.DiscardUnknown(m) } -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 fileDescriptorApi, []int{67, 4} } +var xxx_messageInfo_ExportStorage_Azure proto.InternalMessageInfo type ExportStorage_Workload struct { - Generator string `protobuf:"bytes,1,opt,name=generator,proto3" json:"generator,omitempty"` - Version string `protobuf:"bytes,2,opt,name=version,proto3" json:"version,omitempty"` - Table string `protobuf:"bytes,3,opt,name=table,proto3" json:"table,omitempty"` - Flags []string `protobuf:"bytes,4,rep,name=flags" json:"flags,omitempty"` - Format string `protobuf:"bytes,5,opt,name=format,proto3" json:"format,omitempty"` - BatchBegin int64 `protobuf:"varint,6,opt,name=batch_begin,json=batchBegin,proto3" json:"batch_begin,omitempty"` - BatchEnd int64 `protobuf:"varint,7,opt,name=batch_end,json=batchEnd,proto3" json:"batch_end,omitempty"` + Generator string `protobuf:"bytes,1,opt,name=generator,proto3" json:"generator,omitempty"` + Version string `protobuf:"bytes,2,opt,name=version,proto3" json:"version,omitempty"` + Table string `protobuf:"bytes,3,opt,name=table,proto3" json:"table,omitempty"` + Flags []string `protobuf:"bytes,4,rep,name=flags,proto3" json:"flags,omitempty"` + Format string `protobuf:"bytes,5,opt,name=format,proto3" json:"format,omitempty"` + BatchBegin int64 `protobuf:"varint,6,opt,name=batch_begin,json=batchBegin,proto3" json:"batch_begin,omitempty"` + BatchEnd int64 `protobuf:"varint,7,opt,name=batch_end,json=batchEnd,proto3" json:"batch_end,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{67, 5} +} +func (m *ExportStorage_Workload) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ExportStorage_Workload) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ExportStorage_Workload) XXX_Merge(src proto.Message) { + xxx_messageInfo_ExportStorage_Workload.Merge(dst, src) +} +func (m *ExportStorage_Workload) XXX_Size() int { + return m.Size() +} +func (m *ExportStorage_Workload) XXX_DiscardUnknown() { + xxx_messageInfo_ExportStorage_Workload.DiscardUnknown(m) } -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 fileDescriptorApi, []int{67, 5} } +var xxx_messageInfo_ExportStorage_Workload proto.InternalMessageInfo // WriteBatchRequest is arguments to the WriteBatch() method, to apply the // operations encoded in a BatchRepr. type WriteBatchRequest struct { - RequestHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` // The span of keys encoded in data, duplicated because the header span can // be modified by DistSender and we use this one to fail fast. - DataSpan Span `protobuf:"bytes,2,opt,name=data_span,json=dataSpan" json:"data_span"` + DataSpan Span `protobuf:"bytes,2,opt,name=data_span,json=dataSpan,proto3" json:"data_span"` // A BatchRepr, the serialized form of a RocksDB Batch. - Data []byte `protobuf:"bytes,3,opt,name=data,proto3" json:"data,omitempty"` + Data []byte `protobuf:"bytes,3,opt,name=data,proto3" json:"data,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{68} +} +func (m *WriteBatchRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *WriteBatchRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *WriteBatchRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_WriteBatchRequest.Merge(dst, src) +} +func (m *WriteBatchRequest) XXX_Size() int { + return m.Size() +} +func (m *WriteBatchRequest) XXX_DiscardUnknown() { + xxx_messageInfo_WriteBatchRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_WriteBatchRequest proto.InternalMessageInfo + +// WriteBatchResponse is the response to a WriteBatch() operation. +type WriteBatchResponse struct { + ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{69} +} +func (m *WriteBatchResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *WriteBatchResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil } - -func (m *WriteBatchRequest) Reset() { *m = WriteBatchRequest{} } -func (m *WriteBatchRequest) String() string { return proto.CompactTextString(m) } -func (*WriteBatchRequest) ProtoMessage() {} -func (*WriteBatchRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{68} } - -// WriteBatchResponse is the response to a WriteBatch() operation. -type WriteBatchResponse struct { - ResponseHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` +func (dst *WriteBatchResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_WriteBatchResponse.Merge(dst, src) +} +func (m *WriteBatchResponse) XXX_Size() int { + return m.Size() +} +func (m *WriteBatchResponse) XXX_DiscardUnknown() { + xxx_messageInfo_WriteBatchResponse.DiscardUnknown(m) } -func (m *WriteBatchResponse) Reset() { *m = WriteBatchResponse{} } -func (m *WriteBatchResponse) String() string { return proto.CompactTextString(m) } -func (*WriteBatchResponse) ProtoMessage() {} -func (*WriteBatchResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{69} } +var xxx_messageInfo_WriteBatchResponse proto.InternalMessageInfo // ExportRequest is the argument to the Export() method, to dump a keyrange into // files under a basepath. type ExportRequest struct { - RequestHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` - Storage ExportStorage `protobuf:"bytes,2,opt,name=storage" json:"storage"` - StartTime cockroach_util_hlc.Timestamp `protobuf:"bytes,3,opt,name=start_time,json=startTime" json:"start_time"` - MVCCFilter MVCCFilter `protobuf:"varint,4,opt,name=mvcc_filter,json=mvccFilter,proto3,enum=cockroach.roachpb.MVCCFilter" json:"mvcc_filter,omitempty"` + RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + Storage ExportStorage `protobuf:"bytes,2,opt,name=storage,proto3" json:"storage"` + StartTime hlc.Timestamp `protobuf:"bytes,3,opt,name=start_time,json=startTime,proto3" json:"start_time"` + MVCCFilter MVCCFilter `protobuf:"varint,4,opt,name=mvcc_filter,json=mvccFilter,proto3,enum=cockroach.roachpb.MVCCFilter" json:"mvcc_filter,omitempty"` // Return the exported SST data in the response. ReturnSST bool `protobuf:"varint,5,opt,name=return_sst,json=returnSst,proto3" json:"return_sst,omitempty"` // OmitChecksum, if true, will skip checksumming the sst and leave the // `Sha512` field empty in the response. During a rolling upgrade to 2.1, it // may still be set if the request is served by an old node, but since the // caller has declare they're not going to use it, that's okay. - OmitChecksum bool `protobuf:"varint,6,opt,name=omit_checksum,json=omitChecksum,proto3" json:"omit_checksum,omitempty"` + OmitChecksum bool `protobuf:"varint,6,opt,name=omit_checksum,json=omitChecksum,proto3" json:"omit_checksum,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{70} +} +func (m *ExportRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ExportRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ExportRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_ExportRequest.Merge(dst, src) +} +func (m *ExportRequest) XXX_Size() int { + return m.Size() +} +func (m *ExportRequest) XXX_DiscardUnknown() { + xxx_messageInfo_ExportRequest.DiscardUnknown(m) } -func (m *ExportRequest) Reset() { *m = ExportRequest{} } -func (m *ExportRequest) String() string { return proto.CompactTextString(m) } -func (*ExportRequest) ProtoMessage() {} -func (*ExportRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{70} } +var xxx_messageInfo_ExportRequest proto.InternalMessageInfo type BulkOpSummary struct { - DataSize int64 `protobuf:"varint,1,opt,name=data_size,json=dataSize,proto3" json:"data_size,omitempty"` - Rows int64 `protobuf:"varint,2,opt,name=rows,proto3" json:"rows,omitempty"` - IndexEntries int64 `protobuf:"varint,3,opt,name=index_entries,json=indexEntries,proto3" json:"index_entries,omitempty"` - SystemRecords int64 `protobuf:"varint,4,opt,name=system_records,json=systemRecords,proto3" json:"system_records,omitempty"` + DataSize int64 `protobuf:"varint,1,opt,name=data_size,json=dataSize,proto3" json:"data_size,omitempty"` + Rows int64 `protobuf:"varint,2,opt,name=rows,proto3" json:"rows,omitempty"` + IndexEntries int64 `protobuf:"varint,3,opt,name=index_entries,json=indexEntries,proto3" json:"index_entries,omitempty"` + SystemRecords int64 `protobuf:"varint,4,opt,name=system_records,json=systemRecords,proto3" json:"system_records,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{71} +} +func (m *BulkOpSummary) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *BulkOpSummary) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *BulkOpSummary) XXX_Merge(src proto.Message) { + xxx_messageInfo_BulkOpSummary.Merge(dst, src) +} +func (m *BulkOpSummary) XXX_Size() int { + return m.Size() +} +func (m *BulkOpSummary) XXX_DiscardUnknown() { + xxx_messageInfo_BulkOpSummary.DiscardUnknown(m) } -func (m *BulkOpSummary) Reset() { *m = BulkOpSummary{} } -func (m *BulkOpSummary) String() string { return proto.CompactTextString(m) } -func (*BulkOpSummary) ProtoMessage() {} -func (*BulkOpSummary) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{71} } +var xxx_messageInfo_BulkOpSummary proto.InternalMessageInfo // ExportResponse is the response to an Export() operation. type ExportResponse struct { - ResponseHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` - Files []ExportResponse_File `protobuf:"bytes,2,rep,name=files" json:"files"` - StartTime cockroach_util_hlc.Timestamp `protobuf:"bytes,3,opt,name=start_time,json=startTime" json:"start_time"` + ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + Files []ExportResponse_File `protobuf:"bytes,2,rep,name=files,proto3" json:"files"` + StartTime hlc.Timestamp `protobuf:"bytes,3,opt,name=start_time,json=startTime,proto3" json:"start_time"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{72} +} +func (m *ExportResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ExportResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ExportResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_ExportResponse.Merge(dst, src) +} +func (m *ExportResponse) XXX_Size() int { + return m.Size() +} +func (m *ExportResponse) XXX_DiscardUnknown() { + xxx_messageInfo_ExportResponse.DiscardUnknown(m) } -func (m *ExportResponse) Reset() { *m = ExportResponse{} } -func (m *ExportResponse) String() string { return proto.CompactTextString(m) } -func (*ExportResponse) ProtoMessage() {} -func (*ExportResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{72} } +var xxx_messageInfo_ExportResponse proto.InternalMessageInfo // File describes a keyrange that has been dumped to a file at the given // path. type ExportResponse_File struct { - Span Span `protobuf:"bytes,1,opt,name=span" json:"span"` - Path string `protobuf:"bytes,2,opt,name=path,proto3" json:"path,omitempty"` - Sha512 []byte `protobuf:"bytes,5,opt,name=sha512,proto3" json:"sha512,omitempty"` - Exported BulkOpSummary `protobuf:"bytes,6,opt,name=exported" json:"exported"` - SST []byte `protobuf:"bytes,7,opt,name=sst,proto3" json:"sst,omitempty"` + Span Span `protobuf:"bytes,1,opt,name=span,proto3" json:"span"` + Path string `protobuf:"bytes,2,opt,name=path,proto3" json:"path,omitempty"` + Sha512 []byte `protobuf:"bytes,5,opt,name=sha512,proto3" json:"sha512,omitempty"` + Exported BulkOpSummary `protobuf:"bytes,6,opt,name=exported,proto3" json:"exported"` + SST []byte `protobuf:"bytes,7,opt,name=sst,proto3" json:"sst,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{72, 0} +} +func (m *ExportResponse_File) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ExportResponse_File) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ExportResponse_File) XXX_Merge(src proto.Message) { + xxx_messageInfo_ExportResponse_File.Merge(dst, src) +} +func (m *ExportResponse_File) XXX_Size() int { + return m.Size() +} +func (m *ExportResponse_File) XXX_DiscardUnknown() { + xxx_messageInfo_ExportResponse_File.DiscardUnknown(m) } -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 fileDescriptorApi, []int{72, 0} } +var xxx_messageInfo_ExportResponse_File proto.InternalMessageInfo // ImportRequest is the argument to the Import() method, to bulk load key/value // entries. type ImportRequest struct { - RequestHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` // Files contains an ordered list of files, each containing kv entries to // import. Entries in later files with the same key override earlier ones. - Files []ImportRequest_File `protobuf:"bytes,2,rep,name=files" json:"files"` + Files []ImportRequest_File `protobuf:"bytes,2,rep,name=files,proto3" json:"files"` // DataSpan is the pre-rewrite keyrange of the data in `Files`. - DataSpan Span `protobuf:"bytes,3,opt,name=data_span,json=dataSpan" json:"data_span"` + DataSpan Span `protobuf:"bytes,3,opt,name=data_span,json=dataSpan,proto3" json:"data_span"` // EndTime, if not the zero value, will cause only entries before it to be // imported. - EndTime cockroach_util_hlc.Timestamp `protobuf:"bytes,6,opt,name=end_time,json=endTime" json:"end_time"` + EndTime hlc.Timestamp `protobuf:"bytes,6,opt,name=end_time,json=endTime,proto3" json:"end_time"` // Rekeys contains the descriptors for the data being Imported and the // previous ID for each (which is the ID used in the source data pointed to by // `files`). // TODO(dan): This field is a superset of the information represented by // `key_rewrites` and will supercede it once rekeying of interleaved tables is // fixed. - Rekeys []ImportRequest_TableRekey `protobuf:"bytes,5,rep,name=rekeys" json:"rekeys"` + Rekeys []ImportRequest_TableRekey `protobuf:"bytes,5,rep,name=rekeys,proto3" json:"rekeys"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{73} +} +func (m *ImportRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ImportRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ImportRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_ImportRequest.Merge(dst, src) +} +func (m *ImportRequest) XXX_Size() int { + return m.Size() +} +func (m *ImportRequest) XXX_DiscardUnknown() { + xxx_messageInfo_ImportRequest.DiscardUnknown(m) } -func (m *ImportRequest) Reset() { *m = ImportRequest{} } -func (m *ImportRequest) String() string { return proto.CompactTextString(m) } -func (*ImportRequest) ProtoMessage() {} -func (*ImportRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{73} } +var xxx_messageInfo_ImportRequest proto.InternalMessageInfo type ImportRequest_File struct { - Dir ExportStorage `protobuf:"bytes,1,opt,name=dir" json:"dir"` - Path string `protobuf:"bytes,2,opt,name=path,proto3" json:"path,omitempty"` - Sha512 []byte `protobuf:"bytes,4,opt,name=sha512,proto3" json:"sha512,omitempty"` + Dir ExportStorage `protobuf:"bytes,1,opt,name=dir,proto3" json:"dir"` + Path string `protobuf:"bytes,2,opt,name=path,proto3" json:"path,omitempty"` + Sha512 []byte `protobuf:"bytes,4,opt,name=sha512,proto3" json:"sha512,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{73, 0} +} +func (m *ImportRequest_File) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ImportRequest_File) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ImportRequest_File) XXX_Merge(src proto.Message) { + xxx_messageInfo_ImportRequest_File.Merge(dst, src) +} +func (m *ImportRequest_File) XXX_Size() int { + return m.Size() +} +func (m *ImportRequest_File) XXX_DiscardUnknown() { + xxx_messageInfo_ImportRequest_File.DiscardUnknown(m) } -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 fileDescriptorApi, []int{73, 0} } +var xxx_messageInfo_ImportRequest_File proto.InternalMessageInfo type ImportRequest_TableRekey struct { // OldID is the previous ID of `new_desc`. OldID uint32 `protobuf:"varint,1,opt,name=old_id,json=oldId,proto3" json:"old_id,omitempty"` // NewDesc is an encoded Descriptor message. - NewDesc []byte `protobuf:"bytes,2,opt,name=new_desc,json=newDesc,proto3" json:"new_desc,omitempty"` + NewDesc []byte `protobuf:"bytes,2,opt,name=new_desc,json=newDesc,proto3" json:"new_desc,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ImportRequest_TableRekey) Reset() { *m = ImportRequest_TableRekey{} } +func (m *ImportRequest_TableRekey) String() string { return proto.CompactTextString(m) } +func (*ImportRequest_TableRekey) ProtoMessage() {} +func (*ImportRequest_TableRekey) Descriptor() ([]byte, []int) { + return fileDescriptor_api_a51b49ba855d15ec, []int{73, 1} +} +func (m *ImportRequest_TableRekey) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ImportRequest_TableRekey) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ImportRequest_TableRekey) XXX_Merge(src proto.Message) { + xxx_messageInfo_ImportRequest_TableRekey.Merge(dst, src) +} +func (m *ImportRequest_TableRekey) XXX_Size() int { + return m.Size() +} +func (m *ImportRequest_TableRekey) XXX_DiscardUnknown() { + xxx_messageInfo_ImportRequest_TableRekey.DiscardUnknown(m) } -func (m *ImportRequest_TableRekey) Reset() { *m = ImportRequest_TableRekey{} } -func (m *ImportRequest_TableRekey) String() string { return proto.CompactTextString(m) } -func (*ImportRequest_TableRekey) ProtoMessage() {} -func (*ImportRequest_TableRekey) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{73, 1} } +var xxx_messageInfo_ImportRequest_TableRekey proto.InternalMessageInfo // ImportResponse is the response to a Import() operation. type ImportResponse struct { - ResponseHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` - Imported BulkOpSummary `protobuf:"bytes,3,opt,name=imported" json:"imported"` + ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + Imported BulkOpSummary `protobuf:"bytes,3,opt,name=imported,proto3" json:"imported"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{74} +} +func (m *ImportResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ImportResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ImportResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_ImportResponse.Merge(dst, src) +} +func (m *ImportResponse) XXX_Size() int { + return m.Size() +} +func (m *ImportResponse) XXX_DiscardUnknown() { + xxx_messageInfo_ImportResponse.DiscardUnknown(m) } -func (m *ImportResponse) Reset() { *m = ImportResponse{} } -func (m *ImportResponse) String() string { return proto.CompactTextString(m) } -func (*ImportResponse) ProtoMessage() {} -func (*ImportResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{74} } +var xxx_messageInfo_ImportResponse proto.InternalMessageInfo // AdminScatterRequest is the argument to the AdminScatter() method, which moves // replicas and leaseholders for a selection of ranges. Scatter is best-effort; // ranges that cannot be moved will include an error detail in the response and // won't fail the request. type AdminScatterRequest struct { - RequestHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` - RandomizeLeases bool `protobuf:"varint,2,opt,name=randomize_leases,json=randomizeLeases,proto3" json:"randomize_leases,omitempty"` + RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + RandomizeLeases bool `protobuf:"varint,2,opt,name=randomize_leases,json=randomizeLeases,proto3" json:"randomize_leases,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *AdminScatterRequest) Reset() { *m = AdminScatterRequest{} } -func (m *AdminScatterRequest) String() string { return proto.CompactTextString(m) } -func (*AdminScatterRequest) ProtoMessage() {} -func (*AdminScatterRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{75} } +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_a51b49ba855d15ec, []int{75} +} +func (m *AdminScatterRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *AdminScatterRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *AdminScatterRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_AdminScatterRequest.Merge(dst, src) +} +func (m *AdminScatterRequest) XXX_Size() int { + return m.Size() +} +func (m *AdminScatterRequest) XXX_DiscardUnknown() { + xxx_messageInfo_AdminScatterRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_AdminScatterRequest proto.InternalMessageInfo // ScatterResponse is the response to a Scatter() operation. type AdminScatterResponse struct { - ResponseHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` - Ranges []AdminScatterResponse_Range `protobuf:"bytes,2,rep,name=ranges" json:"ranges"` + ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + Ranges []AdminScatterResponse_Range `protobuf:"bytes,2,rep,name=ranges,proto3" json:"ranges"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{76} +} +func (m *AdminScatterResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *AdminScatterResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *AdminScatterResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_AdminScatterResponse.Merge(dst, src) +} +func (m *AdminScatterResponse) XXX_Size() int { + return m.Size() +} +func (m *AdminScatterResponse) XXX_DiscardUnknown() { + xxx_messageInfo_AdminScatterResponse.DiscardUnknown(m) } -func (m *AdminScatterResponse) Reset() { *m = AdminScatterResponse{} } -func (m *AdminScatterResponse) String() string { return proto.CompactTextString(m) } -func (*AdminScatterResponse) ProtoMessage() {} -func (*AdminScatterResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{76} } +var xxx_messageInfo_AdminScatterResponse proto.InternalMessageInfo type AdminScatterResponse_Range struct { - Span Span `protobuf:"bytes,1,opt,name=span" json:"span"` + Span Span `protobuf:"bytes,1,opt,name=span,proto3" json:"span"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *AdminScatterResponse_Range) Reset() { *m = AdminScatterResponse_Range{} } func (m *AdminScatterResponse_Range) String() string { return proto.CompactTextString(m) } func (*AdminScatterResponse_Range) ProtoMessage() {} func (*AdminScatterResponse_Range) Descriptor() ([]byte, []int) { - return fileDescriptorApi, []int{76, 0} + return fileDescriptor_api_a51b49ba855d15ec, []int{76, 0} +} +func (m *AdminScatterResponse_Range) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *AdminScatterResponse_Range) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *AdminScatterResponse_Range) XXX_Merge(src proto.Message) { + xxx_messageInfo_AdminScatterResponse_Range.Merge(dst, src) +} +func (m *AdminScatterResponse_Range) XXX_Size() int { + return m.Size() +} +func (m *AdminScatterResponse_Range) XXX_DiscardUnknown() { + xxx_messageInfo_AdminScatterResponse_Range.DiscardUnknown(m) } +var xxx_messageInfo_AdminScatterResponse_Range proto.InternalMessageInfo + // AddSSTableRequest is arguments to the AddSSTable() method, to link a file // into the RocksDB log-structured merge-tree. type AddSSTableRequest struct { - RequestHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` - Data []byte `protobuf:"bytes,2,opt,name=data,proto3" json:"data,omitempty"` + RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + Data []byte `protobuf:"bytes,2,opt,name=data,proto3" json:"data,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{77} +} +func (m *AddSSTableRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *AddSSTableRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *AddSSTableRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_AddSSTableRequest.Merge(dst, src) +} +func (m *AddSSTableRequest) XXX_Size() int { + return m.Size() +} +func (m *AddSSTableRequest) XXX_DiscardUnknown() { + xxx_messageInfo_AddSSTableRequest.DiscardUnknown(m) } -func (m *AddSSTableRequest) Reset() { *m = AddSSTableRequest{} } -func (m *AddSSTableRequest) String() string { return proto.CompactTextString(m) } -func (*AddSSTableRequest) ProtoMessage() {} -func (*AddSSTableRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{77} } +var xxx_messageInfo_AddSSTableRequest proto.InternalMessageInfo // AddSSTableResponse is the response to a AddSSTable() operation. type AddSSTableResponse struct { - ResponseHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{78} +} +func (m *AddSSTableResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *AddSSTableResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *AddSSTableResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_AddSSTableResponse.Merge(dst, src) +} +func (m *AddSSTableResponse) XXX_Size() int { + return m.Size() +} +func (m *AddSSTableResponse) XXX_DiscardUnknown() { + xxx_messageInfo_AddSSTableResponse.DiscardUnknown(m) } -func (m *AddSSTableResponse) Reset() { *m = AddSSTableResponse{} } -func (m *AddSSTableResponse) String() string { return proto.CompactTextString(m) } -func (*AddSSTableResponse) ProtoMessage() {} -func (*AddSSTableResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{78} } +var xxx_messageInfo_AddSSTableResponse proto.InternalMessageInfo // RefreshRequest is arguments to the Refresh() method, which verifies // that no write has occurred since the transaction's orig timestamp @@ -1916,53 +4072,157 @@ func (*AddSSTableResponse) Descriptor() ([]byte, []int) { return fileDescriptorA // recently than the original txn timestamp, an error is returned // and the timestamp cache is not updated. type RefreshRequest struct { - RequestHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` // If set, indicates that the write timestamp cache should be // updated. This handles the case of DeleteRange, which must set the // write too old flag on transactions which attempt earlier writes. - Write bool `protobuf:"varint,2,opt,name=write,proto3" json:"write,omitempty"` + Write bool `protobuf:"varint,2,opt,name=write,proto3" json:"write,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{79} +} +func (m *RefreshRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RefreshRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RefreshRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_RefreshRequest.Merge(dst, src) +} +func (m *RefreshRequest) XXX_Size() int { + return m.Size() +} +func (m *RefreshRequest) XXX_DiscardUnknown() { + xxx_messageInfo_RefreshRequest.DiscardUnknown(m) } -func (m *RefreshRequest) Reset() { *m = RefreshRequest{} } -func (m *RefreshRequest) String() string { return proto.CompactTextString(m) } -func (*RefreshRequest) ProtoMessage() {} -func (*RefreshRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{79} } +var xxx_messageInfo_RefreshRequest proto.InternalMessageInfo // RefreshResponse is the response to a Refresh() operation. type RefreshResponse struct { - ResponseHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{80} +} +func (m *RefreshResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RefreshResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RefreshResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_RefreshResponse.Merge(dst, src) +} +func (m *RefreshResponse) XXX_Size() int { + return m.Size() +} +func (m *RefreshResponse) XXX_DiscardUnknown() { + xxx_messageInfo_RefreshResponse.DiscardUnknown(m) } -func (m *RefreshResponse) Reset() { *m = RefreshResponse{} } -func (m *RefreshResponse) String() string { return proto.CompactTextString(m) } -func (*RefreshResponse) ProtoMessage() {} -func (*RefreshResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{80} } +var xxx_messageInfo_RefreshResponse proto.InternalMessageInfo // RefreshRangeRequest is arguments to the RefreshRange() method, which // is similar to RefreshRequest (see comments above), but operates on // a key span instead of a single key. type RefreshRangeRequest struct { - RequestHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` // If set, indicates that the write timestamp cache should be // updated. This handles the case of DeleteRange, which must set the // write too old flag on transactions which attempt earlier writes. - Write bool `protobuf:"varint,2,opt,name=write,proto3" json:"write,omitempty"` + Write bool `protobuf:"varint,2,opt,name=write,proto3" json:"write,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{81} +} +func (m *RefreshRangeRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RefreshRangeRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RefreshRangeRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_RefreshRangeRequest.Merge(dst, src) +} +func (m *RefreshRangeRequest) XXX_Size() int { + return m.Size() +} +func (m *RefreshRangeRequest) XXX_DiscardUnknown() { + xxx_messageInfo_RefreshRangeRequest.DiscardUnknown(m) } -func (m *RefreshRangeRequest) Reset() { *m = RefreshRangeRequest{} } -func (m *RefreshRangeRequest) String() string { return proto.CompactTextString(m) } -func (*RefreshRangeRequest) ProtoMessage() {} -func (*RefreshRangeRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{81} } +var xxx_messageInfo_RefreshRangeRequest proto.InternalMessageInfo // RefreshRangeResponse is the response to a RefreshRange() operation. type RefreshRangeResponse struct { - ResponseHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{82} +} +func (m *RefreshRangeResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RefreshRangeResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RefreshRangeResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_RefreshRangeResponse.Merge(dst, src) +} +func (m *RefreshRangeResponse) XXX_Size() int { + return m.Size() +} +func (m *RefreshRangeResponse) XXX_DiscardUnknown() { + xxx_messageInfo_RefreshRangeResponse.DiscardUnknown(m) } -func (m *RefreshRangeResponse) Reset() { *m = RefreshRangeResponse{} } -func (m *RefreshRangeResponse) String() string { return proto.CompactTextString(m) } -func (*RefreshRangeResponse) ProtoMessage() {} -func (*RefreshRangeResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{82} } +var xxx_messageInfo_RefreshRangeResponse proto.InternalMessageInfo // SubsumeRequest is the argument to the Subsume() method, which freezes a range // for merging with its left-hand neighbor. @@ -1975,22 +4235,48 @@ func (*RefreshRangeResponse) Descriptor() ([]byte, []int) { return fileDescripto // transaction. As a rule of thumb, it is incorrect to call Subsume, except from // its carefully-chosen location within a merge transaction. type SubsumeRequest struct { - RequestHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` // The range descriptor for the left-hand side of the merge. Used by the // right-hand side to sanity-check the validity of the merge. - LeftRange RangeDescriptor `protobuf:"bytes,2,opt,name=left_range,json=leftRange" json:"left_range"` + LeftRange RangeDescriptor `protobuf:"bytes,2,opt,name=left_range,json=leftRange,proto3" json:"left_range"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{83} +} +func (m *SubsumeRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SubsumeRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *SubsumeRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_SubsumeRequest.Merge(dst, src) +} +func (m *SubsumeRequest) XXX_Size() int { + return m.Size() +} +func (m *SubsumeRequest) XXX_DiscardUnknown() { + xxx_messageInfo_SubsumeRequest.DiscardUnknown(m) } -func (m *SubsumeRequest) Reset() { *m = SubsumeRequest{} } -func (m *SubsumeRequest) String() string { return proto.CompactTextString(m) } -func (*SubsumeRequest) ProtoMessage() {} -func (*SubsumeRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{83} } +var xxx_messageInfo_SubsumeRequest proto.InternalMessageInfo // SubsumeResponse is the response to a SubsumeRequest. type SubsumeResponse struct { - ResponseHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` // MVCCStats are the MVCC statistics for the range. - MVCCStats cockroach_storage_engine_enginepb1.MVCCStats `protobuf:"bytes,3,opt,name=mvcc_stats,json=mvccStats" json:"mvcc_stats"` + MVCCStats enginepb.MVCCStats `protobuf:"bytes,3,opt,name=mvcc_stats,json=mvccStats,proto3" json:"mvcc_stats"` // LeaseAppliedIndex is the lease index of the last applied command at the // time that the Subsume request executed. LeaseAppliedIndex uint64 `protobuf:"varint,4,opt,name=lease_applied_index,json=leaseAppliedIndex,proto3" json:"lease_applied_index,omitempty"` @@ -1999,39 +4285,117 @@ type SubsumeResponse struct { // before it stopped responding to requests altogether (in anticipation of // being subsumed). It is suitable for use as the timestamp cache's low water // mark for the keys previously owned by the subsumed range. - FreezeStart cockroach_util_hlc.Timestamp `protobuf:"bytes,5,opt,name=freeze_start,json=freezeStart" json:"freeze_start"` + FreezeStart hlc.Timestamp `protobuf:"bytes,5,opt,name=freeze_start,json=freezeStart,proto3" json:"freeze_start"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{84} +} +func (m *SubsumeResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SubsumeResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *SubsumeResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_SubsumeResponse.Merge(dst, src) +} +func (m *SubsumeResponse) XXX_Size() int { + return m.Size() +} +func (m *SubsumeResponse) XXX_DiscardUnknown() { + xxx_messageInfo_SubsumeResponse.DiscardUnknown(m) } -func (m *SubsumeResponse) Reset() { *m = SubsumeResponse{} } -func (m *SubsumeResponse) String() string { return proto.CompactTextString(m) } -func (*SubsumeResponse) ProtoMessage() {} -func (*SubsumeResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{84} } +var xxx_messageInfo_SubsumeResponse proto.InternalMessageInfo // RangeStatsRequest is the argument to the RangeStats() method. It requests the // MVCC statistics of the receiving range. type RangeStatsRequest struct { - RequestHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{85} +} +func (m *RangeStatsRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RangeStatsRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RangeStatsRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_RangeStatsRequest.Merge(dst, src) +} +func (m *RangeStatsRequest) XXX_Size() int { + return m.Size() +} +func (m *RangeStatsRequest) XXX_DiscardUnknown() { + xxx_messageInfo_RangeStatsRequest.DiscardUnknown(m) } -func (m *RangeStatsRequest) Reset() { *m = RangeStatsRequest{} } -func (m *RangeStatsRequest) String() string { return proto.CompactTextString(m) } -func (*RangeStatsRequest) ProtoMessage() {} -func (*RangeStatsRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{85} } +var xxx_messageInfo_RangeStatsRequest proto.InternalMessageInfo // RangeStatsResponse is the response to a RangeStatsRequest. type RangeStatsResponse struct { - ResponseHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` // MVCCStats are the MVCC statistics for the range that processed the // request. - MVCCStats cockroach_storage_engine_enginepb1.MVCCStats `protobuf:"bytes,2,opt,name=mvcc_stats,json=mvccStats" json:"mvcc_stats"` + MVCCStats enginepb.MVCCStats `protobuf:"bytes,2,opt,name=mvcc_stats,json=mvccStats,proto3" json:"mvcc_stats"` // QueriesPerSecond is the rate of request/s or QPS for the range. - QueriesPerSecond float64 `protobuf:"fixed64,3,opt,name=queries_per_second,json=queriesPerSecond,proto3" json:"queries_per_second,omitempty"` + QueriesPerSecond float64 `protobuf:"fixed64,3,opt,name=queries_per_second,json=queriesPerSecond,proto3" json:"queries_per_second,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{86} +} +func (m *RangeStatsResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RangeStatsResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RangeStatsResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_RangeStatsResponse.Merge(dst, src) +} +func (m *RangeStatsResponse) XXX_Size() int { + return m.Size() +} +func (m *RangeStatsResponse) XXX_DiscardUnknown() { + xxx_messageInfo_RangeStatsResponse.DiscardUnknown(m) } -func (m *RangeStatsResponse) Reset() { *m = RangeStatsResponse{} } -func (m *RangeStatsResponse) String() string { return proto.CompactTextString(m) } -func (*RangeStatsResponse) ProtoMessage() {} -func (*RangeStatsResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{86} } +var xxx_messageInfo_RangeStatsResponse proto.InternalMessageInfo // A RequestUnion contains exactly one of the requests. // The values added here must match those in ResponseUnion. @@ -2081,13 +4445,39 @@ type RequestUnion struct { // *RequestUnion_RefreshRange // *RequestUnion_Subsume // *RequestUnion_RangeStats - Value isRequestUnion_Value `protobuf_oneof:"value"` + Value isRequestUnion_Value `protobuf_oneof:"value"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{87} +} +func (m *RequestUnion) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RequestUnion) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RequestUnion) XXX_Merge(src proto.Message) { + xxx_messageInfo_RequestUnion.Merge(dst, src) +} +func (m *RequestUnion) XXX_Size() int { + return m.Size() +} +func (m *RequestUnion) XXX_DiscardUnknown() { + xxx_messageInfo_RequestUnion.DiscardUnknown(m) } -func (m *RequestUnion) Reset() { *m = RequestUnion{} } -func (m *RequestUnion) String() string { return proto.CompactTextString(m) } -func (*RequestUnion) ProtoMessage() {} -func (*RequestUnion) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{87} } +var xxx_messageInfo_RequestUnion proto.InternalMessageInfo type isRequestUnion_Value interface { isRequestUnion_Value() @@ -2096,127 +4486,127 @@ type isRequestUnion_Value interface { } type RequestUnion_Get struct { - Get *GetRequest `protobuf:"bytes,1,opt,name=get,oneof"` + Get *GetRequest `protobuf:"bytes,1,opt,name=get,proto3,oneof"` } type RequestUnion_Put struct { - Put *PutRequest `protobuf:"bytes,2,opt,name=put,oneof"` + Put *PutRequest `protobuf:"bytes,2,opt,name=put,proto3,oneof"` } type RequestUnion_ConditionalPut struct { - ConditionalPut *ConditionalPutRequest `protobuf:"bytes,3,opt,name=conditional_put,json=conditionalPut,oneof"` + ConditionalPut *ConditionalPutRequest `protobuf:"bytes,3,opt,name=conditional_put,json=conditionalPut,proto3,oneof"` } type RequestUnion_Increment struct { - Increment *IncrementRequest `protobuf:"bytes,4,opt,name=increment,oneof"` + Increment *IncrementRequest `protobuf:"bytes,4,opt,name=increment,proto3,oneof"` } type RequestUnion_Delete struct { - Delete *DeleteRequest `protobuf:"bytes,5,opt,name=delete,oneof"` + Delete *DeleteRequest `protobuf:"bytes,5,opt,name=delete,proto3,oneof"` } type RequestUnion_DeleteRange struct { - DeleteRange *DeleteRangeRequest `protobuf:"bytes,6,opt,name=delete_range,json=deleteRange,oneof"` + DeleteRange *DeleteRangeRequest `protobuf:"bytes,6,opt,name=delete_range,json=deleteRange,proto3,oneof"` } type RequestUnion_ClearRange struct { - ClearRange *ClearRangeRequest `protobuf:"bytes,38,opt,name=clear_range,json=clearRange,oneof"` + ClearRange *ClearRangeRequest `protobuf:"bytes,38,opt,name=clear_range,json=clearRange,proto3,oneof"` } type RequestUnion_Scan struct { - Scan *ScanRequest `protobuf:"bytes,7,opt,name=scan,oneof"` + Scan *ScanRequest `protobuf:"bytes,7,opt,name=scan,proto3,oneof"` } type RequestUnion_BeginTransaction struct { - BeginTransaction *BeginTransactionRequest `protobuf:"bytes,8,opt,name=begin_transaction,json=beginTransaction,oneof"` + BeginTransaction *BeginTransactionRequest `protobuf:"bytes,8,opt,name=begin_transaction,json=beginTransaction,proto3,oneof"` } type RequestUnion_EndTransaction struct { - EndTransaction *EndTransactionRequest `protobuf:"bytes,9,opt,name=end_transaction,json=endTransaction,oneof"` + EndTransaction *EndTransactionRequest `protobuf:"bytes,9,opt,name=end_transaction,json=endTransaction,proto3,oneof"` } type RequestUnion_AdminSplit struct { - AdminSplit *AdminSplitRequest `protobuf:"bytes,10,opt,name=admin_split,json=adminSplit,oneof"` + AdminSplit *AdminSplitRequest `protobuf:"bytes,10,opt,name=admin_split,json=adminSplit,proto3,oneof"` } type RequestUnion_AdminMerge struct { - AdminMerge *AdminMergeRequest `protobuf:"bytes,11,opt,name=admin_merge,json=adminMerge,oneof"` + AdminMerge *AdminMergeRequest `protobuf:"bytes,11,opt,name=admin_merge,json=adminMerge,proto3,oneof"` } type RequestUnion_AdminTransferLease struct { - AdminTransferLease *AdminTransferLeaseRequest `protobuf:"bytes,29,opt,name=admin_transfer_lease,json=adminTransferLease,oneof"` + AdminTransferLease *AdminTransferLeaseRequest `protobuf:"bytes,29,opt,name=admin_transfer_lease,json=adminTransferLease,proto3,oneof"` } type RequestUnion_AdminChangeReplicas struct { - AdminChangeReplicas *AdminChangeReplicasRequest `protobuf:"bytes,35,opt,name=admin_change_replicas,json=adminChangeReplicas,oneof"` + AdminChangeReplicas *AdminChangeReplicasRequest `protobuf:"bytes,35,opt,name=admin_change_replicas,json=adminChangeReplicas,proto3,oneof"` } type RequestUnion_AdminRelocateRange struct { - AdminRelocateRange *AdminRelocateRangeRequest `protobuf:"bytes,45,opt,name=admin_relocate_range,json=adminRelocateRange,oneof"` + AdminRelocateRange *AdminRelocateRangeRequest `protobuf:"bytes,45,opt,name=admin_relocate_range,json=adminRelocateRange,proto3,oneof"` } type RequestUnion_HeartbeatTxn struct { - HeartbeatTxn *HeartbeatTxnRequest `protobuf:"bytes,12,opt,name=heartbeat_txn,json=heartbeatTxn,oneof"` + HeartbeatTxn *HeartbeatTxnRequest `protobuf:"bytes,12,opt,name=heartbeat_txn,json=heartbeatTxn,proto3,oneof"` } type RequestUnion_Gc struct { - Gc *GCRequest `protobuf:"bytes,13,opt,name=gc,oneof"` + Gc *GCRequest `protobuf:"bytes,13,opt,name=gc,proto3,oneof"` } type RequestUnion_PushTxn struct { - PushTxn *PushTxnRequest `protobuf:"bytes,14,opt,name=push_txn,json=pushTxn,oneof"` + PushTxn *PushTxnRequest `protobuf:"bytes,14,opt,name=push_txn,json=pushTxn,proto3,oneof"` } type RequestUnion_ResolveIntent struct { - ResolveIntent *ResolveIntentRequest `protobuf:"bytes,16,opt,name=resolve_intent,json=resolveIntent,oneof"` + ResolveIntent *ResolveIntentRequest `protobuf:"bytes,16,opt,name=resolve_intent,json=resolveIntent,proto3,oneof"` } type RequestUnion_ResolveIntentRange struct { - ResolveIntentRange *ResolveIntentRangeRequest `protobuf:"bytes,17,opt,name=resolve_intent_range,json=resolveIntentRange,oneof"` + ResolveIntentRange *ResolveIntentRangeRequest `protobuf:"bytes,17,opt,name=resolve_intent_range,json=resolveIntentRange,proto3,oneof"` } type RequestUnion_Merge struct { - Merge *MergeRequest `protobuf:"bytes,18,opt,name=merge,oneof"` + Merge *MergeRequest `protobuf:"bytes,18,opt,name=merge,proto3,oneof"` } type RequestUnion_TruncateLog struct { - TruncateLog *TruncateLogRequest `protobuf:"bytes,19,opt,name=truncate_log,json=truncateLog,oneof"` + TruncateLog *TruncateLogRequest `protobuf:"bytes,19,opt,name=truncate_log,json=truncateLog,proto3,oneof"` } type RequestUnion_RequestLease struct { - RequestLease *RequestLeaseRequest `protobuf:"bytes,20,opt,name=request_lease,json=requestLease,oneof"` + RequestLease *RequestLeaseRequest `protobuf:"bytes,20,opt,name=request_lease,json=requestLease,proto3,oneof"` } type RequestUnion_ReverseScan struct { - ReverseScan *ReverseScanRequest `protobuf:"bytes,21,opt,name=reverse_scan,json=reverseScan,oneof"` + ReverseScan *ReverseScanRequest `protobuf:"bytes,21,opt,name=reverse_scan,json=reverseScan,proto3,oneof"` } type RequestUnion_ComputeChecksum struct { - ComputeChecksum *ComputeChecksumRequest `protobuf:"bytes,22,opt,name=compute_checksum,json=computeChecksum,oneof"` + ComputeChecksum *ComputeChecksumRequest `protobuf:"bytes,22,opt,name=compute_checksum,json=computeChecksum,proto3,oneof"` } type RequestUnion_CheckConsistency struct { - CheckConsistency *CheckConsistencyRequest `protobuf:"bytes,24,opt,name=check_consistency,json=checkConsistency,oneof"` + CheckConsistency *CheckConsistencyRequest `protobuf:"bytes,24,opt,name=check_consistency,json=checkConsistency,proto3,oneof"` } type RequestUnion_InitPut struct { - InitPut *InitPutRequest `protobuf:"bytes,26,opt,name=init_put,json=initPut,oneof"` + InitPut *InitPutRequest `protobuf:"bytes,26,opt,name=init_put,json=initPut,proto3,oneof"` } type RequestUnion_TransferLease struct { - TransferLease *TransferLeaseRequest `protobuf:"bytes,28,opt,name=transfer_lease,json=transferLease,oneof"` + TransferLease *TransferLeaseRequest `protobuf:"bytes,28,opt,name=transfer_lease,json=transferLease,proto3,oneof"` } type RequestUnion_LeaseInfo struct { - LeaseInfo *LeaseInfoRequest `protobuf:"bytes,30,opt,name=lease_info,json=leaseInfo,oneof"` + LeaseInfo *LeaseInfoRequest `protobuf:"bytes,30,opt,name=lease_info,json=leaseInfo,proto3,oneof"` } type RequestUnion_WriteBatch struct { - WriteBatch *WriteBatchRequest `protobuf:"bytes,31,opt,name=write_batch,json=writeBatch,oneof"` + WriteBatch *WriteBatchRequest `protobuf:"bytes,31,opt,name=write_batch,json=writeBatch,proto3,oneof"` } type RequestUnion_Export struct { - Export *ExportRequest `protobuf:"bytes,32,opt,name=export,oneof"` + Export *ExportRequest `protobuf:"bytes,32,opt,name=export,proto3,oneof"` } type RequestUnion_Import struct { - Import *ImportRequest `protobuf:"bytes,34,opt,name=import,oneof"` + Import *ImportRequest `protobuf:"bytes,34,opt,name=import,proto3,oneof"` } type RequestUnion_QueryTxn struct { - QueryTxn *QueryTxnRequest `protobuf:"bytes,33,opt,name=query_txn,json=queryTxn,oneof"` + QueryTxn *QueryTxnRequest `protobuf:"bytes,33,opt,name=query_txn,json=queryTxn,proto3,oneof"` } type RequestUnion_QueryIntent struct { - QueryIntent *QueryIntentRequest `protobuf:"bytes,42,opt,name=query_intent,json=queryIntent,oneof"` + QueryIntent *QueryIntentRequest `protobuf:"bytes,42,opt,name=query_intent,json=queryIntent,proto3,oneof"` } type RequestUnion_AdminScatter struct { - AdminScatter *AdminScatterRequest `protobuf:"bytes,36,opt,name=admin_scatter,json=adminScatter,oneof"` + AdminScatter *AdminScatterRequest `protobuf:"bytes,36,opt,name=admin_scatter,json=adminScatter,proto3,oneof"` } type RequestUnion_AddSstable struct { - AddSstable *AddSSTableRequest `protobuf:"bytes,37,opt,name=add_sstable,json=addSstable,oneof"` + AddSstable *AddSSTableRequest `protobuf:"bytes,37,opt,name=add_sstable,json=addSstable,proto3,oneof"` } type RequestUnion_RecomputeStats struct { - RecomputeStats *RecomputeStatsRequest `protobuf:"bytes,39,opt,name=recompute_stats,json=recomputeStats,oneof"` + RecomputeStats *RecomputeStatsRequest `protobuf:"bytes,39,opt,name=recompute_stats,json=recomputeStats,proto3,oneof"` } type RequestUnion_Refresh struct { - Refresh *RefreshRequest `protobuf:"bytes,40,opt,name=refresh,oneof"` + Refresh *RefreshRequest `protobuf:"bytes,40,opt,name=refresh,proto3,oneof"` } type RequestUnion_RefreshRange struct { - RefreshRange *RefreshRangeRequest `protobuf:"bytes,41,opt,name=refresh_range,json=refreshRange,oneof"` + RefreshRange *RefreshRangeRequest `protobuf:"bytes,41,opt,name=refresh_range,json=refreshRange,proto3,oneof"` } type RequestUnion_Subsume struct { - Subsume *SubsumeRequest `protobuf:"bytes,43,opt,name=subsume,oneof"` + Subsume *SubsumeRequest `protobuf:"bytes,43,opt,name=subsume,proto3,oneof"` } type RequestUnion_RangeStats struct { - RangeStats *RangeStatsRequest `protobuf:"bytes,44,opt,name=range_stats,json=rangeStats,oneof"` + RangeStats *RangeStatsRequest `protobuf:"bytes,44,opt,name=range_stats,json=rangeStats,proto3,oneof"` } func (*RequestUnion_Get) isRequestUnion_Value() {} @@ -3160,207 +5550,207 @@ func _RequestUnion_OneofSizer(msg proto.Message) (n int) { switch x := m.Value.(type) { case *RequestUnion_Get: s := proto.Size(x.Get) - n += proto.SizeVarint(1<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *RequestUnion_Put: s := proto.Size(x.Put) - n += proto.SizeVarint(2<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *RequestUnion_ConditionalPut: s := proto.Size(x.ConditionalPut) - n += proto.SizeVarint(3<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *RequestUnion_Increment: s := proto.Size(x.Increment) - n += proto.SizeVarint(4<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *RequestUnion_Delete: s := proto.Size(x.Delete) - n += proto.SizeVarint(5<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *RequestUnion_DeleteRange: s := proto.Size(x.DeleteRange) - n += proto.SizeVarint(6<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *RequestUnion_ClearRange: s := proto.Size(x.ClearRange) - n += proto.SizeVarint(38<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *RequestUnion_Scan: s := proto.Size(x.Scan) - n += proto.SizeVarint(7<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *RequestUnion_BeginTransaction: s := proto.Size(x.BeginTransaction) - n += proto.SizeVarint(8<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *RequestUnion_EndTransaction: s := proto.Size(x.EndTransaction) - n += proto.SizeVarint(9<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *RequestUnion_AdminSplit: s := proto.Size(x.AdminSplit) - n += proto.SizeVarint(10<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *RequestUnion_AdminMerge: s := proto.Size(x.AdminMerge) - n += proto.SizeVarint(11<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *RequestUnion_AdminTransferLease: s := proto.Size(x.AdminTransferLease) - n += proto.SizeVarint(29<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *RequestUnion_AdminChangeReplicas: s := proto.Size(x.AdminChangeReplicas) - n += proto.SizeVarint(35<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *RequestUnion_AdminRelocateRange: s := proto.Size(x.AdminRelocateRange) - n += proto.SizeVarint(45<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *RequestUnion_HeartbeatTxn: s := proto.Size(x.HeartbeatTxn) - n += proto.SizeVarint(12<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *RequestUnion_Gc: s := proto.Size(x.Gc) - n += proto.SizeVarint(13<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *RequestUnion_PushTxn: s := proto.Size(x.PushTxn) - n += proto.SizeVarint(14<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *RequestUnion_ResolveIntent: s := proto.Size(x.ResolveIntent) - n += proto.SizeVarint(16<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *RequestUnion_ResolveIntentRange: s := proto.Size(x.ResolveIntentRange) - n += proto.SizeVarint(17<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *RequestUnion_Merge: s := proto.Size(x.Merge) - n += proto.SizeVarint(18<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *RequestUnion_TruncateLog: s := proto.Size(x.TruncateLog) - n += proto.SizeVarint(19<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *RequestUnion_RequestLease: s := proto.Size(x.RequestLease) - n += proto.SizeVarint(20<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *RequestUnion_ReverseScan: s := proto.Size(x.ReverseScan) - n += proto.SizeVarint(21<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *RequestUnion_ComputeChecksum: s := proto.Size(x.ComputeChecksum) - n += proto.SizeVarint(22<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *RequestUnion_CheckConsistency: s := proto.Size(x.CheckConsistency) - n += proto.SizeVarint(24<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *RequestUnion_InitPut: s := proto.Size(x.InitPut) - n += proto.SizeVarint(26<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *RequestUnion_TransferLease: s := proto.Size(x.TransferLease) - n += proto.SizeVarint(28<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *RequestUnion_LeaseInfo: s := proto.Size(x.LeaseInfo) - n += proto.SizeVarint(30<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *RequestUnion_WriteBatch: s := proto.Size(x.WriteBatch) - n += proto.SizeVarint(31<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *RequestUnion_Export: s := proto.Size(x.Export) - n += proto.SizeVarint(32<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *RequestUnion_Import: s := proto.Size(x.Import) - n += proto.SizeVarint(34<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *RequestUnion_QueryTxn: s := proto.Size(x.QueryTxn) - n += proto.SizeVarint(33<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *RequestUnion_QueryIntent: s := proto.Size(x.QueryIntent) - n += proto.SizeVarint(42<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *RequestUnion_AdminScatter: s := proto.Size(x.AdminScatter) - n += proto.SizeVarint(36<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *RequestUnion_AddSstable: s := proto.Size(x.AddSstable) - n += proto.SizeVarint(37<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *RequestUnion_RecomputeStats: s := proto.Size(x.RecomputeStats) - n += proto.SizeVarint(39<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *RequestUnion_Refresh: s := proto.Size(x.Refresh) - n += proto.SizeVarint(40<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *RequestUnion_RefreshRange: s := proto.Size(x.RefreshRange) - n += proto.SizeVarint(41<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *RequestUnion_Subsume: s := proto.Size(x.Subsume) - n += proto.SizeVarint(43<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *RequestUnion_RangeStats: s := proto.Size(x.RangeStats) - n += proto.SizeVarint(44<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case nil: @@ -3414,13 +5804,39 @@ type ResponseUnion struct { // *ResponseUnion_RefreshRange // *ResponseUnion_Subsume // *ResponseUnion_RangeStats - Value isResponseUnion_Value `protobuf_oneof:"value"` + Value isResponseUnion_Value `protobuf_oneof:"value"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{88} +} +func (m *ResponseUnion) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ResponseUnion) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ResponseUnion) XXX_Merge(src proto.Message) { + xxx_messageInfo_ResponseUnion.Merge(dst, src) +} +func (m *ResponseUnion) XXX_Size() int { + return m.Size() +} +func (m *ResponseUnion) XXX_DiscardUnknown() { + xxx_messageInfo_ResponseUnion.DiscardUnknown(m) } -func (m *ResponseUnion) Reset() { *m = ResponseUnion{} } -func (m *ResponseUnion) String() string { return proto.CompactTextString(m) } -func (*ResponseUnion) ProtoMessage() {} -func (*ResponseUnion) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{88} } +var xxx_messageInfo_ResponseUnion proto.InternalMessageInfo type isResponseUnion_Value interface { isResponseUnion_Value() @@ -3429,124 +5845,124 @@ type isResponseUnion_Value interface { } type ResponseUnion_Get struct { - Get *GetResponse `protobuf:"bytes,1,opt,name=get,oneof"` + Get *GetResponse `protobuf:"bytes,1,opt,name=get,proto3,oneof"` } type ResponseUnion_Put struct { - Put *PutResponse `protobuf:"bytes,2,opt,name=put,oneof"` + Put *PutResponse `protobuf:"bytes,2,opt,name=put,proto3,oneof"` } type ResponseUnion_ConditionalPut struct { - ConditionalPut *ConditionalPutResponse `protobuf:"bytes,3,opt,name=conditional_put,json=conditionalPut,oneof"` + ConditionalPut *ConditionalPutResponse `protobuf:"bytes,3,opt,name=conditional_put,json=conditionalPut,proto3,oneof"` } type ResponseUnion_Increment struct { - Increment *IncrementResponse `protobuf:"bytes,4,opt,name=increment,oneof"` + Increment *IncrementResponse `protobuf:"bytes,4,opt,name=increment,proto3,oneof"` } type ResponseUnion_Delete struct { - Delete *DeleteResponse `protobuf:"bytes,5,opt,name=delete,oneof"` + Delete *DeleteResponse `protobuf:"bytes,5,opt,name=delete,proto3,oneof"` } type ResponseUnion_DeleteRange struct { - DeleteRange *DeleteRangeResponse `protobuf:"bytes,6,opt,name=delete_range,json=deleteRange,oneof"` + DeleteRange *DeleteRangeResponse `protobuf:"bytes,6,opt,name=delete_range,json=deleteRange,proto3,oneof"` } type ResponseUnion_ClearRange struct { - ClearRange *ClearRangeResponse `protobuf:"bytes,38,opt,name=clear_range,json=clearRange,oneof"` + ClearRange *ClearRangeResponse `protobuf:"bytes,38,opt,name=clear_range,json=clearRange,proto3,oneof"` } type ResponseUnion_Scan struct { - Scan *ScanResponse `protobuf:"bytes,7,opt,name=scan,oneof"` + Scan *ScanResponse `protobuf:"bytes,7,opt,name=scan,proto3,oneof"` } type ResponseUnion_BeginTransaction struct { - BeginTransaction *BeginTransactionResponse `protobuf:"bytes,8,opt,name=begin_transaction,json=beginTransaction,oneof"` + BeginTransaction *BeginTransactionResponse `protobuf:"bytes,8,opt,name=begin_transaction,json=beginTransaction,proto3,oneof"` } type ResponseUnion_EndTransaction struct { - EndTransaction *EndTransactionResponse `protobuf:"bytes,9,opt,name=end_transaction,json=endTransaction,oneof"` + EndTransaction *EndTransactionResponse `protobuf:"bytes,9,opt,name=end_transaction,json=endTransaction,proto3,oneof"` } type ResponseUnion_AdminSplit struct { - AdminSplit *AdminSplitResponse `protobuf:"bytes,10,opt,name=admin_split,json=adminSplit,oneof"` + AdminSplit *AdminSplitResponse `protobuf:"bytes,10,opt,name=admin_split,json=adminSplit,proto3,oneof"` } type ResponseUnion_AdminMerge struct { - AdminMerge *AdminMergeResponse `protobuf:"bytes,11,opt,name=admin_merge,json=adminMerge,oneof"` + AdminMerge *AdminMergeResponse `protobuf:"bytes,11,opt,name=admin_merge,json=adminMerge,proto3,oneof"` } type ResponseUnion_AdminTransferLease struct { - AdminTransferLease *AdminTransferLeaseResponse `protobuf:"bytes,29,opt,name=admin_transfer_lease,json=adminTransferLease,oneof"` + AdminTransferLease *AdminTransferLeaseResponse `protobuf:"bytes,29,opt,name=admin_transfer_lease,json=adminTransferLease,proto3,oneof"` } type ResponseUnion_AdminChangeReplicas struct { - AdminChangeReplicas *AdminChangeReplicasResponse `protobuf:"bytes,35,opt,name=admin_change_replicas,json=adminChangeReplicas,oneof"` + AdminChangeReplicas *AdminChangeReplicasResponse `protobuf:"bytes,35,opt,name=admin_change_replicas,json=adminChangeReplicas,proto3,oneof"` } type ResponseUnion_AdminRelocateRange struct { - AdminRelocateRange *AdminRelocateRangeResponse `protobuf:"bytes,45,opt,name=admin_relocate_range,json=adminRelocateRange,oneof"` + AdminRelocateRange *AdminRelocateRangeResponse `protobuf:"bytes,45,opt,name=admin_relocate_range,json=adminRelocateRange,proto3,oneof"` } type ResponseUnion_HeartbeatTxn struct { - HeartbeatTxn *HeartbeatTxnResponse `protobuf:"bytes,12,opt,name=heartbeat_txn,json=heartbeatTxn,oneof"` + HeartbeatTxn *HeartbeatTxnResponse `protobuf:"bytes,12,opt,name=heartbeat_txn,json=heartbeatTxn,proto3,oneof"` } type ResponseUnion_Gc struct { - Gc *GCResponse `protobuf:"bytes,13,opt,name=gc,oneof"` + Gc *GCResponse `protobuf:"bytes,13,opt,name=gc,proto3,oneof"` } type ResponseUnion_PushTxn struct { - PushTxn *PushTxnResponse `protobuf:"bytes,14,opt,name=push_txn,json=pushTxn,oneof"` + PushTxn *PushTxnResponse `protobuf:"bytes,14,opt,name=push_txn,json=pushTxn,proto3,oneof"` } type ResponseUnion_ResolveIntent struct { - ResolveIntent *ResolveIntentResponse `protobuf:"bytes,16,opt,name=resolve_intent,json=resolveIntent,oneof"` + ResolveIntent *ResolveIntentResponse `protobuf:"bytes,16,opt,name=resolve_intent,json=resolveIntent,proto3,oneof"` } type ResponseUnion_ResolveIntentRange struct { - ResolveIntentRange *ResolveIntentRangeResponse `protobuf:"bytes,17,opt,name=resolve_intent_range,json=resolveIntentRange,oneof"` + ResolveIntentRange *ResolveIntentRangeResponse `protobuf:"bytes,17,opt,name=resolve_intent_range,json=resolveIntentRange,proto3,oneof"` } type ResponseUnion_Merge struct { - Merge *MergeResponse `protobuf:"bytes,18,opt,name=merge,oneof"` + Merge *MergeResponse `protobuf:"bytes,18,opt,name=merge,proto3,oneof"` } type ResponseUnion_TruncateLog struct { - TruncateLog *TruncateLogResponse `protobuf:"bytes,19,opt,name=truncate_log,json=truncateLog,oneof"` + TruncateLog *TruncateLogResponse `protobuf:"bytes,19,opt,name=truncate_log,json=truncateLog,proto3,oneof"` } type ResponseUnion_RequestLease struct { - RequestLease *RequestLeaseResponse `protobuf:"bytes,20,opt,name=request_lease,json=requestLease,oneof"` + RequestLease *RequestLeaseResponse `protobuf:"bytes,20,opt,name=request_lease,json=requestLease,proto3,oneof"` } type ResponseUnion_ReverseScan struct { - ReverseScan *ReverseScanResponse `protobuf:"bytes,21,opt,name=reverse_scan,json=reverseScan,oneof"` + ReverseScan *ReverseScanResponse `protobuf:"bytes,21,opt,name=reverse_scan,json=reverseScan,proto3,oneof"` } type ResponseUnion_ComputeChecksum struct { - ComputeChecksum *ComputeChecksumResponse `protobuf:"bytes,22,opt,name=compute_checksum,json=computeChecksum,oneof"` + ComputeChecksum *ComputeChecksumResponse `protobuf:"bytes,22,opt,name=compute_checksum,json=computeChecksum,proto3,oneof"` } type ResponseUnion_CheckConsistency struct { - CheckConsistency *CheckConsistencyResponse `protobuf:"bytes,24,opt,name=check_consistency,json=checkConsistency,oneof"` + CheckConsistency *CheckConsistencyResponse `protobuf:"bytes,24,opt,name=check_consistency,json=checkConsistency,proto3,oneof"` } type ResponseUnion_InitPut struct { - InitPut *InitPutResponse `protobuf:"bytes,26,opt,name=init_put,json=initPut,oneof"` + InitPut *InitPutResponse `protobuf:"bytes,26,opt,name=init_put,json=initPut,proto3,oneof"` } type ResponseUnion_LeaseInfo struct { - LeaseInfo *LeaseInfoResponse `protobuf:"bytes,30,opt,name=lease_info,json=leaseInfo,oneof"` + LeaseInfo *LeaseInfoResponse `protobuf:"bytes,30,opt,name=lease_info,json=leaseInfo,proto3,oneof"` } type ResponseUnion_WriteBatch struct { - WriteBatch *WriteBatchResponse `protobuf:"bytes,31,opt,name=write_batch,json=writeBatch,oneof"` + WriteBatch *WriteBatchResponse `protobuf:"bytes,31,opt,name=write_batch,json=writeBatch,proto3,oneof"` } type ResponseUnion_Export struct { - Export *ExportResponse `protobuf:"bytes,32,opt,name=export,oneof"` + Export *ExportResponse `protobuf:"bytes,32,opt,name=export,proto3,oneof"` } type ResponseUnion_Import struct { - Import *ImportResponse `protobuf:"bytes,34,opt,name=import,oneof"` + Import *ImportResponse `protobuf:"bytes,34,opt,name=import,proto3,oneof"` } type ResponseUnion_QueryTxn struct { - QueryTxn *QueryTxnResponse `protobuf:"bytes,33,opt,name=query_txn,json=queryTxn,oneof"` + QueryTxn *QueryTxnResponse `protobuf:"bytes,33,opt,name=query_txn,json=queryTxn,proto3,oneof"` } type ResponseUnion_QueryIntent struct { - QueryIntent *QueryIntentResponse `protobuf:"bytes,42,opt,name=query_intent,json=queryIntent,oneof"` + QueryIntent *QueryIntentResponse `protobuf:"bytes,42,opt,name=query_intent,json=queryIntent,proto3,oneof"` } type ResponseUnion_AdminScatter struct { - AdminScatter *AdminScatterResponse `protobuf:"bytes,36,opt,name=admin_scatter,json=adminScatter,oneof"` + AdminScatter *AdminScatterResponse `protobuf:"bytes,36,opt,name=admin_scatter,json=adminScatter,proto3,oneof"` } type ResponseUnion_AddSstable struct { - AddSstable *AddSSTableResponse `protobuf:"bytes,37,opt,name=add_sstable,json=addSstable,oneof"` + AddSstable *AddSSTableResponse `protobuf:"bytes,37,opt,name=add_sstable,json=addSstable,proto3,oneof"` } type ResponseUnion_RecomputeStats struct { - RecomputeStats *RecomputeStatsResponse `protobuf:"bytes,39,opt,name=recompute_stats,json=recomputeStats,oneof"` + RecomputeStats *RecomputeStatsResponse `protobuf:"bytes,39,opt,name=recompute_stats,json=recomputeStats,proto3,oneof"` } type ResponseUnion_Refresh struct { - Refresh *RefreshResponse `protobuf:"bytes,40,opt,name=refresh,oneof"` + Refresh *RefreshResponse `protobuf:"bytes,40,opt,name=refresh,proto3,oneof"` } type ResponseUnion_RefreshRange struct { - RefreshRange *RefreshRangeResponse `protobuf:"bytes,41,opt,name=refresh_range,json=refreshRange,oneof"` + RefreshRange *RefreshRangeResponse `protobuf:"bytes,41,opt,name=refresh_range,json=refreshRange,proto3,oneof"` } type ResponseUnion_Subsume struct { - Subsume *SubsumeResponse `protobuf:"bytes,43,opt,name=subsume,oneof"` + Subsume *SubsumeResponse `protobuf:"bytes,43,opt,name=subsume,proto3,oneof"` } type ResponseUnion_RangeStats struct { - RangeStats *RangeStatsResponse `protobuf:"bytes,44,opt,name=range_stats,json=rangeStats,oneof"` + RangeStats *RangeStatsResponse `protobuf:"bytes,44,opt,name=range_stats,json=rangeStats,proto3,oneof"` } func (*ResponseUnion_Get) isResponseUnion_Value() {} @@ -4468,202 +6884,202 @@ func _ResponseUnion_OneofSizer(msg proto.Message) (n int) { switch x := m.Value.(type) { case *ResponseUnion_Get: s := proto.Size(x.Get) - n += proto.SizeVarint(1<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ResponseUnion_Put: s := proto.Size(x.Put) - n += proto.SizeVarint(2<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ResponseUnion_ConditionalPut: s := proto.Size(x.ConditionalPut) - n += proto.SizeVarint(3<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ResponseUnion_Increment: s := proto.Size(x.Increment) - n += proto.SizeVarint(4<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ResponseUnion_Delete: s := proto.Size(x.Delete) - n += proto.SizeVarint(5<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ResponseUnion_DeleteRange: s := proto.Size(x.DeleteRange) - n += proto.SizeVarint(6<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ResponseUnion_ClearRange: s := proto.Size(x.ClearRange) - n += proto.SizeVarint(38<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ResponseUnion_Scan: s := proto.Size(x.Scan) - n += proto.SizeVarint(7<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ResponseUnion_BeginTransaction: s := proto.Size(x.BeginTransaction) - n += proto.SizeVarint(8<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ResponseUnion_EndTransaction: s := proto.Size(x.EndTransaction) - n += proto.SizeVarint(9<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ResponseUnion_AdminSplit: s := proto.Size(x.AdminSplit) - n += proto.SizeVarint(10<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ResponseUnion_AdminMerge: s := proto.Size(x.AdminMerge) - n += proto.SizeVarint(11<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ResponseUnion_AdminTransferLease: s := proto.Size(x.AdminTransferLease) - n += proto.SizeVarint(29<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ResponseUnion_AdminChangeReplicas: s := proto.Size(x.AdminChangeReplicas) - n += proto.SizeVarint(35<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ResponseUnion_AdminRelocateRange: s := proto.Size(x.AdminRelocateRange) - n += proto.SizeVarint(45<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ResponseUnion_HeartbeatTxn: s := proto.Size(x.HeartbeatTxn) - n += proto.SizeVarint(12<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ResponseUnion_Gc: s := proto.Size(x.Gc) - n += proto.SizeVarint(13<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ResponseUnion_PushTxn: s := proto.Size(x.PushTxn) - n += proto.SizeVarint(14<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ResponseUnion_ResolveIntent: s := proto.Size(x.ResolveIntent) - n += proto.SizeVarint(16<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ResponseUnion_ResolveIntentRange: s := proto.Size(x.ResolveIntentRange) - n += proto.SizeVarint(17<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ResponseUnion_Merge: s := proto.Size(x.Merge) - n += proto.SizeVarint(18<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ResponseUnion_TruncateLog: s := proto.Size(x.TruncateLog) - n += proto.SizeVarint(19<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ResponseUnion_RequestLease: s := proto.Size(x.RequestLease) - n += proto.SizeVarint(20<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ResponseUnion_ReverseScan: s := proto.Size(x.ReverseScan) - n += proto.SizeVarint(21<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ResponseUnion_ComputeChecksum: s := proto.Size(x.ComputeChecksum) - n += proto.SizeVarint(22<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ResponseUnion_CheckConsistency: s := proto.Size(x.CheckConsistency) - n += proto.SizeVarint(24<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ResponseUnion_InitPut: s := proto.Size(x.InitPut) - n += proto.SizeVarint(26<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ResponseUnion_LeaseInfo: s := proto.Size(x.LeaseInfo) - n += proto.SizeVarint(30<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ResponseUnion_WriteBatch: s := proto.Size(x.WriteBatch) - n += proto.SizeVarint(31<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ResponseUnion_Export: s := proto.Size(x.Export) - n += proto.SizeVarint(32<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ResponseUnion_Import: s := proto.Size(x.Import) - n += proto.SizeVarint(34<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ResponseUnion_QueryTxn: s := proto.Size(x.QueryTxn) - n += proto.SizeVarint(33<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ResponseUnion_QueryIntent: s := proto.Size(x.QueryIntent) - n += proto.SizeVarint(42<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ResponseUnion_AdminScatter: s := proto.Size(x.AdminScatter) - n += proto.SizeVarint(36<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ResponseUnion_AddSstable: s := proto.Size(x.AddSstable) - n += proto.SizeVarint(37<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ResponseUnion_RecomputeStats: s := proto.Size(x.RecomputeStats) - n += proto.SizeVarint(39<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ResponseUnion_Refresh: s := proto.Size(x.Refresh) - n += proto.SizeVarint(40<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ResponseUnion_RefreshRange: s := proto.Size(x.RefreshRange) - n += proto.SizeVarint(41<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ResponseUnion_Subsume: s := proto.Size(x.Subsume) - n += proto.SizeVarint(43<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ResponseUnion_RangeStats: s := proto.Size(x.RangeStats) - n += proto.SizeVarint(44<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case nil: @@ -4679,9 +7095,9 @@ type Header struct { // timestamp specifies time at which read or writes should be // performed. If the timestamp is set to zero value, its value // is initialized to the wall time of the receiving node. - Timestamp cockroach_util_hlc.Timestamp `protobuf:"bytes,1,opt,name=timestamp" json:"timestamp"` + Timestamp hlc.Timestamp `protobuf:"bytes,1,opt,name=timestamp,proto3" json:"timestamp"` // replica specifies the destination of the request. - Replica ReplicaDescriptor `protobuf:"bytes,2,opt,name=replica" json:"replica"` + Replica ReplicaDescriptor `protobuf:"bytes,2,opt,name=replica,proto3" json:"replica"` // range_id specifies the ID of the Raft consensus group which the key // range belongs to. This is used by the receiving node to route the // request to the correct range. @@ -4701,7 +7117,7 @@ type Header struct { // isolation level set as desired. The response will contain the // fully-initialized transaction with txn ID, priority, initial // timestamp, and maximum timestamp. - Txn *Transaction `protobuf:"bytes,5,opt,name=txn" json:"txn,omitempty"` + Txn *Transaction `protobuf:"bytes,5,opt,name=txn,proto3" json:"txn,omitempty"` // read_consistency specifies the consistency for read // operations. The default is CONSISTENT. This value is ignored for // write operations. @@ -4728,7 +7144,7 @@ type Header struct { ReturnRangeInfo bool `protobuf:"varint,10,opt,name=return_range_info,json=returnRangeInfo,proto3" json:"return_range_info,omitempty"` // gateway_node_id is the ID of the gateway node where the request originated. GatewayNodeID NodeID `protobuf:"varint,11,opt,name=gateway_node_id,json=gatewayNodeId,proto3,casttype=NodeID" json:"gateway_node_id,omitempty"` - ScanOptions *ScanOptions `protobuf:"bytes,12,opt,name=scan_options,json=scanOptions" json:"scan_options,omitempty"` + ScanOptions *ScanOptions `protobuf:"bytes,12,opt,name=scan_options,json=scanOptions,proto3" json:"scan_options,omitempty"` // If set, the request will return to the client before proposing the // request into Raft. All consensus processing will be performed // asynchronously. Because consensus may fail, this means that the @@ -4737,38 +7153,116 @@ type Header struct { // TODO(nvanbenschoten): Handling cases where consensus fails would // be much more straightforward if all transactional requests were // idempotent. We could just re-issue requests. See #26915. - AsyncConsensus bool `protobuf:"varint,13,opt,name=async_consensus,json=asyncConsensus,proto3" json:"async_consensus,omitempty"` + AsyncConsensus bool `protobuf:"varint,13,opt,name=async_consensus,json=asyncConsensus,proto3" json:"async_consensus,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{89} +} +func (m *Header) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Header) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *Header) XXX_Merge(src proto.Message) { + xxx_messageInfo_Header.Merge(dst, src) +} +func (m *Header) XXX_Size() int { + return m.Size() +} +func (m *Header) XXX_DiscardUnknown() { + xxx_messageInfo_Header.DiscardUnknown(m) } -func (m *Header) Reset() { *m = Header{} } -func (m *Header) String() string { return proto.CompactTextString(m) } -func (*Header) ProtoMessage() {} -func (*Header) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{89} } +var xxx_messageInfo_Header proto.InternalMessageInfo // A BatchRequest contains one or more requests to be executed in // parallel, or if applicable (based on write-only commands and // range-locality), as a single update. type BatchRequest struct { - Header `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` - Requests []RequestUnion `protobuf:"bytes,2,rep,name=requests" json:"requests"` + Header `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + Requests []RequestUnion `protobuf:"bytes,2,rep,name=requests,proto3" json:"requests"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *BatchRequest) Reset() { *m = BatchRequest{} } +func (*BatchRequest) ProtoMessage() {} +func (*BatchRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_api_a51b49ba855d15ec, []int{90} +} +func (m *BatchRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *BatchRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *BatchRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_BatchRequest.Merge(dst, src) +} +func (m *BatchRequest) XXX_Size() int { + return m.Size() +} +func (m *BatchRequest) XXX_DiscardUnknown() { + xxx_messageInfo_BatchRequest.DiscardUnknown(m) } -func (m *BatchRequest) Reset() { *m = BatchRequest{} } -func (*BatchRequest) ProtoMessage() {} -func (*BatchRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{90} } +var xxx_messageInfo_BatchRequest proto.InternalMessageInfo // A BatchResponse contains one or more responses, one per request // corresponding to the requests in the matching BatchRequest. The // error in the response header is set to the first error from the // slice of responses, if applicable. type BatchResponse struct { - BatchResponse_Header `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` - Responses []ResponseUnion `protobuf:"bytes,2,rep,name=responses" json:"responses"` + BatchResponse_Header `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + Responses []ResponseUnion `protobuf:"bytes,2,rep,name=responses,proto3" json:"responses"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *BatchResponse) Reset() { *m = BatchResponse{} } +func (*BatchResponse) ProtoMessage() {} +func (*BatchResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_api_a51b49ba855d15ec, []int{91} +} +func (m *BatchResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *BatchResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *BatchResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_BatchResponse.Merge(dst, src) +} +func (m *BatchResponse) XXX_Size() int { + return m.Size() +} +func (m *BatchResponse) XXX_DiscardUnknown() { + xxx_messageInfo_BatchResponse.DiscardUnknown(m) } -func (m *BatchResponse) Reset() { *m = BatchResponse{} } -func (*BatchResponse) ProtoMessage() {} -func (*BatchResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{91} } +var xxx_messageInfo_BatchResponse proto.InternalMessageInfo type BatchResponse_Header struct { // error communicates a structured error (i.e. one originating from a Node) @@ -4783,91 +7277,247 @@ type BatchResponse_Header struct { // checked nor populated (it is reset by the DistSender, which extracts this // error and returns it separately). In effect, nearly no usage of // BatchResponse needs to care about this field. - Error *Error `protobuf:"bytes,1,opt,name=error" json:"error,omitempty"` + Error *Error `protobuf:"bytes,1,opt,name=error,proto3" json:"error,omitempty"` // timestamp is set only for non-transactional responses and denotes the // timestamp at which the batch executed. The timestamp cache is updated // at this timestamp. - Timestamp cockroach_util_hlc.Timestamp `protobuf:"bytes,2,opt,name=Timestamp" json:"Timestamp"` + Timestamp hlc.Timestamp `protobuf:"bytes,2,opt,name=Timestamp,proto3" json:"Timestamp"` // txn is non-nil if the request specified a non-nil // transaction. The transaction timestamp and/or priority may have // been updated, depending on the outcome of the request. - Txn *Transaction `protobuf:"bytes,3,opt,name=txn" json:"txn,omitempty"` + Txn *Transaction `protobuf:"bytes,3,opt,name=txn,proto3" json:"txn,omitempty"` // now is the highest current time from any node contacted during the request. // It can be used by the receiver to update its local HLC. - Now cockroach_util_hlc.Timestamp `protobuf:"bytes,5,opt,name=now" json:"now"` + Now hlc.Timestamp `protobuf:"bytes,5,opt,name=now,proto3" json:"now"` // collected_spans stores trace spans recorded during the execution of this // request. - CollectedSpans []cockroach_util_tracing.RecordedSpan `protobuf:"bytes,6,rep,name=collected_spans,json=collectedSpans" json:"collected_spans"` + CollectedSpans []tracing.RecordedSpan `protobuf:"bytes,6,rep,name=collected_spans,json=collectedSpans,proto3" json:"collected_spans"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{91, 0} +} +func (m *BatchResponse_Header) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *BatchResponse_Header) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *BatchResponse_Header) XXX_Merge(src proto.Message) { + xxx_messageInfo_BatchResponse_Header.Merge(dst, src) +} +func (m *BatchResponse_Header) XXX_Size() int { + return m.Size() +} +func (m *BatchResponse_Header) XXX_DiscardUnknown() { + xxx_messageInfo_BatchResponse_Header.DiscardUnknown(m) } -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 fileDescriptorApi, []int{91, 0} } +var xxx_messageInfo_BatchResponse_Header proto.InternalMessageInfo // RangeFeedRequest is a request that expresses the intention to establish a // RangeFeed stream over the provided span, starting at the specified timestamp. type RangeFeedRequest struct { - Header `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` - Span Span `protobuf:"bytes,2,opt,name=span" json:"span"` + Header `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + Span Span `protobuf:"bytes,2,opt,name=span,proto3" json:"span"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{92} +} +func (m *RangeFeedRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RangeFeedRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RangeFeedRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_RangeFeedRequest.Merge(dst, src) +} +func (m *RangeFeedRequest) XXX_Size() int { + return m.Size() +} +func (m *RangeFeedRequest) XXX_DiscardUnknown() { + xxx_messageInfo_RangeFeedRequest.DiscardUnknown(m) } -func (m *RangeFeedRequest) Reset() { *m = RangeFeedRequest{} } -func (m *RangeFeedRequest) String() string { return proto.CompactTextString(m) } -func (*RangeFeedRequest) ProtoMessage() {} -func (*RangeFeedRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{92} } +var xxx_messageInfo_RangeFeedRequest proto.InternalMessageInfo // RangeFeedValue is a variant of RangeFeedEvent that represents an update to // the specified key with the provided value. type RangeFeedValue struct { - Key Key `protobuf:"bytes,1,opt,name=key,proto3,casttype=Key" json:"key,omitempty"` - Value Value `protobuf:"bytes,2,opt,name=value" json:"value"` + Key Key `protobuf:"bytes,1,opt,name=key,proto3,casttype=Key" json:"key,omitempty"` + Value Value `protobuf:"bytes,2,opt,name=value,proto3" json:"value"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{93} +} +func (m *RangeFeedValue) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RangeFeedValue) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RangeFeedValue) XXX_Merge(src proto.Message) { + xxx_messageInfo_RangeFeedValue.Merge(dst, src) +} +func (m *RangeFeedValue) XXX_Size() int { + return m.Size() +} +func (m *RangeFeedValue) XXX_DiscardUnknown() { + xxx_messageInfo_RangeFeedValue.DiscardUnknown(m) } -func (m *RangeFeedValue) Reset() { *m = RangeFeedValue{} } -func (m *RangeFeedValue) String() string { return proto.CompactTextString(m) } -func (*RangeFeedValue) ProtoMessage() {} -func (*RangeFeedValue) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{93} } +var xxx_messageInfo_RangeFeedValue proto.InternalMessageInfo // RangeFeedCheckpoint is a variant of RangeFeedEvent that represents the // 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. type RangeFeedCheckpoint struct { - Span Span `protobuf:"bytes,1,opt,name=span" json:"span"` - ResolvedTS cockroach_util_hlc.Timestamp `protobuf:"bytes,2,opt,name=resolved_ts,json=resolvedTs" json:"resolved_ts"` + 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"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{94} +} +func (m *RangeFeedCheckpoint) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RangeFeedCheckpoint) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RangeFeedCheckpoint) XXX_Merge(src proto.Message) { + xxx_messageInfo_RangeFeedCheckpoint.Merge(dst, src) +} +func (m *RangeFeedCheckpoint) XXX_Size() int { + return m.Size() +} +func (m *RangeFeedCheckpoint) XXX_DiscardUnknown() { + xxx_messageInfo_RangeFeedCheckpoint.DiscardUnknown(m) } -func (m *RangeFeedCheckpoint) Reset() { *m = RangeFeedCheckpoint{} } -func (m *RangeFeedCheckpoint) String() string { return proto.CompactTextString(m) } -func (*RangeFeedCheckpoint) ProtoMessage() {} -func (*RangeFeedCheckpoint) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{94} } +var xxx_messageInfo_RangeFeedCheckpoint proto.InternalMessageInfo // RangeFeedError is a variant of RangeFeedEvent that indicates that an error // occurred during the processing of the RangeFeed. If emitted, a RangeFeedError // event will always be the final event on a RangeFeed response stream before // it is torn down. type RangeFeedError struct { - Error Error `protobuf:"bytes,1,opt,name=error" json:"error"` + Error Error `protobuf:"bytes,1,opt,name=error,proto3" json:"error"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{95} +} +func (m *RangeFeedError) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RangeFeedError) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RangeFeedError) XXX_Merge(src proto.Message) { + xxx_messageInfo_RangeFeedError.Merge(dst, src) +} +func (m *RangeFeedError) XXX_Size() int { + return m.Size() +} +func (m *RangeFeedError) XXX_DiscardUnknown() { + xxx_messageInfo_RangeFeedError.DiscardUnknown(m) } -func (m *RangeFeedError) Reset() { *m = RangeFeedError{} } -func (m *RangeFeedError) String() string { return proto.CompactTextString(m) } -func (*RangeFeedError) ProtoMessage() {} -func (*RangeFeedError) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{95} } +var xxx_messageInfo_RangeFeedError proto.InternalMessageInfo // RangeFeedEvent is a union of all event types that may be returned on a // RangeFeed response stream. type RangeFeedEvent struct { - Val *RangeFeedValue `protobuf:"bytes,1,opt,name=val" json:"val,omitempty"` - Checkpoint *RangeFeedCheckpoint `protobuf:"bytes,2,opt,name=checkpoint" json:"checkpoint,omitempty"` - Error *RangeFeedError `protobuf:"bytes,3,opt,name=error" json:"error,omitempty"` + Val *RangeFeedValue `protobuf:"bytes,1,opt,name=val,proto3" json:"val,omitempty"` + Checkpoint *RangeFeedCheckpoint `protobuf:"bytes,2,opt,name=checkpoint,proto3" json:"checkpoint,omitempty"` + Error *RangeFeedError `protobuf:"bytes,3,opt,name=error,proto3" json:"error,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_a51b49ba855d15ec, []int{96} +} +func (m *RangeFeedEvent) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RangeFeedEvent) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RangeFeedEvent) XXX_Merge(src proto.Message) { + xxx_messageInfo_RangeFeedEvent.Merge(dst, src) +} +func (m *RangeFeedEvent) XXX_Size() int { + return m.Size() +} +func (m *RangeFeedEvent) XXX_DiscardUnknown() { + xxx_messageInfo_RangeFeedEvent.DiscardUnknown(m) } -func (m *RangeFeedEvent) Reset() { *m = RangeFeedEvent{} } -func (m *RangeFeedEvent) String() string { return proto.CompactTextString(m) } -func (*RangeFeedEvent) ProtoMessage() {} -func (*RangeFeedEvent) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{96} } +var xxx_messageInfo_RangeFeedEvent proto.InternalMessageInfo func init() { proto.RegisterType((*RangeInfo)(nil), "cockroach.roachpb.RangeInfo") @@ -6632,8 +9282,9 @@ var _ grpc.ClientConn // is compatible with the grpc package it is being compiled against. const _ = grpc.SupportPackageIsVersion4 -// Client API for Internal service - +// InternalClient is the client API for Internal service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream. type InternalClient interface { Batch(ctx context.Context, in *BatchRequest, opts ...grpc.CallOption) (*BatchResponse, error) RangeFeed(ctx context.Context, in *RangeFeedRequest, opts ...grpc.CallOption) (Internal_RangeFeedClient, error) @@ -6649,7 +9300,7 @@ func NewInternalClient(cc *grpc.ClientConn) InternalClient { func (c *internalClient) Batch(ctx context.Context, in *BatchRequest, opts ...grpc.CallOption) (*BatchResponse, error) { out := new(BatchResponse) - err := grpc.Invoke(ctx, "/cockroach.roachpb.Internal/Batch", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/cockroach.roachpb.Internal/Batch", in, out, opts...) if err != nil { return nil, err } @@ -6657,7 +9308,7 @@ func (c *internalClient) Batch(ctx context.Context, in *BatchRequest, opts ...gr } func (c *internalClient) RangeFeed(ctx context.Context, in *RangeFeedRequest, opts ...grpc.CallOption) (Internal_RangeFeedClient, error) { - stream, err := grpc.NewClientStream(ctx, &_Internal_serviceDesc.Streams[0], c.cc, "/cockroach.roachpb.Internal/RangeFeed", opts...) + stream, err := c.cc.NewStream(ctx, &_Internal_serviceDesc.Streams[0], "/cockroach.roachpb.Internal/RangeFeed", opts...) if err != nil { return nil, err } @@ -6688,8 +9339,7 @@ func (x *internalRangeFeedClient) Recv() (*RangeFeedEvent, error) { return m, nil } -// Server API for Internal service - +// InternalServer is the server API for Internal service. type InternalServer interface { Batch(context.Context, *BatchRequest) (*BatchResponse, error) RangeFeed(*RangeFeedRequest, Internal_RangeFeedServer) error @@ -10467,7 +13117,7 @@ func (m *RangeStatsResponse) MarshalTo(dAtA []byte) (int, error) { if m.QueriesPerSecond != 0 { dAtA[i] = 0x19 i++ - binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.QueriesPerSecond)))) + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.QueriesPerSecond)))) i += 8 } return i, nil @@ -11802,7 +14452,7 @@ func (m *Header) MarshalTo(dAtA []byte) (int, error) { if m.UserPriority != 0 { dAtA[i] = 0x21 i++ - binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.UserPriority)))) + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.UserPriority)))) i += 8 } if m.Txn != nil { @@ -12199,6 +14849,9 @@ func encodeVarintApi(dAtA []byte, offset int, v uint64) int { return offset + 1 } func (m *RangeInfo) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.Desc.Size() @@ -12209,6 +14862,9 @@ func (m *RangeInfo) Size() (n int) { } func (m *RequestHeader) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Key) @@ -12226,6 +14882,9 @@ func (m *RequestHeader) Size() (n int) { } func (m *ResponseHeader) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Txn != nil { @@ -12252,6 +14911,9 @@ func (m *ResponseHeader) Size() (n int) { } func (m *GetRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.RequestHeader.Size() @@ -12260,6 +14922,9 @@ func (m *GetRequest) Size() (n int) { } func (m *GetResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.ResponseHeader.Size() @@ -12276,6 +14941,9 @@ func (m *GetResponse) Size() (n int) { } func (m *PutRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.RequestHeader.Size() @@ -12292,6 +14960,9 @@ func (m *PutRequest) Size() (n int) { } func (m *PutResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.ResponseHeader.Size() @@ -12300,6 +14971,9 @@ func (m *PutResponse) Size() (n int) { } func (m *ConditionalPutRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.RequestHeader.Size() @@ -12317,6 +14991,9 @@ func (m *ConditionalPutRequest) Size() (n int) { } func (m *ConditionalPutResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.ResponseHeader.Size() @@ -12325,6 +15002,9 @@ func (m *ConditionalPutResponse) Size() (n int) { } func (m *InitPutRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.RequestHeader.Size() @@ -12341,6 +15021,9 @@ func (m *InitPutRequest) Size() (n int) { } func (m *InitPutResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.ResponseHeader.Size() @@ -12349,6 +15032,9 @@ func (m *InitPutResponse) Size() (n int) { } func (m *IncrementRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.RequestHeader.Size() @@ -12360,6 +15046,9 @@ func (m *IncrementRequest) Size() (n int) { } func (m *IncrementResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.ResponseHeader.Size() @@ -12371,6 +15060,9 @@ func (m *IncrementResponse) Size() (n int) { } func (m *DeleteRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.RequestHeader.Size() @@ -12379,6 +15071,9 @@ func (m *DeleteRequest) Size() (n int) { } func (m *DeleteResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.ResponseHeader.Size() @@ -12387,6 +15082,9 @@ func (m *DeleteResponse) Size() (n int) { } func (m *DeleteRangeRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.RequestHeader.Size() @@ -12401,6 +15099,9 @@ func (m *DeleteRangeRequest) Size() (n int) { } func (m *DeleteRangeResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.ResponseHeader.Size() @@ -12415,6 +15116,9 @@ func (m *DeleteRangeResponse) Size() (n int) { } func (m *ClearRangeRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.RequestHeader.Size() @@ -12423,6 +15127,9 @@ func (m *ClearRangeRequest) Size() (n int) { } func (m *ClearRangeResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.ResponseHeader.Size() @@ -12431,6 +15138,9 @@ func (m *ClearRangeResponse) Size() (n int) { } func (m *ScanOptions) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.StopAtRangeBoundary { @@ -12443,6 +15153,9 @@ func (m *ScanOptions) Size() (n int) { } func (m *ScanRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.RequestHeader.Size() @@ -12454,6 +15167,9 @@ func (m *ScanRequest) Size() (n int) { } func (m *ScanResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.ResponseHeader.Size() @@ -12480,6 +15196,9 @@ func (m *ScanResponse) Size() (n int) { } func (m *ReverseScanRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.RequestHeader.Size() @@ -12491,6 +15210,9 @@ func (m *ReverseScanRequest) Size() (n int) { } func (m *ReverseScanResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.ResponseHeader.Size() @@ -12517,6 +15239,9 @@ func (m *ReverseScanResponse) Size() (n int) { } func (m *CheckConsistencyRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.RequestHeader.Size() @@ -12528,6 +15253,9 @@ func (m *CheckConsistencyRequest) Size() (n int) { } func (m *CheckConsistencyResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.ResponseHeader.Size() @@ -12536,6 +15264,9 @@ func (m *CheckConsistencyResponse) Size() (n int) { } func (m *RecomputeStatsRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.RequestHeader.Size() @@ -12547,6 +15278,9 @@ func (m *RecomputeStatsRequest) Size() (n int) { } func (m *RecomputeStatsResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.ResponseHeader.Size() @@ -12557,6 +15291,9 @@ func (m *RecomputeStatsResponse) Size() (n int) { } func (m *BeginTransactionRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.RequestHeader.Size() @@ -12565,6 +15302,9 @@ func (m *BeginTransactionRequest) Size() (n int) { } func (m *BeginTransactionResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.ResponseHeader.Size() @@ -12573,6 +15313,9 @@ func (m *BeginTransactionResponse) Size() (n int) { } func (m *EndTransactionRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.RequestHeader.Size() @@ -12607,6 +15350,9 @@ func (m *EndTransactionRequest) Size() (n int) { } func (m *EndTransactionResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.ResponseHeader.Size() @@ -12618,6 +15364,9 @@ func (m *EndTransactionResponse) Size() (n int) { } func (m *AdminSplitRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.RequestHeader.Size() @@ -12630,6 +15379,9 @@ func (m *AdminSplitRequest) Size() (n int) { } func (m *AdminSplitResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.ResponseHeader.Size() @@ -12638,6 +15390,9 @@ func (m *AdminSplitResponse) Size() (n int) { } func (m *AdminMergeRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.RequestHeader.Size() @@ -12646,6 +15401,9 @@ func (m *AdminMergeRequest) Size() (n int) { } func (m *AdminMergeResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.ResponseHeader.Size() @@ -12654,6 +15412,9 @@ func (m *AdminMergeResponse) Size() (n int) { } func (m *AdminTransferLeaseRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.RequestHeader.Size() @@ -12665,6 +15426,9 @@ func (m *AdminTransferLeaseRequest) Size() (n int) { } func (m *AdminTransferLeaseResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.ResponseHeader.Size() @@ -12673,6 +15437,9 @@ func (m *AdminTransferLeaseResponse) Size() (n int) { } func (m *AdminChangeReplicasRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.RequestHeader.Size() @@ -12690,6 +15457,9 @@ func (m *AdminChangeReplicasRequest) Size() (n int) { } func (m *AdminChangeReplicasResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.ResponseHeader.Size() @@ -12698,6 +15468,9 @@ func (m *AdminChangeReplicasResponse) Size() (n int) { } func (m *AdminRelocateRangeRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.RequestHeader.Size() @@ -12712,6 +15485,9 @@ func (m *AdminRelocateRangeRequest) Size() (n int) { } func (m *AdminRelocateRangeResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.ResponseHeader.Size() @@ -12720,6 +15496,9 @@ func (m *AdminRelocateRangeResponse) Size() (n int) { } func (m *HeartbeatTxnRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.RequestHeader.Size() @@ -12730,6 +15509,9 @@ func (m *HeartbeatTxnRequest) Size() (n int) { } func (m *HeartbeatTxnResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.ResponseHeader.Size() @@ -12738,6 +15520,9 @@ func (m *HeartbeatTxnResponse) Size() (n int) { } func (m *GCRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.RequestHeader.Size() @@ -12756,6 +15541,9 @@ func (m *GCRequest) Size() (n int) { } func (m *GCRequest_GCKey) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Key) @@ -12768,6 +15556,9 @@ func (m *GCRequest_GCKey) Size() (n int) { } func (m *GCResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.ResponseHeader.Size() @@ -12776,6 +15567,9 @@ func (m *GCResponse) Size() (n int) { } func (m *PushTxnRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.RequestHeader.Size() @@ -12798,6 +15592,9 @@ func (m *PushTxnRequest) Size() (n int) { } func (m *PushTxnResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.ResponseHeader.Size() @@ -12808,6 +15605,9 @@ func (m *PushTxnResponse) Size() (n int) { } func (m *QueryTxnRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.RequestHeader.Size() @@ -12827,6 +15627,9 @@ func (m *QueryTxnRequest) Size() (n int) { } func (m *QueryTxnResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.ResponseHeader.Size() @@ -12843,6 +15646,9 @@ func (m *QueryTxnResponse) Size() (n int) { } func (m *QueryIntentRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.RequestHeader.Size() @@ -12856,6 +15662,9 @@ func (m *QueryIntentRequest) Size() (n int) { } func (m *QueryIntentResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.ResponseHeader.Size() @@ -12867,6 +15676,9 @@ func (m *QueryIntentResponse) Size() (n int) { } func (m *ResolveIntentRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.RequestHeader.Size() @@ -12883,6 +15695,9 @@ func (m *ResolveIntentRequest) Size() (n int) { } func (m *ResolveIntentResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.ResponseHeader.Size() @@ -12891,6 +15706,9 @@ func (m *ResolveIntentResponse) Size() (n int) { } func (m *ResolveIntentRangeRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.RequestHeader.Size() @@ -12909,6 +15727,9 @@ func (m *ResolveIntentRangeRequest) Size() (n int) { } func (m *ResolveIntentRangeResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.ResponseHeader.Size() @@ -12917,6 +15738,9 @@ func (m *ResolveIntentRangeResponse) Size() (n int) { } func (m *MergeRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.RequestHeader.Size() @@ -12927,6 +15751,9 @@ func (m *MergeRequest) Size() (n int) { } func (m *MergeResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.ResponseHeader.Size() @@ -12935,6 +15762,9 @@ func (m *MergeResponse) Size() (n int) { } func (m *TruncateLogRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.RequestHeader.Size() @@ -12949,6 +15779,9 @@ func (m *TruncateLogRequest) Size() (n int) { } func (m *TruncateLogResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.ResponseHeader.Size() @@ -12957,6 +15790,9 @@ func (m *TruncateLogResponse) Size() (n int) { } func (m *RequestLeaseRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.RequestHeader.Size() @@ -12973,6 +15809,9 @@ func (m *RequestLeaseRequest) Size() (n int) { } func (m *TransferLeaseRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.RequestHeader.Size() @@ -12985,6 +15824,9 @@ func (m *TransferLeaseRequest) Size() (n int) { } func (m *LeaseInfoRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.RequestHeader.Size() @@ -12993,6 +15835,9 @@ func (m *LeaseInfoRequest) Size() (n int) { } func (m *LeaseInfoResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.ResponseHeader.Size() @@ -13003,6 +15848,9 @@ func (m *LeaseInfoResponse) Size() (n int) { } func (m *RequestLeaseResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.ResponseHeader.Size() @@ -13011,6 +15859,9 @@ func (m *RequestLeaseResponse) Size() (n int) { } func (m *ComputeChecksumRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.RequestHeader.Size() @@ -13025,6 +15876,9 @@ func (m *ComputeChecksumRequest) Size() (n int) { } func (m *ComputeChecksumResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.ResponseHeader.Size() @@ -13035,6 +15889,9 @@ func (m *ComputeChecksumResponse) Size() (n int) { } func (m *ExportStorage) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Provider != 0 { @@ -13064,6 +15921,9 @@ func (m *ExportStorage) Size() (n int) { } func (m *ExportStorage_LocalFilePath) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Path) @@ -13074,6 +15934,9 @@ func (m *ExportStorage_LocalFilePath) Size() (n int) { } func (m *ExportStorage_Http) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.BaseUri) @@ -13084,6 +15947,9 @@ func (m *ExportStorage_Http) Size() (n int) { } func (m *ExportStorage_S3) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Bucket) @@ -13118,6 +15984,9 @@ func (m *ExportStorage_S3) Size() (n int) { } func (m *ExportStorage_GCS) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Bucket) @@ -13144,6 +16013,9 @@ func (m *ExportStorage_GCS) Size() (n int) { } func (m *ExportStorage_Azure) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Container) @@ -13166,6 +16038,9 @@ func (m *ExportStorage_Azure) Size() (n int) { } func (m *ExportStorage_Workload) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Generator) @@ -13200,6 +16075,9 @@ func (m *ExportStorage_Workload) Size() (n int) { } func (m *WriteBatchRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.RequestHeader.Size() @@ -13214,6 +16092,9 @@ func (m *WriteBatchRequest) Size() (n int) { } func (m *WriteBatchResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.ResponseHeader.Size() @@ -13222,6 +16103,9 @@ func (m *WriteBatchResponse) Size() (n int) { } func (m *ExportRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.RequestHeader.Size() @@ -13243,6 +16127,9 @@ func (m *ExportRequest) Size() (n int) { } func (m *BulkOpSummary) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.DataSize != 0 { @@ -13261,6 +16148,9 @@ func (m *BulkOpSummary) Size() (n int) { } func (m *ExportResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.ResponseHeader.Size() @@ -13277,6 +16167,9 @@ func (m *ExportResponse) Size() (n int) { } func (m *ExportResponse_File) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.Span.Size() @@ -13299,6 +16192,9 @@ func (m *ExportResponse_File) Size() (n int) { } func (m *ImportRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.RequestHeader.Size() @@ -13323,6 +16219,9 @@ func (m *ImportRequest) Size() (n int) { } func (m *ImportRequest_File) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.Dir.Size() @@ -13339,6 +16238,9 @@ func (m *ImportRequest_File) Size() (n int) { } func (m *ImportRequest_TableRekey) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.OldID != 0 { @@ -13352,6 +16254,9 @@ func (m *ImportRequest_TableRekey) Size() (n int) { } func (m *ImportResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.ResponseHeader.Size() @@ -13362,6 +16267,9 @@ func (m *ImportResponse) Size() (n int) { } func (m *AdminScatterRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.RequestHeader.Size() @@ -13373,6 +16281,9 @@ func (m *AdminScatterRequest) Size() (n int) { } func (m *AdminScatterResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.ResponseHeader.Size() @@ -13387,6 +16298,9 @@ func (m *AdminScatterResponse) Size() (n int) { } func (m *AdminScatterResponse_Range) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.Span.Size() @@ -13395,6 +16309,9 @@ func (m *AdminScatterResponse_Range) Size() (n int) { } func (m *AddSSTableRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.RequestHeader.Size() @@ -13407,6 +16324,9 @@ func (m *AddSSTableRequest) Size() (n int) { } func (m *AddSSTableResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.ResponseHeader.Size() @@ -13415,6 +16335,9 @@ func (m *AddSSTableResponse) Size() (n int) { } func (m *RefreshRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.RequestHeader.Size() @@ -13426,6 +16349,9 @@ func (m *RefreshRequest) Size() (n int) { } func (m *RefreshResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.ResponseHeader.Size() @@ -13434,6 +16360,9 @@ func (m *RefreshResponse) Size() (n int) { } func (m *RefreshRangeRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.RequestHeader.Size() @@ -13445,6 +16374,9 @@ func (m *RefreshRangeRequest) Size() (n int) { } func (m *RefreshRangeResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.ResponseHeader.Size() @@ -13453,6 +16385,9 @@ func (m *RefreshRangeResponse) Size() (n int) { } func (m *SubsumeRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.RequestHeader.Size() @@ -13463,6 +16398,9 @@ func (m *SubsumeRequest) Size() (n int) { } func (m *SubsumeResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.ResponseHeader.Size() @@ -13478,6 +16416,9 @@ func (m *SubsumeResponse) Size() (n int) { } func (m *RangeStatsRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.RequestHeader.Size() @@ -13486,6 +16427,9 @@ func (m *RangeStatsRequest) Size() (n int) { } func (m *RangeStatsResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.ResponseHeader.Size() @@ -13499,6 +16443,9 @@ func (m *RangeStatsResponse) Size() (n int) { } func (m *RequestUnion) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Value != nil { @@ -13508,6 +16455,9 @@ func (m *RequestUnion) Size() (n int) { } func (m *RequestUnion_Get) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Get != nil { @@ -13517,6 +16467,9 @@ func (m *RequestUnion_Get) Size() (n int) { return n } func (m *RequestUnion_Put) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Put != nil { @@ -13526,6 +16479,9 @@ func (m *RequestUnion_Put) Size() (n int) { return n } func (m *RequestUnion_ConditionalPut) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.ConditionalPut != nil { @@ -13535,6 +16491,9 @@ func (m *RequestUnion_ConditionalPut) Size() (n int) { return n } func (m *RequestUnion_Increment) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Increment != nil { @@ -13544,6 +16503,9 @@ func (m *RequestUnion_Increment) Size() (n int) { return n } func (m *RequestUnion_Delete) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Delete != nil { @@ -13553,6 +16515,9 @@ func (m *RequestUnion_Delete) Size() (n int) { return n } func (m *RequestUnion_DeleteRange) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.DeleteRange != nil { @@ -13562,6 +16527,9 @@ func (m *RequestUnion_DeleteRange) Size() (n int) { return n } func (m *RequestUnion_Scan) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Scan != nil { @@ -13571,6 +16539,9 @@ func (m *RequestUnion_Scan) Size() (n int) { return n } func (m *RequestUnion_BeginTransaction) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.BeginTransaction != nil { @@ -13580,6 +16551,9 @@ func (m *RequestUnion_BeginTransaction) Size() (n int) { return n } func (m *RequestUnion_EndTransaction) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.EndTransaction != nil { @@ -13589,6 +16563,9 @@ func (m *RequestUnion_EndTransaction) Size() (n int) { return n } func (m *RequestUnion_AdminSplit) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.AdminSplit != nil { @@ -13598,6 +16575,9 @@ func (m *RequestUnion_AdminSplit) Size() (n int) { return n } func (m *RequestUnion_AdminMerge) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.AdminMerge != nil { @@ -13607,6 +16587,9 @@ func (m *RequestUnion_AdminMerge) Size() (n int) { return n } func (m *RequestUnion_HeartbeatTxn) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.HeartbeatTxn != nil { @@ -13616,6 +16599,9 @@ func (m *RequestUnion_HeartbeatTxn) Size() (n int) { return n } func (m *RequestUnion_Gc) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Gc != nil { @@ -13625,6 +16611,9 @@ func (m *RequestUnion_Gc) Size() (n int) { return n } func (m *RequestUnion_PushTxn) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.PushTxn != nil { @@ -13634,6 +16623,9 @@ func (m *RequestUnion_PushTxn) Size() (n int) { return n } func (m *RequestUnion_ResolveIntent) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.ResolveIntent != nil { @@ -13643,6 +16635,9 @@ func (m *RequestUnion_ResolveIntent) Size() (n int) { return n } func (m *RequestUnion_ResolveIntentRange) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.ResolveIntentRange != nil { @@ -13652,6 +16647,9 @@ func (m *RequestUnion_ResolveIntentRange) Size() (n int) { return n } func (m *RequestUnion_Merge) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Merge != nil { @@ -13661,6 +16659,9 @@ func (m *RequestUnion_Merge) Size() (n int) { return n } func (m *RequestUnion_TruncateLog) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.TruncateLog != nil { @@ -13670,6 +16671,9 @@ func (m *RequestUnion_TruncateLog) Size() (n int) { return n } func (m *RequestUnion_RequestLease) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.RequestLease != nil { @@ -13679,6 +16683,9 @@ func (m *RequestUnion_RequestLease) Size() (n int) { return n } func (m *RequestUnion_ReverseScan) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.ReverseScan != nil { @@ -13688,6 +16695,9 @@ func (m *RequestUnion_ReverseScan) Size() (n int) { return n } func (m *RequestUnion_ComputeChecksum) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.ComputeChecksum != nil { @@ -13697,6 +16707,9 @@ func (m *RequestUnion_ComputeChecksum) Size() (n int) { return n } func (m *RequestUnion_CheckConsistency) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.CheckConsistency != nil { @@ -13706,6 +16719,9 @@ func (m *RequestUnion_CheckConsistency) Size() (n int) { return n } func (m *RequestUnion_InitPut) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.InitPut != nil { @@ -13715,6 +16731,9 @@ func (m *RequestUnion_InitPut) Size() (n int) { return n } func (m *RequestUnion_TransferLease) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.TransferLease != nil { @@ -13724,6 +16743,9 @@ func (m *RequestUnion_TransferLease) Size() (n int) { return n } func (m *RequestUnion_AdminTransferLease) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.AdminTransferLease != nil { @@ -13733,6 +16755,9 @@ func (m *RequestUnion_AdminTransferLease) Size() (n int) { return n } func (m *RequestUnion_LeaseInfo) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.LeaseInfo != nil { @@ -13742,6 +16767,9 @@ func (m *RequestUnion_LeaseInfo) Size() (n int) { return n } func (m *RequestUnion_WriteBatch) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.WriteBatch != nil { @@ -13751,6 +16779,9 @@ func (m *RequestUnion_WriteBatch) Size() (n int) { return n } func (m *RequestUnion_Export) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Export != nil { @@ -13760,6 +16791,9 @@ func (m *RequestUnion_Export) Size() (n int) { return n } func (m *RequestUnion_QueryTxn) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.QueryTxn != nil { @@ -13769,6 +16803,9 @@ func (m *RequestUnion_QueryTxn) Size() (n int) { return n } func (m *RequestUnion_Import) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Import != nil { @@ -13778,6 +16815,9 @@ func (m *RequestUnion_Import) Size() (n int) { return n } func (m *RequestUnion_AdminChangeReplicas) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.AdminChangeReplicas != nil { @@ -13787,6 +16827,9 @@ func (m *RequestUnion_AdminChangeReplicas) Size() (n int) { return n } func (m *RequestUnion_AdminScatter) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.AdminScatter != nil { @@ -13796,6 +16839,9 @@ func (m *RequestUnion_AdminScatter) Size() (n int) { return n } func (m *RequestUnion_AddSstable) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.AddSstable != nil { @@ -13805,6 +16851,9 @@ func (m *RequestUnion_AddSstable) Size() (n int) { return n } func (m *RequestUnion_ClearRange) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.ClearRange != nil { @@ -13814,6 +16863,9 @@ func (m *RequestUnion_ClearRange) Size() (n int) { return n } func (m *RequestUnion_RecomputeStats) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.RecomputeStats != nil { @@ -13823,6 +16875,9 @@ func (m *RequestUnion_RecomputeStats) Size() (n int) { return n } func (m *RequestUnion_Refresh) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Refresh != nil { @@ -13832,6 +16887,9 @@ func (m *RequestUnion_Refresh) Size() (n int) { return n } func (m *RequestUnion_RefreshRange) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.RefreshRange != nil { @@ -13841,6 +16899,9 @@ func (m *RequestUnion_RefreshRange) Size() (n int) { return n } func (m *RequestUnion_QueryIntent) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.QueryIntent != nil { @@ -13850,6 +16911,9 @@ func (m *RequestUnion_QueryIntent) Size() (n int) { return n } func (m *RequestUnion_Subsume) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Subsume != nil { @@ -13859,6 +16923,9 @@ func (m *RequestUnion_Subsume) Size() (n int) { return n } func (m *RequestUnion_RangeStats) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.RangeStats != nil { @@ -13868,6 +16935,9 @@ func (m *RequestUnion_RangeStats) Size() (n int) { return n } func (m *RequestUnion_AdminRelocateRange) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.AdminRelocateRange != nil { @@ -13877,6 +16947,9 @@ func (m *RequestUnion_AdminRelocateRange) Size() (n int) { return n } func (m *ResponseUnion) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Value != nil { @@ -13886,6 +16959,9 @@ func (m *ResponseUnion) Size() (n int) { } func (m *ResponseUnion_Get) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Get != nil { @@ -13895,6 +16971,9 @@ func (m *ResponseUnion_Get) Size() (n int) { return n } func (m *ResponseUnion_Put) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Put != nil { @@ -13904,6 +16983,9 @@ func (m *ResponseUnion_Put) Size() (n int) { return n } func (m *ResponseUnion_ConditionalPut) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.ConditionalPut != nil { @@ -13913,6 +16995,9 @@ func (m *ResponseUnion_ConditionalPut) Size() (n int) { return n } func (m *ResponseUnion_Increment) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Increment != nil { @@ -13922,6 +17007,9 @@ func (m *ResponseUnion_Increment) Size() (n int) { return n } func (m *ResponseUnion_Delete) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Delete != nil { @@ -13931,6 +17019,9 @@ func (m *ResponseUnion_Delete) Size() (n int) { return n } func (m *ResponseUnion_DeleteRange) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.DeleteRange != nil { @@ -13940,6 +17031,9 @@ func (m *ResponseUnion_DeleteRange) Size() (n int) { return n } func (m *ResponseUnion_Scan) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Scan != nil { @@ -13949,6 +17043,9 @@ func (m *ResponseUnion_Scan) Size() (n int) { return n } func (m *ResponseUnion_BeginTransaction) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.BeginTransaction != nil { @@ -13958,6 +17055,9 @@ func (m *ResponseUnion_BeginTransaction) Size() (n int) { return n } func (m *ResponseUnion_EndTransaction) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.EndTransaction != nil { @@ -13967,6 +17067,9 @@ func (m *ResponseUnion_EndTransaction) Size() (n int) { return n } func (m *ResponseUnion_AdminSplit) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.AdminSplit != nil { @@ -13976,6 +17079,9 @@ func (m *ResponseUnion_AdminSplit) Size() (n int) { return n } func (m *ResponseUnion_AdminMerge) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.AdminMerge != nil { @@ -13985,6 +17091,9 @@ func (m *ResponseUnion_AdminMerge) Size() (n int) { return n } func (m *ResponseUnion_HeartbeatTxn) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.HeartbeatTxn != nil { @@ -13994,6 +17103,9 @@ func (m *ResponseUnion_HeartbeatTxn) Size() (n int) { return n } func (m *ResponseUnion_Gc) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Gc != nil { @@ -14003,6 +17115,9 @@ func (m *ResponseUnion_Gc) Size() (n int) { return n } func (m *ResponseUnion_PushTxn) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.PushTxn != nil { @@ -14012,6 +17127,9 @@ func (m *ResponseUnion_PushTxn) Size() (n int) { return n } func (m *ResponseUnion_ResolveIntent) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.ResolveIntent != nil { @@ -14021,6 +17139,9 @@ func (m *ResponseUnion_ResolveIntent) Size() (n int) { return n } func (m *ResponseUnion_ResolveIntentRange) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.ResolveIntentRange != nil { @@ -14030,6 +17151,9 @@ func (m *ResponseUnion_ResolveIntentRange) Size() (n int) { return n } func (m *ResponseUnion_Merge) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Merge != nil { @@ -14039,6 +17163,9 @@ func (m *ResponseUnion_Merge) Size() (n int) { return n } func (m *ResponseUnion_TruncateLog) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.TruncateLog != nil { @@ -14048,6 +17175,9 @@ func (m *ResponseUnion_TruncateLog) Size() (n int) { return n } func (m *ResponseUnion_RequestLease) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.RequestLease != nil { @@ -14057,6 +17187,9 @@ func (m *ResponseUnion_RequestLease) Size() (n int) { return n } func (m *ResponseUnion_ReverseScan) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.ReverseScan != nil { @@ -14066,6 +17199,9 @@ func (m *ResponseUnion_ReverseScan) Size() (n int) { return n } func (m *ResponseUnion_ComputeChecksum) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.ComputeChecksum != nil { @@ -14075,6 +17211,9 @@ func (m *ResponseUnion_ComputeChecksum) Size() (n int) { return n } func (m *ResponseUnion_CheckConsistency) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.CheckConsistency != nil { @@ -14084,6 +17223,9 @@ func (m *ResponseUnion_CheckConsistency) Size() (n int) { return n } func (m *ResponseUnion_InitPut) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.InitPut != nil { @@ -14093,6 +17235,9 @@ func (m *ResponseUnion_InitPut) Size() (n int) { return n } func (m *ResponseUnion_AdminTransferLease) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.AdminTransferLease != nil { @@ -14102,6 +17247,9 @@ func (m *ResponseUnion_AdminTransferLease) Size() (n int) { return n } func (m *ResponseUnion_LeaseInfo) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.LeaseInfo != nil { @@ -14111,6 +17259,9 @@ func (m *ResponseUnion_LeaseInfo) Size() (n int) { return n } func (m *ResponseUnion_WriteBatch) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.WriteBatch != nil { @@ -14120,6 +17271,9 @@ func (m *ResponseUnion_WriteBatch) Size() (n int) { return n } func (m *ResponseUnion_Export) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Export != nil { @@ -14129,6 +17283,9 @@ func (m *ResponseUnion_Export) Size() (n int) { return n } func (m *ResponseUnion_QueryTxn) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.QueryTxn != nil { @@ -14138,6 +17295,9 @@ func (m *ResponseUnion_QueryTxn) Size() (n int) { return n } func (m *ResponseUnion_Import) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Import != nil { @@ -14147,6 +17307,9 @@ func (m *ResponseUnion_Import) Size() (n int) { return n } func (m *ResponseUnion_AdminChangeReplicas) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.AdminChangeReplicas != nil { @@ -14156,6 +17319,9 @@ func (m *ResponseUnion_AdminChangeReplicas) Size() (n int) { return n } func (m *ResponseUnion_AdminScatter) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.AdminScatter != nil { @@ -14165,6 +17331,9 @@ func (m *ResponseUnion_AdminScatter) Size() (n int) { return n } func (m *ResponseUnion_AddSstable) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.AddSstable != nil { @@ -14174,6 +17343,9 @@ func (m *ResponseUnion_AddSstable) Size() (n int) { return n } func (m *ResponseUnion_ClearRange) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.ClearRange != nil { @@ -14183,6 +17355,9 @@ func (m *ResponseUnion_ClearRange) Size() (n int) { return n } func (m *ResponseUnion_RecomputeStats) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.RecomputeStats != nil { @@ -14192,6 +17367,9 @@ func (m *ResponseUnion_RecomputeStats) Size() (n int) { return n } func (m *ResponseUnion_Refresh) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Refresh != nil { @@ -14201,6 +17379,9 @@ func (m *ResponseUnion_Refresh) Size() (n int) { return n } func (m *ResponseUnion_RefreshRange) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.RefreshRange != nil { @@ -14210,6 +17391,9 @@ func (m *ResponseUnion_RefreshRange) Size() (n int) { return n } func (m *ResponseUnion_QueryIntent) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.QueryIntent != nil { @@ -14219,6 +17403,9 @@ func (m *ResponseUnion_QueryIntent) Size() (n int) { return n } func (m *ResponseUnion_Subsume) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Subsume != nil { @@ -14228,6 +17415,9 @@ func (m *ResponseUnion_Subsume) Size() (n int) { return n } func (m *ResponseUnion_RangeStats) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.RangeStats != nil { @@ -14237,6 +17427,9 @@ func (m *ResponseUnion_RangeStats) Size() (n int) { return n } func (m *ResponseUnion_AdminRelocateRange) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.AdminRelocateRange != nil { @@ -14246,6 +17439,9 @@ func (m *ResponseUnion_AdminRelocateRange) Size() (n int) { return n } func (m *Header) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.Timestamp.Size() @@ -14288,6 +17484,9 @@ func (m *Header) Size() (n int) { } func (m *BatchRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.Header.Size() @@ -14302,6 +17501,9 @@ func (m *BatchRequest) Size() (n int) { } func (m *BatchResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.BatchResponse_Header.Size() @@ -14316,6 +17518,9 @@ func (m *BatchResponse) Size() (n int) { } func (m *BatchResponse_Header) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Error != nil { @@ -14340,6 +17545,9 @@ func (m *BatchResponse_Header) Size() (n int) { } func (m *RangeFeedRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.Header.Size() @@ -14350,6 +17558,9 @@ func (m *RangeFeedRequest) Size() (n int) { } func (m *RangeFeedValue) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Key) @@ -14362,6 +17573,9 @@ func (m *RangeFeedValue) Size() (n int) { } func (m *RangeFeedCheckpoint) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.Span.Size() @@ -14372,6 +17586,9 @@ func (m *RangeFeedCheckpoint) Size() (n int) { } func (m *RangeFeedError) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.Error.Size() @@ -14380,6 +17597,9 @@ func (m *RangeFeedError) Size() (n int) { } func (m *RangeFeedEvent) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Val != nil { @@ -17823,7 +21043,7 @@ func (m *EndTransactionRequest) Unmarshal(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Deadline == nil { - m.Deadline = &cockroach_util_hlc.Timestamp{} + m.Deadline = &hlc.Timestamp{} } if err := m.Deadline.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err @@ -21461,7 +24681,7 @@ func (m *RequestLeaseRequest) Unmarshal(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.MinProposedTS == nil { - m.MinProposedTS = &cockroach_util_hlc.Timestamp{} + m.MinProposedTS = &hlc.Timestamp{} } if err := m.MinProposedTS.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err @@ -26121,7 +29341,7 @@ func (m *RangeStatsResponse) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - v = uint64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 m.QueriesPerSecond = float64(math.Float64frombits(v)) default: @@ -28953,7 +32173,7 @@ func (m *Header) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - v = uint64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 m.UserPriority = UserPriority(math.Float64frombits(v)) case 5: @@ -29563,7 +32783,7 @@ func (m *BatchResponse_Header) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.CollectedSpans = append(m.CollectedSpans, cockroach_util_tracing.RecordedSpan{}) + m.CollectedSpans = append(m.CollectedSpans, tracing.RecordedSpan{}) if err := m.CollectedSpans[len(m.CollectedSpans)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -30254,9 +33474,9 @@ var ( ErrIntOverflowApi = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptorApi) } +func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_a51b49ba855d15ec) } -var fileDescriptorApi = []byte{ +var fileDescriptor_api_a51b49ba855d15ec = []byte{ // 6157 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x3d, 0x4b, 0x6c, 0x1c, 0xc9, 0x75, 0xec, 0x99, 0x21, 0x39, 0xf3, 0xe6, 0xc3, 0x66, 0x51, 0x9f, 0x11, 0xa5, 0x15, 0xa9, 0xd1, diff --git a/pkg/roachpb/app_stats.pb.go b/pkg/roachpb/app_stats.pb.go index b8eabf506926..07c9201f0e9f 100644 --- a/pkg/roachpb/app_stats.pb.go +++ b/pkg/roachpb/app_stats.pb.go @@ -7,7 +7,7 @@ import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" -import binary "encoding/binary" +import encoding_binary "encoding/binary" import io "io" @@ -16,6 +16,12 @@ var _ = proto.Marshal var _ = fmt.Errorf var _ = math.Inf +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package + type StatementStatistics struct { // Count is the total number of times this statement was executed // since the begin of the reporting period. @@ -61,13 +67,39 @@ type StatementStatistics struct { OverheadLat NumericStat `protobuf:"bytes,10,opt,name=overhead_lat,json=overheadLat" json:"overhead_lat"` // SensitiveInfo is info that needs to be scrubbed or redacted before being // sent to the reg cluster. - SensitiveInfo SensitiveInfo `protobuf:"bytes,12,opt,name=sensitive_info,json=sensitiveInfo" json:"sensitive_info"` + SensitiveInfo SensitiveInfo `protobuf:"bytes,12,opt,name=sensitive_info,json=sensitiveInfo" json:"sensitive_info"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *StatementStatistics) Reset() { *m = StatementStatistics{} } -func (m *StatementStatistics) String() string { return proto.CompactTextString(m) } -func (*StatementStatistics) ProtoMessage() {} -func (*StatementStatistics) Descriptor() ([]byte, []int) { return fileDescriptorAppStats, []int{0} } +func (m *StatementStatistics) Reset() { *m = StatementStatistics{} } +func (m *StatementStatistics) String() string { return proto.CompactTextString(m) } +func (*StatementStatistics) ProtoMessage() {} +func (*StatementStatistics) Descriptor() ([]byte, []int) { + return fileDescriptor_app_stats_0e394ba73b607eaf, []int{0} +} +func (m *StatementStatistics) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *StatementStatistics) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *StatementStatistics) XXX_Merge(src proto.Message) { + xxx_messageInfo_StatementStatistics.Merge(dst, src) +} +func (m *StatementStatistics) XXX_Size() int { + return m.Size() +} +func (m *StatementStatistics) XXX_DiscardUnknown() { + xxx_messageInfo_StatementStatistics.DiscardUnknown(m) +} + +var xxx_messageInfo_StatementStatistics proto.InternalMessageInfo type SensitiveInfo struct { // LastErr collects the last error encountered. @@ -75,12 +107,38 @@ type SensitiveInfo struct { LastErr string `protobuf:"bytes,1,opt,name=last_err,json=lastErr" json:"last_err"` // MostRecentPlanDescription is a serialized representation of the logical plan most recently captured for this query. MostRecentPlanDescription ExplainTreePlanNode `protobuf:"bytes,2,opt,name=most_recent_plan_description,json=mostRecentPlanDescription" json:"most_recent_plan_description"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *SensitiveInfo) Reset() { *m = SensitiveInfo{} } +func (m *SensitiveInfo) String() string { return proto.CompactTextString(m) } +func (*SensitiveInfo) ProtoMessage() {} +func (*SensitiveInfo) Descriptor() ([]byte, []int) { + return fileDescriptor_app_stats_0e394ba73b607eaf, []int{1} +} +func (m *SensitiveInfo) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SensitiveInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *SensitiveInfo) XXX_Merge(src proto.Message) { + xxx_messageInfo_SensitiveInfo.Merge(dst, src) +} +func (m *SensitiveInfo) XXX_Size() int { + return m.Size() +} +func (m *SensitiveInfo) XXX_DiscardUnknown() { + xxx_messageInfo_SensitiveInfo.DiscardUnknown(m) } -func (m *SensitiveInfo) Reset() { *m = SensitiveInfo{} } -func (m *SensitiveInfo) String() string { return proto.CompactTextString(m) } -func (*SensitiveInfo) ProtoMessage() {} -func (*SensitiveInfo) Descriptor() ([]byte, []int) { return fileDescriptorAppStats, []int{1} } +var xxx_messageInfo_SensitiveInfo proto.InternalMessageInfo type NumericStat struct { // NumericStat keeps track of two running values --- the running mean and @@ -89,40 +147,116 @@ type NumericStat struct { // method. This is more reliable than keeping track of the sum of // squared values, which is liable to overflow. See // https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance#Online_algorithm - Mean float64 `protobuf:"fixed64,1,opt,name=mean" json:"mean"` - SquaredDiffs float64 `protobuf:"fixed64,2,opt,name=squared_diffs,json=squaredDiffs" json:"squared_diffs"` + Mean float64 `protobuf:"fixed64,1,opt,name=mean" json:"mean"` + SquaredDiffs float64 `protobuf:"fixed64,2,opt,name=squared_diffs,json=squaredDiffs" json:"squared_diffs"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *NumericStat) Reset() { *m = NumericStat{} } -func (m *NumericStat) String() string { return proto.CompactTextString(m) } -func (*NumericStat) ProtoMessage() {} -func (*NumericStat) Descriptor() ([]byte, []int) { return fileDescriptorAppStats, []int{2} } +func (m *NumericStat) Reset() { *m = NumericStat{} } +func (m *NumericStat) String() string { return proto.CompactTextString(m) } +func (*NumericStat) ProtoMessage() {} +func (*NumericStat) Descriptor() ([]byte, []int) { + return fileDescriptor_app_stats_0e394ba73b607eaf, []int{2} +} +func (m *NumericStat) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *NumericStat) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *NumericStat) XXX_Merge(src proto.Message) { + xxx_messageInfo_NumericStat.Merge(dst, src) +} +func (m *NumericStat) XXX_Size() int { + return m.Size() +} +func (m *NumericStat) XXX_DiscardUnknown() { + xxx_messageInfo_NumericStat.DiscardUnknown(m) +} + +var xxx_messageInfo_NumericStat proto.InternalMessageInfo type StatementStatisticsKey struct { - Query string `protobuf:"bytes,1,opt,name=query" json:"query"` - App string `protobuf:"bytes,2,opt,name=app" json:"app"` - DistSQL bool `protobuf:"varint,3,opt,name=distSQL" json:"distSQL"` - Failed bool `protobuf:"varint,4,opt,name=failed" json:"failed"` - Opt bool `protobuf:"varint,5,opt,name=opt" json:"opt"` + Query string `protobuf:"bytes,1,opt,name=query" json:"query"` + App string `protobuf:"bytes,2,opt,name=app" json:"app"` + DistSQL bool `protobuf:"varint,3,opt,name=distSQL" json:"distSQL"` + Failed bool `protobuf:"varint,4,opt,name=failed" json:"failed"` + Opt bool `protobuf:"varint,5,opt,name=opt" json:"opt"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *StatementStatisticsKey) Reset() { *m = StatementStatisticsKey{} } +func (m *StatementStatisticsKey) String() string { return proto.CompactTextString(m) } +func (*StatementStatisticsKey) ProtoMessage() {} +func (*StatementStatisticsKey) Descriptor() ([]byte, []int) { + return fileDescriptor_app_stats_0e394ba73b607eaf, []int{3} +} +func (m *StatementStatisticsKey) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *StatementStatisticsKey) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *StatementStatisticsKey) XXX_Merge(src proto.Message) { + xxx_messageInfo_StatementStatisticsKey.Merge(dst, src) +} +func (m *StatementStatisticsKey) XXX_Size() int { + return m.Size() +} +func (m *StatementStatisticsKey) XXX_DiscardUnknown() { + xxx_messageInfo_StatementStatisticsKey.DiscardUnknown(m) } -func (m *StatementStatisticsKey) Reset() { *m = StatementStatisticsKey{} } -func (m *StatementStatisticsKey) String() string { return proto.CompactTextString(m) } -func (*StatementStatisticsKey) ProtoMessage() {} -func (*StatementStatisticsKey) Descriptor() ([]byte, []int) { return fileDescriptorAppStats, []int{3} } +var xxx_messageInfo_StatementStatisticsKey proto.InternalMessageInfo // CollectedStats wraps collected timings and metadata for some query's execution. type CollectedStatementStatistics struct { - Key StatementStatisticsKey `protobuf:"bytes,1,opt,name=key" json:"key"` - Stats StatementStatistics `protobuf:"bytes,2,opt,name=stats" json:"stats"` + Key StatementStatisticsKey `protobuf:"bytes,1,opt,name=key" json:"key"` + Stats StatementStatistics `protobuf:"bytes,2,opt,name=stats" json:"stats"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *CollectedStatementStatistics) Reset() { *m = CollectedStatementStatistics{} } func (m *CollectedStatementStatistics) String() string { return proto.CompactTextString(m) } func (*CollectedStatementStatistics) ProtoMessage() {} func (*CollectedStatementStatistics) Descriptor() ([]byte, []int) { - return fileDescriptorAppStats, []int{4} + return fileDescriptor_app_stats_0e394ba73b607eaf, []int{4} +} +func (m *CollectedStatementStatistics) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *CollectedStatementStatistics) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *CollectedStatementStatistics) XXX_Merge(src proto.Message) { + xxx_messageInfo_CollectedStatementStatistics.Merge(dst, src) +} +func (m *CollectedStatementStatistics) XXX_Size() int { + return m.Size() } +func (m *CollectedStatementStatistics) XXX_DiscardUnknown() { + xxx_messageInfo_CollectedStatementStatistics.DiscardUnknown(m) +} + +var xxx_messageInfo_CollectedStatementStatistics proto.InternalMessageInfo // ExplainTreePlanNode is a serialized representation of an EXPLAIN tree for a logical plan. type ExplainTreePlanNode struct { @@ -132,25 +266,75 @@ type ExplainTreePlanNode struct { // Often there are many attributes with the same key, e.g. "render". Attrs []*ExplainTreePlanNode_Attr `protobuf:"bytes,2,rep,name=attrs" json:"attrs,omitempty"` // Children are the nodes that feed into this one, e.g. two scans for a join. - Children []*ExplainTreePlanNode `protobuf:"bytes,3,rep,name=children" json:"children,omitempty"` + Children []*ExplainTreePlanNode `protobuf:"bytes,3,rep,name=children" json:"children,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ExplainTreePlanNode) Reset() { *m = ExplainTreePlanNode{} } +func (m *ExplainTreePlanNode) String() string { return proto.CompactTextString(m) } +func (*ExplainTreePlanNode) ProtoMessage() {} +func (*ExplainTreePlanNode) Descriptor() ([]byte, []int) { + return fileDescriptor_app_stats_0e394ba73b607eaf, []int{5} +} +func (m *ExplainTreePlanNode) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ExplainTreePlanNode) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ExplainTreePlanNode) XXX_Merge(src proto.Message) { + xxx_messageInfo_ExplainTreePlanNode.Merge(dst, src) +} +func (m *ExplainTreePlanNode) XXX_Size() int { + return m.Size() +} +func (m *ExplainTreePlanNode) XXX_DiscardUnknown() { + xxx_messageInfo_ExplainTreePlanNode.DiscardUnknown(m) } -func (m *ExplainTreePlanNode) Reset() { *m = ExplainTreePlanNode{} } -func (m *ExplainTreePlanNode) String() string { return proto.CompactTextString(m) } -func (*ExplainTreePlanNode) ProtoMessage() {} -func (*ExplainTreePlanNode) Descriptor() ([]byte, []int) { return fileDescriptorAppStats, []int{5} } +var xxx_messageInfo_ExplainTreePlanNode proto.InternalMessageInfo type ExplainTreePlanNode_Attr struct { - Key string `protobuf:"bytes,1,opt,name=key" json:"key"` - Value string `protobuf:"bytes,2,opt,name=value" json:"value"` + Key string `protobuf:"bytes,1,opt,name=key" json:"key"` + Value string `protobuf:"bytes,2,opt,name=value" json:"value"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *ExplainTreePlanNode_Attr) Reset() { *m = ExplainTreePlanNode_Attr{} } func (m *ExplainTreePlanNode_Attr) String() string { return proto.CompactTextString(m) } func (*ExplainTreePlanNode_Attr) ProtoMessage() {} func (*ExplainTreePlanNode_Attr) Descriptor() ([]byte, []int) { - return fileDescriptorAppStats, []int{5, 0} + return fileDescriptor_app_stats_0e394ba73b607eaf, []int{5, 0} +} +func (m *ExplainTreePlanNode_Attr) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ExplainTreePlanNode_Attr) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ExplainTreePlanNode_Attr) XXX_Merge(src proto.Message) { + xxx_messageInfo_ExplainTreePlanNode_Attr.Merge(dst, src) +} +func (m *ExplainTreePlanNode_Attr) XXX_Size() int { + return m.Size() } +func (m *ExplainTreePlanNode_Attr) XXX_DiscardUnknown() { + xxx_messageInfo_ExplainTreePlanNode_Attr.DiscardUnknown(m) +} + +var xxx_messageInfo_ExplainTreePlanNode_Attr proto.InternalMessageInfo func init() { proto.RegisterType((*StatementStatistics)(nil), "cockroach.sql.StatementStatistics") @@ -299,11 +483,11 @@ func (m *NumericStat) MarshalTo(dAtA []byte) (int, error) { _ = l dAtA[i] = 0x9 i++ - binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.Mean)))) + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.Mean)))) i += 8 dAtA[i] = 0x11 i++ - binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.SquaredDiffs)))) + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.SquaredDiffs)))) i += 8 return i, nil } @@ -474,6 +658,9 @@ func encodeVarintAppStats(dAtA []byte, offset int, v uint64) int { return offset + 1 } func (m *StatementStatistics) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovAppStats(uint64(m.Count)) @@ -501,6 +688,9 @@ func (m *StatementStatistics) Size() (n int) { } func (m *SensitiveInfo) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.LastErr) @@ -511,6 +701,9 @@ func (m *SensitiveInfo) Size() (n int) { } func (m *NumericStat) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 9 @@ -519,6 +712,9 @@ func (m *NumericStat) Size() (n int) { } func (m *StatementStatisticsKey) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Query) @@ -532,6 +728,9 @@ func (m *StatementStatisticsKey) Size() (n int) { } func (m *CollectedStatementStatistics) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.Key.Size() @@ -542,6 +741,9 @@ func (m *CollectedStatementStatistics) Size() (n int) { } func (m *ExplainTreePlanNode) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Name) @@ -562,6 +764,9 @@ func (m *ExplainTreePlanNode) Size() (n int) { } func (m *ExplainTreePlanNode_Attr) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Key) @@ -1105,7 +1310,7 @@ func (m *NumericStat) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - v = uint64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 m.Mean = float64(math.Float64frombits(v)) case 2: @@ -1116,7 +1321,7 @@ func (m *NumericStat) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - v = uint64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 m.SquaredDiffs = float64(math.Float64frombits(v)) default: @@ -1772,9 +1977,9 @@ var ( ErrIntOverflowAppStats = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/app_stats.proto", fileDescriptorAppStats) } +func init() { proto.RegisterFile("roachpb/app_stats.proto", fileDescriptor_app_stats_0e394ba73b607eaf) } -var fileDescriptorAppStats = []byte{ +var fileDescriptor_app_stats_0e394ba73b607eaf = []byte{ // 733 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x55, 0x4d, 0x6f, 0xd3, 0x4a, 0x14, 0xad, 0x5f, 0x92, 0x26, 0xb9, 0x6e, 0xde, 0xd3, 0x9b, 0xbe, 0xd7, 0xe7, 0x17, 0x45, 0x69, diff --git a/pkg/roachpb/batch_test.go b/pkg/roachpb/batch_test.go index ce4841e9268c..fdecf81a5433 100644 --- a/pkg/roachpb/batch_test.go +++ b/pkg/roachpb/batch_test.go @@ -85,15 +85,20 @@ func TestBatchSplit(t *testing.T) { } func TestBatchRequestGetArg(t *testing.T) { + get := RequestUnion{ + Value: &RequestUnion_Get{Get: &GetRequest{}}, + } + end := RequestUnion{ + Value: &RequestUnion_EndTransaction{EndTransaction: &EndTransactionRequest{}}, + } testCases := []struct { bu []RequestUnion expB, expG bool }{ {[]RequestUnion{}, false, false}, - {[]RequestUnion{{&RequestUnion_Get{Get: &GetRequest{}}}}, false, true}, - {[]RequestUnion{{&RequestUnion_EndTransaction{EndTransaction: &EndTransactionRequest{}}}, {&RequestUnion_Get{Get: &GetRequest{}}}}, false, true}, - {[]RequestUnion{{&RequestUnion_EndTransaction{EndTransaction: &EndTransactionRequest{}}}}, true, false}, - {[]RequestUnion{{&RequestUnion_Get{Get: &GetRequest{}}}, {&RequestUnion_EndTransaction{EndTransaction: &EndTransactionRequest{}}}}, true, true}, + {[]RequestUnion{get}, false, true}, + {[]RequestUnion{end, get}, false, true}, + {[]RequestUnion{get, end}, true, true}, } for i, c := range testCases { @@ -168,12 +173,9 @@ func TestIntentSpanIterate(t *testing.T) { span Span resume Span }{ - {&ScanRequest{}, &ScanResponse{}, - Span{Key("a"), Key("c")}, Span{Key("b"), Key("c")}}, - {&ReverseScanRequest{}, &ReverseScanResponse{}, - Span{Key("d"), Key("f")}, Span{Key("d"), Key("e")}}, - {&DeleteRangeRequest{}, &DeleteRangeResponse{}, - Span{Key("g"), Key("i")}, Span{Key("h"), Key("i")}}, + {&ScanRequest{}, &ScanResponse{}, sp("a", "c"), sp("b", "c")}, + {&ReverseScanRequest{}, &ReverseScanResponse{}, sp("d", "f"), sp("d", "e")}, + {&DeleteRangeRequest{}, &DeleteRangeResponse{}, sp("g", "i"), sp("h", "i")}, } // A batch request with a batch response with no ResumeSpan. @@ -214,7 +216,7 @@ func TestIntentSpanIterate(t *testing.T) { if e := 1; len(spans) != e { t.Fatalf("unexpected number of spans: e = %d, found = %d", e, len(spans)) } - if e := (Span{Key("g"), Key("h")}); !reflect.DeepEqual(e, spans[0]) { + if e := sp("g", "h"); !reflect.DeepEqual(e, spans[0]) { t.Fatalf("unexpected spans: e = %+v, found = %+v", e, spans[0]) } } @@ -224,24 +226,16 @@ func TestRefreshSpanIterate(t *testing.T) { req Request resp Response span Span - resume *Span + resume Span }{ - {&ConditionalPutRequest{}, &ConditionalPutResponse{}, - Span{Key: Key("a")}, nil}, - {&PutRequest{}, &PutResponse{}, - Span{Key: Key("a-put")}, nil}, - {&InitPutRequest{}, &InitPutResponse{}, - Span{Key: Key("a-initput")}, nil}, - {&IncrementRequest{}, &IncrementResponse{}, - Span{Key: Key("a-inc")}, nil}, - {&ScanRequest{}, &ScanResponse{}, - Span{Key("a"), Key("c")}, &Span{Key("b"), Key("c")}}, - {&GetRequest{}, &GetResponse{}, - Span{Key: Key("b")}, nil}, - {&ReverseScanRequest{}, &ReverseScanResponse{}, - Span{Key("d"), Key("f")}, &Span{Key("d"), Key("e")}}, - {&DeleteRangeRequest{}, &DeleteRangeResponse{}, - Span{Key("g"), Key("i")}, &Span{Key("h"), Key("i")}}, + {&ConditionalPutRequest{}, &ConditionalPutResponse{}, sp("a", ""), Span{}}, + {&PutRequest{}, &PutResponse{}, sp("a-put", ""), Span{}}, + {&InitPutRequest{}, &InitPutResponse{}, sp("a-initput", ""), Span{}}, + {&IncrementRequest{}, &IncrementResponse{}, sp("a-inc", ""), Span{}}, + {&ScanRequest{}, &ScanResponse{}, sp("a", "c"), sp("b", "c")}, + {&GetRequest{}, &GetResponse{}, sp("b", ""), Span{}}, + {&ReverseScanRequest{}, &ReverseScanResponse{}, sp("d", "f"), sp("d", "e")}, + {&DeleteRangeRequest{}, &DeleteRangeResponse{}, sp("g", "i"), sp("h", "i")}, } // A batch request with a batch response with no ResumeSpan. @@ -280,8 +274,9 @@ func TestRefreshSpanIterate(t *testing.T) { for _, tc := range testCases { tc.req.SetHeader(RequestHeaderFromSpan(tc.span)) ba.Add(tc.req) - if tc.resume != nil { - tc.resp.SetHeader(ResponseHeader{ResumeSpan: tc.resume}) + if tc.resume.Key != nil { + resume := tc.resume + tc.resp.SetHeader(ResponseHeader{ResumeSpan: &resume}) } br.Add(tc.resp) } @@ -290,12 +285,12 @@ func TestRefreshSpanIterate(t *testing.T) { writeSpans = []Span{} ba.RefreshSpanIterate(&br, fn) expReadSpans = []Span{ - {Key("a"), Key("b")}, - {Key: Key("b")}, - {Key("e"), Key("f")}, + sp("a", "b"), + sp("b", ""), + sp("e", "f"), } expWriteSpans = []Span{ - {Key("g"), Key("h")}, + sp("g", "h"), } if !reflect.DeepEqual(expReadSpans, readSpans) { t.Fatalf("unexpected read spans: expected %+v, found = %+v", expReadSpans, readSpans) @@ -378,3 +373,11 @@ func TestBatchResponseCombine(t *testing.T) { t.Fatal(err) } } + +func sp(start, end string) Span { + res := Span{Key: Key(start)} + if end != "" { + res.EndKey = Key(end) + } + return res +} diff --git a/pkg/roachpb/data.go b/pkg/roachpb/data.go index 0438573270ad..16c2771d37b8 100644 --- a/pkg/roachpb/data.go +++ b/pkg/roachpb/data.go @@ -1206,7 +1206,7 @@ func readWithinUncertaintyIntervalRetryTimestamp( if !ok { log.Fatalf(ctx, "missing observed timestamp for node %d found on uncertainty restart. "+ - "err: %s. txn: %s. Observed timestamps: %s", + "err: %s. txn: %s. Observed timestamps: %v", origin, err, txn, txn.ObservedTimestamps) } // Also forward by the existing timestamp. diff --git a/pkg/roachpb/data.pb.go b/pkg/roachpb/data.pb.go index 277257d0838e..322a181fbfa8 100644 --- a/pkg/roachpb/data.pb.go +++ b/pkg/roachpb/data.pb.go @@ -6,9 +6,8 @@ package roachpb import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" -import cockroach_storage_engine_enginepb1 "github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb" -import cockroach_storage_engine_enginepb "github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb" -import cockroach_util_hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" +import enginepb "github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb" +import hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" import github_com_cockroachdb_cockroach_pkg_util_uuid "github.com/cockroachdb/cockroach/pkg/util/uuid" @@ -21,6 +20,12 @@ var _ = proto.Marshal var _ = fmt.Errorf var _ = math.Inf +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package + // ValueType defines a set of type constants placed in the "tag" field of Value // messages. These are defined as a protocol buffer enumeration so that they // can be used portably between our Go and C code. The tags are used by the @@ -80,7 +85,9 @@ var ValueType_value = map[string]int32{ func (x ValueType) String() string { return proto.EnumName(ValueType_name, int32(x)) } -func (ValueType) EnumDescriptor() ([]byte, []int) { return fileDescriptorData, []int{0} } +func (ValueType) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_data_7d67e43646134c7e, []int{0} +} // ReplicaChangeType is a parameter of ChangeReplicasTrigger. type ReplicaChangeType int32 @@ -102,7 +109,9 @@ var ReplicaChangeType_value = map[string]int32{ func (x ReplicaChangeType) String() string { return proto.EnumName(ReplicaChangeType_name, int32(x)) } -func (ReplicaChangeType) EnumDescriptor() ([]byte, []int) { return fileDescriptorData, []int{1} } +func (ReplicaChangeType) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_data_7d67e43646134c7e, []int{1} +} // TransactionStatus specifies possible states for a transaction. type TransactionStatus int32 @@ -140,7 +149,9 @@ var TransactionStatus_value = map[string]int32{ func (x TransactionStatus) String() string { return proto.EnumName(TransactionStatus_name, int32(x)) } -func (TransactionStatus) EnumDescriptor() ([]byte, []int) { return fileDescriptorData, []int{2} } +func (TransactionStatus) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_data_7d67e43646134c7e, []int{2} +} // Span is a key range with an inclusive start Key and an exclusive end Key. type Span struct { @@ -150,12 +161,38 @@ type Span struct { // contains only a single key. Otherwise, it must order strictly after Key. // In such a case, the Span encompasses the key range from Key to EndKey, // including Key and excluding EndKey. - EndKey Key `protobuf:"bytes,4,opt,name=end_key,json=endKey,proto3,casttype=Key" json:"end_key,omitempty"` + EndKey Key `protobuf:"bytes,4,opt,name=end_key,json=endKey,proto3,casttype=Key" json:"end_key,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *Span) Reset() { *m = Span{} } -func (*Span) ProtoMessage() {} -func (*Span) Descriptor() ([]byte, []int) { return fileDescriptorData, []int{0} } +func (m *Span) Reset() { *m = Span{} } +func (*Span) ProtoMessage() {} +func (*Span) Descriptor() ([]byte, []int) { + return fileDescriptor_data_7d67e43646134c7e, []int{0} +} +func (m *Span) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Span) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *Span) XXX_Merge(src proto.Message) { + xxx_messageInfo_Span.Merge(dst, src) +} +func (m *Span) XXX_Size() int { + return m.Size() +} +func (m *Span) XXX_DiscardUnknown() { + xxx_messageInfo_Span.DiscardUnknown(m) +} + +var xxx_messageInfo_Span proto.InternalMessageInfo // Value specifies the value at a key. Multiple values at the same key are // supported based on timestamp. The data stored within a value is typed @@ -175,39 +212,117 @@ type Value struct { // raw_bytes contains the encoded value and checksum. RawBytes []byte `protobuf:"bytes,1,opt,name=raw_bytes,json=rawBytes,proto3" json:"raw_bytes,omitempty"` // Timestamp of value. - Timestamp cockroach_util_hlc.Timestamp `protobuf:"bytes,2,opt,name=timestamp" json:"timestamp"` + Timestamp hlc.Timestamp `protobuf:"bytes,2,opt,name=timestamp,proto3" json:"timestamp"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *Value) Reset() { *m = Value{} } -func (m *Value) String() string { return proto.CompactTextString(m) } -func (*Value) ProtoMessage() {} -func (*Value) Descriptor() ([]byte, []int) { return fileDescriptorData, []int{1} } +func (m *Value) Reset() { *m = Value{} } +func (m *Value) String() string { return proto.CompactTextString(m) } +func (*Value) ProtoMessage() {} +func (*Value) Descriptor() ([]byte, []int) { + return fileDescriptor_data_7d67e43646134c7e, []int{1} +} +func (m *Value) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Value) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *Value) XXX_Merge(src proto.Message) { + xxx_messageInfo_Value.Merge(dst, src) +} +func (m *Value) XXX_Size() int { + return m.Size() +} +func (m *Value) XXX_DiscardUnknown() { + xxx_messageInfo_Value.DiscardUnknown(m) +} + +var xxx_messageInfo_Value proto.InternalMessageInfo // KeyValue is a pair of Key and Value for returned Key/Value pairs // from ScanRequest/ScanResponse. It embeds a Key and a Value. type KeyValue struct { - Key Key `protobuf:"bytes,1,opt,name=key,proto3,casttype=Key" json:"key,omitempty"` - Value Value `protobuf:"bytes,2,opt,name=value" json:"value"` + Key Key `protobuf:"bytes,1,opt,name=key,proto3,casttype=Key" json:"key,omitempty"` + Value Value `protobuf:"bytes,2,opt,name=value,proto3" json:"value"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *KeyValue) Reset() { *m = KeyValue{} } +func (m *KeyValue) String() string { return proto.CompactTextString(m) } +func (*KeyValue) ProtoMessage() {} +func (*KeyValue) Descriptor() ([]byte, []int) { + return fileDescriptor_data_7d67e43646134c7e, []int{2} +} +func (m *KeyValue) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *KeyValue) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *KeyValue) XXX_Merge(src proto.Message) { + xxx_messageInfo_KeyValue.Merge(dst, src) +} +func (m *KeyValue) XXX_Size() int { + return m.Size() +} +func (m *KeyValue) XXX_DiscardUnknown() { + xxx_messageInfo_KeyValue.DiscardUnknown(m) } -func (m *KeyValue) Reset() { *m = KeyValue{} } -func (m *KeyValue) String() string { return proto.CompactTextString(m) } -func (*KeyValue) ProtoMessage() {} -func (*KeyValue) Descriptor() ([]byte, []int) { return fileDescriptorData, []int{2} } +var xxx_messageInfo_KeyValue proto.InternalMessageInfo // A StoreIdent uniquely identifies a store in the cluster. The // StoreIdent is written to the underlying storage engine at a // store-reserved system key (KeyLocalIdent). type StoreIdent struct { - ClusterID github_com_cockroachdb_cockroach_pkg_util_uuid.UUID `protobuf:"bytes,1,opt,name=cluster_id,json=clusterId,proto3,customtype=github.com/cockroachdb/cockroach/pkg/util/uuid.UUID" json:"cluster_id"` - NodeID NodeID `protobuf:"varint,2,opt,name=node_id,json=nodeId,proto3,casttype=NodeID" json:"node_id,omitempty"` - StoreID StoreID `protobuf:"varint,3,opt,name=store_id,json=storeId,proto3,casttype=StoreID" json:"store_id,omitempty"` + ClusterID github_com_cockroachdb_cockroach_pkg_util_uuid.UUID `protobuf:"bytes,1,opt,name=cluster_id,json=clusterId,proto3,customtype=github.com/cockroachdb/cockroach/pkg/util/uuid.UUID" json:"cluster_id"` + NodeID NodeID `protobuf:"varint,2,opt,name=node_id,json=nodeId,proto3,casttype=NodeID" json:"node_id,omitempty"` + StoreID StoreID `protobuf:"varint,3,opt,name=store_id,json=storeId,proto3,casttype=StoreID" json:"store_id,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *StoreIdent) Reset() { *m = StoreIdent{} } -func (m *StoreIdent) String() string { return proto.CompactTextString(m) } -func (*StoreIdent) ProtoMessage() {} -func (*StoreIdent) Descriptor() ([]byte, []int) { return fileDescriptorData, []int{3} } +func (m *StoreIdent) Reset() { *m = StoreIdent{} } +func (m *StoreIdent) String() string { return proto.CompactTextString(m) } +func (*StoreIdent) ProtoMessage() {} +func (*StoreIdent) Descriptor() ([]byte, []int) { + return fileDescriptor_data_7d67e43646134c7e, []int{3} +} +func (m *StoreIdent) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *StoreIdent) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *StoreIdent) XXX_Merge(src proto.Message) { + xxx_messageInfo_StoreIdent.Merge(dst, src) +} +func (m *StoreIdent) XXX_Size() int { + return m.Size() +} +func (m *StoreIdent) XXX_DiscardUnknown() { + xxx_messageInfo_StoreIdent.DiscardUnknown(m) +} + +var xxx_messageInfo_StoreIdent proto.InternalMessageInfo // A SplitTrigger is run after a successful commit of an AdminSplit // command. It provides the updated left hand side of the split's @@ -216,14 +331,40 @@ func (*StoreIdent) Descriptor() ([]byte, []int) { return fileDescriptorData, []i // allows the final bookkeeping for the split to be completed and the // new range put into operation. type SplitTrigger struct { - LeftDesc RangeDescriptor `protobuf:"bytes,1,opt,name=left_desc,json=leftDesc" json:"left_desc"` - RightDesc RangeDescriptor `protobuf:"bytes,2,opt,name=right_desc,json=rightDesc" json:"right_desc"` + LeftDesc RangeDescriptor `protobuf:"bytes,1,opt,name=left_desc,json=leftDesc,proto3" json:"left_desc"` + RightDesc RangeDescriptor `protobuf:"bytes,2,opt,name=right_desc,json=rightDesc,proto3" json:"right_desc"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *SplitTrigger) Reset() { *m = SplitTrigger{} } +func (m *SplitTrigger) String() string { return proto.CompactTextString(m) } +func (*SplitTrigger) ProtoMessage() {} +func (*SplitTrigger) Descriptor() ([]byte, []int) { + return fileDescriptor_data_7d67e43646134c7e, []int{4} +} +func (m *SplitTrigger) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SplitTrigger) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *SplitTrigger) XXX_Merge(src proto.Message) { + xxx_messageInfo_SplitTrigger.Merge(dst, src) +} +func (m *SplitTrigger) XXX_Size() int { + return m.Size() +} +func (m *SplitTrigger) XXX_DiscardUnknown() { + xxx_messageInfo_SplitTrigger.DiscardUnknown(m) } -func (m *SplitTrigger) Reset() { *m = SplitTrigger{} } -func (m *SplitTrigger) String() string { return proto.CompactTextString(m) } -func (*SplitTrigger) ProtoMessage() {} -func (*SplitTrigger) Descriptor() ([]byte, []int) { return fileDescriptorData, []int{4} } +var xxx_messageInfo_SplitTrigger proto.InternalMessageInfo // A MergeTrigger is run after a successful commit of an AdminMerge // command. It provides the updated left hand side of the split's @@ -233,34 +374,86 @@ func (*SplitTrigger) Descriptor() ([]byte, []int) { return fileDescriptorData, [ // (right_desc). This information allows the final bookkeeping for the // merge to be completed and put into operation. type MergeTrigger struct { - LeftDesc RangeDescriptor `protobuf:"bytes,1,opt,name=left_desc,json=leftDesc" json:"left_desc"` - RightDesc RangeDescriptor `protobuf:"bytes,2,opt,name=right_desc,json=rightDesc" json:"right_desc"` - RightMVCCStats cockroach_storage_engine_enginepb1.MVCCStats `protobuf:"bytes,4,opt,name=right_mvcc_stats,json=rightMvccStats" json:"right_mvcc_stats"` + LeftDesc RangeDescriptor `protobuf:"bytes,1,opt,name=left_desc,json=leftDesc,proto3" json:"left_desc"` + RightDesc RangeDescriptor `protobuf:"bytes,2,opt,name=right_desc,json=rightDesc,proto3" json:"right_desc"` + RightMVCCStats enginepb.MVCCStats `protobuf:"bytes,4,opt,name=right_mvcc_stats,json=rightMvccStats,proto3" json:"right_mvcc_stats"` // FreezeStart is a timestamp that is guaranteed to be greater than the // timestamps at which any requests were serviced by the responding replica // before it stopped responding to requests altogether (in anticipation of // being subsumed). It is suitable for use as the timestamp cache's low water // mark for the keys previously owned by the subsumed range. - FreezeStart cockroach_util_hlc.Timestamp `protobuf:"bytes,5,opt,name=freeze_start,json=freezeStart" json:"freeze_start"` + FreezeStart hlc.Timestamp `protobuf:"bytes,5,opt,name=freeze_start,json=freezeStart,proto3" json:"freeze_start"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *MergeTrigger) Reset() { *m = MergeTrigger{} } -func (m *MergeTrigger) String() string { return proto.CompactTextString(m) } -func (*MergeTrigger) ProtoMessage() {} -func (*MergeTrigger) Descriptor() ([]byte, []int) { return fileDescriptorData, []int{5} } +func (m *MergeTrigger) Reset() { *m = MergeTrigger{} } +func (m *MergeTrigger) String() string { return proto.CompactTextString(m) } +func (*MergeTrigger) ProtoMessage() {} +func (*MergeTrigger) Descriptor() ([]byte, []int) { + return fileDescriptor_data_7d67e43646134c7e, []int{5} +} +func (m *MergeTrigger) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *MergeTrigger) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *MergeTrigger) XXX_Merge(src proto.Message) { + xxx_messageInfo_MergeTrigger.Merge(dst, src) +} +func (m *MergeTrigger) XXX_Size() int { + return m.Size() +} +func (m *MergeTrigger) XXX_DiscardUnknown() { + xxx_messageInfo_MergeTrigger.DiscardUnknown(m) +} + +var xxx_messageInfo_MergeTrigger proto.InternalMessageInfo type ChangeReplicasTrigger struct { ChangeType ReplicaChangeType `protobuf:"varint,1,opt,name=change_type,json=changeType,proto3,enum=cockroach.roachpb.ReplicaChangeType" json:"change_type,omitempty"` // The replica being modified. - Replica ReplicaDescriptor `protobuf:"bytes,2,opt,name=replica" json:"replica"` + Replica ReplicaDescriptor `protobuf:"bytes,2,opt,name=replica,proto3" json:"replica"` // The new replica list with this change applied. - UpdatedReplicas []ReplicaDescriptor `protobuf:"bytes,3,rep,name=updated_replicas,json=updatedReplicas" json:"updated_replicas"` - NextReplicaID ReplicaID `protobuf:"varint,4,opt,name=next_replica_id,json=nextReplicaId,proto3,casttype=ReplicaID" json:"next_replica_id,omitempty"` + UpdatedReplicas []ReplicaDescriptor `protobuf:"bytes,3,rep,name=updated_replicas,json=updatedReplicas,proto3" json:"updated_replicas"` + NextReplicaID ReplicaID `protobuf:"varint,4,opt,name=next_replica_id,json=nextReplicaId,proto3,casttype=ReplicaID" json:"next_replica_id,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *ChangeReplicasTrigger) Reset() { *m = ChangeReplicasTrigger{} } -func (*ChangeReplicasTrigger) ProtoMessage() {} -func (*ChangeReplicasTrigger) Descriptor() ([]byte, []int) { return fileDescriptorData, []int{6} } +func (m *ChangeReplicasTrigger) Reset() { *m = ChangeReplicasTrigger{} } +func (*ChangeReplicasTrigger) ProtoMessage() {} +func (*ChangeReplicasTrigger) Descriptor() ([]byte, []int) { + return fileDescriptor_data_7d67e43646134c7e, []int{6} +} +func (m *ChangeReplicasTrigger) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ChangeReplicasTrigger) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ChangeReplicasTrigger) XXX_Merge(src proto.Message) { + xxx_messageInfo_ChangeReplicasTrigger.Merge(dst, src) +} +func (m *ChangeReplicasTrigger) XXX_Size() int { + return m.Size() +} +func (m *ChangeReplicasTrigger) XXX_DiscardUnknown() { + xxx_messageInfo_ChangeReplicasTrigger.DiscardUnknown(m) +} + +var xxx_messageInfo_ChangeReplicasTrigger proto.InternalMessageInfo // ModifiedSpanTrigger indicates that a specific span has been modified. // This can be used to trigger scan-and-gossip for the given span. @@ -272,27 +465,79 @@ type ModifiedSpanTrigger struct { // with heartbeats to extend the expiration timestamp. Changes to the // range lease for the range containing node liveness triggers re-gossip // of the entire node liveness key range. - NodeLivenessSpan *Span `protobuf:"bytes,2,opt,name=node_liveness_span,json=nodeLivenessSpan" json:"node_liveness_span,omitempty"` + NodeLivenessSpan *Span `protobuf:"bytes,2,opt,name=node_liveness_span,json=nodeLivenessSpan,proto3" json:"node_liveness_span,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ModifiedSpanTrigger) Reset() { *m = ModifiedSpanTrigger{} } +func (m *ModifiedSpanTrigger) String() string { return proto.CompactTextString(m) } +func (*ModifiedSpanTrigger) ProtoMessage() {} +func (*ModifiedSpanTrigger) Descriptor() ([]byte, []int) { + return fileDescriptor_data_7d67e43646134c7e, []int{7} +} +func (m *ModifiedSpanTrigger) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ModifiedSpanTrigger) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ModifiedSpanTrigger) XXX_Merge(src proto.Message) { + xxx_messageInfo_ModifiedSpanTrigger.Merge(dst, src) +} +func (m *ModifiedSpanTrigger) XXX_Size() int { + return m.Size() +} +func (m *ModifiedSpanTrigger) XXX_DiscardUnknown() { + xxx_messageInfo_ModifiedSpanTrigger.DiscardUnknown(m) } -func (m *ModifiedSpanTrigger) Reset() { *m = ModifiedSpanTrigger{} } -func (m *ModifiedSpanTrigger) String() string { return proto.CompactTextString(m) } -func (*ModifiedSpanTrigger) ProtoMessage() {} -func (*ModifiedSpanTrigger) Descriptor() ([]byte, []int) { return fileDescriptorData, []int{7} } +var xxx_messageInfo_ModifiedSpanTrigger proto.InternalMessageInfo // InternalCommitTrigger encapsulates all of the internal-only commit triggers. // Only one may be set. type InternalCommitTrigger struct { - SplitTrigger *SplitTrigger `protobuf:"bytes,1,opt,name=split_trigger,json=splitTrigger" json:"split_trigger,omitempty"` - MergeTrigger *MergeTrigger `protobuf:"bytes,2,opt,name=merge_trigger,json=mergeTrigger" json:"merge_trigger,omitempty"` - ChangeReplicasTrigger *ChangeReplicasTrigger `protobuf:"bytes,3,opt,name=change_replicas_trigger,json=changeReplicasTrigger" json:"change_replicas_trigger,omitempty"` - ModifiedSpanTrigger *ModifiedSpanTrigger `protobuf:"bytes,4,opt,name=modified_span_trigger,json=modifiedSpanTrigger" json:"modified_span_trigger,omitempty"` + SplitTrigger *SplitTrigger `protobuf:"bytes,1,opt,name=split_trigger,json=splitTrigger,proto3" json:"split_trigger,omitempty"` + MergeTrigger *MergeTrigger `protobuf:"bytes,2,opt,name=merge_trigger,json=mergeTrigger,proto3" json:"merge_trigger,omitempty"` + ChangeReplicasTrigger *ChangeReplicasTrigger `protobuf:"bytes,3,opt,name=change_replicas_trigger,json=changeReplicasTrigger,proto3" json:"change_replicas_trigger,omitempty"` + ModifiedSpanTrigger *ModifiedSpanTrigger `protobuf:"bytes,4,opt,name=modified_span_trigger,json=modifiedSpanTrigger,proto3" json:"modified_span_trigger,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *InternalCommitTrigger) Reset() { *m = InternalCommitTrigger{} } +func (m *InternalCommitTrigger) String() string { return proto.CompactTextString(m) } +func (*InternalCommitTrigger) ProtoMessage() {} +func (*InternalCommitTrigger) Descriptor() ([]byte, []int) { + return fileDescriptor_data_7d67e43646134c7e, []int{8} +} +func (m *InternalCommitTrigger) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *InternalCommitTrigger) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *InternalCommitTrigger) XXX_Merge(src proto.Message) { + xxx_messageInfo_InternalCommitTrigger.Merge(dst, src) +} +func (m *InternalCommitTrigger) XXX_Size() int { + return m.Size() +} +func (m *InternalCommitTrigger) XXX_DiscardUnknown() { + xxx_messageInfo_InternalCommitTrigger.DiscardUnknown(m) } -func (m *InternalCommitTrigger) Reset() { *m = InternalCommitTrigger{} } -func (m *InternalCommitTrigger) String() string { return proto.CompactTextString(m) } -func (*InternalCommitTrigger) ProtoMessage() {} -func (*InternalCommitTrigger) Descriptor() ([]byte, []int) { return fileDescriptorData, []int{8} } +var xxx_messageInfo_InternalCommitTrigger proto.InternalMessageInfo func (m *InternalCommitTrigger) GetSplitTrigger() *SplitTrigger { if m != nil { @@ -323,14 +568,40 @@ func (m *InternalCommitTrigger) GetModifiedSpanTrigger() *ModifiedSpanTrigger { } type ObservedTimestamp struct { - NodeID NodeID `protobuf:"varint,1,opt,name=node_id,json=nodeId,proto3,casttype=NodeID" json:"node_id,omitempty"` - Timestamp cockroach_util_hlc.Timestamp `protobuf:"bytes,2,opt,name=timestamp" json:"timestamp"` + NodeID NodeID `protobuf:"varint,1,opt,name=node_id,json=nodeId,proto3,casttype=NodeID" json:"node_id,omitempty"` + Timestamp hlc.Timestamp `protobuf:"bytes,2,opt,name=timestamp,proto3" json:"timestamp"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *ObservedTimestamp) Reset() { *m = ObservedTimestamp{} } -func (m *ObservedTimestamp) String() string { return proto.CompactTextString(m) } -func (*ObservedTimestamp) ProtoMessage() {} -func (*ObservedTimestamp) Descriptor() ([]byte, []int) { return fileDescriptorData, []int{9} } +func (m *ObservedTimestamp) Reset() { *m = ObservedTimestamp{} } +func (m *ObservedTimestamp) String() string { return proto.CompactTextString(m) } +func (*ObservedTimestamp) ProtoMessage() {} +func (*ObservedTimestamp) Descriptor() ([]byte, []int) { + return fileDescriptor_data_7d67e43646134c7e, []int{9} +} +func (m *ObservedTimestamp) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ObservedTimestamp) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ObservedTimestamp) XXX_Merge(src proto.Message) { + xxx_messageInfo_ObservedTimestamp.Merge(dst, src) +} +func (m *ObservedTimestamp) XXX_Size() int { + return m.Size() +} +func (m *ObservedTimestamp) XXX_DiscardUnknown() { + xxx_messageInfo_ObservedTimestamp.DiscardUnknown(m) +} + +var xxx_messageInfo_ObservedTimestamp proto.InternalMessageInfo // A Transaction is a unit of work performed on the database. // Cockroach transactions always operate at the serializable isolation @@ -342,11 +613,11 @@ func (*ObservedTimestamp) Descriptor() ([]byte, []int) { return fileDescriptorDa // Transaction.Clone. Failure to do so will result in test failures. type Transaction struct { // The transaction metadata. These are persisted with every intent. - cockroach_storage_engine_enginepb.TxnMeta `protobuf:"bytes,1,opt,name=meta,embedded=meta" json:"meta"` + enginepb.TxnMeta `protobuf:"bytes,1,opt,name=meta,proto3,embedded=meta" json:"meta"` // A free-text identifier for debug purposes. - Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` - Status TransactionStatus `protobuf:"varint,4,opt,name=status,proto3,enum=cockroach.roachpb.TransactionStatus" json:"status,omitempty"` - LastHeartbeat cockroach_util_hlc.Timestamp `protobuf:"bytes,5,opt,name=last_heartbeat,json=lastHeartbeat" json:"last_heartbeat"` + Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` + Status TransactionStatus `protobuf:"varint,4,opt,name=status,proto3,enum=cockroach.roachpb.TransactionStatus" json:"status,omitempty"` + LastHeartbeat hlc.Timestamp `protobuf:"bytes,5,opt,name=last_heartbeat,json=lastHeartbeat,proto3" json:"last_heartbeat"` // The original timestamp at which the transaction started. For serializable // transactions, if the timestamp drifts from the original timestamp, the // transaction will retry unless we manage to "refresh the reads" - see @@ -357,14 +628,14 @@ type Transaction struct { // // Note that writes do not occur at this timestamp; they instead occur at the // provisional commit timestamp, meta.Timestamp. - OrigTimestamp cockroach_util_hlc.Timestamp `protobuf:"bytes,6,opt,name=orig_timestamp,json=origTimestamp" json:"orig_timestamp"` + OrigTimestamp hlc.Timestamp `protobuf:"bytes,6,opt,name=orig_timestamp,json=origTimestamp,proto3" json:"orig_timestamp"` // Initial Timestamp + clock skew. Reads which encounter values with // timestamps between timestamp and max_timestamp trigger a txn // retry error, unless the node being read is listed in observed_timestamps // (in which case no more read uncertainty can occur). // The case max_timestamp < timestamp is possible for transactions which have // been pushed; in this case, max_timestamp should be ignored. - MaxTimestamp cockroach_util_hlc.Timestamp `protobuf:"bytes,7,opt,name=max_timestamp,json=maxTimestamp" json:"max_timestamp"` + MaxTimestamp hlc.Timestamp `protobuf:"bytes,7,opt,name=max_timestamp,json=maxTimestamp,proto3" json:"max_timestamp"` // The refreshed timestamp is the timestamp at which the transaction // can commit without necessitating a serializable restart. This // value is forwarded to the transaction's current timestamp (meta.timestamp) @@ -378,7 +649,7 @@ type Transaction struct { // orig_timestamp needs to be used as a lower bound timestamp for the // time-bound iterator used to resolve intents - i.e. there can be intents to // resolve up to the timestamp that the txn started with. - RefreshedTimestamp cockroach_util_hlc.Timestamp `protobuf:"bytes,15,opt,name=refreshed_timestamp,json=refreshedTimestamp" json:"refreshed_timestamp"` + RefreshedTimestamp hlc.Timestamp `protobuf:"bytes,15,opt,name=refreshed_timestamp,json=refreshedTimestamp,proto3" json:"refreshed_timestamp"` // A list of pairs. The list maps NodeIDs to timestamps // as observed from their local clock during this transaction. The purpose of // this map is to avoid uncertainty related restarts which normally occur @@ -419,7 +690,7 @@ type Transaction struct { // // The list of observed timestamps is kept sorted by NodeID. Use // Transaction.UpdateObservedTimestamp to maintain the sorted order. - ObservedTimestamps []ObservedTimestamp `protobuf:"bytes,8,rep,name=observed_timestamps,json=observedTimestamps" json:"observed_timestamps"` + ObservedTimestamps []ObservedTimestamp `protobuf:"bytes,8,rep,name=observed_timestamps,json=observedTimestamps,proto3" json:"observed_timestamps"` // Writing is true if the transaction has previously sent a Begin transaction // (i.e. if it ever attempted to perform a write, so if it ever attempted to // leave intents (across retries)). The flag will be set even if the BeginTxn @@ -435,35 +706,87 @@ type Transaction struct { // error so that intents can continue to be laid down, minimizing // work required on txn restart. WriteTooOld bool `protobuf:"varint,12,opt,name=write_too_old,json=writeTooOld,proto3" json:"write_too_old,omitempty"` - Intents []Span `protobuf:"bytes,11,rep,name=intents" json:"intents"` + Intents []Span `protobuf:"bytes,11,rep,name=intents,proto3" json:"intents"` // Epoch zero timestamp is used to keep track of the earliest timestamp // that any epoch of the transaction used. This is set only if the // transaction is restarted and the epoch is bumped. It is used during // intent resolution to more efficiently scan for intents. - EpochZeroTimestamp cockroach_util_hlc.Timestamp `protobuf:"bytes,14,opt,name=epoch_zero_timestamp,json=epochZeroTimestamp" json:"epoch_zero_timestamp"` + EpochZeroTimestamp hlc.Timestamp `protobuf:"bytes,14,opt,name=epoch_zero_timestamp,json=epochZeroTimestamp,proto3" json:"epoch_zero_timestamp"` // This flag is set if the transaction's original timestamp was // "leaked" beyond the transaction (i.e. if returned via NOW() or // transaction_timestamp()). If true, this prevents optimizations // which commit at a higher timestamp without resorting to a // client-side retry. - OrigTimestampWasObserved bool `protobuf:"varint,16,opt,name=orig_timestamp_was_observed,json=origTimestampWasObserved,proto3" json:"orig_timestamp_was_observed,omitempty"` + OrigTimestampWasObserved bool `protobuf:"varint,16,opt,name=orig_timestamp_was_observed,json=origTimestampWasObserved,proto3" json:"orig_timestamp_was_observed,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *Transaction) Reset() { *m = Transaction{} } -func (*Transaction) ProtoMessage() {} -func (*Transaction) Descriptor() ([]byte, []int) { return fileDescriptorData, []int{10} } +func (m *Transaction) Reset() { *m = Transaction{} } +func (*Transaction) ProtoMessage() {} +func (*Transaction) Descriptor() ([]byte, []int) { + return fileDescriptor_data_7d67e43646134c7e, []int{10} +} +func (m *Transaction) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Transaction) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *Transaction) XXX_Merge(src proto.Message) { + xxx_messageInfo_Transaction.Merge(dst, src) +} +func (m *Transaction) XXX_Size() int { + return m.Size() +} +func (m *Transaction) XXX_DiscardUnknown() { + xxx_messageInfo_Transaction.DiscardUnknown(m) +} + +var xxx_messageInfo_Transaction proto.InternalMessageInfo // A Intent is a Span together with a Transaction metadata and its status. type Intent struct { - Span `protobuf:"bytes,1,opt,name=span,embedded=span" json:"span"` - Txn cockroach_storage_engine_enginepb.TxnMeta `protobuf:"bytes,2,opt,name=txn" json:"txn"` - Status TransactionStatus `protobuf:"varint,3,opt,name=status,proto3,enum=cockroach.roachpb.TransactionStatus" json:"status,omitempty"` + Span `protobuf:"bytes,1,opt,name=span,proto3,embedded=span" json:"span"` + Txn enginepb.TxnMeta `protobuf:"bytes,2,opt,name=txn,proto3" json:"txn"` + Status TransactionStatus `protobuf:"varint,3,opt,name=status,proto3,enum=cockroach.roachpb.TransactionStatus" json:"status,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Intent) Reset() { *m = Intent{} } +func (m *Intent) String() string { return proto.CompactTextString(m) } +func (*Intent) ProtoMessage() {} +func (*Intent) Descriptor() ([]byte, []int) { + return fileDescriptor_data_7d67e43646134c7e, []int{11} +} +func (m *Intent) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Intent) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *Intent) XXX_Merge(src proto.Message) { + xxx_messageInfo_Intent.Merge(dst, src) +} +func (m *Intent) XXX_Size() int { + return m.Size() +} +func (m *Intent) XXX_DiscardUnknown() { + xxx_messageInfo_Intent.DiscardUnknown(m) } -func (m *Intent) Reset() { *m = Intent{} } -func (m *Intent) String() string { return proto.CompactTextString(m) } -func (*Intent) ProtoMessage() {} -func (*Intent) Descriptor() ([]byte, []int) { return fileDescriptorData, []int{11} } +var xxx_messageInfo_Intent proto.InternalMessageInfo // A SequencedWrite is a point write to a key with a certain sequence number. // @@ -474,13 +797,39 @@ type SequencedWrite struct { // The key that the write was made at. Key Key `protobuf:"bytes,1,opt,name=key,proto3,casttype=Key" json:"key,omitempty"` // The sequence number of the request that created the write. - Sequence int32 `protobuf:"varint,2,opt,name=sequence,proto3" json:"sequence,omitempty"` + Sequence int32 `protobuf:"varint,2,opt,name=sequence,proto3" json:"sequence,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *SequencedWrite) Reset() { *m = SequencedWrite{} } -func (m *SequencedWrite) String() string { return proto.CompactTextString(m) } -func (*SequencedWrite) ProtoMessage() {} -func (*SequencedWrite) Descriptor() ([]byte, []int) { return fileDescriptorData, []int{12} } +func (m *SequencedWrite) Reset() { *m = SequencedWrite{} } +func (m *SequencedWrite) String() string { return proto.CompactTextString(m) } +func (*SequencedWrite) ProtoMessage() {} +func (*SequencedWrite) Descriptor() ([]byte, []int) { + return fileDescriptor_data_7d67e43646134c7e, []int{12} +} +func (m *SequencedWrite) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SequencedWrite) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *SequencedWrite) XXX_Merge(src proto.Message) { + xxx_messageInfo_SequencedWrite.Merge(dst, src) +} +func (m *SequencedWrite) XXX_Size() int { + return m.Size() +} +func (m *SequencedWrite) XXX_DiscardUnknown() { + xxx_messageInfo_SequencedWrite.DiscardUnknown(m) +} + +var xxx_messageInfo_SequencedWrite proto.InternalMessageInfo // Lease contains information about range leases including the // expiration and lease holder. @@ -488,21 +837,21 @@ type Lease struct { // The start is a timestamp at which the lease begins. This value // must be greater than the last lease expiration or the lease request // is considered invalid. - Start cockroach_util_hlc.Timestamp `protobuf:"bytes,1,opt,name=start" json:"start"` + Start hlc.Timestamp `protobuf:"bytes,1,opt,name=start,proto3" json:"start"` // The expiration is a timestamp at which the lease expires. This means that // a new lease can be granted for a later timestamp. - Expiration *cockroach_util_hlc.Timestamp `protobuf:"bytes,2,opt,name=expiration" json:"expiration,omitempty" cockroachdb:"randnullable"` + Expiration *hlc.Timestamp `protobuf:"bytes,2,opt,name=expiration,proto3" json:"expiration,omitempty" cockroachdb:"randnullable"` // The address of the would-be lease holder. - Replica ReplicaDescriptor `protobuf:"bytes,3,opt,name=replica" json:"replica"` + Replica ReplicaDescriptor `protobuf:"bytes,3,opt,name=replica,proto3" json:"replica"` // The start of the lease stasis period. This field is deprecated. - DeprecatedStartStasis *cockroach_util_hlc.Timestamp `protobuf:"bytes,4,opt,name=deprecated_start_stasis,json=deprecatedStartStasis" json:"deprecated_start_stasis,omitempty" cockroachdb:"randnullable"` + DeprecatedStartStasis *hlc.Timestamp `protobuf:"bytes,4,opt,name=deprecated_start_stasis,json=deprecatedStartStasis,proto3" json:"deprecated_start_stasis,omitempty" cockroachdb:"randnullable"` // The current timestamp when this lease has been proposed. Used after a // transfer and after a node restart to enforce that a node only uses leases // proposed after the time of the said transfer or restart. This is nullable // to help with the rollout (such that a lease applied by some nodes before // the rollout and some nodes after the rollout is serialized the same). // TODO(andrei): Make this non-nullable after the rollout. - ProposedTS *cockroach_util_hlc.Timestamp `protobuf:"bytes,5,opt,name=proposed_ts,json=proposedTs" json:"proposed_ts,omitempty"` + ProposedTS *hlc.Timestamp `protobuf:"bytes,5,opt,name=proposed_ts,json=proposedTs,proto3" json:"proposed_ts,omitempty"` // The epoch of the lease holder's node liveness entry. If this value // is non-zero, the start and expiration values are ignored. Epoch int64 `protobuf:"varint,6,opt,name=epoch,proto3" json:"epoch,omitempty"` @@ -515,12 +864,38 @@ type Lease struct { // (see Lease.Equivalent). Two adjacent leases that are equivalent will have // the same sequence number and two adjacent leases that are not equivalent // will have different sequence numbers. - Sequence LeaseSequence `protobuf:"varint,7,opt,name=sequence,proto3,casttype=LeaseSequence" json:"sequence,omitempty"` + Sequence LeaseSequence `protobuf:"varint,7,opt,name=sequence,proto3,casttype=LeaseSequence" json:"sequence,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Lease) Reset() { *m = Lease{} } +func (*Lease) ProtoMessage() {} +func (*Lease) Descriptor() ([]byte, []int) { + return fileDescriptor_data_7d67e43646134c7e, []int{13} +} +func (m *Lease) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Lease) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *Lease) XXX_Merge(src proto.Message) { + xxx_messageInfo_Lease.Merge(dst, src) +} +func (m *Lease) XXX_Size() int { + return m.Size() +} +func (m *Lease) XXX_DiscardUnknown() { + xxx_messageInfo_Lease.DiscardUnknown(m) } -func (m *Lease) Reset() { *m = Lease{} } -func (*Lease) ProtoMessage() {} -func (*Lease) Descriptor() ([]byte, []int) { return fileDescriptorData, []int{13} } +var xxx_messageInfo_Lease proto.InternalMessageInfo // AbortSpanEntry contains information about a transaction which has // been aborted. It's written to a range's AbortSpan if the range @@ -533,27 +908,53 @@ type AbortSpanEntry struct { Key Key `protobuf:"bytes,1,opt,name=key,proto3,casttype=Key" json:"key,omitempty"` // The candidate commit timestamp the transaction record held at the time // it was aborted. - Timestamp cockroach_util_hlc.Timestamp `protobuf:"bytes,2,opt,name=timestamp" json:"timestamp"` + Timestamp hlc.Timestamp `protobuf:"bytes,2,opt,name=timestamp,proto3" json:"timestamp"` // The priority of the transaction. - Priority int32 `protobuf:"varint,3,opt,name=priority,proto3" json:"priority,omitempty"` + Priority int32 `protobuf:"varint,3,opt,name=priority,proto3" json:"priority,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *AbortSpanEntry) Reset() { *m = AbortSpanEntry{} } -func (m *AbortSpanEntry) String() string { return proto.CompactTextString(m) } -func (*AbortSpanEntry) ProtoMessage() {} -func (*AbortSpanEntry) Descriptor() ([]byte, []int) { return fileDescriptorData, []int{14} } +func (m *AbortSpanEntry) Reset() { *m = AbortSpanEntry{} } +func (m *AbortSpanEntry) String() string { return proto.CompactTextString(m) } +func (*AbortSpanEntry) ProtoMessage() {} +func (*AbortSpanEntry) Descriptor() ([]byte, []int) { + return fileDescriptor_data_7d67e43646134c7e, []int{14} +} +func (m *AbortSpanEntry) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *AbortSpanEntry) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *AbortSpanEntry) XXX_Merge(src proto.Message) { + xxx_messageInfo_AbortSpanEntry.Merge(dst, src) +} +func (m *AbortSpanEntry) XXX_Size() int { + return m.Size() +} +func (m *AbortSpanEntry) XXX_DiscardUnknown() { + xxx_messageInfo_AbortSpanEntry.DiscardUnknown(m) +} + +var xxx_messageInfo_AbortSpanEntry proto.InternalMessageInfo // TxnCoordMeta is metadata held by a transaction coordinator. This // message is defined here because it is used in several layers of the // system (internal/client, sql/distsqlrun, kv). type TxnCoordMeta struct { // txn is a copy of the transaction record, updated with each request. - Txn Transaction `protobuf:"bytes,1,opt,name=txn" json:"txn"` + Txn Transaction `protobuf:"bytes,1,opt,name=txn,proto3" json:"txn"` // intents stores key spans affected by this transaction through // this coordinator. These spans allow the coordinator to set the // list of intent spans in the EndTransactionRequest when the // transaction is finalized. - Intents []Span `protobuf:"bytes,2,rep,name=intents" json:"intents"` + Intents []Span `protobuf:"bytes,2,rep,name=intents,proto3" json:"intents"` // command_count indicates how many requests have been sent through // this transaction. Reset on retryable txn errors. CommandCount int32 `protobuf:"varint,3,opt,name=command_count,json=commandCount,proto3" json:"command_count,omitempty"` @@ -568,8 +969,8 @@ type TxnCoordMeta struct { // error is propagated. On success, the transaction's original and // current timestamps are forwarded to the refresh timestamp, and // the transaction can continue. - RefreshReads []Span `protobuf:"bytes,4,rep,name=refresh_reads,json=refreshReads" json:"refresh_reads"` - RefreshWrites []Span `protobuf:"bytes,5,rep,name=refresh_writes,json=refreshWrites" json:"refresh_writes"` + RefreshReads []Span `protobuf:"bytes,4,rep,name=refresh_reads,json=refreshReads,proto3" json:"refresh_reads"` + RefreshWrites []Span `protobuf:"bytes,5,rep,name=refresh_writes,json=refreshWrites,proto3" json:"refresh_writes"` // refresh_invalid indicates that spans were discarded or not collected // (i.e. because of a dist SQL processor running a version before refreshing // was introduced). This is false if all spans encountered during the @@ -585,13 +986,39 @@ type TxnCoordMeta struct { // not yet been resolved. Any client wishing to send a request that // overlaps with them must chain on to their success using a QueryIntent // request. - OutstandingWrites []SequencedWrite `protobuf:"bytes,8,rep,name=outstanding_writes,json=outstandingWrites" json:"outstanding_writes"` + OutstandingWrites []SequencedWrite `protobuf:"bytes,8,rep,name=outstanding_writes,json=outstandingWrites,proto3" json:"outstanding_writes"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *TxnCoordMeta) Reset() { *m = TxnCoordMeta{} } -func (m *TxnCoordMeta) String() string { return proto.CompactTextString(m) } -func (*TxnCoordMeta) ProtoMessage() {} -func (*TxnCoordMeta) Descriptor() ([]byte, []int) { return fileDescriptorData, []int{15} } +func (m *TxnCoordMeta) Reset() { *m = TxnCoordMeta{} } +func (m *TxnCoordMeta) String() string { return proto.CompactTextString(m) } +func (*TxnCoordMeta) ProtoMessage() {} +func (*TxnCoordMeta) Descriptor() ([]byte, []int) { + return fileDescriptor_data_7d67e43646134c7e, []int{15} +} +func (m *TxnCoordMeta) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *TxnCoordMeta) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *TxnCoordMeta) XXX_Merge(src proto.Message) { + xxx_messageInfo_TxnCoordMeta.Merge(dst, src) +} +func (m *TxnCoordMeta) XXX_Size() int { + return m.Size() +} +func (m *TxnCoordMeta) XXX_DiscardUnknown() { + xxx_messageInfo_TxnCoordMeta.DiscardUnknown(m) +} + +var xxx_messageInfo_TxnCoordMeta proto.InternalMessageInfo func init() { proto.RegisterType((*Span)(nil), "cockroach.roachpb.Span") @@ -1815,7 +2242,7 @@ func NewPopulatedObservedTimestamp(r randyData, easy bool) *ObservedTimestamp { if r.Intn(2) == 0 { this.NodeID *= -1 } - v3 := cockroach_util_hlc.NewPopulatedTimestamp(r, easy) + v3 := hlc.NewPopulatedTimestamp(r, easy) this.Timestamp = *v3 if !easy && r.Intn(10) != 0 { } @@ -1824,15 +2251,15 @@ func NewPopulatedObservedTimestamp(r randyData, easy bool) *ObservedTimestamp { func NewPopulatedTransaction(r randyData, easy bool) *Transaction { this := &Transaction{} - v4 := cockroach_storage_engine_enginepb.NewPopulatedTxnMeta(r, easy) + v4 := enginepb.NewPopulatedTxnMeta(r, easy) this.TxnMeta = *v4 this.Name = string(randStringData(r)) this.Status = TransactionStatus([]int32{0, 1, 2}[r.Intn(3)]) - v5 := cockroach_util_hlc.NewPopulatedTimestamp(r, easy) + v5 := hlc.NewPopulatedTimestamp(r, easy) this.LastHeartbeat = *v5 - v6 := cockroach_util_hlc.NewPopulatedTimestamp(r, easy) + v6 := hlc.NewPopulatedTimestamp(r, easy) this.OrigTimestamp = *v6 - v7 := cockroach_util_hlc.NewPopulatedTimestamp(r, easy) + v7 := hlc.NewPopulatedTimestamp(r, easy) this.MaxTimestamp = *v7 if r.Intn(10) != 0 { v8 := r.Intn(5) @@ -1852,9 +2279,9 @@ func NewPopulatedTransaction(r randyData, easy bool) *Transaction { } } this.WriteTooOld = bool(bool(r.Intn(2) == 0)) - v12 := cockroach_util_hlc.NewPopulatedTimestamp(r, easy) + v12 := hlc.NewPopulatedTimestamp(r, easy) this.EpochZeroTimestamp = *v12 - v13 := cockroach_util_hlc.NewPopulatedTimestamp(r, easy) + v13 := hlc.NewPopulatedTimestamp(r, easy) this.RefreshedTimestamp = *v13 this.OrigTimestampWasObserved = bool(bool(r.Intn(2) == 0)) if !easy && r.Intn(10) != 0 { @@ -1864,18 +2291,18 @@ func NewPopulatedTransaction(r randyData, easy bool) *Transaction { func NewPopulatedLease(r randyData, easy bool) *Lease { this := &Lease{} - v14 := cockroach_util_hlc.NewPopulatedTimestamp(r, easy) + v14 := hlc.NewPopulatedTimestamp(r, easy) this.Start = *v14 if r.Intn(10) != 0 { - this.Expiration = cockroach_util_hlc.NewPopulatedTimestamp(r, easy) + this.Expiration = hlc.NewPopulatedTimestamp(r, easy) } v15 := NewPopulatedReplicaDescriptor(r, easy) this.Replica = *v15 if r.Intn(10) != 0 { - this.DeprecatedStartStasis = cockroach_util_hlc.NewPopulatedTimestamp(r, easy) + this.DeprecatedStartStasis = hlc.NewPopulatedTimestamp(r, easy) } if r.Intn(10) != 0 { - this.ProposedTS = cockroach_util_hlc.NewPopulatedTimestamp(r, easy) + this.ProposedTS = hlc.NewPopulatedTimestamp(r, easy) } this.Epoch = int64(r.Int63()) if r.Intn(2) == 0 { @@ -1897,7 +2324,7 @@ func NewPopulatedAbortSpanEntry(r randyData, easy bool) *AbortSpanEntry { for i := 0; i < v16; i++ { this.Key[i] = byte(r.Intn(256)) } - v17 := cockroach_util_hlc.NewPopulatedTimestamp(r, easy) + v17 := hlc.NewPopulatedTimestamp(r, easy) this.Timestamp = *v17 this.Priority = int32(r.Int31()) if r.Intn(2) == 0 { @@ -1981,6 +2408,9 @@ func encodeVarintPopulateData(dAtA []byte, v uint64) []byte { return dAtA } func (m *Span) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Key) @@ -1995,6 +2425,9 @@ func (m *Span) Size() (n int) { } func (m *Value) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.RawBytes) @@ -2007,6 +2440,9 @@ func (m *Value) Size() (n int) { } func (m *KeyValue) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Key) @@ -2019,6 +2455,9 @@ func (m *KeyValue) Size() (n int) { } func (m *StoreIdent) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.ClusterID.Size() @@ -2033,6 +2472,9 @@ func (m *StoreIdent) Size() (n int) { } func (m *SplitTrigger) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.LeftDesc.Size() @@ -2043,6 +2485,9 @@ func (m *SplitTrigger) Size() (n int) { } func (m *MergeTrigger) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.LeftDesc.Size() @@ -2057,6 +2502,9 @@ func (m *MergeTrigger) Size() (n int) { } func (m *ChangeReplicasTrigger) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.ChangeType != 0 { @@ -2077,6 +2525,9 @@ func (m *ChangeReplicasTrigger) Size() (n int) { } func (m *ModifiedSpanTrigger) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.SystemConfigSpan { @@ -2090,6 +2541,9 @@ func (m *ModifiedSpanTrigger) Size() (n int) { } func (m *InternalCommitTrigger) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.SplitTrigger != nil { @@ -2112,6 +2566,9 @@ func (m *InternalCommitTrigger) Size() (n int) { } func (m *ObservedTimestamp) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.NodeID != 0 { @@ -2123,6 +2580,9 @@ func (m *ObservedTimestamp) Size() (n int) { } func (m *Transaction) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.TxnMeta.Size() @@ -2169,6 +2629,9 @@ func (m *Transaction) Size() (n int) { } func (m *Intent) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.Span.Size() @@ -2182,6 +2645,9 @@ func (m *Intent) Size() (n int) { } func (m *SequencedWrite) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Key) @@ -2195,6 +2661,9 @@ func (m *SequencedWrite) Size() (n int) { } func (m *Lease) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.Start.Size() @@ -2223,6 +2692,9 @@ func (m *Lease) Size() (n int) { } func (m *AbortSpanEntry) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Key) @@ -2238,6 +2710,9 @@ func (m *AbortSpanEntry) Size() (n int) { } func (m *TxnCoordMeta) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.Txn.Size() @@ -4271,7 +4746,7 @@ func (m *Lease) Unmarshal(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Expiration == nil { - m.Expiration = &cockroach_util_hlc.Timestamp{} + m.Expiration = &hlc.Timestamp{} } if err := m.Expiration.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err @@ -4334,7 +4809,7 @@ func (m *Lease) Unmarshal(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.DeprecatedStartStasis == nil { - m.DeprecatedStartStasis = &cockroach_util_hlc.Timestamp{} + m.DeprecatedStartStasis = &hlc.Timestamp{} } if err := m.DeprecatedStartStasis.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err @@ -4367,7 +4842,7 @@ func (m *Lease) Unmarshal(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.ProposedTS == nil { - m.ProposedTS = &cockroach_util_hlc.Timestamp{} + m.ProposedTS = &hlc.Timestamp{} } if err := m.ProposedTS.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err @@ -4930,9 +5405,9 @@ var ( ErrIntOverflowData = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/data.proto", fileDescriptorData) } +func init() { proto.RegisterFile("roachpb/data.proto", fileDescriptor_data_7d67e43646134c7e) } -var fileDescriptorData = []byte{ +var fileDescriptor_data_7d67e43646134c7e = []byte{ // 1881 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xcc, 0x58, 0xcb, 0x6f, 0xdb, 0xc8, 0x19, 0x37, 0xf5, 0xb0, 0xa8, 0x4f, 0x0f, 0xd3, 0xe3, 0x38, 0x51, 0xbd, 0xa8, 0x94, 0xd5, 0x16, diff --git a/pkg/roachpb/data_test.go b/pkg/roachpb/data_test.go index cd0dd1670920..9632765606be 100644 --- a/pkg/roachpb/data_test.go +++ b/pkg/roachpb/data_test.go @@ -764,11 +764,16 @@ func TestLeaseEqual(t *testing.T) { ProposedTS *hlc.Timestamp Epoch int64 Sequence LeaseSequence + XXX_NoUnkeyedLiteral struct{} + XXX_sizecache int32 } // Verify that the lease structure does not change unexpectedly. If a compile // error occurs on the following line of code, update the expectedLease // structure AND update Lease.Equal. var _ = expectedLease(Lease{}) + // Appease the linter. + var _ = expectedLease{}.XXX_NoUnkeyedLiteral + var _ = expectedLease{}.XXX_sizecache // Verify that nil == &hlc.Timestamp{} for the Expiration and // DeprecatedStartStasis fields. See #19843. @@ -910,28 +915,28 @@ func TestSpanContains(t *testing.T) { s := Span{Key: []byte("a"), EndKey: []byte("b")} testData := []struct { - start, end []byte + start, end string contains bool }{ // Single keys. - {[]byte("a"), nil, true}, - {[]byte("aa"), nil, true}, - {[]byte("`"), nil, false}, - {[]byte("b"), nil, false}, - {[]byte("c"), nil, false}, + {"a", "", true}, + {"aa", "", true}, + {"`", "", false}, + {"b", "", false}, + {"c", "", false}, // Key ranges. - {[]byte("a"), []byte("b"), true}, - {[]byte("a"), []byte("aa"), true}, - {[]byte("aa"), []byte("b"), true}, - {[]byte("0"), []byte("9"), false}, - {[]byte("`"), []byte("a"), false}, - {[]byte("b"), []byte("bb"), false}, - {[]byte("0"), []byte("bb"), false}, - {[]byte("aa"), []byte("bb"), false}, - {[]byte("b"), []byte("a"), false}, + {"a", "b", true}, + {"a", "aa", true}, + {"aa", "b", true}, + {"0", "9", false}, + {"`", "a", false}, + {"b", "bb", false}, + {"0", "bb", false}, + {"aa", "bb", false}, + {"b", "a", false}, } for i, test := range testData { - if s.Contains(Span{test.start, test.end}) != test.contains { + if s.Contains(sp(test.start, test.end)) != test.contains { t.Errorf("%d: expected span %q-%q within range to be %v", i, test.start, test.end, test.contains) } @@ -974,8 +979,8 @@ func TestSpanSplitOnKey(t *testing.T) { // Simple split. { []byte("bb"), - Span{[]byte("b"), []byte("bb")}, - Span{[]byte("bb"), []byte("c")}, + sp("b", "bb"), + sp("bb", "c"), }, } for testIdx, test := range testData { @@ -1005,7 +1010,7 @@ func TestSpanValid(t *testing.T) { {[]byte("b"), []byte("aa"), false}, } for i, test := range testData { - s := Span{test.start, test.end} + s := Span{Key: test.start, EndKey: test.end} if test.valid != s.Valid() { t.Errorf("%d: expected span %q-%q to return %t for Valid, instead got %t", i, test.start, test.end, test.valid, s.Valid()) diff --git a/pkg/roachpb/errors.go b/pkg/roachpb/errors.go index 111868f23ac4..4293d735ec07 100644 --- a/pkg/roachpb/errors.go +++ b/pkg/roachpb/errors.go @@ -17,6 +17,7 @@ package roachpb import ( "bytes" "fmt" + "strings" "github.com/cockroachdb/cockroach/pkg/util/caller" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -513,10 +514,20 @@ func (e *ReadWithinUncertaintyIntervalError) Error() string { } func (e *ReadWithinUncertaintyIntervalError) message(_ *Error) string { + var ts strings.Builder + ts.WriteByte('[') + for i, ot := range observedTimestampSlice(e.ObservedTimestamps) { + if i > 0 { + ts.WriteByte(' ') + } + fmt.Fprintf(&ts, "{%d %v}", ot.NodeID, ot.Timestamp) + } + ts.WriteByte(']') + return fmt.Sprintf("ReadWithinUncertaintyIntervalError: read at time %s encountered "+ "previous write with future timestamp %s within uncertainty interval `t <= %v`; "+ - "observed timestamps: %v", - e.ReadTimestamp, e.ExistingTimestamp, e.MaxTimestamp, observedTimestampSlice(e.ObservedTimestamps)) + "observed timestamps: %s", + e.ReadTimestamp, e.ExistingTimestamp, e.MaxTimestamp, ts.String()) } var _ ErrorDetailInterface = &ReadWithinUncertaintyIntervalError{} diff --git a/pkg/roachpb/errors.pb.go b/pkg/roachpb/errors.pb.go index 5dab9cefa22b..018fc16b84f9 100644 --- a/pkg/roachpb/errors.pb.go +++ b/pkg/roachpb/errors.pb.go @@ -6,7 +6,7 @@ package roachpb import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" -import cockroach_util_hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" +import hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" import github_com_cockroachdb_cockroach_pkg_util_uuid "github.com/cockroachdb/cockroach/pkg/util/uuid" @@ -19,6 +19,12 @@ var _ = proto.Marshal var _ = fmt.Errorf var _ = math.Inf +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package + // TransactionAbortedReason specifies what caused a TransactionAbortedError. // The reasons below are not necessarily disjoint - they describe where the // error was generated, but generally it's possible that a @@ -105,7 +111,7 @@ func (x *TransactionAbortedReason) UnmarshalJSON(data []byte) error { return nil } func (TransactionAbortedReason) EnumDescriptor() ([]byte, []int) { - return fileDescriptorErrors, []int{0} + return fileDescriptor_errors_9fc5358245dd263f, []int{0} } // TransactionRetryReason specifies what caused a transaction retry. @@ -156,7 +162,9 @@ func (x *TransactionRetryReason) UnmarshalJSON(data []byte) error { *x = TransactionRetryReason(value) return nil } -func (TransactionRetryReason) EnumDescriptor() ([]byte, []int) { return fileDescriptorErrors, []int{1} } +func (TransactionRetryReason) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_errors_9fc5358245dd263f, []int{1} +} // TransactionRestart indicates how an error should be handled in a // transactional context. @@ -205,7 +213,9 @@ func (x *TransactionRestart) UnmarshalJSON(data []byte) error { *x = TransactionRestart(value) return nil } -func (TransactionRestart) EnumDescriptor() ([]byte, []int) { return fileDescriptorErrors, []int{2} } +func (TransactionRestart) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_errors_9fc5358245dd263f, []int{2} +} // Reason specifies what caused the error. type TransactionStatusError_Reason int32 @@ -247,7 +257,7 @@ func (x *TransactionStatusError_Reason) UnmarshalJSON(data []byte) error { return nil } func (TransactionStatusError_Reason) EnumDescriptor() ([]byte, []int) { - return fileDescriptorErrors, []int{9, 0} + return fileDescriptor_errors_9fc5358245dd263f, []int{9, 0} } // Reason specifies what caused the error. @@ -298,7 +308,7 @@ func (x *RangeFeedRetryError_Reason) UnmarshalJSON(data []byte) error { return nil } func (RangeFeedRetryError_Reason) EnumDescriptor() ([]byte, []int) { - return fileDescriptorErrors, []int{29, 0} + return fileDescriptor_errors_9fc5358245dd263f, []int{29, 0} } // A NotLeaseHolderError indicates that the current range is not the lease @@ -318,47 +328,151 @@ type NotLeaseHolderError struct { // regular spiel. Useful because we reuse this error when rejecting a command // because the lease under which its application was attempted is different // than the lease under which it had been proposed. - CustomMsg string `protobuf:"bytes,5,opt,name=custom_msg,json=customMsg" json:"custom_msg"` + CustomMsg string `protobuf:"bytes,5,opt,name=custom_msg,json=customMsg" json:"custom_msg"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *NotLeaseHolderError) Reset() { *m = NotLeaseHolderError{} } +func (m *NotLeaseHolderError) String() string { return proto.CompactTextString(m) } +func (*NotLeaseHolderError) ProtoMessage() {} +func (*NotLeaseHolderError) Descriptor() ([]byte, []int) { + return fileDescriptor_errors_9fc5358245dd263f, []int{0} +} +func (m *NotLeaseHolderError) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *NotLeaseHolderError) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *NotLeaseHolderError) XXX_Merge(src proto.Message) { + xxx_messageInfo_NotLeaseHolderError.Merge(dst, src) +} +func (m *NotLeaseHolderError) XXX_Size() int { + return m.Size() +} +func (m *NotLeaseHolderError) XXX_DiscardUnknown() { + xxx_messageInfo_NotLeaseHolderError.DiscardUnknown(m) } -func (m *NotLeaseHolderError) Reset() { *m = NotLeaseHolderError{} } -func (m *NotLeaseHolderError) String() string { return proto.CompactTextString(m) } -func (*NotLeaseHolderError) ProtoMessage() {} -func (*NotLeaseHolderError) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{0} } +var xxx_messageInfo_NotLeaseHolderError proto.InternalMessageInfo // A NodeUnavailableError indicates that the sending gateway can // not process requests at the time, and that the client should // retry the request with another peer. type NodeUnavailableError struct { + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *NodeUnavailableError) Reset() { *m = NodeUnavailableError{} } -func (m *NodeUnavailableError) String() string { return proto.CompactTextString(m) } -func (*NodeUnavailableError) ProtoMessage() {} -func (*NodeUnavailableError) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{1} } +func (m *NodeUnavailableError) Reset() { *m = NodeUnavailableError{} } +func (m *NodeUnavailableError) String() string { return proto.CompactTextString(m) } +func (*NodeUnavailableError) ProtoMessage() {} +func (*NodeUnavailableError) Descriptor() ([]byte, []int) { + return fileDescriptor_errors_9fc5358245dd263f, []int{1} +} +func (m *NodeUnavailableError) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *NodeUnavailableError) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *NodeUnavailableError) XXX_Merge(src proto.Message) { + xxx_messageInfo_NodeUnavailableError.Merge(dst, src) +} +func (m *NodeUnavailableError) XXX_Size() int { + return m.Size() +} +func (m *NodeUnavailableError) XXX_DiscardUnknown() { + xxx_messageInfo_NodeUnavailableError.DiscardUnknown(m) +} + +var xxx_messageInfo_NodeUnavailableError proto.InternalMessageInfo // An UnsupportedRequestError indicates that the recipient node // does not know how to handle the type of request received. type UnsupportedRequestError struct { + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *UnsupportedRequestError) Reset() { *m = UnsupportedRequestError{} } +func (m *UnsupportedRequestError) String() string { return proto.CompactTextString(m) } +func (*UnsupportedRequestError) ProtoMessage() {} +func (*UnsupportedRequestError) Descriptor() ([]byte, []int) { + return fileDescriptor_errors_9fc5358245dd263f, []int{2} +} +func (m *UnsupportedRequestError) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *UnsupportedRequestError) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *UnsupportedRequestError) XXX_Merge(src proto.Message) { + xxx_messageInfo_UnsupportedRequestError.Merge(dst, src) +} +func (m *UnsupportedRequestError) XXX_Size() int { + return m.Size() +} +func (m *UnsupportedRequestError) XXX_DiscardUnknown() { + xxx_messageInfo_UnsupportedRequestError.DiscardUnknown(m) } -func (m *UnsupportedRequestError) Reset() { *m = UnsupportedRequestError{} } -func (m *UnsupportedRequestError) String() string { return proto.CompactTextString(m) } -func (*UnsupportedRequestError) ProtoMessage() {} -func (*UnsupportedRequestError) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{2} } +var xxx_messageInfo_UnsupportedRequestError proto.InternalMessageInfo // A RangeNotFoundError indicates that a command was sent to a range // which is not hosted on this store. type RangeNotFoundError struct { RangeID RangeID `protobuf:"varint,1,opt,name=range_id,json=rangeId,casttype=RangeID" json:"range_id"` // store_id is nonzero only if the error originated on a Store. - StoreID StoreID `protobuf:"varint,2,opt,name=store_id,json=storeId,casttype=StoreID" json:"store_id"` + StoreID StoreID `protobuf:"varint,2,opt,name=store_id,json=storeId,casttype=StoreID" json:"store_id"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *RangeNotFoundError) Reset() { *m = RangeNotFoundError{} } -func (m *RangeNotFoundError) String() string { return proto.CompactTextString(m) } -func (*RangeNotFoundError) ProtoMessage() {} -func (*RangeNotFoundError) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{3} } +func (m *RangeNotFoundError) Reset() { *m = RangeNotFoundError{} } +func (m *RangeNotFoundError) String() string { return proto.CompactTextString(m) } +func (*RangeNotFoundError) ProtoMessage() {} +func (*RangeNotFoundError) Descriptor() ([]byte, []int) { + return fileDescriptor_errors_9fc5358245dd263f, []int{3} +} +func (m *RangeNotFoundError) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RangeNotFoundError) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RangeNotFoundError) XXX_Merge(src proto.Message) { + xxx_messageInfo_RangeNotFoundError.Merge(dst, src) +} +func (m *RangeNotFoundError) XXX_Size() int { + return m.Size() +} +func (m *RangeNotFoundError) XXX_DiscardUnknown() { + xxx_messageInfo_RangeNotFoundError.DiscardUnknown(m) +} + +var xxx_messageInfo_RangeNotFoundError proto.InternalMessageInfo // A RangeKeyMismatchError indicates that a command was sent to a // range which did not contain the key(s) specified by the command. @@ -372,13 +486,39 @@ type RangeKeyMismatchError struct { // suggested_range is a hint to the sender of a command about the range // they may be looking for. It is only populated when the recipient has // authoritative knowledge of the range requested by the sender. - SuggestedRange *RangeDescriptor `protobuf:"bytes,4,opt,name=suggested_range,json=suggestedRange" json:"suggested_range,omitempty"` + SuggestedRange *RangeDescriptor `protobuf:"bytes,4,opt,name=suggested_range,json=suggestedRange" json:"suggested_range,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *RangeKeyMismatchError) Reset() { *m = RangeKeyMismatchError{} } -func (m *RangeKeyMismatchError) String() string { return proto.CompactTextString(m) } -func (*RangeKeyMismatchError) ProtoMessage() {} -func (*RangeKeyMismatchError) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{4} } +func (m *RangeKeyMismatchError) Reset() { *m = RangeKeyMismatchError{} } +func (m *RangeKeyMismatchError) String() string { return proto.CompactTextString(m) } +func (*RangeKeyMismatchError) ProtoMessage() {} +func (*RangeKeyMismatchError) Descriptor() ([]byte, []int) { + return fileDescriptor_errors_9fc5358245dd263f, []int{4} +} +func (m *RangeKeyMismatchError) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RangeKeyMismatchError) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RangeKeyMismatchError) XXX_Merge(src proto.Message) { + xxx_messageInfo_RangeKeyMismatchError.Merge(dst, src) +} +func (m *RangeKeyMismatchError) XXX_Size() int { + return m.Size() +} +func (m *RangeKeyMismatchError) XXX_DiscardUnknown() { + xxx_messageInfo_RangeKeyMismatchError.DiscardUnknown(m) +} + +var xxx_messageInfo_RangeKeyMismatchError proto.InternalMessageInfo // A ReadWithinUncertaintyIntervalError indicates that a read at timestamp // encountered a write within the uncertainty interval of the reader. @@ -389,20 +529,44 @@ func (*RangeKeyMismatchError) Descriptor() ([]byte, []int) { return fileDescript type ReadWithinUncertaintyIntervalError struct { // This data below is purely informational and used to tailor the // error message. - ReadTimestamp cockroach_util_hlc.Timestamp `protobuf:"bytes,1,opt,name=read_timestamp,json=readTimestamp" json:"read_timestamp"` - ExistingTimestamp cockroach_util_hlc.Timestamp `protobuf:"bytes,2,opt,name=existing_timestamp,json=existingTimestamp" json:"existing_timestamp"` + ReadTimestamp hlc.Timestamp `protobuf:"bytes,1,opt,name=read_timestamp,json=readTimestamp" json:"read_timestamp"` + ExistingTimestamp hlc.Timestamp `protobuf:"bytes,2,opt,name=existing_timestamp,json=existingTimestamp" json:"existing_timestamp"` // The remaining fields may be missing when running in clusters that have // members at below CockroachDB v2.0. - MaxTimestamp *cockroach_util_hlc.Timestamp `protobuf:"bytes,3,opt,name=max_timestamp,json=maxTimestamp" json:"max_timestamp,omitempty"` - ObservedTimestamps []ObservedTimestamp `protobuf:"bytes,4,rep,name=observed_timestamps,json=observedTimestamps" json:"observed_timestamps"` + MaxTimestamp *hlc.Timestamp `protobuf:"bytes,3,opt,name=max_timestamp,json=maxTimestamp" json:"max_timestamp,omitempty"` + ObservedTimestamps []ObservedTimestamp `protobuf:"bytes,4,rep,name=observed_timestamps,json=observedTimestamps" json:"observed_timestamps"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *ReadWithinUncertaintyIntervalError) Reset() { *m = ReadWithinUncertaintyIntervalError{} } func (m *ReadWithinUncertaintyIntervalError) String() string { return proto.CompactTextString(m) } func (*ReadWithinUncertaintyIntervalError) ProtoMessage() {} func (*ReadWithinUncertaintyIntervalError) Descriptor() ([]byte, []int) { - return fileDescriptorErrors, []int{5} + return fileDescriptor_errors_9fc5358245dd263f, []int{5} +} +func (m *ReadWithinUncertaintyIntervalError) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) } +func (m *ReadWithinUncertaintyIntervalError) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ReadWithinUncertaintyIntervalError) XXX_Merge(src proto.Message) { + xxx_messageInfo_ReadWithinUncertaintyIntervalError.Merge(dst, src) +} +func (m *ReadWithinUncertaintyIntervalError) XXX_Size() int { + return m.Size() +} +func (m *ReadWithinUncertaintyIntervalError) XXX_DiscardUnknown() { + xxx_messageInfo_ReadWithinUncertaintyIntervalError.DiscardUnknown(m) +} + +var xxx_messageInfo_ReadWithinUncertaintyIntervalError proto.InternalMessageInfo // A TransactionAbortedError indicates that the client should retry the // transaction (and use a different txn id, as opposed to @@ -415,36 +579,114 @@ func (*ReadWithinUncertaintyIntervalError) Descriptor() ([]byte, []int) { // client; the ID should be checked and then attributes like the timestamp // should be used in creating a new txn. type TransactionAbortedError struct { - Reason TransactionAbortedReason `protobuf:"varint,1,opt,name=reason,enum=cockroach.roachpb.TransactionAbortedReason" json:"reason"` + Reason TransactionAbortedReason `protobuf:"varint,1,opt,name=reason,enum=cockroach.roachpb.TransactionAbortedReason" json:"reason"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *TransactionAbortedError) Reset() { *m = TransactionAbortedError{} } -func (m *TransactionAbortedError) String() string { return proto.CompactTextString(m) } -func (*TransactionAbortedError) ProtoMessage() {} -func (*TransactionAbortedError) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{6} } +func (m *TransactionAbortedError) Reset() { *m = TransactionAbortedError{} } +func (m *TransactionAbortedError) String() string { return proto.CompactTextString(m) } +func (*TransactionAbortedError) ProtoMessage() {} +func (*TransactionAbortedError) Descriptor() ([]byte, []int) { + return fileDescriptor_errors_9fc5358245dd263f, []int{6} +} +func (m *TransactionAbortedError) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *TransactionAbortedError) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *TransactionAbortedError) XXX_Merge(src proto.Message) { + xxx_messageInfo_TransactionAbortedError.Merge(dst, src) +} +func (m *TransactionAbortedError) XXX_Size() int { + return m.Size() +} +func (m *TransactionAbortedError) XXX_DiscardUnknown() { + xxx_messageInfo_TransactionAbortedError.DiscardUnknown(m) +} + +var xxx_messageInfo_TransactionAbortedError proto.InternalMessageInfo // A TransactionPushError indicates that the transaction could not // continue because it encountered a write intent from another // transaction which it was unable to push. type TransactionPushError struct { - PusheeTxn Transaction `protobuf:"bytes,1,opt,name=pushee_txn,json=pusheeTxn" json:"pushee_txn"` + PusheeTxn Transaction `protobuf:"bytes,1,opt,name=pushee_txn,json=pusheeTxn" json:"pushee_txn"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *TransactionPushError) Reset() { *m = TransactionPushError{} } -func (m *TransactionPushError) String() string { return proto.CompactTextString(m) } -func (*TransactionPushError) ProtoMessage() {} -func (*TransactionPushError) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{7} } +func (m *TransactionPushError) Reset() { *m = TransactionPushError{} } +func (m *TransactionPushError) String() string { return proto.CompactTextString(m) } +func (*TransactionPushError) ProtoMessage() {} +func (*TransactionPushError) Descriptor() ([]byte, []int) { + return fileDescriptor_errors_9fc5358245dd263f, []int{7} +} +func (m *TransactionPushError) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *TransactionPushError) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *TransactionPushError) XXX_Merge(src proto.Message) { + xxx_messageInfo_TransactionPushError.Merge(dst, src) +} +func (m *TransactionPushError) XXX_Size() int { + return m.Size() +} +func (m *TransactionPushError) XXX_DiscardUnknown() { + xxx_messageInfo_TransactionPushError.DiscardUnknown(m) +} + +var xxx_messageInfo_TransactionPushError proto.InternalMessageInfo // A TransactionRetryError indicates that the transaction must be // retried, usually with an increased transaction timestamp. type TransactionRetryError struct { - Reason TransactionRetryReason `protobuf:"varint,1,opt,name=reason,enum=cockroach.roachpb.TransactionRetryReason" json:"reason"` + Reason TransactionRetryReason `protobuf:"varint,1,opt,name=reason,enum=cockroach.roachpb.TransactionRetryReason" json:"reason"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *TransactionRetryError) Reset() { *m = TransactionRetryError{} } +func (m *TransactionRetryError) String() string { return proto.CompactTextString(m) } +func (*TransactionRetryError) ProtoMessage() {} +func (*TransactionRetryError) Descriptor() ([]byte, []int) { + return fileDescriptor_errors_9fc5358245dd263f, []int{8} +} +func (m *TransactionRetryError) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *TransactionRetryError) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *TransactionRetryError) XXX_Merge(src proto.Message) { + xxx_messageInfo_TransactionRetryError.Merge(dst, src) +} +func (m *TransactionRetryError) XXX_Size() int { + return m.Size() +} +func (m *TransactionRetryError) XXX_DiscardUnknown() { + xxx_messageInfo_TransactionRetryError.DiscardUnknown(m) } -func (m *TransactionRetryError) Reset() { *m = TransactionRetryError{} } -func (m *TransactionRetryError) String() string { return proto.CompactTextString(m) } -func (*TransactionRetryError) ProtoMessage() {} -func (*TransactionRetryError) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{8} } +var xxx_messageInfo_TransactionRetryError proto.InternalMessageInfo // A TransactionStatusError indicates that the transaction status is // incompatible with the requested operation. This might mean the @@ -453,14 +695,40 @@ func (*TransactionRetryError) Descriptor() ([]byte, []int) { return fileDescript // regression in transaction epoch or timestamp, both of which may // only monotonically increase. type TransactionStatusError struct { - Msg string `protobuf:"bytes,1,opt,name=msg" json:"msg"` - Reason TransactionStatusError_Reason `protobuf:"varint,2,opt,name=reason,enum=cockroach.roachpb.TransactionStatusError_Reason" json:"reason"` + Msg string `protobuf:"bytes,1,opt,name=msg" json:"msg"` + Reason TransactionStatusError_Reason `protobuf:"varint,2,opt,name=reason,enum=cockroach.roachpb.TransactionStatusError_Reason" json:"reason"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *TransactionStatusError) Reset() { *m = TransactionStatusError{} } -func (m *TransactionStatusError) String() string { return proto.CompactTextString(m) } -func (*TransactionStatusError) ProtoMessage() {} -func (*TransactionStatusError) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{9} } +func (m *TransactionStatusError) Reset() { *m = TransactionStatusError{} } +func (m *TransactionStatusError) String() string { return proto.CompactTextString(m) } +func (*TransactionStatusError) ProtoMessage() {} +func (*TransactionStatusError) Descriptor() ([]byte, []int) { + return fileDescriptor_errors_9fc5358245dd263f, []int{9} +} +func (m *TransactionStatusError) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *TransactionStatusError) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *TransactionStatusError) XXX_Merge(src proto.Message) { + xxx_messageInfo_TransactionStatusError.Merge(dst, src) +} +func (m *TransactionStatusError) XXX_Size() int { + return m.Size() +} +func (m *TransactionStatusError) XXX_DiscardUnknown() { + xxx_messageInfo_TransactionStatusError.DiscardUnknown(m) +} + +var xxx_messageInfo_TransactionStatusError proto.InternalMessageInfo // A WriteIntentError indicates that one or more write intent // belonging to another transaction were encountered leading to a @@ -468,76 +736,232 @@ func (*TransactionStatusError) Descriptor() ([]byte, []int) { return fileDescrip // was encountered are set, as are the txn records for the intents' // transactions. type WriteIntentError struct { - Intents []Intent `protobuf:"bytes,1,rep,name=intents" json:"intents"` + Intents []Intent `protobuf:"bytes,1,rep,name=intents" json:"intents"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *WriteIntentError) Reset() { *m = WriteIntentError{} } +func (m *WriteIntentError) String() string { return proto.CompactTextString(m) } +func (*WriteIntentError) ProtoMessage() {} +func (*WriteIntentError) Descriptor() ([]byte, []int) { + return fileDescriptor_errors_9fc5358245dd263f, []int{10} +} +func (m *WriteIntentError) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *WriteIntentError) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *WriteIntentError) XXX_Merge(src proto.Message) { + xxx_messageInfo_WriteIntentError.Merge(dst, src) +} +func (m *WriteIntentError) XXX_Size() int { + return m.Size() +} +func (m *WriteIntentError) XXX_DiscardUnknown() { + xxx_messageInfo_WriteIntentError.DiscardUnknown(m) } -func (m *WriteIntentError) Reset() { *m = WriteIntentError{} } -func (m *WriteIntentError) String() string { return proto.CompactTextString(m) } -func (*WriteIntentError) ProtoMessage() {} -func (*WriteIntentError) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{10} } +var xxx_messageInfo_WriteIntentError proto.InternalMessageInfo // A WriteTooOldError indicates that a write encountered a versioned // value newer than its timestamp, making it impossible to rewrite // history. The write is instead done at actual timestamp, which is // the timestamp of the existing version+1. type WriteTooOldError struct { - Timestamp cockroach_util_hlc.Timestamp `protobuf:"bytes,1,opt,name=timestamp" json:"timestamp"` - ActualTimestamp cockroach_util_hlc.Timestamp `protobuf:"bytes,2,opt,name=actual_timestamp,json=actualTimestamp" json:"actual_timestamp"` + Timestamp hlc.Timestamp `protobuf:"bytes,1,opt,name=timestamp" json:"timestamp"` + ActualTimestamp hlc.Timestamp `protobuf:"bytes,2,opt,name=actual_timestamp,json=actualTimestamp" json:"actual_timestamp"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *WriteTooOldError) Reset() { *m = WriteTooOldError{} } -func (m *WriteTooOldError) String() string { return proto.CompactTextString(m) } -func (*WriteTooOldError) ProtoMessage() {} -func (*WriteTooOldError) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{11} } +func (m *WriteTooOldError) Reset() { *m = WriteTooOldError{} } +func (m *WriteTooOldError) String() string { return proto.CompactTextString(m) } +func (*WriteTooOldError) ProtoMessage() {} +func (*WriteTooOldError) Descriptor() ([]byte, []int) { + return fileDescriptor_errors_9fc5358245dd263f, []int{11} +} +func (m *WriteTooOldError) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *WriteTooOldError) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *WriteTooOldError) XXX_Merge(src proto.Message) { + xxx_messageInfo_WriteTooOldError.Merge(dst, src) +} +func (m *WriteTooOldError) XXX_Size() int { + return m.Size() +} +func (m *WriteTooOldError) XXX_DiscardUnknown() { + xxx_messageInfo_WriteTooOldError.DiscardUnknown(m) +} + +var xxx_messageInfo_WriteTooOldError proto.InternalMessageInfo // An OpRequiresTxnError indicates that a command required to be // carried out in a transactional context but was not. // For example, a Scan which spans ranges requires a transaction. // The operation should be retried inside of a transaction. type OpRequiresTxnError struct { + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *OpRequiresTxnError) Reset() { *m = OpRequiresTxnError{} } -func (m *OpRequiresTxnError) String() string { return proto.CompactTextString(m) } -func (*OpRequiresTxnError) ProtoMessage() {} -func (*OpRequiresTxnError) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{12} } +func (m *OpRequiresTxnError) Reset() { *m = OpRequiresTxnError{} } +func (m *OpRequiresTxnError) String() string { return proto.CompactTextString(m) } +func (*OpRequiresTxnError) ProtoMessage() {} +func (*OpRequiresTxnError) Descriptor() ([]byte, []int) { + return fileDescriptor_errors_9fc5358245dd263f, []int{12} +} +func (m *OpRequiresTxnError) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *OpRequiresTxnError) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *OpRequiresTxnError) XXX_Merge(src proto.Message) { + xxx_messageInfo_OpRequiresTxnError.Merge(dst, src) +} +func (m *OpRequiresTxnError) XXX_Size() int { + return m.Size() +} +func (m *OpRequiresTxnError) XXX_DiscardUnknown() { + xxx_messageInfo_OpRequiresTxnError.DiscardUnknown(m) +} + +var xxx_messageInfo_OpRequiresTxnError proto.InternalMessageInfo // A ConditionFailedError indicates that the expected value // of a ConditionalPutRequest was not found, either // because it was missing or was not equal. The error will // contain the actual value found. type ConditionFailedError struct { - ActualValue *Value `protobuf:"bytes,1,opt,name=actual_value,json=actualValue" json:"actual_value,omitempty"` + ActualValue *Value `protobuf:"bytes,1,opt,name=actual_value,json=actualValue" json:"actual_value,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ConditionFailedError) Reset() { *m = ConditionFailedError{} } +func (m *ConditionFailedError) String() string { return proto.CompactTextString(m) } +func (*ConditionFailedError) ProtoMessage() {} +func (*ConditionFailedError) Descriptor() ([]byte, []int) { + return fileDescriptor_errors_9fc5358245dd263f, []int{13} +} +func (m *ConditionFailedError) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ConditionFailedError) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ConditionFailedError) XXX_Merge(src proto.Message) { + xxx_messageInfo_ConditionFailedError.Merge(dst, src) +} +func (m *ConditionFailedError) XXX_Size() int { + return m.Size() +} +func (m *ConditionFailedError) XXX_DiscardUnknown() { + xxx_messageInfo_ConditionFailedError.DiscardUnknown(m) } -func (m *ConditionFailedError) Reset() { *m = ConditionFailedError{} } -func (m *ConditionFailedError) String() string { return proto.CompactTextString(m) } -func (*ConditionFailedError) ProtoMessage() {} -func (*ConditionFailedError) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{13} } +var xxx_messageInfo_ConditionFailedError proto.InternalMessageInfo // A LeaseRejectedError indicates that the requested replica could // not acquire the desired lease because of an existing range lease. type LeaseRejectedError struct { - Message string `protobuf:"bytes,1,opt,name=message" json:"message"` - Requested Lease `protobuf:"bytes,2,opt,name=requested" json:"requested"` - Existing Lease `protobuf:"bytes,3,opt,name=existing" json:"existing"` + Message string `protobuf:"bytes,1,opt,name=message" json:"message"` + Requested Lease `protobuf:"bytes,2,opt,name=requested" json:"requested"` + Existing Lease `protobuf:"bytes,3,opt,name=existing" json:"existing"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *LeaseRejectedError) Reset() { *m = LeaseRejectedError{} } +func (m *LeaseRejectedError) String() string { return proto.CompactTextString(m) } +func (*LeaseRejectedError) ProtoMessage() {} +func (*LeaseRejectedError) Descriptor() ([]byte, []int) { + return fileDescriptor_errors_9fc5358245dd263f, []int{14} +} +func (m *LeaseRejectedError) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *LeaseRejectedError) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *LeaseRejectedError) XXX_Merge(src proto.Message) { + xxx_messageInfo_LeaseRejectedError.Merge(dst, src) +} +func (m *LeaseRejectedError) XXX_Size() int { + return m.Size() +} +func (m *LeaseRejectedError) XXX_DiscardUnknown() { + xxx_messageInfo_LeaseRejectedError.DiscardUnknown(m) } -func (m *LeaseRejectedError) Reset() { *m = LeaseRejectedError{} } -func (m *LeaseRejectedError) String() string { return proto.CompactTextString(m) } -func (*LeaseRejectedError) ProtoMessage() {} -func (*LeaseRejectedError) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{14} } +var xxx_messageInfo_LeaseRejectedError proto.InternalMessageInfo // A SendError indicates that a message could not be delivered to // the desired recipient(s). type SendError struct { - Message string `protobuf:"bytes,1,opt,name=message" json:"message"` + Message string `protobuf:"bytes,1,opt,name=message" json:"message"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *SendError) Reset() { *m = SendError{} } -func (m *SendError) String() string { return proto.CompactTextString(m) } -func (*SendError) ProtoMessage() {} -func (*SendError) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{15} } +func (m *SendError) Reset() { *m = SendError{} } +func (m *SendError) String() string { return proto.CompactTextString(m) } +func (*SendError) ProtoMessage() {} +func (*SendError) Descriptor() ([]byte, []int) { + return fileDescriptor_errors_9fc5358245dd263f, []int{15} +} +func (m *SendError) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SendError) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *SendError) XXX_Merge(src proto.Message) { + xxx_messageInfo_SendError.Merge(dst, src) +} +func (m *SendError) XXX_Size() int { + return m.Size() +} +func (m *SendError) XXX_DiscardUnknown() { + xxx_messageInfo_SendError.DiscardUnknown(m) +} + +var xxx_messageInfo_SendError proto.InternalMessageInfo // An AmbiguousResultError indicates that a request may have succeeded or // failed, but the response was not received and the final result is ambiguous. @@ -545,23 +969,75 @@ type AmbiguousResultError struct { Message string `protobuf:"bytes,1,opt,name=message" json:"message"` // This can be set to give extra information about which error was converted // into an AmbiguousResultError. Useful for tests. - WrappedErr *Error `protobuf:"bytes,2,opt,name=wrapped_err,json=wrappedErr" json:"wrapped_err,omitempty"` + WrappedErr *Error `protobuf:"bytes,2,opt,name=wrapped_err,json=wrappedErr" json:"wrapped_err,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *AmbiguousResultError) Reset() { *m = AmbiguousResultError{} } +func (m *AmbiguousResultError) String() string { return proto.CompactTextString(m) } +func (*AmbiguousResultError) ProtoMessage() {} +func (*AmbiguousResultError) Descriptor() ([]byte, []int) { + return fileDescriptor_errors_9fc5358245dd263f, []int{16} +} +func (m *AmbiguousResultError) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *AmbiguousResultError) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *AmbiguousResultError) XXX_Merge(src proto.Message) { + xxx_messageInfo_AmbiguousResultError.Merge(dst, src) +} +func (m *AmbiguousResultError) XXX_Size() int { + return m.Size() +} +func (m *AmbiguousResultError) XXX_DiscardUnknown() { + xxx_messageInfo_AmbiguousResultError.DiscardUnknown(m) } -func (m *AmbiguousResultError) Reset() { *m = AmbiguousResultError{} } -func (m *AmbiguousResultError) String() string { return proto.CompactTextString(m) } -func (*AmbiguousResultError) ProtoMessage() {} -func (*AmbiguousResultError) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{16} } +var xxx_messageInfo_AmbiguousResultError proto.InternalMessageInfo // A RaftGroupDeletedError indicates a raft group has been deleted for // the replica. type RaftGroupDeletedError struct { + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *RaftGroupDeletedError) Reset() { *m = RaftGroupDeletedError{} } -func (m *RaftGroupDeletedError) String() string { return proto.CompactTextString(m) } -func (*RaftGroupDeletedError) ProtoMessage() {} -func (*RaftGroupDeletedError) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{17} } +func (m *RaftGroupDeletedError) Reset() { *m = RaftGroupDeletedError{} } +func (m *RaftGroupDeletedError) String() string { return proto.CompactTextString(m) } +func (*RaftGroupDeletedError) ProtoMessage() {} +func (*RaftGroupDeletedError) Descriptor() ([]byte, []int) { + return fileDescriptor_errors_9fc5358245dd263f, []int{17} +} +func (m *RaftGroupDeletedError) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RaftGroupDeletedError) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RaftGroupDeletedError) XXX_Merge(src proto.Message) { + xxx_messageInfo_RaftGroupDeletedError.Merge(dst, src) +} +func (m *RaftGroupDeletedError) XXX_Size() int { + return m.Size() +} +func (m *RaftGroupDeletedError) XXX_DiscardUnknown() { + xxx_messageInfo_RaftGroupDeletedError.DiscardUnknown(m) +} + +var xxx_messageInfo_RaftGroupDeletedError proto.InternalMessageInfo // A ReplicaCorruptionError indicates that the replica has experienced // an error which puts its integrity at risk. @@ -569,37 +1045,115 @@ type ReplicaCorruptionError struct { ErrorMsg string `protobuf:"bytes,1,opt,name=error_msg,json=errorMsg" json:"error_msg"` // processed indicates that the error has been taken into account and // necessary steps will be taken. For now, required for testing. - Processed bool `protobuf:"varint,2,opt,name=processed" json:"processed"` + Processed bool `protobuf:"varint,2,opt,name=processed" json:"processed"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ReplicaCorruptionError) Reset() { *m = ReplicaCorruptionError{} } +func (m *ReplicaCorruptionError) String() string { return proto.CompactTextString(m) } +func (*ReplicaCorruptionError) ProtoMessage() {} +func (*ReplicaCorruptionError) Descriptor() ([]byte, []int) { + return fileDescriptor_errors_9fc5358245dd263f, []int{18} +} +func (m *ReplicaCorruptionError) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ReplicaCorruptionError) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ReplicaCorruptionError) XXX_Merge(src proto.Message) { + xxx_messageInfo_ReplicaCorruptionError.Merge(dst, src) +} +func (m *ReplicaCorruptionError) XXX_Size() int { + return m.Size() +} +func (m *ReplicaCorruptionError) XXX_DiscardUnknown() { + xxx_messageInfo_ReplicaCorruptionError.DiscardUnknown(m) } -func (m *ReplicaCorruptionError) Reset() { *m = ReplicaCorruptionError{} } -func (m *ReplicaCorruptionError) String() string { return proto.CompactTextString(m) } -func (*ReplicaCorruptionError) ProtoMessage() {} -func (*ReplicaCorruptionError) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{18} } +var xxx_messageInfo_ReplicaCorruptionError proto.InternalMessageInfo // ReplicaTooOldError is sent in response to a raft message when the // recipient of the raft message believes the sender of the raft // message to have been removed from the raft group type ReplicaTooOldError struct { // replica_id is the ID of the replica that is too old. - ReplicaID ReplicaID `protobuf:"varint,1,opt,name=replica_id,json=replicaId,casttype=ReplicaID" json:"replica_id"` + ReplicaID ReplicaID `protobuf:"varint,1,opt,name=replica_id,json=replicaId,casttype=ReplicaID" json:"replica_id"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *ReplicaTooOldError) Reset() { *m = ReplicaTooOldError{} } -func (m *ReplicaTooOldError) String() string { return proto.CompactTextString(m) } -func (*ReplicaTooOldError) ProtoMessage() {} -func (*ReplicaTooOldError) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{19} } +func (m *ReplicaTooOldError) Reset() { *m = ReplicaTooOldError{} } +func (m *ReplicaTooOldError) String() string { return proto.CompactTextString(m) } +func (*ReplicaTooOldError) ProtoMessage() {} +func (*ReplicaTooOldError) Descriptor() ([]byte, []int) { + return fileDescriptor_errors_9fc5358245dd263f, []int{19} +} +func (m *ReplicaTooOldError) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ReplicaTooOldError) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ReplicaTooOldError) XXX_Merge(src proto.Message) { + xxx_messageInfo_ReplicaTooOldError.Merge(dst, src) +} +func (m *ReplicaTooOldError) XXX_Size() int { + return m.Size() +} +func (m *ReplicaTooOldError) XXX_DiscardUnknown() { + xxx_messageInfo_ReplicaTooOldError.DiscardUnknown(m) +} + +var xxx_messageInfo_ReplicaTooOldError proto.InternalMessageInfo // A StoreNotFoundError indicates that a command was sent to a store // which is not hosted on this node. type StoreNotFoundError struct { - StoreID StoreID `protobuf:"varint,1,opt,name=store_id,json=storeId,casttype=StoreID" json:"store_id"` + StoreID StoreID `protobuf:"varint,1,opt,name=store_id,json=storeId,casttype=StoreID" json:"store_id"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *StoreNotFoundError) Reset() { *m = StoreNotFoundError{} } -func (m *StoreNotFoundError) String() string { return proto.CompactTextString(m) } -func (*StoreNotFoundError) ProtoMessage() {} -func (*StoreNotFoundError) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{20} } +func (m *StoreNotFoundError) Reset() { *m = StoreNotFoundError{} } +func (m *StoreNotFoundError) String() string { return proto.CompactTextString(m) } +func (*StoreNotFoundError) ProtoMessage() {} +func (*StoreNotFoundError) Descriptor() ([]byte, []int) { + return fileDescriptor_errors_9fc5358245dd263f, []int{20} +} +func (m *StoreNotFoundError) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *StoreNotFoundError) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *StoreNotFoundError) XXX_Merge(src proto.Message) { + xxx_messageInfo_StoreNotFoundError.Merge(dst, src) +} +func (m *StoreNotFoundError) XXX_Size() int { + return m.Size() +} +func (m *StoreNotFoundError) XXX_DiscardUnknown() { + xxx_messageInfo_StoreNotFoundError.DiscardUnknown(m) +} + +var xxx_messageInfo_StoreNotFoundError proto.InternalMessageInfo // UnhandledRetryableError tells the recipient that a KV request must be // retried. In case the request was transactional, the whole transaction needs @@ -615,13 +1169,39 @@ type UnhandledRetryableError struct { // The underlying storage error that is being marshaled. // pErr.TransactionRestart is expected to be set, and the error // detail is one of the retryable ones. - PErr Error `protobuf:"bytes,1,opt,name=pErr" json:"pErr"` + PErr Error `protobuf:"bytes,1,opt,name=pErr" json:"pErr"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *UnhandledRetryableError) Reset() { *m = UnhandledRetryableError{} } -func (m *UnhandledRetryableError) String() string { return proto.CompactTextString(m) } -func (*UnhandledRetryableError) ProtoMessage() {} -func (*UnhandledRetryableError) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{21} } +func (m *UnhandledRetryableError) Reset() { *m = UnhandledRetryableError{} } +func (m *UnhandledRetryableError) String() string { return proto.CompactTextString(m) } +func (*UnhandledRetryableError) ProtoMessage() {} +func (*UnhandledRetryableError) Descriptor() ([]byte, []int) { + return fileDescriptor_errors_9fc5358245dd263f, []int{21} +} +func (m *UnhandledRetryableError) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *UnhandledRetryableError) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *UnhandledRetryableError) XXX_Merge(src proto.Message) { + xxx_messageInfo_UnhandledRetryableError.Merge(dst, src) +} +func (m *UnhandledRetryableError) XXX_Size() int { + return m.Size() +} +func (m *UnhandledRetryableError) XXX_DiscardUnknown() { + xxx_messageInfo_UnhandledRetryableError.DiscardUnknown(m) +} + +var xxx_messageInfo_UnhandledRetryableError proto.InternalMessageInfo // HandledRetryableTxnError is an error detail representing a retryable error // that has been "handled" by the TxnCoordSender. This error is produced by the @@ -643,78 +1223,230 @@ type HandledRetryableTxnError struct { // original cause of this method, this can either be the same Transaction as // before, but with an incremented epoch and timestamp, or a completely new // Transaction. - Transaction Transaction `protobuf:"bytes,3,opt,name=transaction" json:"transaction"` + Transaction Transaction `protobuf:"bytes,3,opt,name=transaction" json:"transaction"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *HandledRetryableTxnError) Reset() { *m = HandledRetryableTxnError{} } +func (m *HandledRetryableTxnError) String() string { return proto.CompactTextString(m) } +func (*HandledRetryableTxnError) ProtoMessage() {} +func (*HandledRetryableTxnError) Descriptor() ([]byte, []int) { + return fileDescriptor_errors_9fc5358245dd263f, []int{22} +} +func (m *HandledRetryableTxnError) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *HandledRetryableTxnError) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *HandledRetryableTxnError) XXX_Merge(src proto.Message) { + xxx_messageInfo_HandledRetryableTxnError.Merge(dst, src) +} +func (m *HandledRetryableTxnError) XXX_Size() int { + return m.Size() +} +func (m *HandledRetryableTxnError) XXX_DiscardUnknown() { + xxx_messageInfo_HandledRetryableTxnError.DiscardUnknown(m) } -func (m *HandledRetryableTxnError) Reset() { *m = HandledRetryableTxnError{} } -func (m *HandledRetryableTxnError) String() string { return proto.CompactTextString(m) } -func (*HandledRetryableTxnError) ProtoMessage() {} -func (*HandledRetryableTxnError) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{22} } +var xxx_messageInfo_HandledRetryableTxnError proto.InternalMessageInfo // TxnAlreadyEncounteredErrorError indicates that an operation tried to use a // transaction that already received an error from a previous request. Once that // happens, client.Txn rejects future requests. type TxnAlreadyEncounteredErrorError struct { + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *TxnAlreadyEncounteredErrorError) Reset() { *m = TxnAlreadyEncounteredErrorError{} } func (m *TxnAlreadyEncounteredErrorError) String() string { return proto.CompactTextString(m) } func (*TxnAlreadyEncounteredErrorError) ProtoMessage() {} func (*TxnAlreadyEncounteredErrorError) Descriptor() ([]byte, []int) { - return fileDescriptorErrors, []int{23} + return fileDescriptor_errors_9fc5358245dd263f, []int{23} +} +func (m *TxnAlreadyEncounteredErrorError) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) } +func (m *TxnAlreadyEncounteredErrorError) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *TxnAlreadyEncounteredErrorError) XXX_Merge(src proto.Message) { + xxx_messageInfo_TxnAlreadyEncounteredErrorError.Merge(dst, src) +} +func (m *TxnAlreadyEncounteredErrorError) XXX_Size() int { + return m.Size() +} +func (m *TxnAlreadyEncounteredErrorError) XXX_DiscardUnknown() { + xxx_messageInfo_TxnAlreadyEncounteredErrorError.DiscardUnknown(m) +} + +var xxx_messageInfo_TxnAlreadyEncounteredErrorError proto.InternalMessageInfo // An IntegerOverflowError indicates that an operation was aborted because // it would have caused an integeter overflow. type IntegerOverflowError struct { - Key Key `protobuf:"bytes,1,opt,name=key,casttype=Key" json:"key,omitempty"` - CurrentValue int64 `protobuf:"varint,2,opt,name=current_value,json=currentValue" json:"current_value"` - IncrementValue int64 `protobuf:"varint,3,opt,name=increment_value,json=incrementValue" json:"increment_value"` + Key Key `protobuf:"bytes,1,opt,name=key,casttype=Key" json:"key,omitempty"` + CurrentValue int64 `protobuf:"varint,2,opt,name=current_value,json=currentValue" json:"current_value"` + IncrementValue int64 `protobuf:"varint,3,opt,name=increment_value,json=incrementValue" json:"increment_value"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *IntegerOverflowError) Reset() { *m = IntegerOverflowError{} } +func (m *IntegerOverflowError) String() string { return proto.CompactTextString(m) } +func (*IntegerOverflowError) ProtoMessage() {} +func (*IntegerOverflowError) Descriptor() ([]byte, []int) { + return fileDescriptor_errors_9fc5358245dd263f, []int{24} +} +func (m *IntegerOverflowError) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *IntegerOverflowError) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *IntegerOverflowError) XXX_Merge(src proto.Message) { + xxx_messageInfo_IntegerOverflowError.Merge(dst, src) +} +func (m *IntegerOverflowError) XXX_Size() int { + return m.Size() +} +func (m *IntegerOverflowError) XXX_DiscardUnknown() { + xxx_messageInfo_IntegerOverflowError.DiscardUnknown(m) } -func (m *IntegerOverflowError) Reset() { *m = IntegerOverflowError{} } -func (m *IntegerOverflowError) String() string { return proto.CompactTextString(m) } -func (*IntegerOverflowError) ProtoMessage() {} -func (*IntegerOverflowError) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{24} } +var xxx_messageInfo_IntegerOverflowError proto.InternalMessageInfo // A MixedSuccessError indicates that some portion of the batch // request may have succeeded, but the batch as a whole failed with // the wrapped error. type MixedSuccessError struct { - Wrapped *Error `protobuf:"bytes,1,opt,name=wrapped" json:"wrapped,omitempty"` + Wrapped *Error `protobuf:"bytes,1,opt,name=wrapped" json:"wrapped,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *MixedSuccessError) Reset() { *m = MixedSuccessError{} } -func (m *MixedSuccessError) String() string { return proto.CompactTextString(m) } -func (*MixedSuccessError) ProtoMessage() {} -func (*MixedSuccessError) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{25} } +func (m *MixedSuccessError) Reset() { *m = MixedSuccessError{} } +func (m *MixedSuccessError) String() string { return proto.CompactTextString(m) } +func (*MixedSuccessError) ProtoMessage() {} +func (*MixedSuccessError) Descriptor() ([]byte, []int) { + return fileDescriptor_errors_9fc5358245dd263f, []int{25} +} +func (m *MixedSuccessError) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *MixedSuccessError) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *MixedSuccessError) XXX_Merge(src proto.Message) { + xxx_messageInfo_MixedSuccessError.Merge(dst, src) +} +func (m *MixedSuccessError) XXX_Size() int { + return m.Size() +} +func (m *MixedSuccessError) XXX_DiscardUnknown() { + xxx_messageInfo_MixedSuccessError.DiscardUnknown(m) +} + +var xxx_messageInfo_MixedSuccessError proto.InternalMessageInfo // A BatchTimestampBeforeGCError indicates that a request's timestamp was // before the GC threshold. type BatchTimestampBeforeGCError struct { - Timestamp cockroach_util_hlc.Timestamp `protobuf:"bytes,1,opt,name=Timestamp" json:"Timestamp"` - Threshold cockroach_util_hlc.Timestamp `protobuf:"bytes,2,opt,name=Threshold" json:"Threshold"` + Timestamp hlc.Timestamp `protobuf:"bytes,1,opt,name=Timestamp" json:"Timestamp"` + Threshold hlc.Timestamp `protobuf:"bytes,2,opt,name=Threshold" json:"Threshold"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *BatchTimestampBeforeGCError) Reset() { *m = BatchTimestampBeforeGCError{} } func (m *BatchTimestampBeforeGCError) String() string { return proto.CompactTextString(m) } func (*BatchTimestampBeforeGCError) ProtoMessage() {} func (*BatchTimestampBeforeGCError) Descriptor() ([]byte, []int) { - return fileDescriptorErrors, []int{26} + return fileDescriptor_errors_9fc5358245dd263f, []int{26} +} +func (m *BatchTimestampBeforeGCError) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *BatchTimestampBeforeGCError) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *BatchTimestampBeforeGCError) XXX_Merge(src proto.Message) { + xxx_messageInfo_BatchTimestampBeforeGCError.Merge(dst, src) } +func (m *BatchTimestampBeforeGCError) XXX_Size() int { + return m.Size() +} +func (m *BatchTimestampBeforeGCError) XXX_DiscardUnknown() { + xxx_messageInfo_BatchTimestampBeforeGCError.DiscardUnknown(m) +} + +var xxx_messageInfo_BatchTimestampBeforeGCError proto.InternalMessageInfo // An IntentMissingError indicates that a QueryIntent request expected // an intent to be present at its specified key but the intent was // not there. type IntentMissingError struct { // The non-matching intent that was found at that key, if any. - WrongIntent *Intent `protobuf:"bytes,1,opt,name=wrong_intent,json=wrongIntent" json:"wrong_intent,omitempty"` + WrongIntent *Intent `protobuf:"bytes,1,opt,name=wrong_intent,json=wrongIntent" json:"wrong_intent,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *IntentMissingError) Reset() { *m = IntentMissingError{} } -func (m *IntentMissingError) String() string { return proto.CompactTextString(m) } -func (*IntentMissingError) ProtoMessage() {} -func (*IntentMissingError) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{27} } +func (m *IntentMissingError) Reset() { *m = IntentMissingError{} } +func (m *IntentMissingError) String() string { return proto.CompactTextString(m) } +func (*IntentMissingError) ProtoMessage() {} +func (*IntentMissingError) Descriptor() ([]byte, []int) { + return fileDescriptor_errors_9fc5358245dd263f, []int{27} +} +func (m *IntentMissingError) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *IntentMissingError) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *IntentMissingError) XXX_Merge(src proto.Message) { + xxx_messageInfo_IntentMissingError.Merge(dst, src) +} +func (m *IntentMissingError) XXX_Size() int { + return m.Size() +} +func (m *IntentMissingError) XXX_DiscardUnknown() { + xxx_messageInfo_IntentMissingError.DiscardUnknown(m) +} + +var xxx_messageInfo_IntentMissingError proto.InternalMessageInfo // A MergeInProgressError indicates that the request could not be completed // because the replica is being merged into its left-hand neighbor. The request @@ -722,23 +1454,75 @@ func (*IntentMissingError) Descriptor() ([]byte, []int) { return fileDescriptorE // // This error is handled by the Store and should not escape to higher levels. type MergeInProgressError struct { + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *MergeInProgressError) Reset() { *m = MergeInProgressError{} } +func (m *MergeInProgressError) String() string { return proto.CompactTextString(m) } +func (*MergeInProgressError) ProtoMessage() {} +func (*MergeInProgressError) Descriptor() ([]byte, []int) { + return fileDescriptor_errors_9fc5358245dd263f, []int{28} +} +func (m *MergeInProgressError) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *MergeInProgressError) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *MergeInProgressError) XXX_Merge(src proto.Message) { + xxx_messageInfo_MergeInProgressError.Merge(dst, src) +} +func (m *MergeInProgressError) XXX_Size() int { + return m.Size() +} +func (m *MergeInProgressError) XXX_DiscardUnknown() { + xxx_messageInfo_MergeInProgressError.DiscardUnknown(m) } -func (m *MergeInProgressError) Reset() { *m = MergeInProgressError{} } -func (m *MergeInProgressError) String() string { return proto.CompactTextString(m) } -func (*MergeInProgressError) ProtoMessage() {} -func (*MergeInProgressError) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{28} } +var xxx_messageInfo_MergeInProgressError proto.InternalMessageInfo // A RangeFeedRetryError indicates that a rangefeed was disconnected, often // because of a range lifecycle event, and can be retried. type RangeFeedRetryError struct { - Reason RangeFeedRetryError_Reason `protobuf:"varint,1,opt,name=reason,enum=cockroach.roachpb.RangeFeedRetryError_Reason" json:"reason"` + Reason RangeFeedRetryError_Reason `protobuf:"varint,1,opt,name=reason,enum=cockroach.roachpb.RangeFeedRetryError_Reason" json:"reason"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *RangeFeedRetryError) Reset() { *m = RangeFeedRetryError{} } +func (m *RangeFeedRetryError) String() string { return proto.CompactTextString(m) } +func (*RangeFeedRetryError) ProtoMessage() {} +func (*RangeFeedRetryError) Descriptor() ([]byte, []int) { + return fileDescriptor_errors_9fc5358245dd263f, []int{29} +} +func (m *RangeFeedRetryError) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RangeFeedRetryError) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RangeFeedRetryError) XXX_Merge(src proto.Message) { + xxx_messageInfo_RangeFeedRetryError.Merge(dst, src) +} +func (m *RangeFeedRetryError) XXX_Size() int { + return m.Size() +} +func (m *RangeFeedRetryError) XXX_DiscardUnknown() { + xxx_messageInfo_RangeFeedRetryError.DiscardUnknown(m) } -func (m *RangeFeedRetryError) Reset() { *m = RangeFeedRetryError{} } -func (m *RangeFeedRetryError) String() string { return proto.CompactTextString(m) } -func (*RangeFeedRetryError) ProtoMessage() {} -func (*RangeFeedRetryError) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{29} } +var xxx_messageInfo_RangeFeedRetryError proto.InternalMessageInfo // ErrorDetail is a union type containing all available errors. type ErrorDetail struct { @@ -772,13 +1556,39 @@ type ErrorDetail struct { // *ErrorDetail_IntentMissing // *ErrorDetail_MergeInProgress // *ErrorDetail_RangefeedRetry - Value isErrorDetail_Value `protobuf_oneof:"value"` + Value isErrorDetail_Value `protobuf_oneof:"value"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *ErrorDetail) Reset() { *m = ErrorDetail{} } -func (m *ErrorDetail) String() string { return proto.CompactTextString(m) } -func (*ErrorDetail) ProtoMessage() {} -func (*ErrorDetail) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{30} } +func (m *ErrorDetail) Reset() { *m = ErrorDetail{} } +func (m *ErrorDetail) String() string { return proto.CompactTextString(m) } +func (*ErrorDetail) ProtoMessage() {} +func (*ErrorDetail) Descriptor() ([]byte, []int) { + return fileDescriptor_errors_9fc5358245dd263f, []int{30} +} +func (m *ErrorDetail) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ErrorDetail) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ErrorDetail) XXX_Merge(src proto.Message) { + xxx_messageInfo_ErrorDetail.Merge(dst, src) +} +func (m *ErrorDetail) XXX_Size() int { + return m.Size() +} +func (m *ErrorDetail) XXX_DiscardUnknown() { + xxx_messageInfo_ErrorDetail.DiscardUnknown(m) +} + +var xxx_messageInfo_ErrorDetail proto.InternalMessageInfo type isErrorDetail_Value interface { isErrorDetail_Value() @@ -1552,147 +2362,147 @@ func _ErrorDetail_OneofSizer(msg proto.Message) (n int) { switch x := m.Value.(type) { case *ErrorDetail_NotLeaseHolder: s := proto.Size(x.NotLeaseHolder) - n += proto.SizeVarint(1<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ErrorDetail_RangeNotFound: s := proto.Size(x.RangeNotFound) - n += proto.SizeVarint(2<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ErrorDetail_RangeKeyMismatch: s := proto.Size(x.RangeKeyMismatch) - n += proto.SizeVarint(3<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ErrorDetail_ReadWithinUncertaintyInterval: s := proto.Size(x.ReadWithinUncertaintyInterval) - n += proto.SizeVarint(4<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ErrorDetail_TransactionAborted: s := proto.Size(x.TransactionAborted) - n += proto.SizeVarint(5<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ErrorDetail_TransactionPush: s := proto.Size(x.TransactionPush) - n += proto.SizeVarint(6<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ErrorDetail_TransactionRetry: s := proto.Size(x.TransactionRetry) - n += proto.SizeVarint(7<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ErrorDetail_TransactionStatus: s := proto.Size(x.TransactionStatus) - n += proto.SizeVarint(8<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ErrorDetail_WriteIntent: s := proto.Size(x.WriteIntent) - n += proto.SizeVarint(9<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ErrorDetail_WriteTooOld: s := proto.Size(x.WriteTooOld) - n += proto.SizeVarint(10<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ErrorDetail_OpRequiresTxn: s := proto.Size(x.OpRequiresTxn) - n += proto.SizeVarint(11<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ErrorDetail_ConditionFailed: s := proto.Size(x.ConditionFailed) - n += proto.SizeVarint(12<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ErrorDetail_LeaseRejected: s := proto.Size(x.LeaseRejected) - n += proto.SizeVarint(13<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ErrorDetail_NodeUnavailable: s := proto.Size(x.NodeUnavailable) - n += proto.SizeVarint(14<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ErrorDetail_Send: s := proto.Size(x.Send) - n += proto.SizeVarint(15<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ErrorDetail_RaftGroupDeleted: s := proto.Size(x.RaftGroupDeleted) - n += proto.SizeVarint(16<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ErrorDetail_ReplicaCorruption: s := proto.Size(x.ReplicaCorruption) - n += proto.SizeVarint(17<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ErrorDetail_ReplicaTooOld: s := proto.Size(x.ReplicaTooOld) - n += proto.SizeVarint(18<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ErrorDetail_AmbiguousResult: s := proto.Size(x.AmbiguousResult) - n += proto.SizeVarint(26<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ErrorDetail_StoreNotFound: s := proto.Size(x.StoreNotFound) - n += proto.SizeVarint(27<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ErrorDetail_HandledRetryableTxnError: s := proto.Size(x.HandledRetryableTxnError) - n += proto.SizeVarint(28<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ErrorDetail_IntegerOverflow: s := proto.Size(x.IntegerOverflow) - n += proto.SizeVarint(31<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ErrorDetail_UnsupportedRequest: s := proto.Size(x.UnsupportedRequest) - n += proto.SizeVarint(32<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ErrorDetail_MixedSuccess: s := proto.Size(x.MixedSuccess) - n += proto.SizeVarint(33<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ErrorDetail_TimestampBefore: s := proto.Size(x.TimestampBefore) - n += proto.SizeVarint(34<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ErrorDetail_TxnAlreadyEncounteredError: s := proto.Size(x.TxnAlreadyEncounteredError) - n += proto.SizeVarint(35<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ErrorDetail_IntentMissing: s := proto.Size(x.IntentMissing) - n += proto.SizeVarint(36<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ErrorDetail_MergeInProgress: s := proto.Size(x.MergeInProgress) - n += proto.SizeVarint(37<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *ErrorDetail_RangefeedRetry: s := proto.Size(x.RangefeedRetry) - n += proto.SizeVarint(38<<3 | proto.WireBytes) + n += 2 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case nil: @@ -1706,13 +2516,39 @@ func _ErrorDetail_OneofSizer(msg proto.Message) (n int) { // primitive field would break compatibility with proto3, where primitive fields // are no longer allowed to be nullable. type ErrPosition struct { - Index int32 `protobuf:"varint,1,opt,name=index" json:"index"` + Index int32 `protobuf:"varint,1,opt,name=index" json:"index"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *ErrPosition) Reset() { *m = ErrPosition{} } -func (m *ErrPosition) String() string { return proto.CompactTextString(m) } -func (*ErrPosition) ProtoMessage() {} -func (*ErrPosition) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{31} } +func (m *ErrPosition) Reset() { *m = ErrPosition{} } +func (m *ErrPosition) String() string { return proto.CompactTextString(m) } +func (*ErrPosition) ProtoMessage() {} +func (*ErrPosition) Descriptor() ([]byte, []int) { + return fileDescriptor_errors_9fc5358245dd263f, []int{31} +} +func (m *ErrPosition) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ErrPosition) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ErrPosition) XXX_Merge(src proto.Message) { + xxx_messageInfo_ErrPosition.Merge(dst, src) +} +func (m *ErrPosition) XXX_Size() int { + return m.Size() +} +func (m *ErrPosition) XXX_DiscardUnknown() { + xxx_messageInfo_ErrPosition.DiscardUnknown(m) +} + +var xxx_messageInfo_ErrPosition proto.InternalMessageInfo // Error is a generic representation including a string message // and information about retryability. @@ -1737,12 +2573,38 @@ type Error struct { Index *ErrPosition `protobuf:"bytes,7,opt,name=index" json:"index,omitempty"` // now is the current time at the node sending the response, // which can be used by the receiver to update its local HLC. - Now cockroach_util_hlc.Timestamp `protobuf:"bytes,8,opt,name=now" json:"now"` + Now hlc.Timestamp `protobuf:"bytes,8,opt,name=now" json:"now"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Error) Reset() { *m = Error{} } +func (*Error) ProtoMessage() {} +func (*Error) Descriptor() ([]byte, []int) { + return fileDescriptor_errors_9fc5358245dd263f, []int{32} +} +func (m *Error) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Error) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *Error) XXX_Merge(src proto.Message) { + xxx_messageInfo_Error.Merge(dst, src) +} +func (m *Error) XXX_Size() int { + return m.Size() +} +func (m *Error) XXX_DiscardUnknown() { + xxx_messageInfo_Error.DiscardUnknown(m) } -func (m *Error) Reset() { *m = Error{} } -func (*Error) ProtoMessage() {} -func (*Error) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{32} } +var xxx_messageInfo_Error proto.InternalMessageInfo func init() { proto.RegisterType((*NotLeaseHolderError)(nil), "cockroach.roachpb.NotLeaseHolderError") @@ -4729,6 +5591,9 @@ func encodeVarintErrors(dAtA []byte, offset int, v uint64) int { return offset + 1 } func (m *NotLeaseHolderError) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.Replica.Size() @@ -4748,18 +5613,27 @@ func (m *NotLeaseHolderError) Size() (n int) { } func (m *NodeUnavailableError) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l return n } func (m *UnsupportedRequestError) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l return n } func (m *RangeNotFoundError) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovErrors(uint64(m.RangeID)) @@ -4768,6 +5642,9 @@ func (m *RangeNotFoundError) Size() (n int) { } func (m *RangeKeyMismatchError) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.RequestStartKey != nil { @@ -4790,6 +5667,9 @@ func (m *RangeKeyMismatchError) Size() (n int) { } func (m *ReadWithinUncertaintyIntervalError) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.ReadTimestamp.Size() @@ -4810,6 +5690,9 @@ func (m *ReadWithinUncertaintyIntervalError) Size() (n int) { } func (m *TransactionAbortedError) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovErrors(uint64(m.Reason)) @@ -4817,6 +5700,9 @@ func (m *TransactionAbortedError) Size() (n int) { } func (m *TransactionPushError) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.PusheeTxn.Size() @@ -4825,6 +5711,9 @@ func (m *TransactionPushError) Size() (n int) { } func (m *TransactionRetryError) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovErrors(uint64(m.Reason)) @@ -4832,6 +5721,9 @@ func (m *TransactionRetryError) Size() (n int) { } func (m *TransactionStatusError) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Msg) @@ -4841,6 +5733,9 @@ func (m *TransactionStatusError) Size() (n int) { } func (m *WriteIntentError) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.Intents) > 0 { @@ -4853,6 +5748,9 @@ func (m *WriteIntentError) Size() (n int) { } func (m *WriteTooOldError) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.Timestamp.Size() @@ -4863,12 +5761,18 @@ func (m *WriteTooOldError) Size() (n int) { } func (m *OpRequiresTxnError) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l return n } func (m *ConditionFailedError) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.ActualValue != nil { @@ -4879,6 +5783,9 @@ func (m *ConditionFailedError) Size() (n int) { } func (m *LeaseRejectedError) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Message) @@ -4891,6 +5798,9 @@ func (m *LeaseRejectedError) Size() (n int) { } func (m *SendError) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Message) @@ -4899,6 +5809,9 @@ func (m *SendError) Size() (n int) { } func (m *AmbiguousResultError) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Message) @@ -4911,12 +5824,18 @@ func (m *AmbiguousResultError) Size() (n int) { } func (m *RaftGroupDeletedError) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l return n } func (m *ReplicaCorruptionError) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.ErrorMsg) @@ -4926,6 +5845,9 @@ func (m *ReplicaCorruptionError) Size() (n int) { } func (m *ReplicaTooOldError) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovErrors(uint64(m.ReplicaID)) @@ -4933,6 +5855,9 @@ func (m *ReplicaTooOldError) Size() (n int) { } func (m *StoreNotFoundError) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovErrors(uint64(m.StoreID)) @@ -4940,6 +5865,9 @@ func (m *StoreNotFoundError) Size() (n int) { } func (m *UnhandledRetryableError) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.PErr.Size() @@ -4948,6 +5876,9 @@ func (m *UnhandledRetryableError) Size() (n int) { } func (m *HandledRetryableTxnError) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Msg) @@ -4960,12 +5891,18 @@ func (m *HandledRetryableTxnError) Size() (n int) { } func (m *TxnAlreadyEncounteredErrorError) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l return n } func (m *IntegerOverflowError) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Key != nil { @@ -4978,6 +5915,9 @@ func (m *IntegerOverflowError) Size() (n int) { } func (m *MixedSuccessError) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Wrapped != nil { @@ -4988,6 +5928,9 @@ func (m *MixedSuccessError) Size() (n int) { } func (m *BatchTimestampBeforeGCError) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.Timestamp.Size() @@ -4998,6 +5941,9 @@ func (m *BatchTimestampBeforeGCError) Size() (n int) { } func (m *IntentMissingError) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.WrongIntent != nil { @@ -5008,12 +5954,18 @@ func (m *IntentMissingError) Size() (n int) { } func (m *MergeInProgressError) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l return n } func (m *RangeFeedRetryError) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovErrors(uint64(m.Reason)) @@ -5021,6 +5973,9 @@ func (m *RangeFeedRetryError) Size() (n int) { } func (m *ErrorDetail) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Value != nil { @@ -5030,6 +5985,9 @@ func (m *ErrorDetail) Size() (n int) { } func (m *ErrorDetail_NotLeaseHolder) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.NotLeaseHolder != nil { @@ -5039,6 +5997,9 @@ func (m *ErrorDetail_NotLeaseHolder) Size() (n int) { return n } func (m *ErrorDetail_RangeNotFound) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.RangeNotFound != nil { @@ -5048,6 +6009,9 @@ func (m *ErrorDetail_RangeNotFound) Size() (n int) { return n } func (m *ErrorDetail_RangeKeyMismatch) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.RangeKeyMismatch != nil { @@ -5057,6 +6021,9 @@ func (m *ErrorDetail_RangeKeyMismatch) Size() (n int) { return n } func (m *ErrorDetail_ReadWithinUncertaintyInterval) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.ReadWithinUncertaintyInterval != nil { @@ -5066,6 +6033,9 @@ func (m *ErrorDetail_ReadWithinUncertaintyInterval) Size() (n int) { return n } func (m *ErrorDetail_TransactionAborted) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.TransactionAborted != nil { @@ -5075,6 +6045,9 @@ func (m *ErrorDetail_TransactionAborted) Size() (n int) { return n } func (m *ErrorDetail_TransactionPush) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.TransactionPush != nil { @@ -5084,6 +6057,9 @@ func (m *ErrorDetail_TransactionPush) Size() (n int) { return n } func (m *ErrorDetail_TransactionRetry) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.TransactionRetry != nil { @@ -5093,6 +6069,9 @@ func (m *ErrorDetail_TransactionRetry) Size() (n int) { return n } func (m *ErrorDetail_TransactionStatus) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.TransactionStatus != nil { @@ -5102,6 +6081,9 @@ func (m *ErrorDetail_TransactionStatus) Size() (n int) { return n } func (m *ErrorDetail_WriteIntent) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.WriteIntent != nil { @@ -5111,6 +6093,9 @@ func (m *ErrorDetail_WriteIntent) Size() (n int) { return n } func (m *ErrorDetail_WriteTooOld) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.WriteTooOld != nil { @@ -5120,6 +6105,9 @@ func (m *ErrorDetail_WriteTooOld) Size() (n int) { return n } func (m *ErrorDetail_OpRequiresTxn) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.OpRequiresTxn != nil { @@ -5129,6 +6117,9 @@ func (m *ErrorDetail_OpRequiresTxn) Size() (n int) { return n } func (m *ErrorDetail_ConditionFailed) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.ConditionFailed != nil { @@ -5138,6 +6129,9 @@ func (m *ErrorDetail_ConditionFailed) Size() (n int) { return n } func (m *ErrorDetail_LeaseRejected) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.LeaseRejected != nil { @@ -5147,6 +6141,9 @@ func (m *ErrorDetail_LeaseRejected) Size() (n int) { return n } func (m *ErrorDetail_NodeUnavailable) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.NodeUnavailable != nil { @@ -5156,6 +6153,9 @@ func (m *ErrorDetail_NodeUnavailable) Size() (n int) { return n } func (m *ErrorDetail_Send) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Send != nil { @@ -5165,6 +6165,9 @@ func (m *ErrorDetail_Send) Size() (n int) { return n } func (m *ErrorDetail_RaftGroupDeleted) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.RaftGroupDeleted != nil { @@ -5174,6 +6177,9 @@ func (m *ErrorDetail_RaftGroupDeleted) Size() (n int) { return n } func (m *ErrorDetail_ReplicaCorruption) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.ReplicaCorruption != nil { @@ -5183,6 +6189,9 @@ func (m *ErrorDetail_ReplicaCorruption) Size() (n int) { return n } func (m *ErrorDetail_ReplicaTooOld) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.ReplicaTooOld != nil { @@ -5192,6 +6201,9 @@ func (m *ErrorDetail_ReplicaTooOld) Size() (n int) { return n } func (m *ErrorDetail_AmbiguousResult) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.AmbiguousResult != nil { @@ -5201,6 +6213,9 @@ func (m *ErrorDetail_AmbiguousResult) Size() (n int) { return n } func (m *ErrorDetail_StoreNotFound) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.StoreNotFound != nil { @@ -5210,6 +6225,9 @@ func (m *ErrorDetail_StoreNotFound) Size() (n int) { return n } func (m *ErrorDetail_HandledRetryableTxnError) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.HandledRetryableTxnError != nil { @@ -5219,6 +6237,9 @@ func (m *ErrorDetail_HandledRetryableTxnError) Size() (n int) { return n } func (m *ErrorDetail_IntegerOverflow) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.IntegerOverflow != nil { @@ -5228,6 +6249,9 @@ func (m *ErrorDetail_IntegerOverflow) Size() (n int) { return n } func (m *ErrorDetail_UnsupportedRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.UnsupportedRequest != nil { @@ -5237,6 +6261,9 @@ func (m *ErrorDetail_UnsupportedRequest) Size() (n int) { return n } func (m *ErrorDetail_MixedSuccess) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.MixedSuccess != nil { @@ -5246,6 +6273,9 @@ func (m *ErrorDetail_MixedSuccess) Size() (n int) { return n } func (m *ErrorDetail_TimestampBefore) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.TimestampBefore != nil { @@ -5255,6 +6285,9 @@ func (m *ErrorDetail_TimestampBefore) Size() (n int) { return n } func (m *ErrorDetail_TxnAlreadyEncounteredError) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.TxnAlreadyEncounteredError != nil { @@ -5264,6 +6297,9 @@ func (m *ErrorDetail_TxnAlreadyEncounteredError) Size() (n int) { return n } func (m *ErrorDetail_IntentMissing) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.IntentMissing != nil { @@ -5273,6 +6309,9 @@ func (m *ErrorDetail_IntentMissing) Size() (n int) { return n } func (m *ErrorDetail_MergeInProgress) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.MergeInProgress != nil { @@ -5282,6 +6321,9 @@ func (m *ErrorDetail_MergeInProgress) Size() (n int) { return n } func (m *ErrorDetail_RangefeedRetry) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.RangefeedRetry != nil { @@ -5291,6 +6333,9 @@ func (m *ErrorDetail_RangefeedRetry) Size() (n int) { return n } func (m *ErrPosition) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovErrors(uint64(m.Index)) @@ -5298,6 +6343,9 @@ func (m *ErrPosition) Size() (n int) { } func (m *Error) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Message) @@ -6008,7 +7056,7 @@ func (m *ReadWithinUncertaintyIntervalError) Unmarshal(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.MaxTimestamp == nil { - m.MaxTimestamp = &cockroach_util_hlc.Timestamp{} + m.MaxTimestamp = &hlc.Timestamp{} } if err := m.MaxTimestamp.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err @@ -9501,9 +10549,9 @@ var ( ErrIntOverflowErrors = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/errors.proto", fileDescriptorErrors) } +func init() { proto.RegisterFile("roachpb/errors.proto", fileDescriptor_errors_9fc5358245dd263f) } -var fileDescriptorErrors = []byte{ +var fileDescriptor_errors_9fc5358245dd263f = []byte{ // 2767 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x58, 0xcb, 0x73, 0xdb, 0xd6, 0xd5, 0x27, 0x24, 0x4a, 0x94, 0x8e, 0x5e, 0xd0, 0xb5, 0x22, 0xc3, 0x72, 0x4c, 0xd9, 0x72, 0x1e, diff --git a/pkg/roachpb/internal.pb.go b/pkg/roachpb/internal.pb.go index d51ff42bf047..9ec3f0bba922 100644 --- a/pkg/roachpb/internal.pb.go +++ b/pkg/roachpb/internal.pb.go @@ -7,7 +7,7 @@ import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" -import binary "encoding/binary" +import encoding_binary "encoding/binary" import io "io" @@ -16,6 +16,12 @@ var _ = proto.Marshal var _ = fmt.Errorf var _ = math.Inf +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package + // InternalTimeSeriesData is a collection of data samples for some // measurable value, where each sample is taken over a uniform time // interval. @@ -61,7 +67,7 @@ type InternalTimeSeriesData struct { SampleDurationNanos int64 `protobuf:"varint,2,opt,name=sample_duration_nanos,json=sampleDurationNanos" json:"sample_duration_nanos"` // The data samples for this metric if this data was written in the old // row format. - Samples []InternalTimeSeriesSample `protobuf:"bytes,3,rep,name=samples" json:"samples"` + Samples []InternalTimeSeriesSample `protobuf:"bytes,3,rep,name=samples" json:"samples"` // Deprecated: Do not use. // Columnar array containing the ordered offsets of the samples in this // data set. Offset []int32 `protobuf:"varint,4,rep,packed,name=offset" json:"offset,omitempty"` @@ -92,13 +98,39 @@ type InternalTimeSeriesData struct { // Columnar array containing the variance of measurements that were taken // during this sample period. If this column is elided, its value for all // samples is zero. - Variance []float64 `protobuf:"fixed64,11,rep,packed,name=variance" json:"variance,omitempty"` + Variance []float64 `protobuf:"fixed64,11,rep,packed,name=variance" json:"variance,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *InternalTimeSeriesData) Reset() { *m = InternalTimeSeriesData{} } +func (m *InternalTimeSeriesData) String() string { return proto.CompactTextString(m) } +func (*InternalTimeSeriesData) ProtoMessage() {} +func (*InternalTimeSeriesData) Descriptor() ([]byte, []int) { + return fileDescriptor_internal_1f616e5f0cc948ec, []int{0} +} +func (m *InternalTimeSeriesData) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *InternalTimeSeriesData) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *InternalTimeSeriesData) XXX_Merge(src proto.Message) { + xxx_messageInfo_InternalTimeSeriesData.Merge(dst, src) +} +func (m *InternalTimeSeriesData) XXX_Size() int { + return m.Size() +} +func (m *InternalTimeSeriesData) XXX_DiscardUnknown() { + xxx_messageInfo_InternalTimeSeriesData.DiscardUnknown(m) } -func (m *InternalTimeSeriesData) Reset() { *m = InternalTimeSeriesData{} } -func (m *InternalTimeSeriesData) String() string { return proto.CompactTextString(m) } -func (*InternalTimeSeriesData) ProtoMessage() {} -func (*InternalTimeSeriesData) Descriptor() ([]byte, []int) { return fileDescriptorInternal, []int{0} } +var xxx_messageInfo_InternalTimeSeriesData proto.InternalMessageInfo // A InternalTimeSeriesSample represents data gathered from multiple // measurements of a variable value over a given period of time. The @@ -132,13 +164,39 @@ type InternalTimeSeriesSample struct { // Maximum encountered measurement in this sample. Max *float64 `protobuf:"fixed64,8,opt,name=max" json:"max,omitempty"` // Minimum encountered measurement in this sample. - Min *float64 `protobuf:"fixed64,9,opt,name=min" json:"min,omitempty"` + Min *float64 `protobuf:"fixed64,9,opt,name=min" json:"min,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *InternalTimeSeriesSample) Reset() { *m = InternalTimeSeriesSample{} } -func (m *InternalTimeSeriesSample) String() string { return proto.CompactTextString(m) } -func (*InternalTimeSeriesSample) ProtoMessage() {} -func (*InternalTimeSeriesSample) Descriptor() ([]byte, []int) { return fileDescriptorInternal, []int{1} } +func (m *InternalTimeSeriesSample) Reset() { *m = InternalTimeSeriesSample{} } +func (m *InternalTimeSeriesSample) String() string { return proto.CompactTextString(m) } +func (*InternalTimeSeriesSample) ProtoMessage() {} +func (*InternalTimeSeriesSample) Descriptor() ([]byte, []int) { + return fileDescriptor_internal_1f616e5f0cc948ec, []int{1} +} +func (m *InternalTimeSeriesSample) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *InternalTimeSeriesSample) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *InternalTimeSeriesSample) XXX_Merge(src proto.Message) { + xxx_messageInfo_InternalTimeSeriesSample.Merge(dst, src) +} +func (m *InternalTimeSeriesSample) XXX_Size() int { + return m.Size() +} +func (m *InternalTimeSeriesSample) XXX_DiscardUnknown() { + xxx_messageInfo_InternalTimeSeriesSample.DiscardUnknown(m) +} + +var xxx_messageInfo_InternalTimeSeriesSample proto.InternalMessageInfo func init() { proto.RegisterType((*InternalTimeSeriesData)(nil), "cockroach.roachpb.InternalTimeSeriesData") @@ -201,7 +259,7 @@ func (m *InternalTimeSeriesData) MarshalTo(dAtA []byte) (int, error) { i = encodeVarintInternal(dAtA, i, uint64(len(m.Last)*8)) for _, num := range m.Last { f3 := math.Float64bits(float64(num)) - binary.LittleEndian.PutUint64(dAtA[i:], uint64(f3)) + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(f3)) i += 8 } } @@ -228,7 +286,7 @@ func (m *InternalTimeSeriesData) MarshalTo(dAtA []byte) (int, error) { i = encodeVarintInternal(dAtA, i, uint64(len(m.Sum)*8)) for _, num := range m.Sum { f6 := math.Float64bits(float64(num)) - binary.LittleEndian.PutUint64(dAtA[i:], uint64(f6)) + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(f6)) i += 8 } } @@ -238,7 +296,7 @@ func (m *InternalTimeSeriesData) MarshalTo(dAtA []byte) (int, error) { i = encodeVarintInternal(dAtA, i, uint64(len(m.Max)*8)) for _, num := range m.Max { f7 := math.Float64bits(float64(num)) - binary.LittleEndian.PutUint64(dAtA[i:], uint64(f7)) + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(f7)) i += 8 } } @@ -248,7 +306,7 @@ func (m *InternalTimeSeriesData) MarshalTo(dAtA []byte) (int, error) { i = encodeVarintInternal(dAtA, i, uint64(len(m.Min)*8)) for _, num := range m.Min { f8 := math.Float64bits(float64(num)) - binary.LittleEndian.PutUint64(dAtA[i:], uint64(f8)) + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(f8)) i += 8 } } @@ -258,7 +316,7 @@ func (m *InternalTimeSeriesData) MarshalTo(dAtA []byte) (int, error) { i = encodeVarintInternal(dAtA, i, uint64(len(m.First)*8)) for _, num := range m.First { f9 := math.Float64bits(float64(num)) - binary.LittleEndian.PutUint64(dAtA[i:], uint64(f9)) + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(f9)) i += 8 } } @@ -268,7 +326,7 @@ func (m *InternalTimeSeriesData) MarshalTo(dAtA []byte) (int, error) { i = encodeVarintInternal(dAtA, i, uint64(len(m.Variance)*8)) for _, num := range m.Variance { f10 := math.Float64bits(float64(num)) - binary.LittleEndian.PutUint64(dAtA[i:], uint64(f10)) + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(f10)) i += 8 } } @@ -298,18 +356,18 @@ func (m *InternalTimeSeriesSample) MarshalTo(dAtA []byte) (int, error) { i = encodeVarintInternal(dAtA, i, uint64(m.Count)) dAtA[i] = 0x39 i++ - binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.Sum)))) + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.Sum)))) i += 8 if m.Max != nil { dAtA[i] = 0x41 i++ - binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(*m.Max)))) + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(*m.Max)))) i += 8 } if m.Min != nil { dAtA[i] = 0x49 i++ - binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(*m.Min)))) + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(*m.Min)))) i += 8 } return i, nil @@ -325,6 +383,9 @@ func encodeVarintInternal(dAtA []byte, offset int, v uint64) int { return offset + 1 } func (m *InternalTimeSeriesData) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovInternal(uint64(m.StartTimestampNanos)) @@ -371,6 +432,9 @@ func (m *InternalTimeSeriesData) Size() (n int) { } func (m *InternalTimeSeriesSample) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovInternal(uint64(m.Offset)) @@ -537,6 +601,17 @@ func (m *InternalTimeSeriesData) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + var count int + for _, integer := range dAtA { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.Offset) == 0 { + m.Offset = make([]int32, 0, elementCount) + } for iNdEx < postIndex { var v int32 for shift := uint(0); ; shift += 7 { @@ -564,7 +639,7 @@ func (m *InternalTimeSeriesData) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - v = uint64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 v2 := float64(math.Float64frombits(v)) m.Last = append(m.Last, v2) @@ -591,12 +666,17 @@ func (m *InternalTimeSeriesData) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + elementCount = packedLen / 8 + if elementCount != 0 && len(m.Last) == 0 { + m.Last = make([]float64, 0, elementCount) + } for iNdEx < postIndex { var v uint64 if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - v = uint64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 v2 := float64(math.Float64frombits(v)) m.Last = append(m.Last, v2) @@ -645,6 +725,17 @@ func (m *InternalTimeSeriesData) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + var count int + for _, integer := range dAtA { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.Count) == 0 { + m.Count = make([]uint32, 0, elementCount) + } for iNdEx < postIndex { var v uint32 for shift := uint(0); ; shift += 7 { @@ -672,7 +763,7 @@ func (m *InternalTimeSeriesData) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - v = uint64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 v2 := float64(math.Float64frombits(v)) m.Sum = append(m.Sum, v2) @@ -699,12 +790,17 @@ func (m *InternalTimeSeriesData) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + elementCount = packedLen / 8 + if elementCount != 0 && len(m.Sum) == 0 { + m.Sum = make([]float64, 0, elementCount) + } for iNdEx < postIndex { var v uint64 if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - v = uint64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 v2 := float64(math.Float64frombits(v)) m.Sum = append(m.Sum, v2) @@ -718,7 +814,7 @@ func (m *InternalTimeSeriesData) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - v = uint64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 v2 := float64(math.Float64frombits(v)) m.Max = append(m.Max, v2) @@ -745,12 +841,17 @@ func (m *InternalTimeSeriesData) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + elementCount = packedLen / 8 + if elementCount != 0 && len(m.Max) == 0 { + m.Max = make([]float64, 0, elementCount) + } for iNdEx < postIndex { var v uint64 if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - v = uint64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 v2 := float64(math.Float64frombits(v)) m.Max = append(m.Max, v2) @@ -764,7 +865,7 @@ func (m *InternalTimeSeriesData) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - v = uint64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 v2 := float64(math.Float64frombits(v)) m.Min = append(m.Min, v2) @@ -791,12 +892,17 @@ func (m *InternalTimeSeriesData) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + elementCount = packedLen / 8 + if elementCount != 0 && len(m.Min) == 0 { + m.Min = make([]float64, 0, elementCount) + } for iNdEx < postIndex { var v uint64 if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - v = uint64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 v2 := float64(math.Float64frombits(v)) m.Min = append(m.Min, v2) @@ -810,7 +916,7 @@ func (m *InternalTimeSeriesData) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - v = uint64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 v2 := float64(math.Float64frombits(v)) m.First = append(m.First, v2) @@ -837,12 +943,17 @@ func (m *InternalTimeSeriesData) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + elementCount = packedLen / 8 + if elementCount != 0 && len(m.First) == 0 { + m.First = make([]float64, 0, elementCount) + } for iNdEx < postIndex { var v uint64 if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - v = uint64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 v2 := float64(math.Float64frombits(v)) m.First = append(m.First, v2) @@ -856,7 +967,7 @@ func (m *InternalTimeSeriesData) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - v = uint64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 v2 := float64(math.Float64frombits(v)) m.Variance = append(m.Variance, v2) @@ -883,12 +994,17 @@ func (m *InternalTimeSeriesData) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + elementCount = packedLen / 8 + if elementCount != 0 && len(m.Variance) == 0 { + m.Variance = make([]float64, 0, elementCount) + } for iNdEx < postIndex { var v uint64 if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - v = uint64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 v2 := float64(math.Float64frombits(v)) m.Variance = append(m.Variance, v2) @@ -992,7 +1108,7 @@ func (m *InternalTimeSeriesSample) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - v = uint64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 m.Sum = float64(math.Float64frombits(v)) case 8: @@ -1003,7 +1119,7 @@ func (m *InternalTimeSeriesSample) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - v = uint64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 v2 := float64(math.Float64frombits(v)) m.Max = &v2 @@ -1015,7 +1131,7 @@ func (m *InternalTimeSeriesSample) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - v = uint64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 v2 := float64(math.Float64frombits(v)) m.Min = &v2 @@ -1145,9 +1261,9 @@ var ( ErrIntOverflowInternal = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/internal.proto", fileDescriptorInternal) } +func init() { proto.RegisterFile("roachpb/internal.proto", fileDescriptor_internal_1f616e5f0cc948ec) } -var fileDescriptorInternal = []byte{ +var fileDescriptor_internal_1f616e5f0cc948ec = []byte{ // 387 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x74, 0x91, 0xb1, 0x8e, 0xda, 0x30, 0x00, 0x86, 0x31, 0x26, 0x40, 0x8d, 0x90, 0xa8, 0x4b, 0x23, 0x0b, 0x55, 0x69, 0xca, 0x64, 0xa9, diff --git a/pkg/roachpb/internal_raft.pb.go b/pkg/roachpb/internal_raft.pb.go index 937685d8e6a4..7677b6e40c7b 100644 --- a/pkg/roachpb/internal_raft.pb.go +++ b/pkg/roachpb/internal_raft.pb.go @@ -6,7 +6,7 @@ package roachpb import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" -import cockroach_util_hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" +import hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" import io "io" @@ -15,6 +15,12 @@ var _ = proto.Marshal var _ = fmt.Errorf var _ = math.Inf +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package + // RaftTruncatedState contains metadata about the truncated portion of the raft log. // Raft requires access to the term of the last truncated log entry even after the // rest of the entry has been discarded. @@ -22,23 +28,75 @@ type RaftTruncatedState struct { // The highest index that has been removed from the log. Index uint64 `protobuf:"varint,1,opt,name=index" json:"index"` // The term corresponding to 'index'. - Term uint64 `protobuf:"varint,2,opt,name=term" json:"term"` + Term uint64 `protobuf:"varint,2,opt,name=term" json:"term"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *RaftTruncatedState) Reset() { *m = RaftTruncatedState{} } +func (m *RaftTruncatedState) String() string { return proto.CompactTextString(m) } +func (*RaftTruncatedState) ProtoMessage() {} +func (*RaftTruncatedState) Descriptor() ([]byte, []int) { + return fileDescriptor_internal_raft_8611ae4325a43066, []int{0} +} +func (m *RaftTruncatedState) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RaftTruncatedState) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RaftTruncatedState) XXX_Merge(src proto.Message) { + xxx_messageInfo_RaftTruncatedState.Merge(dst, src) +} +func (m *RaftTruncatedState) XXX_Size() int { + return m.Size() +} +func (m *RaftTruncatedState) XXX_DiscardUnknown() { + xxx_messageInfo_RaftTruncatedState.DiscardUnknown(m) } -func (m *RaftTruncatedState) Reset() { *m = RaftTruncatedState{} } -func (m *RaftTruncatedState) String() string { return proto.CompactTextString(m) } -func (*RaftTruncatedState) ProtoMessage() {} -func (*RaftTruncatedState) Descriptor() ([]byte, []int) { return fileDescriptorInternalRaft, []int{0} } +var xxx_messageInfo_RaftTruncatedState proto.InternalMessageInfo // RaftTombstone contains information about a replica that has been deleted. type RaftTombstone struct { - NextReplicaID ReplicaID `protobuf:"varint,1,opt,name=next_replica_id,json=nextReplicaId,casttype=ReplicaID" json:"next_replica_id"` + NextReplicaID ReplicaID `protobuf:"varint,1,opt,name=next_replica_id,json=nextReplicaId,casttype=ReplicaID" json:"next_replica_id"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *RaftTombstone) Reset() { *m = RaftTombstone{} } -func (m *RaftTombstone) String() string { return proto.CompactTextString(m) } -func (*RaftTombstone) ProtoMessage() {} -func (*RaftTombstone) Descriptor() ([]byte, []int) { return fileDescriptorInternalRaft, []int{1} } +func (m *RaftTombstone) Reset() { *m = RaftTombstone{} } +func (m *RaftTombstone) String() string { return proto.CompactTextString(m) } +func (*RaftTombstone) ProtoMessage() {} +func (*RaftTombstone) Descriptor() ([]byte, []int) { + return fileDescriptor_internal_raft_8611ae4325a43066, []int{1} +} +func (m *RaftTombstone) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RaftTombstone) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RaftTombstone) XXX_Merge(src proto.Message) { + xxx_messageInfo_RaftTombstone.Merge(dst, src) +} +func (m *RaftTombstone) XXX_Size() int { + return m.Size() +} +func (m *RaftTombstone) XXX_DiscardUnknown() { + xxx_messageInfo_RaftTombstone.DiscardUnknown(m) +} + +var xxx_messageInfo_RaftTombstone proto.InternalMessageInfo // RaftSnapshotData is the payload of a raftpb.Snapshot. It contains a raw copy of // all of the range's data and metadata, including the raft log, abort span, etc. @@ -48,26 +106,76 @@ type RaftSnapshotData struct { KV []RaftSnapshotData_KeyValue `protobuf:"bytes,2,rep,name=KV" json:"KV"` // These are really raftpb.Entry, but we model them as raw bytes to avoid // roundtripping through memory. - LogEntries [][]byte `protobuf:"bytes,3,rep,name=log_entries,json=logEntries" json:"log_entries,omitempty"` + LogEntries [][]byte `protobuf:"bytes,3,rep,name=log_entries,json=logEntries" json:"log_entries,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *RaftSnapshotData) Reset() { *m = RaftSnapshotData{} } +func (m *RaftSnapshotData) String() string { return proto.CompactTextString(m) } +func (*RaftSnapshotData) ProtoMessage() {} +func (*RaftSnapshotData) Descriptor() ([]byte, []int) { + return fileDescriptor_internal_raft_8611ae4325a43066, []int{2} +} +func (m *RaftSnapshotData) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RaftSnapshotData) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RaftSnapshotData) XXX_Merge(src proto.Message) { + xxx_messageInfo_RaftSnapshotData.Merge(dst, src) +} +func (m *RaftSnapshotData) XXX_Size() int { + return m.Size() +} +func (m *RaftSnapshotData) XXX_DiscardUnknown() { + xxx_messageInfo_RaftSnapshotData.DiscardUnknown(m) } -func (m *RaftSnapshotData) Reset() { *m = RaftSnapshotData{} } -func (m *RaftSnapshotData) String() string { return proto.CompactTextString(m) } -func (*RaftSnapshotData) ProtoMessage() {} -func (*RaftSnapshotData) Descriptor() ([]byte, []int) { return fileDescriptorInternalRaft, []int{2} } +var xxx_messageInfo_RaftSnapshotData proto.InternalMessageInfo type RaftSnapshotData_KeyValue struct { - Key []byte `protobuf:"bytes,1,opt,name=key" json:"key,omitempty"` - Value []byte `protobuf:"bytes,2,opt,name=value" json:"value,omitempty"` - Timestamp cockroach_util_hlc.Timestamp `protobuf:"bytes,3,opt,name=timestamp" json:"timestamp"` + Key []byte `protobuf:"bytes,1,opt,name=key" json:"key,omitempty"` + Value []byte `protobuf:"bytes,2,opt,name=value" json:"value,omitempty"` + Timestamp hlc.Timestamp `protobuf:"bytes,3,opt,name=timestamp" json:"timestamp"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *RaftSnapshotData_KeyValue) Reset() { *m = RaftSnapshotData_KeyValue{} } func (m *RaftSnapshotData_KeyValue) String() string { return proto.CompactTextString(m) } func (*RaftSnapshotData_KeyValue) ProtoMessage() {} func (*RaftSnapshotData_KeyValue) Descriptor() ([]byte, []int) { - return fileDescriptorInternalRaft, []int{2, 0} + return fileDescriptor_internal_raft_8611ae4325a43066, []int{2, 0} +} +func (m *RaftSnapshotData_KeyValue) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) } +func (m *RaftSnapshotData_KeyValue) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RaftSnapshotData_KeyValue) XXX_Merge(src proto.Message) { + xxx_messageInfo_RaftSnapshotData_KeyValue.Merge(dst, src) +} +func (m *RaftSnapshotData_KeyValue) XXX_Size() int { + return m.Size() +} +func (m *RaftSnapshotData_KeyValue) XXX_DiscardUnknown() { + xxx_messageInfo_RaftSnapshotData_KeyValue.DiscardUnknown(m) +} + +var xxx_messageInfo_RaftSnapshotData_KeyValue proto.InternalMessageInfo func init() { proto.RegisterType((*RaftTruncatedState)(nil), "cockroach.roachpb.RaftTruncatedState") @@ -322,6 +430,9 @@ func encodeVarintPopulateInternalRaft(dAtA []byte, v uint64) []byte { return dAtA } func (m *RaftTruncatedState) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovInternalRaft(uint64(m.Index)) @@ -330,6 +441,9 @@ func (m *RaftTruncatedState) Size() (n int) { } func (m *RaftTombstone) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovInternalRaft(uint64(m.NextReplicaID)) @@ -337,6 +451,9 @@ func (m *RaftTombstone) Size() (n int) { } func (m *RaftSnapshotData) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.RangeDescriptor.Size() @@ -357,6 +474,9 @@ func (m *RaftSnapshotData) Size() (n int) { } func (m *RaftSnapshotData_KeyValue) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Key != nil { @@ -929,9 +1049,11 @@ var ( ErrIntOverflowInternalRaft = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/internal_raft.proto", fileDescriptorInternalRaft) } +func init() { + proto.RegisterFile("roachpb/internal_raft.proto", fileDescriptor_internal_raft_8611ae4325a43066) +} -var fileDescriptorInternalRaft = []byte{ +var fileDescriptor_internal_raft_8611ae4325a43066 = []byte{ // 447 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x6c, 0x52, 0xb1, 0x8e, 0xd3, 0x4c, 0x10, 0x8e, 0xed, 0x44, 0xff, 0xdd, 0x26, 0xd1, 0xe5, 0x5f, 0x9d, 0x90, 0x15, 0x84, 0x1d, 0x5c, diff --git a/pkg/roachpb/io-formats.pb.go b/pkg/roachpb/io-formats.pb.go index da129a3f23aa..3f46fbe01376 100644 --- a/pkg/roachpb/io-formats.pb.go +++ b/pkg/roachpb/io-formats.pb.go @@ -14,6 +14,12 @@ var _ = proto.Marshal var _ = fmt.Errorf var _ = math.Inf +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package + type IOFileFormat_FileFormat int32 const ( @@ -59,7 +65,7 @@ func (x *IOFileFormat_FileFormat) UnmarshalJSON(data []byte) error { return nil } func (IOFileFormat_FileFormat) EnumDescriptor() ([]byte, []int) { - return fileDescriptorIoFormats, []int{0, 0} + return fileDescriptor_io_formats_2eb6f547f3889c94, []int{0, 0} } type IOFileFormat_Compression int32 @@ -101,7 +107,7 @@ func (x *IOFileFormat_Compression) UnmarshalJSON(data []byte) error { return nil } func (IOFileFormat_Compression) EnumDescriptor() ([]byte, []int) { - return fileDescriptorIoFormats, []int{0, 1} + return fileDescriptor_io_formats_2eb6f547f3889c94, []int{0, 1} } type MySQLOutfileOptions_Enclose int32 @@ -140,22 +146,48 @@ func (x *MySQLOutfileOptions_Enclose) UnmarshalJSON(data []byte) error { return nil } func (MySQLOutfileOptions_Enclose) EnumDescriptor() ([]byte, []int) { - return fileDescriptorIoFormats, []int{2, 0} + return fileDescriptor_io_formats_2eb6f547f3889c94, []int{2, 0} } type IOFileFormat struct { - Format IOFileFormat_FileFormat `protobuf:"varint,1,opt,name=format,enum=cockroach.roachpb.IOFileFormat_FileFormat" json:"format"` - Csv CSVOptions `protobuf:"bytes,2,opt,name=csv" json:"csv"` - MysqlOut MySQLOutfileOptions `protobuf:"bytes,3,opt,name=mysql_out,json=mysqlOut" json:"mysql_out"` - PgCopy PgCopyOptions `protobuf:"bytes,4,opt,name=pg_copy,json=pgCopy" json:"pg_copy"` - PgDump PgDumpOptions `protobuf:"bytes,6,opt,name=pg_dump,json=pgDump" json:"pg_dump"` - Compression IOFileFormat_Compression `protobuf:"varint,5,opt,name=compression,enum=cockroach.roachpb.IOFileFormat_Compression" json:"compression"` + Format IOFileFormat_FileFormat `protobuf:"varint,1,opt,name=format,enum=cockroach.roachpb.IOFileFormat_FileFormat" json:"format"` + Csv CSVOptions `protobuf:"bytes,2,opt,name=csv" json:"csv"` + MysqlOut MySQLOutfileOptions `protobuf:"bytes,3,opt,name=mysql_out,json=mysqlOut" json:"mysql_out"` + PgCopy PgCopyOptions `protobuf:"bytes,4,opt,name=pg_copy,json=pgCopy" json:"pg_copy"` + PgDump PgDumpOptions `protobuf:"bytes,6,opt,name=pg_dump,json=pgDump" json:"pg_dump"` + Compression IOFileFormat_Compression `protobuf:"varint,5,opt,name=compression,enum=cockroach.roachpb.IOFileFormat_Compression" json:"compression"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *IOFileFormat) Reset() { *m = IOFileFormat{} } -func (m *IOFileFormat) String() string { return proto.CompactTextString(m) } -func (*IOFileFormat) ProtoMessage() {} -func (*IOFileFormat) Descriptor() ([]byte, []int) { return fileDescriptorIoFormats, []int{0} } +func (m *IOFileFormat) Reset() { *m = IOFileFormat{} } +func (m *IOFileFormat) String() string { return proto.CompactTextString(m) } +func (*IOFileFormat) ProtoMessage() {} +func (*IOFileFormat) Descriptor() ([]byte, []int) { + return fileDescriptor_io_formats_2eb6f547f3889c94, []int{0} +} +func (m *IOFileFormat) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *IOFileFormat) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *IOFileFormat) XXX_Merge(src proto.Message) { + xxx_messageInfo_IOFileFormat.Merge(dst, src) +} +func (m *IOFileFormat) XXX_Size() int { + return m.Size() +} +func (m *IOFileFormat) XXX_DiscardUnknown() { + xxx_messageInfo_IOFileFormat.DiscardUnknown(m) +} + +var xxx_messageInfo_IOFileFormat proto.InternalMessageInfo // CSVOptions describe the format of csv data (delimiter, comment, etc). type CSVOptions struct { @@ -166,13 +198,39 @@ type CSVOptions struct { // null_encoding, if not nil, is the string which identifies a NULL. Can be the empty string. NullEncoding *string `protobuf:"bytes,3,opt,name=null_encoding,json=nullEncoding" json:"null_encoding,omitempty"` // skip the first N lines of the input (e.g. to ignore column headers) when reading. - Skip uint32 `protobuf:"varint,4,opt,name=skip" json:"skip"` + Skip uint32 `protobuf:"varint,4,opt,name=skip" json:"skip"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *CSVOptions) Reset() { *m = CSVOptions{} } -func (m *CSVOptions) String() string { return proto.CompactTextString(m) } -func (*CSVOptions) ProtoMessage() {} -func (*CSVOptions) Descriptor() ([]byte, []int) { return fileDescriptorIoFormats, []int{1} } +func (m *CSVOptions) Reset() { *m = CSVOptions{} } +func (m *CSVOptions) String() string { return proto.CompactTextString(m) } +func (*CSVOptions) ProtoMessage() {} +func (*CSVOptions) Descriptor() ([]byte, []int) { + return fileDescriptor_io_formats_2eb6f547f3889c94, []int{1} +} +func (m *CSVOptions) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *CSVOptions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *CSVOptions) XXX_Merge(src proto.Message) { + xxx_messageInfo_CSVOptions.Merge(dst, src) +} +func (m *CSVOptions) XXX_Size() int { + return m.Size() +} +func (m *CSVOptions) XXX_DiscardUnknown() { + xxx_messageInfo_CSVOptions.DiscardUnknown(m) +} + +var xxx_messageInfo_CSVOptions proto.InternalMessageInfo // MySQLOutfileOptions describe the format of mysql's outfile. type MySQLOutfileOptions struct { @@ -187,13 +245,39 @@ type MySQLOutfileOptions struct { // has_escape indicates that an escape character is set (mysql's default is not). HasEscape bool `protobuf:"varint,5,opt,name=has_escape,json=hasEscape" json:"has_escape"` // escape is the character used to prefix the other delimiters (--fields-escaped-by) - Escape int32 `protobuf:"varint,6,opt,name=escape" json:"escape"` + Escape int32 `protobuf:"varint,6,opt,name=escape" json:"escape"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *MySQLOutfileOptions) Reset() { *m = MySQLOutfileOptions{} } +func (m *MySQLOutfileOptions) String() string { return proto.CompactTextString(m) } +func (*MySQLOutfileOptions) ProtoMessage() {} +func (*MySQLOutfileOptions) Descriptor() ([]byte, []int) { + return fileDescriptor_io_formats_2eb6f547f3889c94, []int{2} +} +func (m *MySQLOutfileOptions) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *MySQLOutfileOptions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *MySQLOutfileOptions) XXX_Merge(src proto.Message) { + xxx_messageInfo_MySQLOutfileOptions.Merge(dst, src) +} +func (m *MySQLOutfileOptions) XXX_Size() int { + return m.Size() +} +func (m *MySQLOutfileOptions) XXX_DiscardUnknown() { + xxx_messageInfo_MySQLOutfileOptions.DiscardUnknown(m) } -func (m *MySQLOutfileOptions) Reset() { *m = MySQLOutfileOptions{} } -func (m *MySQLOutfileOptions) String() string { return proto.CompactTextString(m) } -func (*MySQLOutfileOptions) ProtoMessage() {} -func (*MySQLOutfileOptions) Descriptor() ([]byte, []int) { return fileDescriptorIoFormats, []int{2} } +var xxx_messageInfo_MySQLOutfileOptions proto.InternalMessageInfo // PgCopyOptions describe the format of postgresql's COPY TO STDOUT. type PgCopyOptions struct { @@ -202,24 +286,76 @@ type PgCopyOptions struct { // null is the NULL value (NULL) Null string `protobuf:"bytes,2,opt,name=null" json:"null"` // maxRowSize is the maximum row size - MaxRowSize int32 `protobuf:"varint,3,opt,name=maxRowSize" json:"maxRowSize"` + MaxRowSize int32 `protobuf:"varint,3,opt,name=maxRowSize" json:"maxRowSize"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *PgCopyOptions) Reset() { *m = PgCopyOptions{} } -func (m *PgCopyOptions) String() string { return proto.CompactTextString(m) } -func (*PgCopyOptions) ProtoMessage() {} -func (*PgCopyOptions) Descriptor() ([]byte, []int) { return fileDescriptorIoFormats, []int{3} } +func (m *PgCopyOptions) Reset() { *m = PgCopyOptions{} } +func (m *PgCopyOptions) String() string { return proto.CompactTextString(m) } +func (*PgCopyOptions) ProtoMessage() {} +func (*PgCopyOptions) Descriptor() ([]byte, []int) { + return fileDescriptor_io_formats_2eb6f547f3889c94, []int{3} +} +func (m *PgCopyOptions) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *PgCopyOptions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *PgCopyOptions) XXX_Merge(src proto.Message) { + xxx_messageInfo_PgCopyOptions.Merge(dst, src) +} +func (m *PgCopyOptions) XXX_Size() int { + return m.Size() +} +func (m *PgCopyOptions) XXX_DiscardUnknown() { + xxx_messageInfo_PgCopyOptions.DiscardUnknown(m) +} + +var xxx_messageInfo_PgCopyOptions proto.InternalMessageInfo // PgDumpOptions describe the format of postgresql's pg_dump. type PgDumpOptions struct { // maxRowSize is the maximum row size - MaxRowSize int32 `protobuf:"varint,1,opt,name=maxRowSize" json:"maxRowSize"` + MaxRowSize int32 `protobuf:"varint,1,opt,name=maxRowSize" json:"maxRowSize"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *PgDumpOptions) Reset() { *m = PgDumpOptions{} } -func (m *PgDumpOptions) String() string { return proto.CompactTextString(m) } -func (*PgDumpOptions) ProtoMessage() {} -func (*PgDumpOptions) Descriptor() ([]byte, []int) { return fileDescriptorIoFormats, []int{4} } +func (m *PgDumpOptions) Reset() { *m = PgDumpOptions{} } +func (m *PgDumpOptions) String() string { return proto.CompactTextString(m) } +func (*PgDumpOptions) ProtoMessage() {} +func (*PgDumpOptions) Descriptor() ([]byte, []int) { + return fileDescriptor_io_formats_2eb6f547f3889c94, []int{4} +} +func (m *PgDumpOptions) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *PgDumpOptions) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *PgDumpOptions) XXX_Merge(src proto.Message) { + xxx_messageInfo_PgDumpOptions.Merge(dst, src) +} +func (m *PgDumpOptions) XXX_Size() int { + return m.Size() +} +func (m *PgDumpOptions) XXX_DiscardUnknown() { + xxx_messageInfo_PgDumpOptions.DiscardUnknown(m) +} + +var xxx_messageInfo_PgDumpOptions proto.InternalMessageInfo func init() { proto.RegisterType((*IOFileFormat)(nil), "cockroach.roachpb.IOFileFormat") @@ -420,6 +556,9 @@ func encodeVarintIoFormats(dAtA []byte, offset int, v uint64) int { return offset + 1 } func (m *IOFileFormat) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovIoFormats(uint64(m.Format)) @@ -436,6 +575,9 @@ func (m *IOFileFormat) Size() (n int) { } func (m *CSVOptions) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovIoFormats(uint64(m.Comma)) @@ -449,6 +591,9 @@ func (m *CSVOptions) Size() (n int) { } func (m *MySQLOutfileOptions) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovIoFormats(uint64(m.RowSeparator)) @@ -461,6 +606,9 @@ func (m *MySQLOutfileOptions) Size() (n int) { } func (m *PgCopyOptions) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovIoFormats(uint64(m.Delimiter)) @@ -471,6 +619,9 @@ func (m *PgCopyOptions) Size() (n int) { } func (m *PgDumpOptions) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovIoFormats(uint64(m.MaxRowSize)) @@ -1291,9 +1442,11 @@ var ( ErrIntOverflowIoFormats = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/io-formats.proto", fileDescriptorIoFormats) } +func init() { + proto.RegisterFile("roachpb/io-formats.proto", fileDescriptor_io_formats_2eb6f547f3889c94) +} -var fileDescriptorIoFormats = []byte{ +var fileDescriptor_io_formats_2eb6f547f3889c94 = []byte{ // 660 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x53, 0x4f, 0x6f, 0xd3, 0x4e, 0x10, 0xcd, 0xe6, 0xbf, 0x27, 0x49, 0x7f, 0xfb, 0x5b, 0x38, 0x58, 0x15, 0x98, 0x60, 0x10, 0x2a, diff --git a/pkg/roachpb/metadata.pb.go b/pkg/roachpb/metadata.pb.go index 5f2873004059..50784b31bb84 100644 --- a/pkg/roachpb/metadata.pb.go +++ b/pkg/roachpb/metadata.pb.go @@ -6,11 +6,11 @@ package roachpb import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" -import cockroach_util "github.com/cockroachdb/cockroach/pkg/util" +import util "github.com/cockroachdb/cockroach/pkg/util" import bytes "bytes" -import binary "encoding/binary" +import encoding_binary "encoding/binary" import io "io" @@ -19,25 +19,83 @@ var _ = proto.Marshal var _ = fmt.Errorf var _ = math.Inf +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package + // Attributes specifies a list of arbitrary strings describing // node topology, store type, and machine capabilities. type Attributes struct { - Attrs []string `protobuf:"bytes,1,rep,name=attrs" json:"attrs,omitempty" yaml:"attrs,flow"` + Attrs []string `protobuf:"bytes,1,rep,name=attrs" json:"attrs,omitempty" yaml:"attrs,flow"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *Attributes) Reset() { *m = Attributes{} } -func (*Attributes) ProtoMessage() {} -func (*Attributes) Descriptor() ([]byte, []int) { return fileDescriptorMetadata, []int{0} } +func (m *Attributes) Reset() { *m = Attributes{} } +func (*Attributes) ProtoMessage() {} +func (*Attributes) Descriptor() ([]byte, []int) { + return fileDescriptor_metadata_a25ba028f2b3ac41, []int{0} +} +func (m *Attributes) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Attributes) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *Attributes) XXX_Merge(src proto.Message) { + xxx_messageInfo_Attributes.Merge(dst, src) +} +func (m *Attributes) XXX_Size() int { + return m.Size() +} +func (m *Attributes) XXX_DiscardUnknown() { + xxx_messageInfo_Attributes.DiscardUnknown(m) +} + +var xxx_messageInfo_Attributes proto.InternalMessageInfo // ReplicationTarget identifies a node/store pair. type ReplicationTarget struct { - NodeID NodeID `protobuf:"varint,1,opt,name=node_id,json=nodeId,casttype=NodeID" json:"node_id"` - StoreID StoreID `protobuf:"varint,2,opt,name=store_id,json=storeId,casttype=StoreID" json:"store_id"` + NodeID NodeID `protobuf:"varint,1,opt,name=node_id,json=nodeId,casttype=NodeID" json:"node_id"` + StoreID StoreID `protobuf:"varint,2,opt,name=store_id,json=storeId,casttype=StoreID" json:"store_id"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *ReplicationTarget) Reset() { *m = ReplicationTarget{} } -func (*ReplicationTarget) ProtoMessage() {} -func (*ReplicationTarget) Descriptor() ([]byte, []int) { return fileDescriptorMetadata, []int{1} } +func (m *ReplicationTarget) Reset() { *m = ReplicationTarget{} } +func (*ReplicationTarget) ProtoMessage() {} +func (*ReplicationTarget) Descriptor() ([]byte, []int) { + return fileDescriptor_metadata_a25ba028f2b3ac41, []int{1} +} +func (m *ReplicationTarget) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ReplicationTarget) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ReplicationTarget) XXX_Merge(src proto.Message) { + xxx_messageInfo_ReplicationTarget.Merge(dst, src) +} +func (m *ReplicationTarget) XXX_Size() int { + return m.Size() +} +func (m *ReplicationTarget) XXX_DiscardUnknown() { + xxx_messageInfo_ReplicationTarget.DiscardUnknown(m) +} + +var xxx_messageInfo_ReplicationTarget proto.InternalMessageInfo // ReplicaDescriptor describes a replica location by node ID // (corresponds to a host:port via lookup on gossip network) and store @@ -48,23 +106,75 @@ type ReplicaDescriptor struct { // replica_id uniquely identifies a replica instance. If a range is removed from // a store and then re-added to the same store, the new instance will have a // higher replica_id. - ReplicaID ReplicaID `protobuf:"varint,3,opt,name=replica_id,json=replicaId,casttype=ReplicaID" json:"replica_id"` + ReplicaID ReplicaID `protobuf:"varint,3,opt,name=replica_id,json=replicaId,casttype=ReplicaID" json:"replica_id"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ReplicaDescriptor) Reset() { *m = ReplicaDescriptor{} } +func (*ReplicaDescriptor) ProtoMessage() {} +func (*ReplicaDescriptor) Descriptor() ([]byte, []int) { + return fileDescriptor_metadata_a25ba028f2b3ac41, []int{2} +} +func (m *ReplicaDescriptor) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ReplicaDescriptor) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ReplicaDescriptor) XXX_Merge(src proto.Message) { + xxx_messageInfo_ReplicaDescriptor.Merge(dst, src) +} +func (m *ReplicaDescriptor) XXX_Size() int { + return m.Size() +} +func (m *ReplicaDescriptor) XXX_DiscardUnknown() { + xxx_messageInfo_ReplicaDescriptor.DiscardUnknown(m) } -func (m *ReplicaDescriptor) Reset() { *m = ReplicaDescriptor{} } -func (*ReplicaDescriptor) ProtoMessage() {} -func (*ReplicaDescriptor) Descriptor() ([]byte, []int) { return fileDescriptorMetadata, []int{2} } +var xxx_messageInfo_ReplicaDescriptor proto.InternalMessageInfo // ReplicaIdent uniquely identifies a specific replica. type ReplicaIdent struct { - RangeID RangeID `protobuf:"varint,1,opt,name=range_id,json=rangeId,casttype=RangeID" json:"range_id"` - Replica ReplicaDescriptor `protobuf:"bytes,2,opt,name=replica" json:"replica"` + RangeID RangeID `protobuf:"varint,1,opt,name=range_id,json=rangeId,casttype=RangeID" json:"range_id"` + Replica ReplicaDescriptor `protobuf:"bytes,2,opt,name=replica" json:"replica"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *ReplicaIdent) Reset() { *m = ReplicaIdent{} } -func (m *ReplicaIdent) String() string { return proto.CompactTextString(m) } -func (*ReplicaIdent) ProtoMessage() {} -func (*ReplicaIdent) Descriptor() ([]byte, []int) { return fileDescriptorMetadata, []int{3} } +func (m *ReplicaIdent) Reset() { *m = ReplicaIdent{} } +func (m *ReplicaIdent) String() string { return proto.CompactTextString(m) } +func (*ReplicaIdent) ProtoMessage() {} +func (*ReplicaIdent) Descriptor() ([]byte, []int) { + return fileDescriptor_metadata_a25ba028f2b3ac41, []int{3} +} +func (m *ReplicaIdent) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ReplicaIdent) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ReplicaIdent) XXX_Merge(src proto.Message) { + xxx_messageInfo_ReplicaIdent.Merge(dst, src) +} +func (m *ReplicaIdent) XXX_Size() int { + return m.Size() +} +func (m *ReplicaIdent) XXX_DiscardUnknown() { + xxx_messageInfo_ReplicaIdent.DiscardUnknown(m) +} + +var xxx_messageInfo_ReplicaIdent proto.InternalMessageInfo // RangeDescriptor is the value stored in a range metadata key. // A range is described using an inclusive start key, a non-inclusive end key, @@ -94,27 +204,79 @@ type RangeDescriptor struct { // Cockroach prior to v2.1. To maintain backwards compatibility with these old // versions of Cockroach, we cannot enable the gogoproto.nullable option, as // we need to be able to encode this mesage with the generation field unset. - Generation *int64 `protobuf:"varint,6,opt,name=generation" json:"generation,omitempty"` + Generation *int64 `protobuf:"varint,6,opt,name=generation" json:"generation,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *RangeDescriptor) Reset() { *m = RangeDescriptor{} } +func (*RangeDescriptor) ProtoMessage() {} +func (*RangeDescriptor) Descriptor() ([]byte, []int) { + return fileDescriptor_metadata_a25ba028f2b3ac41, []int{4} +} +func (m *RangeDescriptor) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RangeDescriptor) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RangeDescriptor) XXX_Merge(src proto.Message) { + xxx_messageInfo_RangeDescriptor.Merge(dst, src) +} +func (m *RangeDescriptor) XXX_Size() int { + return m.Size() +} +func (m *RangeDescriptor) XXX_DiscardUnknown() { + xxx_messageInfo_RangeDescriptor.DiscardUnknown(m) } -func (m *RangeDescriptor) Reset() { *m = RangeDescriptor{} } -func (*RangeDescriptor) ProtoMessage() {} -func (*RangeDescriptor) Descriptor() ([]byte, []int) { return fileDescriptorMetadata, []int{4} } +var xxx_messageInfo_RangeDescriptor proto.InternalMessageInfo // Percentiles contains a handful of hard-coded percentiles meant to summarize // a distribution. type Percentiles struct { - P10 float64 `protobuf:"fixed64,1,opt,name=p10" json:"p10"` - P25 float64 `protobuf:"fixed64,2,opt,name=p25" json:"p25"` - P50 float64 `protobuf:"fixed64,3,opt,name=p50" json:"p50"` - P75 float64 `protobuf:"fixed64,4,opt,name=p75" json:"p75"` - P90 float64 `protobuf:"fixed64,5,opt,name=p90" json:"p90"` - PMax float64 `protobuf:"fixed64,6,opt,name=pMax" json:"pMax"` + P10 float64 `protobuf:"fixed64,1,opt,name=p10" json:"p10"` + P25 float64 `protobuf:"fixed64,2,opt,name=p25" json:"p25"` + P50 float64 `protobuf:"fixed64,3,opt,name=p50" json:"p50"` + P75 float64 `protobuf:"fixed64,4,opt,name=p75" json:"p75"` + P90 float64 `protobuf:"fixed64,5,opt,name=p90" json:"p90"` + PMax float64 `protobuf:"fixed64,6,opt,name=pMax" json:"pMax"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Percentiles) Reset() { *m = Percentiles{} } +func (*Percentiles) ProtoMessage() {} +func (*Percentiles) Descriptor() ([]byte, []int) { + return fileDescriptor_metadata_a25ba028f2b3ac41, []int{5} +} +func (m *Percentiles) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Percentiles) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *Percentiles) XXX_Merge(src proto.Message) { + xxx_messageInfo_Percentiles.Merge(dst, src) +} +func (m *Percentiles) XXX_Size() int { + return m.Size() +} +func (m *Percentiles) XXX_DiscardUnknown() { + xxx_messageInfo_Percentiles.DiscardUnknown(m) } -func (m *Percentiles) Reset() { *m = Percentiles{} } -func (*Percentiles) ProtoMessage() {} -func (*Percentiles) Descriptor() ([]byte, []int) { return fileDescriptorMetadata, []int{5} } +var xxx_messageInfo_Percentiles proto.InternalMessageInfo // StoreCapacity contains capacity information for a storage device. type StoreCapacity struct { @@ -146,91 +308,273 @@ type StoreCapacity struct { // bytes_per_replica and writes_per_replica contain percentiles for the // number of bytes and writes-per-second to each replica in the store. // This information can be used for rebalancing decisions. - BytesPerReplica Percentiles `protobuf:"bytes,6,opt,name=bytes_per_replica,json=bytesPerReplica" json:"bytes_per_replica"` - WritesPerReplica Percentiles `protobuf:"bytes,7,opt,name=writes_per_replica,json=writesPerReplica" json:"writes_per_replica"` + BytesPerReplica Percentiles `protobuf:"bytes,6,opt,name=bytes_per_replica,json=bytesPerReplica" json:"bytes_per_replica"` + WritesPerReplica Percentiles `protobuf:"bytes,7,opt,name=writes_per_replica,json=writesPerReplica" json:"writes_per_replica"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *StoreCapacity) Reset() { *m = StoreCapacity{} } +func (*StoreCapacity) ProtoMessage() {} +func (*StoreCapacity) Descriptor() ([]byte, []int) { + return fileDescriptor_metadata_a25ba028f2b3ac41, []int{6} +} +func (m *StoreCapacity) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *StoreCapacity) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *StoreCapacity) XXX_Merge(src proto.Message) { + xxx_messageInfo_StoreCapacity.Merge(dst, src) +} +func (m *StoreCapacity) XXX_Size() int { + return m.Size() +} +func (m *StoreCapacity) XXX_DiscardUnknown() { + xxx_messageInfo_StoreCapacity.DiscardUnknown(m) } -func (m *StoreCapacity) Reset() { *m = StoreCapacity{} } -func (*StoreCapacity) ProtoMessage() {} -func (*StoreCapacity) Descriptor() ([]byte, []int) { return fileDescriptorMetadata, []int{6} } +var xxx_messageInfo_StoreCapacity proto.InternalMessageInfo // NodeDescriptor holds details on node physical/network topology. type NodeDescriptor struct { - NodeID NodeID `protobuf:"varint,1,opt,name=node_id,json=nodeId,casttype=NodeID" json:"node_id"` - Address cockroach_util.UnresolvedAddr `protobuf:"bytes,2,opt,name=address" json:"address"` - Attrs Attributes `protobuf:"bytes,3,opt,name=attrs" json:"attrs"` - Locality Locality `protobuf:"bytes,4,opt,name=locality" json:"locality"` - ServerVersion Version `protobuf:"bytes,5,opt,name=ServerVersion" json:"ServerVersion"` - BuildTag string `protobuf:"bytes,6,opt,name=build_tag,json=buildTag" json:"build_tag"` - StartedAt int64 `protobuf:"varint,7,opt,name=started_at,json=startedAt" json:"started_at"` - LocalityAddress []LocalityAddress `protobuf:"bytes,8,rep,name=locality_address,json=localityAddress" json:"locality_address"` + NodeID NodeID `protobuf:"varint,1,opt,name=node_id,json=nodeId,casttype=NodeID" json:"node_id"` + Address util.UnresolvedAddr `protobuf:"bytes,2,opt,name=address" json:"address"` + Attrs Attributes `protobuf:"bytes,3,opt,name=attrs" json:"attrs"` + Locality Locality `protobuf:"bytes,4,opt,name=locality" json:"locality"` + ServerVersion Version `protobuf:"bytes,5,opt,name=ServerVersion" json:"ServerVersion"` + BuildTag string `protobuf:"bytes,6,opt,name=build_tag,json=buildTag" json:"build_tag"` + StartedAt int64 `protobuf:"varint,7,opt,name=started_at,json=startedAt" json:"started_at"` + LocalityAddress []LocalityAddress `protobuf:"bytes,8,rep,name=locality_address,json=localityAddress" json:"locality_address"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *NodeDescriptor) Reset() { *m = NodeDescriptor{} } +func (m *NodeDescriptor) String() string { return proto.CompactTextString(m) } +func (*NodeDescriptor) ProtoMessage() {} +func (*NodeDescriptor) Descriptor() ([]byte, []int) { + return fileDescriptor_metadata_a25ba028f2b3ac41, []int{7} +} +func (m *NodeDescriptor) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *NodeDescriptor) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *NodeDescriptor) XXX_Merge(src proto.Message) { + xxx_messageInfo_NodeDescriptor.Merge(dst, src) +} +func (m *NodeDescriptor) XXX_Size() int { + return m.Size() +} +func (m *NodeDescriptor) XXX_DiscardUnknown() { + xxx_messageInfo_NodeDescriptor.DiscardUnknown(m) } -func (m *NodeDescriptor) Reset() { *m = NodeDescriptor{} } -func (m *NodeDescriptor) String() string { return proto.CompactTextString(m) } -func (*NodeDescriptor) ProtoMessage() {} -func (*NodeDescriptor) Descriptor() ([]byte, []int) { return fileDescriptorMetadata, []int{7} } +var xxx_messageInfo_NodeDescriptor proto.InternalMessageInfo // LocalityAddress holds the private address accessible only from other nodes // in the corresponding locality. type LocalityAddress struct { - Address cockroach_util.UnresolvedAddr `protobuf:"bytes,1,opt,name=address" json:"address"` - LocalityTier Tier `protobuf:"bytes,2,opt,name=locality_tier,json=localityTier" json:"locality_tier"` + Address util.UnresolvedAddr `protobuf:"bytes,1,opt,name=address" json:"address"` + LocalityTier Tier `protobuf:"bytes,2,opt,name=locality_tier,json=localityTier" json:"locality_tier"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *LocalityAddress) Reset() { *m = LocalityAddress{} } +func (m *LocalityAddress) String() string { return proto.CompactTextString(m) } +func (*LocalityAddress) ProtoMessage() {} +func (*LocalityAddress) Descriptor() ([]byte, []int) { + return fileDescriptor_metadata_a25ba028f2b3ac41, []int{8} +} +func (m *LocalityAddress) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *LocalityAddress) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *LocalityAddress) XXX_Merge(src proto.Message) { + xxx_messageInfo_LocalityAddress.Merge(dst, src) +} +func (m *LocalityAddress) XXX_Size() int { + return m.Size() +} +func (m *LocalityAddress) XXX_DiscardUnknown() { + xxx_messageInfo_LocalityAddress.DiscardUnknown(m) } -func (m *LocalityAddress) Reset() { *m = LocalityAddress{} } -func (m *LocalityAddress) String() string { return proto.CompactTextString(m) } -func (*LocalityAddress) ProtoMessage() {} -func (*LocalityAddress) Descriptor() ([]byte, []int) { return fileDescriptorMetadata, []int{8} } +var xxx_messageInfo_LocalityAddress proto.InternalMessageInfo // StoreDescriptor holds store information including store attributes, node // descriptor and store capacity. type StoreDescriptor struct { - StoreID StoreID `protobuf:"varint,1,opt,name=store_id,json=storeId,casttype=StoreID" json:"store_id"` - Attrs Attributes `protobuf:"bytes,2,opt,name=attrs" json:"attrs"` - Node NodeDescriptor `protobuf:"bytes,3,opt,name=node" json:"node"` - Capacity StoreCapacity `protobuf:"bytes,4,opt,name=capacity" json:"capacity"` + StoreID StoreID `protobuf:"varint,1,opt,name=store_id,json=storeId,casttype=StoreID" json:"store_id"` + Attrs Attributes `protobuf:"bytes,2,opt,name=attrs" json:"attrs"` + Node NodeDescriptor `protobuf:"bytes,3,opt,name=node" json:"node"` + Capacity StoreCapacity `protobuf:"bytes,4,opt,name=capacity" json:"capacity"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *StoreDescriptor) Reset() { *m = StoreDescriptor{} } +func (m *StoreDescriptor) String() string { return proto.CompactTextString(m) } +func (*StoreDescriptor) ProtoMessage() {} +func (*StoreDescriptor) Descriptor() ([]byte, []int) { + return fileDescriptor_metadata_a25ba028f2b3ac41, []int{9} +} +func (m *StoreDescriptor) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *StoreDescriptor) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *StoreDescriptor) XXX_Merge(src proto.Message) { + xxx_messageInfo_StoreDescriptor.Merge(dst, src) +} +func (m *StoreDescriptor) XXX_Size() int { + return m.Size() +} +func (m *StoreDescriptor) XXX_DiscardUnknown() { + xxx_messageInfo_StoreDescriptor.DiscardUnknown(m) } -func (m *StoreDescriptor) Reset() { *m = StoreDescriptor{} } -func (m *StoreDescriptor) String() string { return proto.CompactTextString(m) } -func (*StoreDescriptor) ProtoMessage() {} -func (*StoreDescriptor) Descriptor() ([]byte, []int) { return fileDescriptorMetadata, []int{9} } +var xxx_messageInfo_StoreDescriptor proto.InternalMessageInfo // StoreDeadReplicas holds a storeID and a list of dead replicas on that store. // Used to let the range lease holder know about corrupted or otherwise // destroyed replicas that should be transferred to a different store. type StoreDeadReplicas struct { - StoreID StoreID `protobuf:"varint,1,opt,name=store_id,json=storeId,casttype=StoreID" json:"store_id"` - Replicas []ReplicaIdent `protobuf:"bytes,2,rep,name=replicas" json:"replicas"` + StoreID StoreID `protobuf:"varint,1,opt,name=store_id,json=storeId,casttype=StoreID" json:"store_id"` + Replicas []ReplicaIdent `protobuf:"bytes,2,rep,name=replicas" json:"replicas"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *StoreDeadReplicas) Reset() { *m = StoreDeadReplicas{} } -func (m *StoreDeadReplicas) String() string { return proto.CompactTextString(m) } -func (*StoreDeadReplicas) ProtoMessage() {} -func (*StoreDeadReplicas) Descriptor() ([]byte, []int) { return fileDescriptorMetadata, []int{10} } +func (m *StoreDeadReplicas) Reset() { *m = StoreDeadReplicas{} } +func (m *StoreDeadReplicas) String() string { return proto.CompactTextString(m) } +func (*StoreDeadReplicas) ProtoMessage() {} +func (*StoreDeadReplicas) Descriptor() ([]byte, []int) { + return fileDescriptor_metadata_a25ba028f2b3ac41, []int{10} +} +func (m *StoreDeadReplicas) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *StoreDeadReplicas) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *StoreDeadReplicas) XXX_Merge(src proto.Message) { + xxx_messageInfo_StoreDeadReplicas.Merge(dst, src) +} +func (m *StoreDeadReplicas) XXX_Size() int { + return m.Size() +} +func (m *StoreDeadReplicas) XXX_DiscardUnknown() { + xxx_messageInfo_StoreDeadReplicas.DiscardUnknown(m) +} + +var xxx_messageInfo_StoreDeadReplicas proto.InternalMessageInfo // Locality is an ordered set of key value Tiers that describe a node's // location. The tier keys should be the same across all nodes. type Locality struct { - Tiers []Tier `protobuf:"bytes,1,rep,name=tiers" json:"tiers"` + Tiers []Tier `protobuf:"bytes,1,rep,name=tiers" json:"tiers"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Locality) Reset() { *m = Locality{} } +func (*Locality) ProtoMessage() {} +func (*Locality) Descriptor() ([]byte, []int) { + return fileDescriptor_metadata_a25ba028f2b3ac41, []int{11} +} +func (m *Locality) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Locality) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *Locality) XXX_Merge(src proto.Message) { + xxx_messageInfo_Locality.Merge(dst, src) +} +func (m *Locality) XXX_Size() int { + return m.Size() +} +func (m *Locality) XXX_DiscardUnknown() { + xxx_messageInfo_Locality.DiscardUnknown(m) } -func (m *Locality) Reset() { *m = Locality{} } -func (*Locality) ProtoMessage() {} -func (*Locality) Descriptor() ([]byte, []int) { return fileDescriptorMetadata, []int{11} } +var xxx_messageInfo_Locality proto.InternalMessageInfo // Tier represents one level of the locality hierarchy. type Tier struct { // Key is the name of tier and should match all other nodes. Key string `protobuf:"bytes,1,opt,name=key" json:"key"` // Value is node specific value corresponding to the key. - Value string `protobuf:"bytes,2,opt,name=value" json:"value"` + Value string `protobuf:"bytes,2,opt,name=value" json:"value"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Tier) Reset() { *m = Tier{} } +func (*Tier) ProtoMessage() {} +func (*Tier) Descriptor() ([]byte, []int) { + return fileDescriptor_metadata_a25ba028f2b3ac41, []int{12} +} +func (m *Tier) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Tier) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *Tier) XXX_Merge(src proto.Message) { + xxx_messageInfo_Tier.Merge(dst, src) +} +func (m *Tier) XXX_Size() int { + return m.Size() +} +func (m *Tier) XXX_DiscardUnknown() { + xxx_messageInfo_Tier.DiscardUnknown(m) } -func (m *Tier) Reset() { *m = Tier{} } -func (*Tier) ProtoMessage() {} -func (*Tier) Descriptor() ([]byte, []int) { return fileDescriptorMetadata, []int{12} } +var xxx_messageInfo_Tier proto.InternalMessageInfo type Version struct { // The names "major" and "minor" are reserved in C in @@ -242,12 +586,38 @@ type Version struct { // The unstable version is used to migrate during development. // Users of stable, public releases will only use binaries // with unstable set to 0. - Unstable int32 `protobuf:"varint,4,opt,name=unstable" json:"unstable"` + Unstable int32 `protobuf:"varint,4,opt,name=unstable" json:"unstable"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Version) Reset() { *m = Version{} } +func (*Version) ProtoMessage() {} +func (*Version) Descriptor() ([]byte, []int) { + return fileDescriptor_metadata_a25ba028f2b3ac41, []int{13} +} +func (m *Version) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Version) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *Version) XXX_Merge(src proto.Message) { + xxx_messageInfo_Version.Merge(dst, src) +} +func (m *Version) XXX_Size() int { + return m.Size() +} +func (m *Version) XXX_DiscardUnknown() { + xxx_messageInfo_Version.DiscardUnknown(m) } -func (m *Version) Reset() { *m = Version{} } -func (*Version) ProtoMessage() {} -func (*Version) Descriptor() ([]byte, []int) { return fileDescriptorMetadata, []int{13} } +var xxx_messageInfo_Version proto.InternalMessageInfo func init() { proto.RegisterType((*Attributes)(nil), "cockroach.roachpb.Attributes") @@ -555,27 +925,27 @@ func (m *Percentiles) MarshalTo(dAtA []byte) (int, error) { _ = l dAtA[i] = 0x9 i++ - binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.P10)))) + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.P10)))) i += 8 dAtA[i] = 0x11 i++ - binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.P25)))) + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.P25)))) i += 8 dAtA[i] = 0x19 i++ - binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.P50)))) + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.P50)))) i += 8 dAtA[i] = 0x21 i++ - binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.P75)))) + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.P75)))) i += 8 dAtA[i] = 0x29 i++ - binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.P90)))) + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.P90)))) i += 8 dAtA[i] = 0x31 i++ - binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.PMax)))) + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.PMax)))) i += 8 return i, nil } @@ -609,7 +979,7 @@ func (m *StoreCapacity) MarshalTo(dAtA []byte) (int, error) { i = encodeVarintMetadata(dAtA, i, uint64(m.LeaseCount)) dAtA[i] = 0x29 i++ - binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.WritesPerSecond)))) + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.WritesPerSecond)))) i += 8 dAtA[i] = 0x32 i++ @@ -635,7 +1005,7 @@ func (m *StoreCapacity) MarshalTo(dAtA []byte) (int, error) { i = encodeVarintMetadata(dAtA, i, uint64(m.LogicalBytes)) dAtA[i] = 0x51 i++ - binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.QueriesPerSecond)))) + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.QueriesPerSecond)))) i += 8 return i, nil } @@ -1055,6 +1425,9 @@ func encodeVarintPopulateMetadata(dAtA []byte, v uint64) []byte { return dAtA } func (m *Attributes) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.Attrs) > 0 { @@ -1067,6 +1440,9 @@ func (m *Attributes) Size() (n int) { } func (m *ReplicationTarget) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovMetadata(uint64(m.NodeID)) @@ -1075,6 +1451,9 @@ func (m *ReplicationTarget) Size() (n int) { } func (m *ReplicaDescriptor) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovMetadata(uint64(m.NodeID)) @@ -1084,6 +1463,9 @@ func (m *ReplicaDescriptor) Size() (n int) { } func (m *ReplicaIdent) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovMetadata(uint64(m.RangeID)) @@ -1093,6 +1475,9 @@ func (m *ReplicaIdent) Size() (n int) { } func (m *RangeDescriptor) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovMetadata(uint64(m.RangeID)) @@ -1118,6 +1503,9 @@ func (m *RangeDescriptor) Size() (n int) { } func (m *Percentiles) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 9 @@ -1130,6 +1518,9 @@ func (m *Percentiles) Size() (n int) { } func (m *StoreCapacity) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovMetadata(uint64(m.Capacity)) @@ -1148,6 +1539,9 @@ func (m *StoreCapacity) Size() (n int) { } func (m *NodeDescriptor) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovMetadata(uint64(m.NodeID)) @@ -1172,6 +1566,9 @@ func (m *NodeDescriptor) Size() (n int) { } func (m *LocalityAddress) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.Address.Size() @@ -1182,6 +1579,9 @@ func (m *LocalityAddress) Size() (n int) { } func (m *StoreDescriptor) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovMetadata(uint64(m.StoreID)) @@ -1195,6 +1595,9 @@ func (m *StoreDescriptor) Size() (n int) { } func (m *StoreDeadReplicas) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovMetadata(uint64(m.StoreID)) @@ -1208,6 +1611,9 @@ func (m *StoreDeadReplicas) Size() (n int) { } func (m *Locality) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.Tiers) > 0 { @@ -1220,6 +1626,9 @@ func (m *Locality) Size() (n int) { } func (m *Tier) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Key) @@ -1230,6 +1639,9 @@ func (m *Tier) Size() (n int) { } func (m *Version) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovMetadata(uint64(m.Major)) @@ -1863,7 +2275,7 @@ func (m *Percentiles) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - v = uint64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 m.P10 = float64(math.Float64frombits(v)) case 2: @@ -1874,7 +2286,7 @@ func (m *Percentiles) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - v = uint64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 m.P25 = float64(math.Float64frombits(v)) case 3: @@ -1885,7 +2297,7 @@ func (m *Percentiles) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - v = uint64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 m.P50 = float64(math.Float64frombits(v)) case 4: @@ -1896,7 +2308,7 @@ func (m *Percentiles) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - v = uint64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 m.P75 = float64(math.Float64frombits(v)) case 5: @@ -1907,7 +2319,7 @@ func (m *Percentiles) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - v = uint64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 m.P90 = float64(math.Float64frombits(v)) case 6: @@ -1918,7 +2330,7 @@ func (m *Percentiles) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - v = uint64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 m.PMax = float64(math.Float64frombits(v)) default: @@ -2055,7 +2467,7 @@ func (m *StoreCapacity) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - v = uint64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 m.WritesPerSecond = float64(math.Float64frombits(v)) case 6: @@ -2164,7 +2576,7 @@ func (m *StoreCapacity) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - v = uint64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 m.QueriesPerSecond = float64(math.Float64frombits(v)) default: @@ -3245,9 +3657,9 @@ var ( ErrIntOverflowMetadata = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/metadata.proto", fileDescriptorMetadata) } +func init() { proto.RegisterFile("roachpb/metadata.proto", fileDescriptor_metadata_a25ba028f2b3ac41) } -var fileDescriptorMetadata = []byte{ +var fileDescriptor_metadata_a25ba028f2b3ac41 = []byte{ // 1162 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x56, 0xcf, 0x6f, 0x1b, 0x45, 0x14, 0xce, 0xda, 0xeb, 0x78, 0xfd, 0x1a, 0x93, 0x7a, 0x04, 0xc5, 0x32, 0xc2, 0x4e, 0x16, 0x2a, diff --git a/pkg/roachpb/version_test.go b/pkg/roachpb/version_test.go index 9b3f3b2bc380..38e96b75abc1 100644 --- a/pkg/roachpb/version_test.go +++ b/pkg/roachpb/version_test.go @@ -21,23 +21,31 @@ import ( ) func TestVersionLess(t *testing.T) { + v := func(major, minor, patch, unstable int32) Version { + return Version{ + Major: major, + Minor: minor, + Patch: patch, + Unstable: unstable, + } + } testData := []struct { v1, v2 Version less bool }{ {v1: Version{}, v2: Version{}, less: false}, - {v1: Version{0, 0, 0, 0}, v2: Version{0, 0, 0, 1}, less: true}, - {v1: Version{0, 0, 0, 2}, v2: Version{0, 0, 0, 1}, less: false}, - {v1: Version{0, 0, 1, 0}, v2: Version{0, 0, 0, 1}, less: false}, - {v1: Version{0, 0, 1, 0}, v2: Version{0, 0, 0, 2}, less: false}, - {v1: Version{0, 0, 1, 1}, v2: Version{0, 0, 1, 1}, less: false}, - {v1: Version{0, 0, 1, 0}, v2: Version{0, 0, 1, 1}, less: true}, - {v1: Version{0, 1, 1, 0}, v2: Version{0, 1, 0, 1}, less: false}, - {v1: Version{0, 1, 0, 1}, v2: Version{0, 1, 1, 0}, less: true}, - {v1: Version{1, 0, 0, 0}, v2: Version{1, 1, 0, 0}, less: true}, - {v1: Version{1, 1, 0, 1}, v2: Version{1, 1, 0, 0}, less: false}, - {v1: Version{1, 1, 0, 1}, v2: Version{1, 2, 0, 0}, less: true}, - {v1: Version{2, 0, 0, 0}, v2: Version{3, 0, 0, 0}, less: true}, + {v1: v(0, 0, 0, 0), v2: v(0, 0, 0, 1), less: true}, + {v1: v(0, 0, 0, 2), v2: v(0, 0, 0, 1), less: false}, + {v1: v(0, 0, 1, 0), v2: v(0, 0, 0, 1), less: false}, + {v1: v(0, 0, 1, 0), v2: v(0, 0, 0, 2), less: false}, + {v1: v(0, 0, 1, 1), v2: v(0, 0, 1, 1), less: false}, + {v1: v(0, 0, 1, 0), v2: v(0, 0, 1, 1), less: true}, + {v1: v(0, 1, 1, 0), v2: v(0, 1, 0, 1), less: false}, + {v1: v(0, 1, 0, 1), v2: v(0, 1, 1, 0), less: true}, + {v1: v(1, 0, 0, 0), v2: v(1, 1, 0, 0), less: true}, + {v1: v(1, 1, 0, 1), v2: v(1, 1, 0, 0), less: false}, + {v1: v(1, 1, 0, 1), v2: v(1, 2, 0, 0), less: true}, + {v1: v(2, 0, 0, 0), v2: v(3, 0, 0, 0), less: true}, } for _, test := range testData { diff --git a/pkg/rpc/heartbeat.pb.go b/pkg/rpc/heartbeat.pb.go index a51ef0047de7..47dd9c5ba869 100644 --- a/pkg/rpc/heartbeat.pb.go +++ b/pkg/rpc/heartbeat.pb.go @@ -1,28 +1,19 @@ // Code generated by protoc-gen-gogo. DO NOT EDIT. // source: rpc/heartbeat.proto -/* - Package rpc is a generated protocol buffer package. - - It is generated from these files: - rpc/heartbeat.proto - - It has these top-level messages: - RemoteOffset - PingRequest - PingResponse -*/ package rpc import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" -import cockroach_roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" +import roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" import github_com_cockroachdb_cockroach_pkg_util_uuid "github.com/cockroachdb/cockroach/pkg/util/uuid" -import context "context" -import grpc "google.golang.org/grpc" +import ( + context "context" + grpc "google.golang.org/grpc" +) import io "io" @@ -51,12 +42,38 @@ type RemoteOffset struct { // The maximum error of the measured offset, in nanoseconds. Uncertainty int64 `protobuf:"varint,2,opt,name=uncertainty" json:"uncertainty"` // Measurement time, in nanoseconds from unix epoch. - MeasuredAt int64 `protobuf:"varint,3,opt,name=measured_at,json=measuredAt" json:"measured_at"` + MeasuredAt int64 `protobuf:"varint,3,opt,name=measured_at,json=measuredAt" json:"measured_at"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *RemoteOffset) Reset() { *m = RemoteOffset{} } -func (*RemoteOffset) ProtoMessage() {} -func (*RemoteOffset) Descriptor() ([]byte, []int) { return fileDescriptorHeartbeat, []int{0} } +func (m *RemoteOffset) Reset() { *m = RemoteOffset{} } +func (*RemoteOffset) ProtoMessage() {} +func (*RemoteOffset) Descriptor() ([]byte, []int) { + return fileDescriptor_heartbeat_108eca8199aea192, []int{0} +} +func (m *RemoteOffset) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RemoteOffset) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RemoteOffset) XXX_Merge(src proto.Message) { + xxx_messageInfo_RemoteOffset.Merge(dst, src) +} +func (m *RemoteOffset) XXX_Size() int { + return m.Size() +} +func (m *RemoteOffset) XXX_DiscardUnknown() { + xxx_messageInfo_RemoteOffset.DiscardUnknown(m) +} + +var xxx_messageInfo_RemoteOffset proto.InternalMessageInfo // A PingRequest specifies the string to echo in response. // Fields are exported so that they will be serialized in the rpc call. @@ -70,27 +87,79 @@ type PingRequest struct { // The configured maximum clock offset (in nanoseconds) on the server. MaxOffsetNanos int64 `protobuf:"varint,4,opt,name=max_offset_nanos,json=maxOffsetNanos" json:"max_offset_nanos"` // Cluster ID to prevent connections between nodes in different clusters. - ClusterID *github_com_cockroachdb_cockroach_pkg_util_uuid.UUID `protobuf:"bytes,5,opt,name=cluster_id,json=clusterId,customtype=github.com/cockroachdb/cockroach/pkg/util/uuid.UUID" json:"cluster_id,omitempty"` - ServerVersion cockroach_roachpb.Version `protobuf:"bytes,6,opt,name=server_version,json=serverVersion" json:"server_version"` + ClusterID *github_com_cockroachdb_cockroach_pkg_util_uuid.UUID `protobuf:"bytes,5,opt,name=cluster_id,json=clusterId,customtype=github.com/cockroachdb/cockroach/pkg/util/uuid.UUID" json:"cluster_id,omitempty"` + ServerVersion roachpb.Version `protobuf:"bytes,6,opt,name=server_version,json=serverVersion" json:"server_version"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *PingRequest) Reset() { *m = PingRequest{} } +func (m *PingRequest) String() string { return proto.CompactTextString(m) } +func (*PingRequest) ProtoMessage() {} +func (*PingRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_heartbeat_108eca8199aea192, []int{1} +} +func (m *PingRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *PingRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *PingRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_PingRequest.Merge(dst, src) +} +func (m *PingRequest) XXX_Size() int { + return m.Size() +} +func (m *PingRequest) XXX_DiscardUnknown() { + xxx_messageInfo_PingRequest.DiscardUnknown(m) } -func (m *PingRequest) Reset() { *m = PingRequest{} } -func (m *PingRequest) String() string { return proto.CompactTextString(m) } -func (*PingRequest) ProtoMessage() {} -func (*PingRequest) Descriptor() ([]byte, []int) { return fileDescriptorHeartbeat, []int{1} } +var xxx_messageInfo_PingRequest proto.InternalMessageInfo // A PingResponse contains the echoed ping request string. type PingResponse struct { // An echo of value sent with PingRequest. - Pong string `protobuf:"bytes,1,opt,name=pong" json:"pong"` - ServerTime int64 `protobuf:"varint,2,opt,name=server_time,json=serverTime" json:"server_time"` - ServerVersion cockroach_roachpb.Version `protobuf:"bytes,3,opt,name=server_version,json=serverVersion" json:"server_version"` + Pong string `protobuf:"bytes,1,opt,name=pong" json:"pong"` + ServerTime int64 `protobuf:"varint,2,opt,name=server_time,json=serverTime" json:"server_time"` + ServerVersion roachpb.Version `protobuf:"bytes,3,opt,name=server_version,json=serverVersion" json:"server_version"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *PingResponse) Reset() { *m = PingResponse{} } -func (m *PingResponse) String() string { return proto.CompactTextString(m) } -func (*PingResponse) ProtoMessage() {} -func (*PingResponse) Descriptor() ([]byte, []int) { return fileDescriptorHeartbeat, []int{2} } +func (m *PingResponse) Reset() { *m = PingResponse{} } +func (m *PingResponse) String() string { return proto.CompactTextString(m) } +func (*PingResponse) ProtoMessage() {} +func (*PingResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_heartbeat_108eca8199aea192, []int{2} +} +func (m *PingResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *PingResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *PingResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_PingResponse.Merge(dst, src) +} +func (m *PingResponse) XXX_Size() int { + return m.Size() +} +func (m *PingResponse) XXX_DiscardUnknown() { + xxx_messageInfo_PingResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_PingResponse proto.InternalMessageInfo func init() { proto.RegisterType((*RemoteOffset)(nil), "cockroach.rpc.RemoteOffset") @@ -106,8 +175,9 @@ var _ grpc.ClientConn // is compatible with the grpc package it is being compiled against. const _ = grpc.SupportPackageIsVersion4 -// Client API for Heartbeat service - +// HeartbeatClient is the client API for Heartbeat service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream. type HeartbeatClient interface { Ping(ctx context.Context, in *PingRequest, opts ...grpc.CallOption) (*PingResponse, error) } @@ -122,15 +192,14 @@ func NewHeartbeatClient(cc *grpc.ClientConn) HeartbeatClient { func (c *heartbeatClient) Ping(ctx context.Context, in *PingRequest, opts ...grpc.CallOption) (*PingResponse, error) { out := new(PingResponse) - err := grpc.Invoke(ctx, "/cockroach.rpc.Heartbeat/Ping", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/cockroach.rpc.Heartbeat/Ping", in, out, opts...) if err != nil { return nil, err } return out, nil } -// Server API for Heartbeat service - +// HeartbeatServer is the server API for Heartbeat service. type HeartbeatServer interface { Ping(context.Context, *PingRequest) (*PingResponse, error) } @@ -170,8 +239,9 @@ var _Heartbeat_serviceDesc = grpc.ServiceDesc{ Metadata: "rpc/heartbeat.proto", } -// Client API for TestingHeartbeatStream service - +// TestingHeartbeatStreamClient is the client API for TestingHeartbeatStream service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream. type TestingHeartbeatStreamClient interface { PingStream(ctx context.Context, opts ...grpc.CallOption) (TestingHeartbeatStream_PingStreamClient, error) } @@ -185,7 +255,7 @@ func NewTestingHeartbeatStreamClient(cc *grpc.ClientConn) TestingHeartbeatStream } func (c *testingHeartbeatStreamClient) PingStream(ctx context.Context, opts ...grpc.CallOption) (TestingHeartbeatStream_PingStreamClient, error) { - stream, err := grpc.NewClientStream(ctx, &_TestingHeartbeatStream_serviceDesc.Streams[0], c.cc, "/cockroach.rpc.TestingHeartbeatStream/PingStream", opts...) + stream, err := c.cc.NewStream(ctx, &_TestingHeartbeatStream_serviceDesc.Streams[0], "/cockroach.rpc.TestingHeartbeatStream/PingStream", opts...) if err != nil { return nil, err } @@ -215,8 +285,7 @@ func (x *testingHeartbeatStreamPingStreamClient) Recv() (*PingResponse, error) { return m, nil } -// Server API for TestingHeartbeatStream service - +// TestingHeartbeatStreamServer is the server API for TestingHeartbeatStream service. type TestingHeartbeatStreamServer interface { PingStream(TestingHeartbeatStream_PingStreamServer) error } @@ -391,6 +460,9 @@ func encodeVarintHeartbeat(dAtA []byte, offset int, v uint64) int { return offset + 1 } func (m *RemoteOffset) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovHeartbeat(uint64(m.Offset)) @@ -400,6 +472,9 @@ func (m *RemoteOffset) Size() (n int) { } func (m *PingRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Ping) @@ -419,6 +494,9 @@ func (m *PingRequest) Size() (n int) { } func (m *PingResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Pong) @@ -1001,9 +1079,9 @@ var ( ErrIntOverflowHeartbeat = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("rpc/heartbeat.proto", fileDescriptorHeartbeat) } +func init() { proto.RegisterFile("rpc/heartbeat.proto", fileDescriptor_heartbeat_108eca8199aea192) } -var fileDescriptorHeartbeat = []byte{ +var fileDescriptor_heartbeat_108eca8199aea192 = []byte{ // 499 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x92, 0xc1, 0x6e, 0xd3, 0x40, 0x10, 0x86, 0xb3, 0x8d, 0xa9, 0x94, 0x49, 0x5a, 0xa1, 0x05, 0x55, 0x56, 0x0a, 0x4e, 0x15, 0x09, diff --git a/pkg/server/admin_test.go b/pkg/server/admin_test.go index 98c1f63b17c0..d4de6a962faf 100644 --- a/pkg/server/admin_test.go +++ b/pkg/server/admin_test.go @@ -1334,7 +1334,7 @@ func TestAdminAPIQueryPlan(t *testing.T) { for _, exp := range testCase.exp { if !strings.Contains(res.DistSQLPhysicalQueryPlan, exp) { - t.Errorf("%d: expected response %s to contain %s", i, res, exp) + t.Errorf("%d: expected response %v to contain %s", i, res, exp) } } } diff --git a/pkg/server/diagnosticspb/diagnostics.pb.go b/pkg/server/diagnosticspb/diagnostics.pb.go index 70a232ca3c10..1fb1dad746cd 100644 --- a/pkg/server/diagnosticspb/diagnostics.pb.go +++ b/pkg/server/diagnosticspb/diagnostics.pb.go @@ -1,35 +1,19 @@ // Code generated by protoc-gen-gogo. DO NOT EDIT. // source: server/diagnosticspb/diagnostics.proto -/* - Package diagnosticspb is a generated protocol buffer package. - - It is generated from these files: - server/diagnosticspb/diagnostics.proto - - It has these top-level messages: - DiagnosticReport - NodeInfo - StoreInfo - CPUInfo - HardwareInfo - OSInfo - MemInfo -*/ package diagnosticspb import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" -import cockroach_config "github.com/cockroachdb/cockroach/pkg/config" -import cockroach_sql "github.com/cockroachdb/cockroach/pkg/roachpb" -import cockroach_sql_sqlbase1 "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" -import cockroach_roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" +import config "github.com/cockroachdb/cockroach/pkg/config" +import roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" +import sqlbase "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" import github_com_cockroachdb_cockroach_pkg_roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" -import sortkeys "github.com/gogo/protobuf/sortkeys" -import binary "encoding/binary" +import github_com_gogo_protobuf_sortkeys "github.com/gogo/protobuf/sortkeys" +import encoding_binary "encoding/binary" import io "io" @@ -45,102 +29,289 @@ var _ = math.Inf const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package type DiagnosticReport struct { - Node NodeInfo `protobuf:"bytes,1,opt,name=node" json:"node"` - Stores []StoreInfo `protobuf:"bytes,2,rep,name=stores" json:"stores"` - Schema []cockroach_sql_sqlbase1.TableDescriptor `protobuf:"bytes,3,rep,name=schema" json:"schema"` - SqlStats []cockroach_sql.CollectedStatementStatistics `protobuf:"bytes,4,rep,name=sql_stats,json=sqlStats" json:"sql_stats"` - AlteredSettings map[string]string `protobuf:"bytes,6,rep,name=altered_settings,json=alteredSettings" json:"altered_settings,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` - ZoneConfigs map[int64]cockroach_config.ZoneConfig `protobuf:"bytes,8,rep,name=zone_configs,json=zoneConfigs" json:"zone_configs" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value"` - FeatureUsage map[string]int32 `protobuf:"bytes,9,rep,name=feature_usage,json=featureUsage" json:"feature_usage" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` - LegacyUnimplementedErrors map[string]int64 `protobuf:"bytes,5,rep,name=legacy_unimplemented_errors,json=legacyUnimplementedErrors" json:"legacy_unimplemented_errors,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` - LegacyErrorCounts map[string]int64 `protobuf:"bytes,7,rep,name=legacy_error_counts,json=legacyErrorCounts" json:"legacy_error_counts,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` + Node NodeInfo `protobuf:"bytes,1,opt,name=node,proto3" json:"node"` + Stores []StoreInfo `protobuf:"bytes,2,rep,name=stores,proto3" json:"stores"` + Schema []sqlbase.TableDescriptor `protobuf:"bytes,3,rep,name=schema,proto3" json:"schema"` + SqlStats []roachpb.CollectedStatementStatistics `protobuf:"bytes,4,rep,name=sql_stats,json=sqlStats,proto3" json:"sql_stats"` + AlteredSettings map[string]string `protobuf:"bytes,6,rep,name=altered_settings,json=alteredSettings,proto3" json:"altered_settings,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + ZoneConfigs map[int64]config.ZoneConfig `protobuf:"bytes,8,rep,name=zone_configs,json=zoneConfigs,proto3" json:"zone_configs" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + FeatureUsage map[string]int32 `protobuf:"bytes,9,rep,name=feature_usage,json=featureUsage,proto3" json:"feature_usage" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` + LegacyUnimplementedErrors map[string]int64 `protobuf:"bytes,5,rep,name=legacy_unimplemented_errors,json=legacyUnimplementedErrors,proto3" json:"legacy_unimplemented_errors,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` + LegacyErrorCounts map[string]int64 `protobuf:"bytes,7,rep,name=legacy_error_counts,json=legacyErrorCounts,proto3" json:"legacy_error_counts,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *DiagnosticReport) Reset() { *m = DiagnosticReport{} } -func (m *DiagnosticReport) String() string { return proto.CompactTextString(m) } -func (*DiagnosticReport) ProtoMessage() {} -func (*DiagnosticReport) Descriptor() ([]byte, []int) { return fileDescriptorDiagnostics, []int{0} } +func (m *DiagnosticReport) Reset() { *m = DiagnosticReport{} } +func (m *DiagnosticReport) String() string { return proto.CompactTextString(m) } +func (*DiagnosticReport) ProtoMessage() {} +func (*DiagnosticReport) Descriptor() ([]byte, []int) { + return fileDescriptor_diagnostics_d68dd0a3f6635a47, []int{0} +} +func (m *DiagnosticReport) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *DiagnosticReport) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *DiagnosticReport) XXX_Merge(src proto.Message) { + xxx_messageInfo_DiagnosticReport.Merge(dst, src) +} +func (m *DiagnosticReport) XXX_Size() int { + return m.Size() +} +func (m *DiagnosticReport) XXX_DiscardUnknown() { + xxx_messageInfo_DiagnosticReport.DiscardUnknown(m) +} + +var xxx_messageInfo_DiagnosticReport proto.InternalMessageInfo type NodeInfo struct { - NodeID github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,1,opt,name=node_id,json=nodeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"node_id,omitempty"` - Bytes int64 `protobuf:"varint,2,opt,name=bytes,proto3" json:"bytes,omitempty"` - KeyCount int64 `protobuf:"varint,3,opt,name=key_count,json=keyCount,proto3" json:"key_count,omitempty"` - RangeCount int64 `protobuf:"varint,4,opt,name=range_count,json=rangeCount,proto3" json:"range_count,omitempty"` - Locality cockroach_roachpb.Locality `protobuf:"bytes,5,opt,name=locality" json:"locality"` - Hardware HardwareInfo `protobuf:"bytes,6,opt,name=hardware" json:"hardware"` - Os OSInfo `protobuf:"bytes,7,opt,name=os" json:"os"` + NodeID github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,1,opt,name=node_id,json=nodeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"node_id,omitempty"` + Bytes int64 `protobuf:"varint,2,opt,name=bytes,proto3" json:"bytes,omitempty"` + KeyCount int64 `protobuf:"varint,3,opt,name=key_count,json=keyCount,proto3" json:"key_count,omitempty"` + RangeCount int64 `protobuf:"varint,4,opt,name=range_count,json=rangeCount,proto3" json:"range_count,omitempty"` + Locality roachpb.Locality `protobuf:"bytes,5,opt,name=locality,proto3" json:"locality"` + Hardware HardwareInfo `protobuf:"bytes,6,opt,name=hardware,proto3" json:"hardware"` + Os OSInfo `protobuf:"bytes,7,opt,name=os,proto3" json:"os"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *NodeInfo) Reset() { *m = NodeInfo{} } -func (m *NodeInfo) String() string { return proto.CompactTextString(m) } -func (*NodeInfo) ProtoMessage() {} -func (*NodeInfo) Descriptor() ([]byte, []int) { return fileDescriptorDiagnostics, []int{1} } +func (m *NodeInfo) Reset() { *m = NodeInfo{} } +func (m *NodeInfo) String() string { return proto.CompactTextString(m) } +func (*NodeInfo) ProtoMessage() {} +func (*NodeInfo) Descriptor() ([]byte, []int) { + return fileDescriptor_diagnostics_d68dd0a3f6635a47, []int{1} +} +func (m *NodeInfo) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *NodeInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *NodeInfo) XXX_Merge(src proto.Message) { + xxx_messageInfo_NodeInfo.Merge(dst, src) +} +func (m *NodeInfo) XXX_Size() int { + return m.Size() +} +func (m *NodeInfo) XXX_DiscardUnknown() { + xxx_messageInfo_NodeInfo.DiscardUnknown(m) +} + +var xxx_messageInfo_NodeInfo proto.InternalMessageInfo type StoreInfo struct { - NodeID github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,1,opt,name=node_id,json=nodeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"node_id,omitempty"` - StoreID github_com_cockroachdb_cockroach_pkg_roachpb.StoreID `protobuf:"varint,2,opt,name=store_id,json=storeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.StoreID" json:"store_id,omitempty"` - Bytes int64 `protobuf:"varint,3,opt,name=bytes,proto3" json:"bytes,omitempty"` - KeyCount int64 `protobuf:"varint,4,opt,name=key_count,json=keyCount,proto3" json:"key_count,omitempty"` - RangeCount int64 `protobuf:"varint,5,opt,name=range_count,json=rangeCount,proto3" json:"range_count,omitempty"` - Capacity int64 `protobuf:"varint,6,opt,name=capacity,proto3" json:"capacity,omitempty"` - Available int64 `protobuf:"varint,7,opt,name=available,proto3" json:"available,omitempty"` - Used int64 `protobuf:"varint,8,opt,name=used,proto3" json:"used,omitempty"` + NodeID github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,1,opt,name=node_id,json=nodeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"node_id,omitempty"` + StoreID github_com_cockroachdb_cockroach_pkg_roachpb.StoreID `protobuf:"varint,2,opt,name=store_id,json=storeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.StoreID" json:"store_id,omitempty"` + Bytes int64 `protobuf:"varint,3,opt,name=bytes,proto3" json:"bytes,omitempty"` + KeyCount int64 `protobuf:"varint,4,opt,name=key_count,json=keyCount,proto3" json:"key_count,omitempty"` + RangeCount int64 `protobuf:"varint,5,opt,name=range_count,json=rangeCount,proto3" json:"range_count,omitempty"` + Capacity int64 `protobuf:"varint,6,opt,name=capacity,proto3" json:"capacity,omitempty"` + Available int64 `protobuf:"varint,7,opt,name=available,proto3" json:"available,omitempty"` + Used int64 `protobuf:"varint,8,opt,name=used,proto3" json:"used,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *StoreInfo) Reset() { *m = StoreInfo{} } +func (m *StoreInfo) String() string { return proto.CompactTextString(m) } +func (*StoreInfo) ProtoMessage() {} +func (*StoreInfo) Descriptor() ([]byte, []int) { + return fileDescriptor_diagnostics_d68dd0a3f6635a47, []int{2} +} +func (m *StoreInfo) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *StoreInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *StoreInfo) XXX_Merge(src proto.Message) { + xxx_messageInfo_StoreInfo.Merge(dst, src) +} +func (m *StoreInfo) XXX_Size() int { + return m.Size() +} +func (m *StoreInfo) XXX_DiscardUnknown() { + xxx_messageInfo_StoreInfo.DiscardUnknown(m) } -func (m *StoreInfo) Reset() { *m = StoreInfo{} } -func (m *StoreInfo) String() string { return proto.CompactTextString(m) } -func (*StoreInfo) ProtoMessage() {} -func (*StoreInfo) Descriptor() ([]byte, []int) { return fileDescriptorDiagnostics, []int{2} } +var xxx_messageInfo_StoreInfo proto.InternalMessageInfo type CPUInfo struct { - Numcpu int32 `protobuf:"varint,1,opt,name=numcpu,proto3" json:"numcpu,omitempty"` - Sockets int32 `protobuf:"varint,2,opt,name=sockets,proto3" json:"sockets,omitempty"` - Cores int32 `protobuf:"varint,3,opt,name=cores,proto3" json:"cores,omitempty"` - Model string `protobuf:"bytes,4,opt,name=model,proto3" json:"model,omitempty"` - Mhz float32 `protobuf:"fixed32,5,opt,name=mhz,proto3" json:"mhz,omitempty"` - Features []string `protobuf:"bytes,6,rep,name=features" json:"features,omitempty"` + Numcpu int32 `protobuf:"varint,1,opt,name=numcpu,proto3" json:"numcpu,omitempty"` + Sockets int32 `protobuf:"varint,2,opt,name=sockets,proto3" json:"sockets,omitempty"` + Cores int32 `protobuf:"varint,3,opt,name=cores,proto3" json:"cores,omitempty"` + Model string `protobuf:"bytes,4,opt,name=model,proto3" json:"model,omitempty"` + Mhz float32 `protobuf:"fixed32,5,opt,name=mhz,proto3" json:"mhz,omitempty"` + Features []string `protobuf:"bytes,6,rep,name=features,proto3" json:"features,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *CPUInfo) Reset() { *m = CPUInfo{} } -func (m *CPUInfo) String() string { return proto.CompactTextString(m) } -func (*CPUInfo) ProtoMessage() {} -func (*CPUInfo) Descriptor() ([]byte, []int) { return fileDescriptorDiagnostics, []int{3} } +func (m *CPUInfo) Reset() { *m = CPUInfo{} } +func (m *CPUInfo) String() string { return proto.CompactTextString(m) } +func (*CPUInfo) ProtoMessage() {} +func (*CPUInfo) Descriptor() ([]byte, []int) { + return fileDescriptor_diagnostics_d68dd0a3f6635a47, []int{3} +} +func (m *CPUInfo) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *CPUInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *CPUInfo) XXX_Merge(src proto.Message) { + xxx_messageInfo_CPUInfo.Merge(dst, src) +} +func (m *CPUInfo) XXX_Size() int { + return m.Size() +} +func (m *CPUInfo) XXX_DiscardUnknown() { + xxx_messageInfo_CPUInfo.DiscardUnknown(m) +} + +var xxx_messageInfo_CPUInfo proto.InternalMessageInfo type HardwareInfo struct { - Virtualization string `protobuf:"bytes,1,opt,name=virtualization,proto3" json:"virtualization,omitempty"` - Cpu CPUInfo `protobuf:"bytes,2,opt,name=cpu" json:"cpu"` - Mem MemInfo `protobuf:"bytes,3,opt,name=mem" json:"mem"` - Loadavg15 float32 `protobuf:"fixed32,4,opt,name=loadavg15,proto3" json:"loadavg15,omitempty"` + Virtualization string `protobuf:"bytes,1,opt,name=virtualization,proto3" json:"virtualization,omitempty"` + Cpu CPUInfo `protobuf:"bytes,2,opt,name=cpu,proto3" json:"cpu"` + Mem MemInfo `protobuf:"bytes,3,opt,name=mem,proto3" json:"mem"` + Loadavg15 float32 `protobuf:"fixed32,4,opt,name=loadavg15,proto3" json:"loadavg15,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *HardwareInfo) Reset() { *m = HardwareInfo{} } +func (m *HardwareInfo) String() string { return proto.CompactTextString(m) } +func (*HardwareInfo) ProtoMessage() {} +func (*HardwareInfo) Descriptor() ([]byte, []int) { + return fileDescriptor_diagnostics_d68dd0a3f6635a47, []int{4} +} +func (m *HardwareInfo) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *HardwareInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *HardwareInfo) XXX_Merge(src proto.Message) { + xxx_messageInfo_HardwareInfo.Merge(dst, src) +} +func (m *HardwareInfo) XXX_Size() int { + return m.Size() +} +func (m *HardwareInfo) XXX_DiscardUnknown() { + xxx_messageInfo_HardwareInfo.DiscardUnknown(m) } -func (m *HardwareInfo) Reset() { *m = HardwareInfo{} } -func (m *HardwareInfo) String() string { return proto.CompactTextString(m) } -func (*HardwareInfo) ProtoMessage() {} -func (*HardwareInfo) Descriptor() ([]byte, []int) { return fileDescriptorDiagnostics, []int{4} } +var xxx_messageInfo_HardwareInfo proto.InternalMessageInfo type OSInfo struct { - Family string `protobuf:"bytes,1,opt,name=family,proto3" json:"family,omitempty"` - Platform string `protobuf:"bytes,2,opt,name=platform,proto3" json:"platform,omitempty"` - Version string `protobuf:"bytes,3,opt,name=version,proto3" json:"version,omitempty"` + Family string `protobuf:"bytes,1,opt,name=family,proto3" json:"family,omitempty"` + Platform string `protobuf:"bytes,2,opt,name=platform,proto3" json:"platform,omitempty"` + Version string `protobuf:"bytes,3,opt,name=version,proto3" json:"version,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *OSInfo) Reset() { *m = OSInfo{} } +func (m *OSInfo) String() string { return proto.CompactTextString(m) } +func (*OSInfo) ProtoMessage() {} +func (*OSInfo) Descriptor() ([]byte, []int) { + return fileDescriptor_diagnostics_d68dd0a3f6635a47, []int{5} +} +func (m *OSInfo) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *OSInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *OSInfo) XXX_Merge(src proto.Message) { + xxx_messageInfo_OSInfo.Merge(dst, src) +} +func (m *OSInfo) XXX_Size() int { + return m.Size() +} +func (m *OSInfo) XXX_DiscardUnknown() { + xxx_messageInfo_OSInfo.DiscardUnknown(m) } -func (m *OSInfo) Reset() { *m = OSInfo{} } -func (m *OSInfo) String() string { return proto.CompactTextString(m) } -func (*OSInfo) ProtoMessage() {} -func (*OSInfo) Descriptor() ([]byte, []int) { return fileDescriptorDiagnostics, []int{5} } +var xxx_messageInfo_OSInfo proto.InternalMessageInfo type MemInfo struct { - Total uint64 `protobuf:"varint,1,opt,name=total,proto3" json:"total,omitempty"` - Available uint64 `protobuf:"varint,2,opt,name=available,proto3" json:"available,omitempty"` + Total uint64 `protobuf:"varint,1,opt,name=total,proto3" json:"total,omitempty"` + Available uint64 `protobuf:"varint,2,opt,name=available,proto3" json:"available,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *MemInfo) Reset() { *m = MemInfo{} } +func (m *MemInfo) String() string { return proto.CompactTextString(m) } +func (*MemInfo) ProtoMessage() {} +func (*MemInfo) Descriptor() ([]byte, []int) { + return fileDescriptor_diagnostics_d68dd0a3f6635a47, []int{6} +} +func (m *MemInfo) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *MemInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *MemInfo) XXX_Merge(src proto.Message) { + xxx_messageInfo_MemInfo.Merge(dst, src) +} +func (m *MemInfo) XXX_Size() int { + return m.Size() +} +func (m *MemInfo) XXX_DiscardUnknown() { + xxx_messageInfo_MemInfo.DiscardUnknown(m) } -func (m *MemInfo) Reset() { *m = MemInfo{} } -func (m *MemInfo) String() string { return proto.CompactTextString(m) } -func (*MemInfo) ProtoMessage() {} -func (*MemInfo) Descriptor() ([]byte, []int) { return fileDescriptorDiagnostics, []int{6} } +var xxx_messageInfo_MemInfo proto.InternalMessageInfo func init() { proto.RegisterType((*DiagnosticReport)(nil), "cockroach.server.diagnosticspb.DiagnosticReport") + proto.RegisterMapType((map[string]string)(nil), "cockroach.server.diagnosticspb.DiagnosticReport.AlteredSettingsEntry") + proto.RegisterMapType((map[string]int32)(nil), "cockroach.server.diagnosticspb.DiagnosticReport.FeatureUsageEntry") + proto.RegisterMapType((map[string]int64)(nil), "cockroach.server.diagnosticspb.DiagnosticReport.LegacyErrorCountsEntry") + proto.RegisterMapType((map[string]int64)(nil), "cockroach.server.diagnosticspb.DiagnosticReport.LegacyUnimplementedErrorsEntry") + proto.RegisterMapType((map[int64]config.ZoneConfig)(nil), "cockroach.server.diagnosticspb.DiagnosticReport.ZoneConfigsEntry") proto.RegisterType((*NodeInfo)(nil), "cockroach.server.diagnosticspb.NodeInfo") proto.RegisterType((*StoreInfo)(nil), "cockroach.server.diagnosticspb.StoreInfo") proto.RegisterType((*CPUInfo)(nil), "cockroach.server.diagnosticspb.CPUInfo") @@ -212,7 +383,7 @@ func (m *DiagnosticReport) MarshalTo(dAtA []byte) (int, error) { for k := range m.LegacyUnimplementedErrors { keysForLegacyUnimplementedErrors = append(keysForLegacyUnimplementedErrors, string(k)) } - sortkeys.Strings(keysForLegacyUnimplementedErrors) + github_com_gogo_protobuf_sortkeys.Strings(keysForLegacyUnimplementedErrors) for _, k := range keysForLegacyUnimplementedErrors { dAtA[i] = 0x2a i++ @@ -233,7 +404,7 @@ func (m *DiagnosticReport) MarshalTo(dAtA []byte) (int, error) { for k := range m.AlteredSettings { keysForAlteredSettings = append(keysForAlteredSettings, string(k)) } - sortkeys.Strings(keysForAlteredSettings) + github_com_gogo_protobuf_sortkeys.Strings(keysForAlteredSettings) for _, k := range keysForAlteredSettings { dAtA[i] = 0x32 i++ @@ -255,7 +426,7 @@ func (m *DiagnosticReport) MarshalTo(dAtA []byte) (int, error) { for k := range m.LegacyErrorCounts { keysForLegacyErrorCounts = append(keysForLegacyErrorCounts, string(k)) } - sortkeys.Strings(keysForLegacyErrorCounts) + github_com_gogo_protobuf_sortkeys.Strings(keysForLegacyErrorCounts) for _, k := range keysForLegacyErrorCounts { dAtA[i] = 0x3a i++ @@ -276,7 +447,7 @@ func (m *DiagnosticReport) MarshalTo(dAtA []byte) (int, error) { for k := range m.ZoneConfigs { keysForZoneConfigs = append(keysForZoneConfigs, int64(k)) } - sortkeys.Int64s(keysForZoneConfigs) + github_com_gogo_protobuf_sortkeys.Int64s(keysForZoneConfigs) for _, k := range keysForZoneConfigs { dAtA[i] = 0x42 i++ @@ -306,7 +477,7 @@ func (m *DiagnosticReport) MarshalTo(dAtA []byte) (int, error) { for k := range m.FeatureUsage { keysForFeatureUsage = append(keysForFeatureUsage, string(k)) } - sortkeys.Strings(keysForFeatureUsage) + github_com_gogo_protobuf_sortkeys.Strings(keysForFeatureUsage) for _, k := range keysForFeatureUsage { dAtA[i] = 0x4a i++ @@ -484,7 +655,7 @@ func (m *CPUInfo) MarshalTo(dAtA []byte) (int, error) { if m.Mhz != 0 { dAtA[i] = 0x2d i++ - binary.LittleEndian.PutUint32(dAtA[i:], uint32(math.Float32bits(float32(m.Mhz)))) + encoding_binary.LittleEndian.PutUint32(dAtA[i:], uint32(math.Float32bits(float32(m.Mhz)))) i += 4 } if len(m.Features) > 0 { @@ -545,7 +716,7 @@ func (m *HardwareInfo) MarshalTo(dAtA []byte) (int, error) { if m.Loadavg15 != 0 { dAtA[i] = 0x25 i++ - binary.LittleEndian.PutUint32(dAtA[i:], uint32(math.Float32bits(float32(m.Loadavg15)))) + encoding_binary.LittleEndian.PutUint32(dAtA[i:], uint32(math.Float32bits(float32(m.Loadavg15)))) i += 4 } return i, nil @@ -625,6 +796,9 @@ func encodeVarintDiagnostics(dAtA []byte, offset int, v uint64) int { return offset + 1 } func (m *DiagnosticReport) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.Node.Size() @@ -692,6 +866,9 @@ func (m *DiagnosticReport) Size() (n int) { } func (m *NodeInfo) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.NodeID != 0 { @@ -716,6 +893,9 @@ func (m *NodeInfo) Size() (n int) { } func (m *StoreInfo) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.NodeID != 0 { @@ -746,6 +926,9 @@ func (m *StoreInfo) Size() (n int) { } func (m *CPUInfo) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Numcpu != 0 { @@ -774,6 +957,9 @@ func (m *CPUInfo) Size() (n int) { } func (m *HardwareInfo) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Virtualization) @@ -791,6 +977,9 @@ func (m *HardwareInfo) Size() (n int) { } func (m *OSInfo) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Family) @@ -809,6 +998,9 @@ func (m *OSInfo) Size() (n int) { } func (m *MemInfo) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Total != 0 { @@ -949,7 +1141,7 @@ func (m *DiagnosticReport) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Schema = append(m.Schema, cockroach_sql_sqlbase1.TableDescriptor{}) + m.Schema = append(m.Schema, sqlbase.TableDescriptor{}) if err := m.Schema[len(m.Schema)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -980,7 +1172,7 @@ func (m *DiagnosticReport) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.SqlStats = append(m.SqlStats, cockroach_sql.CollectedStatementStatistics{}) + m.SqlStats = append(m.SqlStats, roachpb.CollectedStatementStatistics{}) if err := m.SqlStats[len(m.SqlStats)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -1344,10 +1536,10 @@ func (m *DiagnosticReport) Unmarshal(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.ZoneConfigs == nil { - m.ZoneConfigs = make(map[int64]cockroach_config.ZoneConfig) + m.ZoneConfigs = make(map[int64]config.ZoneConfig) } var mapkey int64 - mapvalue := &cockroach_config.ZoneConfig{} + mapvalue := &config.ZoneConfig{} for iNdEx < postIndex { entryPreIndex := iNdEx var wire uint64 @@ -1407,7 +1599,7 @@ func (m *DiagnosticReport) Unmarshal(dAtA []byte) error { if postmsgIndex > l { return io.ErrUnexpectedEOF } - mapvalue = &cockroach_config.ZoneConfig{} + mapvalue = &config.ZoneConfig{} if err := mapvalue.Unmarshal(dAtA[iNdEx:postmsgIndex]); err != nil { return err } @@ -2098,7 +2290,7 @@ func (m *CPUInfo) Unmarshal(dAtA []byte) error { if (iNdEx + 4) > l { return io.ErrUnexpectedEOF } - v = uint32(binary.LittleEndian.Uint32(dAtA[iNdEx:])) + v = uint32(encoding_binary.LittleEndian.Uint32(dAtA[iNdEx:])) iNdEx += 4 m.Mhz = float32(math.Float32frombits(v)) case 6: @@ -2277,7 +2469,7 @@ func (m *HardwareInfo) Unmarshal(dAtA []byte) error { if (iNdEx + 4) > l { return io.ErrUnexpectedEOF } - v = uint32(binary.LittleEndian.Uint32(dAtA[iNdEx:])) + v = uint32(encoding_binary.LittleEndian.Uint32(dAtA[iNdEx:])) iNdEx += 4 m.Loadavg15 = float32(math.Float32frombits(v)) default: @@ -2631,9 +2823,11 @@ var ( ErrIntOverflowDiagnostics = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("server/diagnosticspb/diagnostics.proto", fileDescriptorDiagnostics) } +func init() { + proto.RegisterFile("server/diagnosticspb/diagnostics.proto", fileDescriptor_diagnostics_d68dd0a3f6635a47) +} -var fileDescriptorDiagnostics = []byte{ +var fileDescriptor_diagnostics_d68dd0a3f6635a47 = []byte{ // 1083 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x56, 0x4f, 0x6f, 0xe3, 0x44, 0x14, 0x5f, 0xc7, 0xf9, 0x3b, 0xdd, 0x85, 0x76, 0xa8, 0x8a, 0x49, 0xab, 0x74, 0x95, 0x43, 0xb7, diff --git a/pkg/server/node.go b/pkg/server/node.go index eb2f7116d473..75b90ab0d388 100644 --- a/pkg/server/node.go +++ b/pkg/server/node.go @@ -373,7 +373,7 @@ func (n *Node) bootstrap( func (n *Node) onClusterVersionChange(cv cluster.ClusterVersion) { ctx := n.AnnotateCtx(context.Background()) if err := n.stores.OnClusterVersionChange(ctx, cv); err != nil { - log.Fatal(ctx, errors.Wrapf(err, "updating cluster version to %s", cv)) + log.Fatal(ctx, errors.Wrapf(err, "updating cluster version to %v", cv)) } } diff --git a/pkg/server/serverpb/admin.pb.go b/pkg/server/serverpb/admin.pb.go index 32ac0be96227..ad1dc0596051 100644 --- a/pkg/server/serverpb/admin.pb.go +++ b/pkg/server/serverpb/admin.pb.go @@ -1,152 +1,29 @@ // Code generated by protoc-gen-gogo. DO NOT EDIT. // source: server/serverpb/admin.proto -/* - Package serverpb is a generated protocol buffer package. - - It is generated from these files: - server/serverpb/admin.proto - server/serverpb/authentication.proto - server/serverpb/init.proto - server/serverpb/status.proto - - It has these top-level messages: - DatabasesRequest - DatabasesResponse - DatabaseDetailsRequest - DatabaseDetailsResponse - TableDetailsRequest - TableDetailsResponse - TableStatsRequest - TableStatsResponse - NonTableStatsRequest - NonTableStatsResponse - UsersRequest - UsersResponse - EventsRequest - EventsResponse - SetUIDataRequest - SetUIDataResponse - GetUIDataRequest - GetUIDataResponse - ClusterRequest - ClusterResponse - DrainRequest - DrainResponse - DecommissionStatusRequest - DecommissionRequest - DecommissionStatusResponse - SettingsRequest - SettingsResponse - HealthRequest - HealthResponse - LivenessRequest - LivenessResponse - JobsRequest - JobsResponse - LocationsRequest - LocationsResponse - RangeLogRequest - RangeLogResponse - QueryPlanRequest - QueryPlanResponse - DataDistributionRequest - DataDistributionResponse - MetricMetadataRequest - MetricMetadataResponse - EnqueueRangeRequest - EnqueueRangeResponse - UserLoginRequest - UserLoginResponse - UserLogoutRequest - UserLogoutResponse - SessionCookie - BootstrapRequest - BootstrapResponse - CertificatesRequest - CertificateDetails - CertificatesResponse - DetailsRequest - SystemInfo - DetailsResponse - NodesRequest - NodesResponse - NodeRequest - RaftState - RangeProblems - RangeStatistics - PrettySpan - RangeInfo - RangesRequest - RangesResponse - GossipRequest - TraceEvent - AllocatorDryRun - AllocatorRangeRequest - AllocatorRangeResponse - AllocatorRequest - AllocatorResponse - JSONResponse - LogsRequest - LogEntriesResponse - LogFilesListRequest - LogFilesListResponse - LogFileRequest - StacksRequest - File - GetFilesRequest - GetFilesResponse - ProfileRequest - MetricsRequest - RaftRangeNode - RaftRangeError - RaftRangeStatus - RaftDebugRequest - RaftDebugResponse - ActiveQuery - ListSessionsRequest - Session - ListSessionsError - ListSessionsResponse - CancelQueryRequest - CancelQueryResponse - CancelSessionRequest - CancelSessionResponse - SpanStatsRequest - SpanStatsResponse - ProblemRangesRequest - ProblemRangesResponse - RangeRequest - RangeResponse - DiagnosticsRequest - StoresRequest - StoreDetails - StoresResponse - StatementsRequest - StatementsResponse -*/ package serverpb import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" -import cockroach_config "github.com/cockroachdb/cockroach/pkg/config" -import cockroach_sql_jobs_jobspb "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" -import cockroach_storage_engine_enginepb1 "github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb" -import cockroach_storage "github.com/cockroachdb/cockroach/pkg/storage/storagepb" -import cockroach_storage2 "github.com/cockroachdb/cockroach/pkg/storage/storagepb" -import cockroach_util_metric "github.com/cockroachdb/cockroach/pkg/util/metric" +import config "github.com/cockroachdb/cockroach/pkg/config" +import jobspb "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" +import enginepb "github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb" +import storagepb "github.com/cockroachdb/cockroach/pkg/storage/storagepb" +import metric "github.com/cockroachdb/cockroach/pkg/util/metric" import time "time" import github_com_cockroachdb_cockroach_pkg_roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" import github_com_cockroachdb_cockroach_pkg_sql_sqlbase "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" -import context "context" -import grpc "google.golang.org/grpc" +import ( + context "context" + grpc "google.golang.org/grpc" +) -import sortkeys "github.com/gogo/protobuf/sortkeys" -import binary "encoding/binary" -import types "github.com/gogo/protobuf/types" +import github_com_gogo_protobuf_sortkeys "github.com/gogo/protobuf/sortkeys" +import encoding_binary "encoding/binary" +import github_com_gogo_protobuf_types "github.com/gogo/protobuf/types" import io "io" @@ -194,7 +71,9 @@ var ZoneConfigurationLevel_value = map[string]int32{ func (x ZoneConfigurationLevel) String() string { return proto.EnumName(ZoneConfigurationLevel_name, int32(x)) } -func (ZoneConfigurationLevel) EnumDescriptor() ([]byte, []int) { return fileDescriptorAdmin, []int{0} } +func (ZoneConfigurationLevel) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_admin_b98220ab75a6bb7c, []int{0} +} type DrainMode int32 @@ -219,95 +98,251 @@ var DrainMode_value = map[string]int32{ func (x DrainMode) String() string { return proto.EnumName(DrainMode_name, int32(x)) } -func (DrainMode) EnumDescriptor() ([]byte, []int) { return fileDescriptorAdmin, []int{1} } +func (DrainMode) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_admin_b98220ab75a6bb7c, []int{1} +} // DatabasesRequest requests a list of databases. type DatabasesRequest struct { + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *DatabasesRequest) Reset() { *m = DatabasesRequest{} } -func (m *DatabasesRequest) String() string { return proto.CompactTextString(m) } -func (*DatabasesRequest) ProtoMessage() {} -func (*DatabasesRequest) Descriptor() ([]byte, []int) { return fileDescriptorAdmin, []int{0} } +func (m *DatabasesRequest) Reset() { *m = DatabasesRequest{} } +func (m *DatabasesRequest) String() string { return proto.CompactTextString(m) } +func (*DatabasesRequest) ProtoMessage() {} +func (*DatabasesRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_admin_b98220ab75a6bb7c, []int{0} +} +func (m *DatabasesRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *DatabasesRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *DatabasesRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_DatabasesRequest.Merge(dst, src) +} +func (m *DatabasesRequest) XXX_Size() int { + return m.Size() +} +func (m *DatabasesRequest) XXX_DiscardUnknown() { + xxx_messageInfo_DatabasesRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_DatabasesRequest proto.InternalMessageInfo // DatabasesResponse contains a list of databases. type DatabasesResponse struct { - Databases []string `protobuf:"bytes,1,rep,name=databases" json:"databases,omitempty"` + Databases []string `protobuf:"bytes,1,rep,name=databases,proto3" json:"databases,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *DatabasesResponse) Reset() { *m = DatabasesResponse{} } +func (m *DatabasesResponse) String() string { return proto.CompactTextString(m) } +func (*DatabasesResponse) ProtoMessage() {} +func (*DatabasesResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_admin_b98220ab75a6bb7c, []int{1} +} +func (m *DatabasesResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *DatabasesResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *DatabasesResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_DatabasesResponse.Merge(dst, src) +} +func (m *DatabasesResponse) XXX_Size() int { + return m.Size() +} +func (m *DatabasesResponse) XXX_DiscardUnknown() { + xxx_messageInfo_DatabasesResponse.DiscardUnknown(m) } -func (m *DatabasesResponse) Reset() { *m = DatabasesResponse{} } -func (m *DatabasesResponse) String() string { return proto.CompactTextString(m) } -func (*DatabasesResponse) ProtoMessage() {} -func (*DatabasesResponse) Descriptor() ([]byte, []int) { return fileDescriptorAdmin, []int{1} } +var xxx_messageInfo_DatabasesResponse proto.InternalMessageInfo // DatabaseDetailsRequest requests detailed information about the specified // database type DatabaseDetailsRequest struct { // database is the name of the database we are querying. - Database string `protobuf:"bytes,1,opt,name=database,proto3" json:"database,omitempty"` + Database string `protobuf:"bytes,1,opt,name=database,proto3" json:"database,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *DatabaseDetailsRequest) Reset() { *m = DatabaseDetailsRequest{} } -func (m *DatabaseDetailsRequest) String() string { return proto.CompactTextString(m) } -func (*DatabaseDetailsRequest) ProtoMessage() {} -func (*DatabaseDetailsRequest) Descriptor() ([]byte, []int) { return fileDescriptorAdmin, []int{2} } +func (m *DatabaseDetailsRequest) Reset() { *m = DatabaseDetailsRequest{} } +func (m *DatabaseDetailsRequest) String() string { return proto.CompactTextString(m) } +func (*DatabaseDetailsRequest) ProtoMessage() {} +func (*DatabaseDetailsRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_admin_b98220ab75a6bb7c, []int{2} +} +func (m *DatabaseDetailsRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *DatabaseDetailsRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *DatabaseDetailsRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_DatabaseDetailsRequest.Merge(dst, src) +} +func (m *DatabaseDetailsRequest) XXX_Size() int { + return m.Size() +} +func (m *DatabaseDetailsRequest) XXX_DiscardUnknown() { + xxx_messageInfo_DatabaseDetailsRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_DatabaseDetailsRequest proto.InternalMessageInfo // DatabaseDetailsResponse contains grant information and table names for a // database. type DatabaseDetailsResponse struct { // grants are the results of SHOW GRANTS for this database. - Grants []DatabaseDetailsResponse_Grant `protobuf:"bytes,1,rep,name=grants" json:"grants"` + Grants []DatabaseDetailsResponse_Grant `protobuf:"bytes,1,rep,name=grants,proto3" json:"grants"` // table_names contains the names of all tables (SHOW TABLES) in this // database. - TableNames []string `protobuf:"bytes,2,rep,name=table_names,json=tableNames" json:"table_names,omitempty"` + TableNames []string `protobuf:"bytes,2,rep,name=table_names,json=tableNames,proto3" json:"table_names,omitempty"` // descriptor_id is an identifier used to uniquely identify this database. // It can be used to find events pertaining to this database by filtering on // the 'target_id' field of events. DescriptorID int64 `protobuf:"varint,3,opt,name=descriptor_id,json=descriptorId,proto3" json:"descriptor_id,omitempty"` // The zone configuration in effect for this database. - ZoneConfig cockroach_config.ZoneConfig `protobuf:"bytes,4,opt,name=zone_config,json=zoneConfig" json:"zone_config"` + ZoneConfig config.ZoneConfig `protobuf:"bytes,4,opt,name=zone_config,json=zoneConfig,proto3" json:"zone_config"` // The level at which this object's zone configuration is set. - ZoneConfigLevel ZoneConfigurationLevel `protobuf:"varint,5,opt,name=zone_config_level,json=zoneConfigLevel,proto3,enum=cockroach.server.serverpb.ZoneConfigurationLevel" json:"zone_config_level,omitempty"` + ZoneConfigLevel ZoneConfigurationLevel `protobuf:"varint,5,opt,name=zone_config_level,json=zoneConfigLevel,proto3,enum=cockroach.server.serverpb.ZoneConfigurationLevel" json:"zone_config_level,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *DatabaseDetailsResponse) Reset() { *m = DatabaseDetailsResponse{} } -func (m *DatabaseDetailsResponse) String() string { return proto.CompactTextString(m) } -func (*DatabaseDetailsResponse) ProtoMessage() {} -func (*DatabaseDetailsResponse) Descriptor() ([]byte, []int) { return fileDescriptorAdmin, []int{3} } +func (m *DatabaseDetailsResponse) Reset() { *m = DatabaseDetailsResponse{} } +func (m *DatabaseDetailsResponse) String() string { return proto.CompactTextString(m) } +func (*DatabaseDetailsResponse) ProtoMessage() {} +func (*DatabaseDetailsResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_admin_b98220ab75a6bb7c, []int{3} +} +func (m *DatabaseDetailsResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *DatabaseDetailsResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *DatabaseDetailsResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_DatabaseDetailsResponse.Merge(dst, src) +} +func (m *DatabaseDetailsResponse) XXX_Size() int { + return m.Size() +} +func (m *DatabaseDetailsResponse) XXX_DiscardUnknown() { + xxx_messageInfo_DatabaseDetailsResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_DatabaseDetailsResponse proto.InternalMessageInfo type DatabaseDetailsResponse_Grant struct { // user is the user that this grant applies to. User string `protobuf:"bytes,1,opt,name=user,proto3" json:"user,omitempty"` // privileges are the abilities this grant gives to the user. - Privileges []string `protobuf:"bytes,2,rep,name=privileges" json:"privileges,omitempty"` + Privileges []string `protobuf:"bytes,2,rep,name=privileges,proto3" json:"privileges,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *DatabaseDetailsResponse_Grant) Reset() { *m = DatabaseDetailsResponse_Grant{} } func (m *DatabaseDetailsResponse_Grant) String() string { return proto.CompactTextString(m) } func (*DatabaseDetailsResponse_Grant) ProtoMessage() {} func (*DatabaseDetailsResponse_Grant) Descriptor() ([]byte, []int) { - return fileDescriptorAdmin, []int{3, 0} + return fileDescriptor_admin_b98220ab75a6bb7c, []int{3, 0} +} +func (m *DatabaseDetailsResponse_Grant) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *DatabaseDetailsResponse_Grant) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *DatabaseDetailsResponse_Grant) XXX_Merge(src proto.Message) { + xxx_messageInfo_DatabaseDetailsResponse_Grant.Merge(dst, src) +} +func (m *DatabaseDetailsResponse_Grant) XXX_Size() int { + return m.Size() +} +func (m *DatabaseDetailsResponse_Grant) XXX_DiscardUnknown() { + xxx_messageInfo_DatabaseDetailsResponse_Grant.DiscardUnknown(m) } +var xxx_messageInfo_DatabaseDetailsResponse_Grant proto.InternalMessageInfo + // TableDetailsRequest is a request for detailed information about a table. type TableDetailsRequest struct { // database is the database that contains the table we're interested in. Database string `protobuf:"bytes,1,opt,name=database,proto3" json:"database,omitempty"` // table is the name of the table that we're querying. - Table string `protobuf:"bytes,2,opt,name=table,proto3" json:"table,omitempty"` + Table string `protobuf:"bytes,2,opt,name=table,proto3" json:"table,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *TableDetailsRequest) Reset() { *m = TableDetailsRequest{} } -func (m *TableDetailsRequest) String() string { return proto.CompactTextString(m) } -func (*TableDetailsRequest) ProtoMessage() {} -func (*TableDetailsRequest) Descriptor() ([]byte, []int) { return fileDescriptorAdmin, []int{4} } +func (m *TableDetailsRequest) Reset() { *m = TableDetailsRequest{} } +func (m *TableDetailsRequest) String() string { return proto.CompactTextString(m) } +func (*TableDetailsRequest) ProtoMessage() {} +func (*TableDetailsRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_admin_b98220ab75a6bb7c, []int{4} +} +func (m *TableDetailsRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *TableDetailsRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *TableDetailsRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_TableDetailsRequest.Merge(dst, src) +} +func (m *TableDetailsRequest) XXX_Size() int { + return m.Size() +} +func (m *TableDetailsRequest) XXX_DiscardUnknown() { + xxx_messageInfo_TableDetailsRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_TableDetailsRequest proto.InternalMessageInfo // TableDetailsResponse contains grants, column names, and indexes for // a table. type TableDetailsResponse struct { - Grants []TableDetailsResponse_Grant `protobuf:"bytes,1,rep,name=grants" json:"grants"` - Columns []TableDetailsResponse_Column `protobuf:"bytes,2,rep,name=columns" json:"columns"` - Indexes []TableDetailsResponse_Index `protobuf:"bytes,3,rep,name=indexes" json:"indexes"` + Grants []TableDetailsResponse_Grant `protobuf:"bytes,1,rep,name=grants,proto3" json:"grants"` + Columns []TableDetailsResponse_Column `protobuf:"bytes,2,rep,name=columns,proto3" json:"columns"` + Indexes []TableDetailsResponse_Index `protobuf:"bytes,3,rep,name=indexes,proto3" json:"indexes"` // range_count is the size of the table in ranges. This provides a rough // estimate of the storage requirements for the table. // TODO(mrtracy): The TableStats method also returns a range_count field which @@ -321,35 +356,85 @@ type TableDetailsResponse struct { // executed. CreateTableStatement string `protobuf:"bytes,5,opt,name=create_table_statement,json=createTableStatement,proto3" json:"create_table_statement,omitempty"` // The zone configuration in effect for this table. - ZoneConfig cockroach_config.ZoneConfig `protobuf:"bytes,6,opt,name=zone_config,json=zoneConfig" json:"zone_config"` + ZoneConfig config.ZoneConfig `protobuf:"bytes,6,opt,name=zone_config,json=zoneConfig,proto3" json:"zone_config"` // The level at which this object's zone configuration is set. ZoneConfigLevel ZoneConfigurationLevel `protobuf:"varint,7,opt,name=zone_config_level,json=zoneConfigLevel,proto3,enum=cockroach.server.serverpb.ZoneConfigurationLevel" json:"zone_config_level,omitempty"` // descriptor_id is an identifier used to uniquely identify this table. // It can be used to find events pertaining to this table by filtering on // the 'target_id' field of events. - DescriptorID int64 `protobuf:"varint,8,opt,name=descriptor_id,json=descriptorId,proto3" json:"descriptor_id,omitempty"` + DescriptorID int64 `protobuf:"varint,8,opt,name=descriptor_id,json=descriptorId,proto3" json:"descriptor_id,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *TableDetailsResponse) Reset() { *m = TableDetailsResponse{} } -func (m *TableDetailsResponse) String() string { return proto.CompactTextString(m) } -func (*TableDetailsResponse) ProtoMessage() {} -func (*TableDetailsResponse) Descriptor() ([]byte, []int) { return fileDescriptorAdmin, []int{5} } +func (m *TableDetailsResponse) Reset() { *m = TableDetailsResponse{} } +func (m *TableDetailsResponse) String() string { return proto.CompactTextString(m) } +func (*TableDetailsResponse) ProtoMessage() {} +func (*TableDetailsResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_admin_b98220ab75a6bb7c, []int{5} +} +func (m *TableDetailsResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *TableDetailsResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *TableDetailsResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_TableDetailsResponse.Merge(dst, src) +} +func (m *TableDetailsResponse) XXX_Size() int { + return m.Size() +} +func (m *TableDetailsResponse) XXX_DiscardUnknown() { + xxx_messageInfo_TableDetailsResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_TableDetailsResponse proto.InternalMessageInfo // Grant is an entry from SHOW GRANTS. type TableDetailsResponse_Grant struct { // user is the user that this grant applies to. User string `protobuf:"bytes,1,opt,name=user,proto3" json:"user,omitempty"` // privileges are the abilities this grant gives to the user. - Privileges []string `protobuf:"bytes,2,rep,name=privileges" json:"privileges,omitempty"` + Privileges []string `protobuf:"bytes,2,rep,name=privileges,proto3" json:"privileges,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *TableDetailsResponse_Grant) Reset() { *m = TableDetailsResponse_Grant{} } func (m *TableDetailsResponse_Grant) String() string { return proto.CompactTextString(m) } func (*TableDetailsResponse_Grant) ProtoMessage() {} func (*TableDetailsResponse_Grant) Descriptor() ([]byte, []int) { - return fileDescriptorAdmin, []int{5, 0} + return fileDescriptor_admin_b98220ab75a6bb7c, []int{5, 0} +} +func (m *TableDetailsResponse_Grant) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *TableDetailsResponse_Grant) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *TableDetailsResponse_Grant) XXX_Merge(src proto.Message) { + xxx_messageInfo_TableDetailsResponse_Grant.Merge(dst, src) +} +func (m *TableDetailsResponse_Grant) XXX_Size() int { + return m.Size() +} +func (m *TableDetailsResponse_Grant) XXX_DiscardUnknown() { + xxx_messageInfo_TableDetailsResponse_Grant.DiscardUnknown(m) } +var xxx_messageInfo_TableDetailsResponse_Grant proto.InternalMessageInfo + type TableDetailsResponse_Column struct { // name is the name of the column. Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` @@ -362,16 +447,40 @@ type TableDetailsResponse_Column struct { // generation_expression is the generator expression if the column is computed. GenerationExpression string `protobuf:"bytes,5,opt,name=generation_expression,json=generationExpression,proto3" json:"generation_expression,omitempty"` // hidden is whether this column is hidden. - Hidden bool `protobuf:"varint,6,opt,name=hidden,proto3" json:"hidden,omitempty"` + Hidden bool `protobuf:"varint,6,opt,name=hidden,proto3" json:"hidden,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *TableDetailsResponse_Column) Reset() { *m = TableDetailsResponse_Column{} } func (m *TableDetailsResponse_Column) String() string { return proto.CompactTextString(m) } func (*TableDetailsResponse_Column) ProtoMessage() {} func (*TableDetailsResponse_Column) Descriptor() ([]byte, []int) { - return fileDescriptorAdmin, []int{5, 1} + return fileDescriptor_admin_b98220ab75a6bb7c, []int{5, 1} +} +func (m *TableDetailsResponse_Column) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *TableDetailsResponse_Column) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *TableDetailsResponse_Column) XXX_Merge(src proto.Message) { + xxx_messageInfo_TableDetailsResponse_Column.Merge(dst, src) +} +func (m *TableDetailsResponse_Column) XXX_Size() int { + return m.Size() +} +func (m *TableDetailsResponse_Column) XXX_DiscardUnknown() { + xxx_messageInfo_TableDetailsResponse_Column.DiscardUnknown(m) } +var xxx_messageInfo_TableDetailsResponse_Column proto.InternalMessageInfo + type TableDetailsResponse_Index struct { // name is the name of this index. Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` @@ -386,15 +495,39 @@ type TableDetailsResponse_Index struct { // storing is an internal variable that's passed along. Storing bool `protobuf:"varint,6,opt,name=storing,proto3" json:"storing,omitempty"` // implicit is an internal variable that's passed along. - Implicit bool `protobuf:"varint,7,opt,name=implicit,proto3" json:"implicit,omitempty"` + Implicit bool `protobuf:"varint,7,opt,name=implicit,proto3" json:"implicit,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *TableDetailsResponse_Index) Reset() { *m = TableDetailsResponse_Index{} } func (m *TableDetailsResponse_Index) String() string { return proto.CompactTextString(m) } func (*TableDetailsResponse_Index) ProtoMessage() {} func (*TableDetailsResponse_Index) Descriptor() ([]byte, []int) { - return fileDescriptorAdmin, []int{5, 2} + return fileDescriptor_admin_b98220ab75a6bb7c, []int{5, 2} +} +func (m *TableDetailsResponse_Index) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *TableDetailsResponse_Index) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil } +func (dst *TableDetailsResponse_Index) XXX_Merge(src proto.Message) { + xxx_messageInfo_TableDetailsResponse_Index.Merge(dst, src) +} +func (m *TableDetailsResponse_Index) XXX_Size() int { + return m.Size() +} +func (m *TableDetailsResponse_Index) XXX_DiscardUnknown() { + xxx_messageInfo_TableDetailsResponse_Index.DiscardUnknown(m) +} + +var xxx_messageInfo_TableDetailsResponse_Index proto.InternalMessageInfo // TableStatsRequest is a request for detailed, computationally expensive // information about a table. @@ -402,13 +535,39 @@ type TableStatsRequest struct { // database is the database that contains the table we're interested in. Database string `protobuf:"bytes,1,opt,name=database,proto3" json:"database,omitempty"` // table is the name of the table that we're querying. - Table string `protobuf:"bytes,2,opt,name=table,proto3" json:"table,omitempty"` + Table string `protobuf:"bytes,2,opt,name=table,proto3" json:"table,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *TableStatsRequest) Reset() { *m = TableStatsRequest{} } -func (m *TableStatsRequest) String() string { return proto.CompactTextString(m) } -func (*TableStatsRequest) ProtoMessage() {} -func (*TableStatsRequest) Descriptor() ([]byte, []int) { return fileDescriptorAdmin, []int{6} } +func (m *TableStatsRequest) Reset() { *m = TableStatsRequest{} } +func (m *TableStatsRequest) String() string { return proto.CompactTextString(m) } +func (*TableStatsRequest) ProtoMessage() {} +func (*TableStatsRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_admin_b98220ab75a6bb7c, []int{6} +} +func (m *TableStatsRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *TableStatsRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *TableStatsRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_TableStatsRequest.Merge(dst, src) +} +func (m *TableStatsRequest) XXX_Size() int { + return m.Size() +} +func (m *TableStatsRequest) XXX_DiscardUnknown() { + xxx_messageInfo_TableStatsRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_TableStatsRequest proto.InternalMessageInfo // TableStatsResponse contains detailed, computationally expensive information // about a table. @@ -426,19 +585,45 @@ type TableStatsResponse struct { NodeCount int64 `protobuf:"varint,3,opt,name=node_count,json=nodeCount,proto3" json:"node_count,omitempty"` // stats is the summation of MVCCStats for all replicas of this table // across the cluster. - Stats cockroach_storage_engine_enginepb1.MVCCStats `protobuf:"bytes,4,opt,name=stats" json:"stats"` + Stats enginepb.MVCCStats `protobuf:"bytes,4,opt,name=stats,proto3" json:"stats"` // approximate_disk_bytes is an approximation of the disk space (in bytes) // used for all replicas of this table across the cluster. ApproximateDiskBytes uint64 `protobuf:"varint,6,opt,name=approximate_disk_bytes,json=approximateDiskBytes,proto3" json:"approximate_disk_bytes,omitempty"` // A list of nodes which should contain data for this table (according to // cluster metadata), but could not be contacted during this request. - MissingNodes []TableStatsResponse_MissingNode `protobuf:"bytes,5,rep,name=missing_nodes,json=missingNodes" json:"missing_nodes"` + MissingNodes []TableStatsResponse_MissingNode `protobuf:"bytes,5,rep,name=missing_nodes,json=missingNodes,proto3" json:"missing_nodes"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *TableStatsResponse) Reset() { *m = TableStatsResponse{} } -func (m *TableStatsResponse) String() string { return proto.CompactTextString(m) } -func (*TableStatsResponse) ProtoMessage() {} -func (*TableStatsResponse) Descriptor() ([]byte, []int) { return fileDescriptorAdmin, []int{7} } +func (m *TableStatsResponse) Reset() { *m = TableStatsResponse{} } +func (m *TableStatsResponse) String() string { return proto.CompactTextString(m) } +func (*TableStatsResponse) ProtoMessage() {} +func (*TableStatsResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_admin_b98220ab75a6bb7c, []int{7} +} +func (m *TableStatsResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *TableStatsResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *TableStatsResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_TableStatsResponse.Merge(dst, src) +} +func (m *TableStatsResponse) XXX_Size() int { + return m.Size() +} +func (m *TableStatsResponse) XXX_DiscardUnknown() { + xxx_messageInfo_TableStatsResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_TableStatsResponse proto.InternalMessageInfo // MissingNode represents information on a node which should contain data // for this table, but could not be contacted during this request. @@ -446,70 +631,224 @@ type TableStatsResponse_MissingNode struct { // The ID of the missing node. NodeID string `protobuf:"bytes,1,opt,name=node_id,json=nodeId,proto3" json:"node_id,omitempty"` // The error message that resulted when the query sent to this node failed. - ErrorMessage string `protobuf:"bytes,2,opt,name=error_message,json=errorMessage,proto3" json:"error_message,omitempty"` + ErrorMessage string `protobuf:"bytes,2,opt,name=error_message,json=errorMessage,proto3" json:"error_message,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *TableStatsResponse_MissingNode) Reset() { *m = TableStatsResponse_MissingNode{} } func (m *TableStatsResponse_MissingNode) String() string { return proto.CompactTextString(m) } func (*TableStatsResponse_MissingNode) ProtoMessage() {} func (*TableStatsResponse_MissingNode) Descriptor() ([]byte, []int) { - return fileDescriptorAdmin, []int{7, 0} + return fileDescriptor_admin_b98220ab75a6bb7c, []int{7, 0} +} +func (m *TableStatsResponse_MissingNode) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *TableStatsResponse_MissingNode) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil } +func (dst *TableStatsResponse_MissingNode) XXX_Merge(src proto.Message) { + xxx_messageInfo_TableStatsResponse_MissingNode.Merge(dst, src) +} +func (m *TableStatsResponse_MissingNode) XXX_Size() int { + return m.Size() +} +func (m *TableStatsResponse_MissingNode) XXX_DiscardUnknown() { + xxx_messageInfo_TableStatsResponse_MissingNode.DiscardUnknown(m) +} + +var xxx_messageInfo_TableStatsResponse_MissingNode proto.InternalMessageInfo // NonTableStatsRequest requests statistics on cluster data ranges that do not // belong to SQL tables. type NonTableStatsRequest struct { + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *NonTableStatsRequest) Reset() { *m = NonTableStatsRequest{} } -func (m *NonTableStatsRequest) String() string { return proto.CompactTextString(m) } -func (*NonTableStatsRequest) ProtoMessage() {} -func (*NonTableStatsRequest) Descriptor() ([]byte, []int) { return fileDescriptorAdmin, []int{8} } +func (m *NonTableStatsRequest) Reset() { *m = NonTableStatsRequest{} } +func (m *NonTableStatsRequest) String() string { return proto.CompactTextString(m) } +func (*NonTableStatsRequest) ProtoMessage() {} +func (*NonTableStatsRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_admin_b98220ab75a6bb7c, []int{8} +} +func (m *NonTableStatsRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *NonTableStatsRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *NonTableStatsRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_NonTableStatsRequest.Merge(dst, src) +} +func (m *NonTableStatsRequest) XXX_Size() int { + return m.Size() +} +func (m *NonTableStatsRequest) XXX_DiscardUnknown() { + xxx_messageInfo_NonTableStatsRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_NonTableStatsRequest proto.InternalMessageInfo // NonTableStatsResponse returns statistics on various cluster data ranges // that do not belong to SQL tables. The statistics for each range are returned // as a TableStatsResponse. type NonTableStatsResponse struct { // Information on time series ranges. - TimeSeriesStats *TableStatsResponse `protobuf:"bytes,1,opt,name=time_series_stats,json=timeSeriesStats" json:"time_series_stats,omitempty"` + TimeSeriesStats *TableStatsResponse `protobuf:"bytes,1,opt,name=time_series_stats,json=timeSeriesStats,proto3" json:"time_series_stats,omitempty"` // Information for remaining (non-table, non-time-series) ranges. - InternalUseStats *TableStatsResponse `protobuf:"bytes,2,opt,name=internal_use_stats,json=internalUseStats" json:"internal_use_stats,omitempty"` + InternalUseStats *TableStatsResponse `protobuf:"bytes,2,opt,name=internal_use_stats,json=internalUseStats,proto3" json:"internal_use_stats,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *NonTableStatsResponse) Reset() { *m = NonTableStatsResponse{} } +func (m *NonTableStatsResponse) String() string { return proto.CompactTextString(m) } +func (*NonTableStatsResponse) ProtoMessage() {} +func (*NonTableStatsResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_admin_b98220ab75a6bb7c, []int{9} +} +func (m *NonTableStatsResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *NonTableStatsResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *NonTableStatsResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_NonTableStatsResponse.Merge(dst, src) +} +func (m *NonTableStatsResponse) XXX_Size() int { + return m.Size() +} +func (m *NonTableStatsResponse) XXX_DiscardUnknown() { + xxx_messageInfo_NonTableStatsResponse.DiscardUnknown(m) } -func (m *NonTableStatsResponse) Reset() { *m = NonTableStatsResponse{} } -func (m *NonTableStatsResponse) String() string { return proto.CompactTextString(m) } -func (*NonTableStatsResponse) ProtoMessage() {} -func (*NonTableStatsResponse) Descriptor() ([]byte, []int) { return fileDescriptorAdmin, []int{9} } +var xxx_messageInfo_NonTableStatsResponse proto.InternalMessageInfo // UsersRequest requests a list of users. type UsersRequest struct { + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *UsersRequest) Reset() { *m = UsersRequest{} } -func (m *UsersRequest) String() string { return proto.CompactTextString(m) } -func (*UsersRequest) ProtoMessage() {} -func (*UsersRequest) Descriptor() ([]byte, []int) { return fileDescriptorAdmin, []int{10} } +func (m *UsersRequest) Reset() { *m = UsersRequest{} } +func (m *UsersRequest) String() string { return proto.CompactTextString(m) } +func (*UsersRequest) ProtoMessage() {} +func (*UsersRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_admin_b98220ab75a6bb7c, []int{10} +} +func (m *UsersRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *UsersRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *UsersRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_UsersRequest.Merge(dst, src) +} +func (m *UsersRequest) XXX_Size() int { + return m.Size() +} +func (m *UsersRequest) XXX_DiscardUnknown() { + xxx_messageInfo_UsersRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_UsersRequest proto.InternalMessageInfo // UsersResponse returns a list of users. type UsersResponse struct { // usernames is a list of users for the CockroachDB cluster. - Users []UsersResponse_User `protobuf:"bytes,1,rep,name=users" json:"users"` + Users []UsersResponse_User `protobuf:"bytes,1,rep,name=users,proto3" json:"users"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *UsersResponse) Reset() { *m = UsersResponse{} } -func (m *UsersResponse) String() string { return proto.CompactTextString(m) } -func (*UsersResponse) ProtoMessage() {} -func (*UsersResponse) Descriptor() ([]byte, []int) { return fileDescriptorAdmin, []int{11} } +func (m *UsersResponse) Reset() { *m = UsersResponse{} } +func (m *UsersResponse) String() string { return proto.CompactTextString(m) } +func (*UsersResponse) ProtoMessage() {} +func (*UsersResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_admin_b98220ab75a6bb7c, []int{11} +} +func (m *UsersResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *UsersResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *UsersResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_UsersResponse.Merge(dst, src) +} +func (m *UsersResponse) XXX_Size() int { + return m.Size() +} +func (m *UsersResponse) XXX_DiscardUnknown() { + xxx_messageInfo_UsersResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_UsersResponse proto.InternalMessageInfo // User is a CockroachDB user. type UsersResponse_User struct { - Username string `protobuf:"bytes,1,opt,name=username,proto3" json:"username,omitempty"` + Username string `protobuf:"bytes,1,opt,name=username,proto3" json:"username,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *UsersResponse_User) Reset() { *m = UsersResponse_User{} } +func (m *UsersResponse_User) String() string { return proto.CompactTextString(m) } +func (*UsersResponse_User) ProtoMessage() {} +func (*UsersResponse_User) Descriptor() ([]byte, []int) { + return fileDescriptor_admin_b98220ab75a6bb7c, []int{11, 0} +} +func (m *UsersResponse_User) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *UsersResponse_User) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *UsersResponse_User) XXX_Merge(src proto.Message) { + xxx_messageInfo_UsersResponse_User.Merge(dst, src) +} +func (m *UsersResponse_User) XXX_Size() int { + return m.Size() +} +func (m *UsersResponse_User) XXX_DiscardUnknown() { + xxx_messageInfo_UsersResponse_User.DiscardUnknown(m) } -func (m *UsersResponse_User) Reset() { *m = UsersResponse_User{} } -func (m *UsersResponse_User) String() string { return proto.CompactTextString(m) } -func (*UsersResponse_User) ProtoMessage() {} -func (*UsersResponse_User) Descriptor() ([]byte, []int) { return fileDescriptorAdmin, []int{11, 0} } +var xxx_messageInfo_UsersResponse_User proto.InternalMessageInfo // EventsRequest is a request for event log entries, optionally filtered // by the specified event type and/or target_id. @@ -520,28 +859,80 @@ type EventsRequest struct { // this is omitted or set to 0, the default maximum number of results are // returned. When set to > 0, at most only that number of results are // returned. When set to < 0, an unlimited number of results are returned. - Limit int32 `protobuf:"varint,3,opt,name=limit,proto3" json:"limit,omitempty"` + Limit int32 `protobuf:"varint,3,opt,name=limit,proto3" json:"limit,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *EventsRequest) Reset() { *m = EventsRequest{} } -func (m *EventsRequest) String() string { return proto.CompactTextString(m) } -func (*EventsRequest) ProtoMessage() {} -func (*EventsRequest) Descriptor() ([]byte, []int) { return fileDescriptorAdmin, []int{12} } +func (m *EventsRequest) Reset() { *m = EventsRequest{} } +func (m *EventsRequest) String() string { return proto.CompactTextString(m) } +func (*EventsRequest) ProtoMessage() {} +func (*EventsRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_admin_b98220ab75a6bb7c, []int{12} +} +func (m *EventsRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *EventsRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *EventsRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_EventsRequest.Merge(dst, src) +} +func (m *EventsRequest) XXX_Size() int { + return m.Size() +} +func (m *EventsRequest) XXX_DiscardUnknown() { + xxx_messageInfo_EventsRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_EventsRequest proto.InternalMessageInfo // EventsResponse contains a set of event log entries. This is always limited // to the latest N entries (N is enforced in the associated endpoint). type EventsResponse struct { - Events []EventsResponse_Event `protobuf:"bytes,1,rep,name=events" json:"events"` + Events []EventsResponse_Event `protobuf:"bytes,1,rep,name=events,proto3" json:"events"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *EventsResponse) Reset() { *m = EventsResponse{} } -func (m *EventsResponse) String() string { return proto.CompactTextString(m) } -func (*EventsResponse) ProtoMessage() {} -func (*EventsResponse) Descriptor() ([]byte, []int) { return fileDescriptorAdmin, []int{13} } +func (m *EventsResponse) Reset() { *m = EventsResponse{} } +func (m *EventsResponse) String() string { return proto.CompactTextString(m) } +func (*EventsResponse) ProtoMessage() {} +func (*EventsResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_admin_b98220ab75a6bb7c, []int{13} +} +func (m *EventsResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *EventsResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *EventsResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_EventsResponse.Merge(dst, src) +} +func (m *EventsResponse) XXX_Size() int { + return m.Size() +} +func (m *EventsResponse) XXX_DiscardUnknown() { + xxx_messageInfo_EventsResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_EventsResponse proto.InternalMessageInfo type EventsResponse_Event struct { // timestamp is the time at which the event occurred. - Timestamp time.Time `protobuf:"bytes,1,opt,name=timestamp,stdtime" json:"timestamp"` + Timestamp time.Time `protobuf:"bytes,1,opt,name=timestamp,proto3,stdtime" json:"timestamp"` // event_type is the type of the event (e.g. "create_table", "drop_table". EventType string `protobuf:"bytes,2,opt,name=event_type,json=eventType,proto3" json:"event_type,omitempty"` // target_id is the target for this event. @@ -552,79 +943,261 @@ type EventsResponse_Event struct { // depending on the event. Info string `protobuf:"bytes,5,opt,name=info,proto3" json:"info,omitempty"` // unique_id is a unique identifier for this event. - UniqueID []byte `protobuf:"bytes,6,opt,name=unique_id,json=uniqueId,proto3" json:"unique_id,omitempty"` + UniqueID []byte `protobuf:"bytes,6,opt,name=unique_id,json=uniqueId,proto3" json:"unique_id,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *EventsResponse_Event) Reset() { *m = EventsResponse_Event{} } +func (m *EventsResponse_Event) String() string { return proto.CompactTextString(m) } +func (*EventsResponse_Event) ProtoMessage() {} +func (*EventsResponse_Event) Descriptor() ([]byte, []int) { + return fileDescriptor_admin_b98220ab75a6bb7c, []int{13, 0} +} +func (m *EventsResponse_Event) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *EventsResponse_Event) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *EventsResponse_Event) XXX_Merge(src proto.Message) { + xxx_messageInfo_EventsResponse_Event.Merge(dst, src) +} +func (m *EventsResponse_Event) XXX_Size() int { + return m.Size() +} +func (m *EventsResponse_Event) XXX_DiscardUnknown() { + xxx_messageInfo_EventsResponse_Event.DiscardUnknown(m) } -func (m *EventsResponse_Event) Reset() { *m = EventsResponse_Event{} } -func (m *EventsResponse_Event) String() string { return proto.CompactTextString(m) } -func (*EventsResponse_Event) ProtoMessage() {} -func (*EventsResponse_Event) Descriptor() ([]byte, []int) { return fileDescriptorAdmin, []int{13, 0} } +var xxx_messageInfo_EventsResponse_Event proto.InternalMessageInfo // SetUIDataRequest stores the given key/value pairs in the system.ui table. type SetUIDataRequest struct { // key_values is a map of keys to bytes values. Each key will be stored // with its corresponding value as a separate row in system.ui. - KeyValues map[string][]byte `protobuf:"bytes,1,rep,name=key_values,json=keyValues" json:"key_values,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + KeyValues map[string][]byte `protobuf:"bytes,1,rep,name=key_values,json=keyValues,proto3" json:"key_values,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *SetUIDataRequest) Reset() { *m = SetUIDataRequest{} } +func (m *SetUIDataRequest) String() string { return proto.CompactTextString(m) } +func (*SetUIDataRequest) ProtoMessage() {} +func (*SetUIDataRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_admin_b98220ab75a6bb7c, []int{14} +} +func (m *SetUIDataRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SetUIDataRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *SetUIDataRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_SetUIDataRequest.Merge(dst, src) +} +func (m *SetUIDataRequest) XXX_Size() int { + return m.Size() +} +func (m *SetUIDataRequest) XXX_DiscardUnknown() { + xxx_messageInfo_SetUIDataRequest.DiscardUnknown(m) } -func (m *SetUIDataRequest) Reset() { *m = SetUIDataRequest{} } -func (m *SetUIDataRequest) String() string { return proto.CompactTextString(m) } -func (*SetUIDataRequest) ProtoMessage() {} -func (*SetUIDataRequest) Descriptor() ([]byte, []int) { return fileDescriptorAdmin, []int{14} } +var xxx_messageInfo_SetUIDataRequest proto.InternalMessageInfo // SetUIDataResponse is currently an empty response. type SetUIDataResponse struct { + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *SetUIDataResponse) Reset() { *m = SetUIDataResponse{} } -func (m *SetUIDataResponse) String() string { return proto.CompactTextString(m) } -func (*SetUIDataResponse) ProtoMessage() {} -func (*SetUIDataResponse) Descriptor() ([]byte, []int) { return fileDescriptorAdmin, []int{15} } +func (m *SetUIDataResponse) Reset() { *m = SetUIDataResponse{} } +func (m *SetUIDataResponse) String() string { return proto.CompactTextString(m) } +func (*SetUIDataResponse) ProtoMessage() {} +func (*SetUIDataResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_admin_b98220ab75a6bb7c, []int{15} +} +func (m *SetUIDataResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SetUIDataResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *SetUIDataResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_SetUIDataResponse.Merge(dst, src) +} +func (m *SetUIDataResponse) XXX_Size() int { + return m.Size() +} +func (m *SetUIDataResponse) XXX_DiscardUnknown() { + xxx_messageInfo_SetUIDataResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_SetUIDataResponse proto.InternalMessageInfo // GETUIDataRequest requests the values for the given keys from the system.ui // table. type GetUIDataRequest struct { - Keys []string `protobuf:"bytes,1,rep,name=keys" json:"keys,omitempty"` + Keys []string `protobuf:"bytes,1,rep,name=keys,proto3" json:"keys,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *GetUIDataRequest) Reset() { *m = GetUIDataRequest{} } +func (m *GetUIDataRequest) String() string { return proto.CompactTextString(m) } +func (*GetUIDataRequest) ProtoMessage() {} +func (*GetUIDataRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_admin_b98220ab75a6bb7c, []int{16} +} +func (m *GetUIDataRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *GetUIDataRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *GetUIDataRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_GetUIDataRequest.Merge(dst, src) +} +func (m *GetUIDataRequest) XXX_Size() int { + return m.Size() +} +func (m *GetUIDataRequest) XXX_DiscardUnknown() { + xxx_messageInfo_GetUIDataRequest.DiscardUnknown(m) } -func (m *GetUIDataRequest) Reset() { *m = GetUIDataRequest{} } -func (m *GetUIDataRequest) String() string { return proto.CompactTextString(m) } -func (*GetUIDataRequest) ProtoMessage() {} -func (*GetUIDataRequest) Descriptor() ([]byte, []int) { return fileDescriptorAdmin, []int{16} } +var xxx_messageInfo_GetUIDataRequest proto.InternalMessageInfo // GetUIDataResponse contains the requested values and the times at which // the values were last updated. type GetUIDataResponse struct { // key_values maps keys to their retrieved values. If this doesn't contain a // a requested key, that key was not found. - KeyValues map[string]GetUIDataResponse_Value `protobuf:"bytes,1,rep,name=key_values,json=keyValues" json:"key_values" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value"` + KeyValues map[string]GetUIDataResponse_Value `protobuf:"bytes,1,rep,name=key_values,json=keyValues,proto3" json:"key_values" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *GetUIDataResponse) Reset() { *m = GetUIDataResponse{} } +func (m *GetUIDataResponse) String() string { return proto.CompactTextString(m) } +func (*GetUIDataResponse) ProtoMessage() {} +func (*GetUIDataResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_admin_b98220ab75a6bb7c, []int{17} +} +func (m *GetUIDataResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *GetUIDataResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *GetUIDataResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_GetUIDataResponse.Merge(dst, src) +} +func (m *GetUIDataResponse) XXX_Size() int { + return m.Size() +} +func (m *GetUIDataResponse) XXX_DiscardUnknown() { + xxx_messageInfo_GetUIDataResponse.DiscardUnknown(m) } -func (m *GetUIDataResponse) Reset() { *m = GetUIDataResponse{} } -func (m *GetUIDataResponse) String() string { return proto.CompactTextString(m) } -func (*GetUIDataResponse) ProtoMessage() {} -func (*GetUIDataResponse) Descriptor() ([]byte, []int) { return fileDescriptorAdmin, []int{17} } +var xxx_messageInfo_GetUIDataResponse proto.InternalMessageInfo type GetUIDataResponse_Value struct { // value is the value of the requested key. Value []byte `protobuf:"bytes,1,opt,name=value,proto3" json:"value,omitempty"` // last_updated is the time at which the value was last updated. - LastUpdated time.Time `protobuf:"bytes,2,opt,name=last_updated,json=lastUpdated,stdtime" json:"last_updated"` + LastUpdated time.Time `protobuf:"bytes,2,opt,name=last_updated,json=lastUpdated,proto3,stdtime" json:"last_updated"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *GetUIDataResponse_Value) Reset() { *m = GetUIDataResponse_Value{} } -func (m *GetUIDataResponse_Value) String() string { return proto.CompactTextString(m) } -func (*GetUIDataResponse_Value) ProtoMessage() {} -func (*GetUIDataResponse_Value) Descriptor() ([]byte, []int) { return fileDescriptorAdmin, []int{17, 0} } +func (m *GetUIDataResponse_Value) Reset() { *m = GetUIDataResponse_Value{} } +func (m *GetUIDataResponse_Value) String() string { return proto.CompactTextString(m) } +func (*GetUIDataResponse_Value) ProtoMessage() {} +func (*GetUIDataResponse_Value) Descriptor() ([]byte, []int) { + return fileDescriptor_admin_b98220ab75a6bb7c, []int{17, 0} +} +func (m *GetUIDataResponse_Value) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *GetUIDataResponse_Value) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *GetUIDataResponse_Value) XXX_Merge(src proto.Message) { + xxx_messageInfo_GetUIDataResponse_Value.Merge(dst, src) +} +func (m *GetUIDataResponse_Value) XXX_Size() int { + return m.Size() +} +func (m *GetUIDataResponse_Value) XXX_DiscardUnknown() { + xxx_messageInfo_GetUIDataResponse_Value.DiscardUnknown(m) +} + +var xxx_messageInfo_GetUIDataResponse_Value proto.InternalMessageInfo // ClusterRequest requests metadata for the cluster. type ClusterRequest struct { + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *ClusterRequest) Reset() { *m = ClusterRequest{} } -func (m *ClusterRequest) String() string { return proto.CompactTextString(m) } -func (*ClusterRequest) ProtoMessage() {} -func (*ClusterRequest) Descriptor() ([]byte, []int) { return fileDescriptorAdmin, []int{18} } +func (m *ClusterRequest) Reset() { *m = ClusterRequest{} } +func (m *ClusterRequest) String() string { return proto.CompactTextString(m) } +func (*ClusterRequest) ProtoMessage() {} +func (*ClusterRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_admin_b98220ab75a6bb7c, []int{18} +} +func (m *ClusterRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ClusterRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ClusterRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_ClusterRequest.Merge(dst, src) +} +func (m *ClusterRequest) XXX_Size() int { + return m.Size() +} +func (m *ClusterRequest) XXX_DiscardUnknown() { + xxx_messageInfo_ClusterRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_ClusterRequest proto.InternalMessageInfo // ClusterResponse contains metadata for the cluster. type ClusterResponse struct { @@ -633,13 +1206,39 @@ type ClusterResponse struct { // True if diagnostics reporting is enabled for the cluster. ReportingEnabled bool `protobuf:"varint,2,opt,name=reporting_enabled,json=reportingEnabled,proto3" json:"reporting_enabled,omitempty"` // True if enterprise features are enabled for the cluster. - EnterpriseEnabled bool `protobuf:"varint,3,opt,name=enterprise_enabled,json=enterpriseEnabled,proto3" json:"enterprise_enabled,omitempty"` + EnterpriseEnabled bool `protobuf:"varint,3,opt,name=enterprise_enabled,json=enterpriseEnabled,proto3" json:"enterprise_enabled,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *ClusterResponse) Reset() { *m = ClusterResponse{} } -func (m *ClusterResponse) String() string { return proto.CompactTextString(m) } -func (*ClusterResponse) ProtoMessage() {} -func (*ClusterResponse) Descriptor() ([]byte, []int) { return fileDescriptorAdmin, []int{19} } +func (m *ClusterResponse) Reset() { *m = ClusterResponse{} } +func (m *ClusterResponse) String() string { return proto.CompactTextString(m) } +func (*ClusterResponse) ProtoMessage() {} +func (*ClusterResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_admin_b98220ab75a6bb7c, []int{19} +} +func (m *ClusterResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ClusterResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ClusterResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_ClusterResponse.Merge(dst, src) +} +func (m *ClusterResponse) XXX_Size() int { + return m.Size() +} +func (m *ClusterResponse) XXX_DiscardUnknown() { + xxx_messageInfo_ClusterResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_ClusterResponse proto.InternalMessageInfo // DrainRequest requests the server to enter the specified draining mode. The // server first deactivates all the modes specified in 'off' and then activates @@ -649,237 +1248,727 @@ type DrainRequest struct { // proxying enum fields (yet: // https://github.com/grpc-ecosystem/grpc-gateway/issues/5) and it fails in // pretty dramatic ways (panics the server). - On []int32 `protobuf:"varint,1,rep,packed,name=on" json:"on,omitempty"` - Off []int32 `protobuf:"varint,2,rep,packed,name=off" json:"off,omitempty"` + On []int32 `protobuf:"varint,1,rep,packed,name=on,proto3" json:"on,omitempty"` + Off []int32 `protobuf:"varint,2,rep,packed,name=off,proto3" json:"off,omitempty"` // When true, terminates the process after the given drain modes have been // activated. - Shutdown bool `protobuf:"varint,3,opt,name=shutdown,proto3" json:"shutdown,omitempty"` + Shutdown bool `protobuf:"varint,3,opt,name=shutdown,proto3" json:"shutdown,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *DrainRequest) Reset() { *m = DrainRequest{} } +func (m *DrainRequest) String() string { return proto.CompactTextString(m) } +func (*DrainRequest) ProtoMessage() {} +func (*DrainRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_admin_b98220ab75a6bb7c, []int{20} +} +func (m *DrainRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *DrainRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *DrainRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_DrainRequest.Merge(dst, src) +} +func (m *DrainRequest) XXX_Size() int { + return m.Size() +} +func (m *DrainRequest) XXX_DiscardUnknown() { + xxx_messageInfo_DrainRequest.DiscardUnknown(m) } -func (m *DrainRequest) Reset() { *m = DrainRequest{} } -func (m *DrainRequest) String() string { return proto.CompactTextString(m) } -func (*DrainRequest) ProtoMessage() {} -func (*DrainRequest) Descriptor() ([]byte, []int) { return fileDescriptorAdmin, []int{20} } +var xxx_messageInfo_DrainRequest proto.InternalMessageInfo // DrainResponse is the response to a successful DrainRequest and lists the // modes which are activated after having processing the request. type DrainResponse struct { - On []int32 `protobuf:"varint,1,rep,packed,name=on" json:"on,omitempty"` + On []int32 `protobuf:"varint,1,rep,packed,name=on,proto3" json:"on,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *DrainResponse) Reset() { *m = DrainResponse{} } -func (m *DrainResponse) String() string { return proto.CompactTextString(m) } -func (*DrainResponse) ProtoMessage() {} -func (*DrainResponse) Descriptor() ([]byte, []int) { return fileDescriptorAdmin, []int{21} } +func (m *DrainResponse) Reset() { *m = DrainResponse{} } +func (m *DrainResponse) String() string { return proto.CompactTextString(m) } +func (*DrainResponse) ProtoMessage() {} +func (*DrainResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_admin_b98220ab75a6bb7c, []int{21} +} +func (m *DrainResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *DrainResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *DrainResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_DrainResponse.Merge(dst, src) +} +func (m *DrainResponse) XXX_Size() int { + return m.Size() +} +func (m *DrainResponse) XXX_DiscardUnknown() { + xxx_messageInfo_DrainResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_DrainResponse proto.InternalMessageInfo // DecommissionStatusRequest requests the decommissioning status for the // specified or, if none are specified, all nodes. type DecommissionStatusRequest struct { - NodeIDs []github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,1,rep,packed,name=node_ids,json=nodeIds,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"node_ids,omitempty"` + NodeIDs []github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,1,rep,packed,name=node_ids,json=nodeIds,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"node_ids,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *DecommissionStatusRequest) Reset() { *m = DecommissionStatusRequest{} } -func (m *DecommissionStatusRequest) String() string { return proto.CompactTextString(m) } -func (*DecommissionStatusRequest) ProtoMessage() {} -func (*DecommissionStatusRequest) Descriptor() ([]byte, []int) { return fileDescriptorAdmin, []int{22} } +func (m *DecommissionStatusRequest) Reset() { *m = DecommissionStatusRequest{} } +func (m *DecommissionStatusRequest) String() string { return proto.CompactTextString(m) } +func (*DecommissionStatusRequest) ProtoMessage() {} +func (*DecommissionStatusRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_admin_b98220ab75a6bb7c, []int{22} +} +func (m *DecommissionStatusRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *DecommissionStatusRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *DecommissionStatusRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_DecommissionStatusRequest.Merge(dst, src) +} +func (m *DecommissionStatusRequest) XXX_Size() int { + return m.Size() +} +func (m *DecommissionStatusRequest) XXX_DiscardUnknown() { + xxx_messageInfo_DecommissionStatusRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_DecommissionStatusRequest proto.InternalMessageInfo // DecommissionRequest requests the server to set the Decommissioning flag on // all nodes specified by 'node_id' to the value of 'decommissioning'. // // If no 'node_id' is given, targets the recipient node. type DecommissionRequest struct { - NodeIDs []github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,1,rep,packed,name=node_ids,json=nodeIds,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"node_ids,omitempty"` - Decommissioning bool `protobuf:"varint,2,opt,name=decommissioning,proto3" json:"decommissioning,omitempty"` + NodeIDs []github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,1,rep,packed,name=node_ids,json=nodeIds,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"node_ids,omitempty"` + Decommissioning bool `protobuf:"varint,2,opt,name=decommissioning,proto3" json:"decommissioning,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *DecommissionRequest) Reset() { *m = DecommissionRequest{} } +func (m *DecommissionRequest) String() string { return proto.CompactTextString(m) } +func (*DecommissionRequest) ProtoMessage() {} +func (*DecommissionRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_admin_b98220ab75a6bb7c, []int{23} +} +func (m *DecommissionRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *DecommissionRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *DecommissionRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_DecommissionRequest.Merge(dst, src) +} +func (m *DecommissionRequest) XXX_Size() int { + return m.Size() +} +func (m *DecommissionRequest) XXX_DiscardUnknown() { + xxx_messageInfo_DecommissionRequest.DiscardUnknown(m) } -func (m *DecommissionRequest) Reset() { *m = DecommissionRequest{} } -func (m *DecommissionRequest) String() string { return proto.CompactTextString(m) } -func (*DecommissionRequest) ProtoMessage() {} -func (*DecommissionRequest) Descriptor() ([]byte, []int) { return fileDescriptorAdmin, []int{23} } +var xxx_messageInfo_DecommissionRequest proto.InternalMessageInfo // DecommissionStatusResponse lists decommissioning statuses for a number of NodeIDs. type DecommissionStatusResponse struct { // Status of all affected nodes. - Status []DecommissionStatusResponse_Status `protobuf:"bytes,2,rep,name=status" json:"status"` + Status []DecommissionStatusResponse_Status `protobuf:"bytes,2,rep,name=status,proto3" json:"status"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *DecommissionStatusResponse) Reset() { *m = DecommissionStatusResponse{} } +func (m *DecommissionStatusResponse) String() string { return proto.CompactTextString(m) } +func (*DecommissionStatusResponse) ProtoMessage() {} +func (*DecommissionStatusResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_admin_b98220ab75a6bb7c, []int{24} +} +func (m *DecommissionStatusResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *DecommissionStatusResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *DecommissionStatusResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_DecommissionStatusResponse.Merge(dst, src) +} +func (m *DecommissionStatusResponse) XXX_Size() int { + return m.Size() +} +func (m *DecommissionStatusResponse) XXX_DiscardUnknown() { + xxx_messageInfo_DecommissionStatusResponse.DiscardUnknown(m) } -func (m *DecommissionStatusResponse) Reset() { *m = DecommissionStatusResponse{} } -func (m *DecommissionStatusResponse) String() string { return proto.CompactTextString(m) } -func (*DecommissionStatusResponse) ProtoMessage() {} -func (*DecommissionStatusResponse) Descriptor() ([]byte, []int) { return fileDescriptorAdmin, []int{24} } +var xxx_messageInfo_DecommissionStatusResponse proto.InternalMessageInfo type DecommissionStatusResponse_Status struct { NodeID github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,1,opt,name=node_id,json=nodeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"node_id,omitempty"` IsLive bool `protobuf:"varint,2,opt,name=is_live,json=isLive,proto3" json:"is_live,omitempty"` // The number of replicas on the node, computed by scanning meta2 ranges. - ReplicaCount int64 `protobuf:"varint,3,opt,name=replica_count,json=replicaCount,proto3" json:"replica_count,omitempty"` - Decommissioning bool `protobuf:"varint,4,opt,name=decommissioning,proto3" json:"decommissioning,omitempty"` - Draining bool `protobuf:"varint,5,opt,name=draining,proto3" json:"draining,omitempty"` + ReplicaCount int64 `protobuf:"varint,3,opt,name=replica_count,json=replicaCount,proto3" json:"replica_count,omitempty"` + Decommissioning bool `protobuf:"varint,4,opt,name=decommissioning,proto3" json:"decommissioning,omitempty"` + Draining bool `protobuf:"varint,5,opt,name=draining,proto3" json:"draining,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *DecommissionStatusResponse_Status) Reset() { *m = DecommissionStatusResponse_Status{} } func (m *DecommissionStatusResponse_Status) String() string { return proto.CompactTextString(m) } func (*DecommissionStatusResponse_Status) ProtoMessage() {} func (*DecommissionStatusResponse_Status) Descriptor() ([]byte, []int) { - return fileDescriptorAdmin, []int{24, 0} + return fileDescriptor_admin_b98220ab75a6bb7c, []int{24, 0} +} +func (m *DecommissionStatusResponse_Status) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *DecommissionStatusResponse_Status) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *DecommissionStatusResponse_Status) XXX_Merge(src proto.Message) { + xxx_messageInfo_DecommissionStatusResponse_Status.Merge(dst, src) +} +func (m *DecommissionStatusResponse_Status) XXX_Size() int { + return m.Size() } +func (m *DecommissionStatusResponse_Status) XXX_DiscardUnknown() { + xxx_messageInfo_DecommissionStatusResponse_Status.DiscardUnknown(m) +} + +var xxx_messageInfo_DecommissionStatusResponse_Status proto.InternalMessageInfo // SettingsRequest inquires what are the current settings in the cluster. type SettingsRequest struct { // The array of setting names to retrieve. // An empty keys array means "all". - Keys []string `protobuf:"bytes,1,rep,name=keys" json:"keys,omitempty"` + Keys []string `protobuf:"bytes,1,rep,name=keys,proto3" json:"keys,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *SettingsRequest) Reset() { *m = SettingsRequest{} } +func (m *SettingsRequest) String() string { return proto.CompactTextString(m) } +func (*SettingsRequest) ProtoMessage() {} +func (*SettingsRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_admin_b98220ab75a6bb7c, []int{25} +} +func (m *SettingsRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SettingsRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *SettingsRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_SettingsRequest.Merge(dst, src) +} +func (m *SettingsRequest) XXX_Size() int { + return m.Size() +} +func (m *SettingsRequest) XXX_DiscardUnknown() { + xxx_messageInfo_SettingsRequest.DiscardUnknown(m) } -func (m *SettingsRequest) Reset() { *m = SettingsRequest{} } -func (m *SettingsRequest) String() string { return proto.CompactTextString(m) } -func (*SettingsRequest) ProtoMessage() {} -func (*SettingsRequest) Descriptor() ([]byte, []int) { return fileDescriptorAdmin, []int{25} } +var xxx_messageInfo_SettingsRequest proto.InternalMessageInfo // SettingsResponse is the response to SettingsRequest. type SettingsResponse struct { - KeyValues map[string]SettingsResponse_Value `protobuf:"bytes,1,rep,name=key_values,json=keyValues" json:"key_values" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value"` + KeyValues map[string]SettingsResponse_Value `protobuf:"bytes,1,rep,name=key_values,json=keyValues,proto3" json:"key_values" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *SettingsResponse) Reset() { *m = SettingsResponse{} } -func (m *SettingsResponse) String() string { return proto.CompactTextString(m) } -func (*SettingsResponse) ProtoMessage() {} -func (*SettingsResponse) Descriptor() ([]byte, []int) { return fileDescriptorAdmin, []int{26} } +func (m *SettingsResponse) Reset() { *m = SettingsResponse{} } +func (m *SettingsResponse) String() string { return proto.CompactTextString(m) } +func (*SettingsResponse) ProtoMessage() {} +func (*SettingsResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_admin_b98220ab75a6bb7c, []int{26} +} +func (m *SettingsResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SettingsResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *SettingsResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_SettingsResponse.Merge(dst, src) +} +func (m *SettingsResponse) XXX_Size() int { + return m.Size() +} +func (m *SettingsResponse) XXX_DiscardUnknown() { + xxx_messageInfo_SettingsResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_SettingsResponse proto.InternalMessageInfo type SettingsResponse_Value struct { - Value string `protobuf:"bytes,1,opt,name=value,proto3" json:"value,omitempty"` - Type string `protobuf:"bytes,2,opt,name=type,proto3" json:"type,omitempty"` - Description string `protobuf:"bytes,3,opt,name=description,proto3" json:"description,omitempty"` + Value string `protobuf:"bytes,1,opt,name=value,proto3" json:"value,omitempty"` + Type string `protobuf:"bytes,2,opt,name=type,proto3" json:"type,omitempty"` + Description string `protobuf:"bytes,3,opt,name=description,proto3" json:"description,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *SettingsResponse_Value) Reset() { *m = SettingsResponse_Value{} } -func (m *SettingsResponse_Value) String() string { return proto.CompactTextString(m) } -func (*SettingsResponse_Value) ProtoMessage() {} -func (*SettingsResponse_Value) Descriptor() ([]byte, []int) { return fileDescriptorAdmin, []int{26, 0} } +func (m *SettingsResponse_Value) Reset() { *m = SettingsResponse_Value{} } +func (m *SettingsResponse_Value) String() string { return proto.CompactTextString(m) } +func (*SettingsResponse_Value) ProtoMessage() {} +func (*SettingsResponse_Value) Descriptor() ([]byte, []int) { + return fileDescriptor_admin_b98220ab75a6bb7c, []int{26, 0} +} +func (m *SettingsResponse_Value) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SettingsResponse_Value) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *SettingsResponse_Value) XXX_Merge(src proto.Message) { + xxx_messageInfo_SettingsResponse_Value.Merge(dst, src) +} +func (m *SettingsResponse_Value) XXX_Size() int { + return m.Size() +} +func (m *SettingsResponse_Value) XXX_DiscardUnknown() { + xxx_messageInfo_SettingsResponse_Value.DiscardUnknown(m) +} + +var xxx_messageInfo_SettingsResponse_Value proto.InternalMessageInfo // HealthRequest inquires whether the addressed node is healthy. type HealthRequest struct { + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *HealthRequest) Reset() { *m = HealthRequest{} } +func (m *HealthRequest) String() string { return proto.CompactTextString(m) } +func (*HealthRequest) ProtoMessage() {} +func (*HealthRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_admin_b98220ab75a6bb7c, []int{27} +} +func (m *HealthRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *HealthRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *HealthRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_HealthRequest.Merge(dst, src) +} +func (m *HealthRequest) XXX_Size() int { + return m.Size() +} +func (m *HealthRequest) XXX_DiscardUnknown() { + xxx_messageInfo_HealthRequest.DiscardUnknown(m) } -func (m *HealthRequest) Reset() { *m = HealthRequest{} } -func (m *HealthRequest) String() string { return proto.CompactTextString(m) } -func (*HealthRequest) ProtoMessage() {} -func (*HealthRequest) Descriptor() ([]byte, []int) { return fileDescriptorAdmin, []int{27} } +var xxx_messageInfo_HealthRequest proto.InternalMessageInfo // HealthResponse is the response to HealthRequest. It currently does not // contain any information. The request fails however if the node is not live. // TODO(tschottdorf): Should it contain information, and if so, what? type HealthResponse struct { + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *HealthResponse) Reset() { *m = HealthResponse{} } +func (m *HealthResponse) String() string { return proto.CompactTextString(m) } +func (*HealthResponse) ProtoMessage() {} +func (*HealthResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_admin_b98220ab75a6bb7c, []int{28} +} +func (m *HealthResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *HealthResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *HealthResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_HealthResponse.Merge(dst, src) +} +func (m *HealthResponse) XXX_Size() int { + return m.Size() +} +func (m *HealthResponse) XXX_DiscardUnknown() { + xxx_messageInfo_HealthResponse.DiscardUnknown(m) } -func (m *HealthResponse) Reset() { *m = HealthResponse{} } -func (m *HealthResponse) String() string { return proto.CompactTextString(m) } -func (*HealthResponse) ProtoMessage() {} -func (*HealthResponse) Descriptor() ([]byte, []int) { return fileDescriptorAdmin, []int{28} } +var xxx_messageInfo_HealthResponse proto.InternalMessageInfo // LivenessRequest requests liveness data for all nodes on the cluster. type LivenessRequest struct { + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *LivenessRequest) Reset() { *m = LivenessRequest{} } -func (m *LivenessRequest) String() string { return proto.CompactTextString(m) } -func (*LivenessRequest) ProtoMessage() {} -func (*LivenessRequest) Descriptor() ([]byte, []int) { return fileDescriptorAdmin, []int{29} } +func (m *LivenessRequest) Reset() { *m = LivenessRequest{} } +func (m *LivenessRequest) String() string { return proto.CompactTextString(m) } +func (*LivenessRequest) ProtoMessage() {} +func (*LivenessRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_admin_b98220ab75a6bb7c, []int{29} +} +func (m *LivenessRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *LivenessRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *LivenessRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_LivenessRequest.Merge(dst, src) +} +func (m *LivenessRequest) XXX_Size() int { + return m.Size() +} +func (m *LivenessRequest) XXX_DiscardUnknown() { + xxx_messageInfo_LivenessRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_LivenessRequest proto.InternalMessageInfo // LivenessResponse contains the liveness status of each node on the cluster. type LivenessResponse struct { - Livenesses []cockroach_storage.Liveness `protobuf:"bytes,1,rep,name=livenesses" json:"livenesses"` - Statuses map[github_com_cockroachdb_cockroach_pkg_roachpb.NodeID]cockroach_storage.NodeLivenessStatus `protobuf:"bytes,2,rep,name=statuses,castkey=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"statuses" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3,enum=cockroach.storage.NodeLivenessStatus"` + Livenesses []storagepb.Liveness `protobuf:"bytes,1,rep,name=livenesses,proto3" json:"livenesses"` + Statuses map[github_com_cockroachdb_cockroach_pkg_roachpb.NodeID]storagepb.NodeLivenessStatus `protobuf:"bytes,2,rep,name=statuses,proto3,castkey=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"statuses" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3,enum=cockroach.storage.NodeLivenessStatus"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *LivenessResponse) Reset() { *m = LivenessResponse{} } +func (m *LivenessResponse) String() string { return proto.CompactTextString(m) } +func (*LivenessResponse) ProtoMessage() {} +func (*LivenessResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_admin_b98220ab75a6bb7c, []int{30} +} +func (m *LivenessResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *LivenessResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *LivenessResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_LivenessResponse.Merge(dst, src) +} +func (m *LivenessResponse) XXX_Size() int { + return m.Size() +} +func (m *LivenessResponse) XXX_DiscardUnknown() { + xxx_messageInfo_LivenessResponse.DiscardUnknown(m) } -func (m *LivenessResponse) Reset() { *m = LivenessResponse{} } -func (m *LivenessResponse) String() string { return proto.CompactTextString(m) } -func (*LivenessResponse) ProtoMessage() {} -func (*LivenessResponse) Descriptor() ([]byte, []int) { return fileDescriptorAdmin, []int{30} } +var xxx_messageInfo_LivenessResponse proto.InternalMessageInfo // JobsRequest requests system job information of the given status and type. type JobsRequest struct { - Limit int32 `protobuf:"varint,1,opt,name=limit,proto3" json:"limit,omitempty"` - Status string `protobuf:"bytes,2,opt,name=status,proto3" json:"status,omitempty"` - Type cockroach_sql_jobs_jobspb.Type `protobuf:"varint,3,opt,name=type,proto3,enum=cockroach.sql.jobs.jobspb.Type" json:"type,omitempty"` + Limit int32 `protobuf:"varint,1,opt,name=limit,proto3" json:"limit,omitempty"` + Status string `protobuf:"bytes,2,opt,name=status,proto3" json:"status,omitempty"` + Type jobspb.Type `protobuf:"varint,3,opt,name=type,proto3,enum=cockroach.sql.jobs.jobspb.Type" json:"type,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *JobsRequest) Reset() { *m = JobsRequest{} } -func (m *JobsRequest) String() string { return proto.CompactTextString(m) } -func (*JobsRequest) ProtoMessage() {} -func (*JobsRequest) Descriptor() ([]byte, []int) { return fileDescriptorAdmin, []int{31} } +func (m *JobsRequest) Reset() { *m = JobsRequest{} } +func (m *JobsRequest) String() string { return proto.CompactTextString(m) } +func (*JobsRequest) ProtoMessage() {} +func (*JobsRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_admin_b98220ab75a6bb7c, []int{31} +} +func (m *JobsRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *JobsRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *JobsRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_JobsRequest.Merge(dst, src) +} +func (m *JobsRequest) XXX_Size() int { + return m.Size() +} +func (m *JobsRequest) XXX_DiscardUnknown() { + xxx_messageInfo_JobsRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_JobsRequest proto.InternalMessageInfo // JobsResponse contains the job record for each matching job. type JobsResponse struct { - Jobs []JobsResponse_Job `protobuf:"bytes,1,rep,name=jobs" json:"jobs"` + Jobs []JobsResponse_Job `protobuf:"bytes,1,rep,name=jobs,proto3" json:"jobs"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *JobsResponse) Reset() { *m = JobsResponse{} } -func (m *JobsResponse) String() string { return proto.CompactTextString(m) } -func (*JobsResponse) ProtoMessage() {} -func (*JobsResponse) Descriptor() ([]byte, []int) { return fileDescriptorAdmin, []int{32} } +func (m *JobsResponse) Reset() { *m = JobsResponse{} } +func (m *JobsResponse) String() string { return proto.CompactTextString(m) } +func (*JobsResponse) ProtoMessage() {} +func (*JobsResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_admin_b98220ab75a6bb7c, []int{32} +} +func (m *JobsResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *JobsResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *JobsResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_JobsResponse.Merge(dst, src) +} +func (m *JobsResponse) XXX_Size() int { + return m.Size() +} +func (m *JobsResponse) XXX_DiscardUnknown() { + xxx_messageInfo_JobsResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_JobsResponse proto.InternalMessageInfo type JobsResponse_Job struct { ID int64 `protobuf:"varint,1,opt,name=id,proto3" json:"id,omitempty"` Type string `protobuf:"bytes,2,opt,name=type,proto3" json:"type,omitempty"` Description string `protobuf:"bytes,3,opt,name=description,proto3" json:"description,omitempty"` Username string `protobuf:"bytes,4,opt,name=username,proto3" json:"username,omitempty"` - DescriptorIDs []github_com_cockroachdb_cockroach_pkg_sql_sqlbase.ID `protobuf:"varint,5,rep,packed,name=descriptor_ids,json=descriptorIds,casttype=github.com/cockroachdb/cockroach/pkg/sql/sqlbase.ID" json:"descriptor_ids,omitempty"` + DescriptorIDs []github_com_cockroachdb_cockroach_pkg_sql_sqlbase.ID `protobuf:"varint,5,rep,packed,name=descriptor_ids,json=descriptorIds,proto3,casttype=github.com/cockroachdb/cockroach/pkg/sql/sqlbase.ID" json:"descriptor_ids,omitempty"` Status string `protobuf:"bytes,6,opt,name=status,proto3" json:"status,omitempty"` - Created *time.Time `protobuf:"bytes,7,opt,name=created,stdtime" json:"created,omitempty"` - Started *time.Time `protobuf:"bytes,8,opt,name=started,stdtime" json:"started,omitempty"` - Finished *time.Time `protobuf:"bytes,9,opt,name=finished,stdtime" json:"finished,omitempty"` - Modified *time.Time `protobuf:"bytes,10,opt,name=modified,stdtime" json:"modified,omitempty"` + Created *time.Time `protobuf:"bytes,7,opt,name=created,proto3,stdtime" json:"created,omitempty"` + Started *time.Time `protobuf:"bytes,8,opt,name=started,proto3,stdtime" json:"started,omitempty"` + Finished *time.Time `protobuf:"bytes,9,opt,name=finished,proto3,stdtime" json:"finished,omitempty"` + Modified *time.Time `protobuf:"bytes,10,opt,name=modified,proto3,stdtime" json:"modified,omitempty"` FractionCompleted float32 `protobuf:"fixed32,11,opt,name=fraction_completed,json=fractionCompleted,proto3" json:"fraction_completed,omitempty"` Error string `protobuf:"bytes,12,opt,name=error,proto3" json:"error,omitempty"` // highwater_timestamp is the highwater timestamp returned as normal // timestamp. This is appropriate for display to humans. - HighwaterTimestamp *time.Time `protobuf:"bytes,13,opt,name=highwater_timestamp,json=highwaterTimestamp,stdtime" json:"highwater_timestamp,omitempty"` + HighwaterTimestamp *time.Time `protobuf:"bytes,13,opt,name=highwater_timestamp,json=highwaterTimestamp,proto3,stdtime" json:"highwater_timestamp,omitempty"` // highwater_decimal is the highwater timestamp in the proprietary decimal // form used by logical timestamps internally. This is appropriate to pass // to a "AS OF SYSTEM TIME" SQL statement. - HighwaterDecimal string `protobuf:"bytes,14,opt,name=highwater_decimal,json=highwaterDecimal,proto3" json:"highwater_decimal,omitempty"` - RunningStatus string `protobuf:"bytes,15,opt,name=running_status,json=runningStatus,proto3" json:"running_status,omitempty"` + HighwaterDecimal string `protobuf:"bytes,14,opt,name=highwater_decimal,json=highwaterDecimal,proto3" json:"highwater_decimal,omitempty"` + RunningStatus string `protobuf:"bytes,15,opt,name=running_status,json=runningStatus,proto3" json:"running_status,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *JobsResponse_Job) Reset() { *m = JobsResponse_Job{} } +func (m *JobsResponse_Job) String() string { return proto.CompactTextString(m) } +func (*JobsResponse_Job) ProtoMessage() {} +func (*JobsResponse_Job) Descriptor() ([]byte, []int) { + return fileDescriptor_admin_b98220ab75a6bb7c, []int{32, 0} +} +func (m *JobsResponse_Job) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *JobsResponse_Job) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *JobsResponse_Job) XXX_Merge(src proto.Message) { + xxx_messageInfo_JobsResponse_Job.Merge(dst, src) +} +func (m *JobsResponse_Job) XXX_Size() int { + return m.Size() +} +func (m *JobsResponse_Job) XXX_DiscardUnknown() { + xxx_messageInfo_JobsResponse_Job.DiscardUnknown(m) +} + +var xxx_messageInfo_JobsResponse_Job proto.InternalMessageInfo + +// LocationsRequest requests system locality location information. +type LocationsRequest struct { + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *LocationsRequest) Reset() { *m = LocationsRequest{} } +func (m *LocationsRequest) String() string { return proto.CompactTextString(m) } +func (*LocationsRequest) ProtoMessage() {} +func (*LocationsRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_admin_b98220ab75a6bb7c, []int{33} +} +func (m *LocationsRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *LocationsRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *LocationsRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_LocationsRequest.Merge(dst, src) +} +func (m *LocationsRequest) XXX_Size() int { + return m.Size() +} +func (m *LocationsRequest) XXX_DiscardUnknown() { + xxx_messageInfo_LocationsRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_LocationsRequest proto.InternalMessageInfo + +// JobsResponse contains the job record for each matching job. +type LocationsResponse struct { + Locations []LocationsResponse_Location `protobuf:"bytes,1,rep,name=locations,proto3" json:"locations"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *LocationsResponse) Reset() { *m = LocationsResponse{} } +func (m *LocationsResponse) String() string { return proto.CompactTextString(m) } +func (*LocationsResponse) ProtoMessage() {} +func (*LocationsResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_admin_b98220ab75a6bb7c, []int{34} +} +func (m *LocationsResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *LocationsResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *LocationsResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_LocationsResponse.Merge(dst, src) } - -func (m *JobsResponse_Job) Reset() { *m = JobsResponse_Job{} } -func (m *JobsResponse_Job) String() string { return proto.CompactTextString(m) } -func (*JobsResponse_Job) ProtoMessage() {} -func (*JobsResponse_Job) Descriptor() ([]byte, []int) { return fileDescriptorAdmin, []int{32, 0} } - -// LocationsRequest requests system locality location information. -type LocationsRequest struct { +func (m *LocationsResponse) XXX_Size() int { + return m.Size() } - -func (m *LocationsRequest) Reset() { *m = LocationsRequest{} } -func (m *LocationsRequest) String() string { return proto.CompactTextString(m) } -func (*LocationsRequest) ProtoMessage() {} -func (*LocationsRequest) Descriptor() ([]byte, []int) { return fileDescriptorAdmin, []int{33} } - -// JobsResponse contains the job record for each matching job. -type LocationsResponse struct { - Locations []LocationsResponse_Location `protobuf:"bytes,1,rep,name=locations" json:"locations"` +func (m *LocationsResponse) XXX_DiscardUnknown() { + xxx_messageInfo_LocationsResponse.DiscardUnknown(m) } -func (m *LocationsResponse) Reset() { *m = LocationsResponse{} } -func (m *LocationsResponse) String() string { return proto.CompactTextString(m) } -func (*LocationsResponse) ProtoMessage() {} -func (*LocationsResponse) Descriptor() ([]byte, []int) { return fileDescriptorAdmin, []int{34} } +var xxx_messageInfo_LocationsResponse proto.InternalMessageInfo type LocationsResponse_Location struct { - LocalityKey string `protobuf:"bytes,1,opt,name=locality_key,json=localityKey,proto3" json:"locality_key,omitempty"` - LocalityValue string `protobuf:"bytes,2,opt,name=locality_value,json=localityValue,proto3" json:"locality_value,omitempty"` - Latitude float64 `protobuf:"fixed64,3,opt,name=latitude,proto3" json:"latitude,omitempty"` - Longitude float64 `protobuf:"fixed64,4,opt,name=longitude,proto3" json:"longitude,omitempty"` + LocalityKey string `protobuf:"bytes,1,opt,name=locality_key,json=localityKey,proto3" json:"locality_key,omitempty"` + LocalityValue string `protobuf:"bytes,2,opt,name=locality_value,json=localityValue,proto3" json:"locality_value,omitempty"` + Latitude float64 `protobuf:"fixed64,3,opt,name=latitude,proto3" json:"latitude,omitempty"` + Longitude float64 `protobuf:"fixed64,4,opt,name=longitude,proto3" json:"longitude,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *LocationsResponse_Location) Reset() { *m = LocationsResponse_Location{} } func (m *LocationsResponse_Location) String() string { return proto.CompactTextString(m) } func (*LocationsResponse_Location) ProtoMessage() {} func (*LocationsResponse_Location) Descriptor() ([]byte, []int) { - return fileDescriptorAdmin, []int{34, 0} + return fileDescriptor_admin_b98220ab75a6bb7c, []int{34, 0} +} +func (m *LocationsResponse_Location) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *LocationsResponse_Location) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *LocationsResponse_Location) XXX_Merge(src proto.Message) { + xxx_messageInfo_LocationsResponse_Location.Merge(dst, src) +} +func (m *LocationsResponse_Location) XXX_Size() int { + return m.Size() +} +func (m *LocationsResponse_Location) XXX_DiscardUnknown() { + xxx_messageInfo_LocationsResponse_Location.DiscardUnknown(m) } +var xxx_messageInfo_LocationsResponse_Location proto.InternalMessageInfo + // RangeLogRequest request the history of a range from the range log. type RangeLogRequest struct { // TODO(tamird): use [(gogoproto.customname) = "RangeID"] below. Need to @@ -890,153 +1979,483 @@ type RangeLogRequest struct { // this is omitted or set to 0, the default maximum number of results are // returned. When set to > 0, at most only that number of results are // returned. When set to < 0, an unlimited number of results are returned. - Limit int32 `protobuf:"varint,2,opt,name=limit,proto3" json:"limit,omitempty"` + Limit int32 `protobuf:"varint,2,opt,name=limit,proto3" json:"limit,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *RangeLogRequest) Reset() { *m = RangeLogRequest{} } +func (m *RangeLogRequest) String() string { return proto.CompactTextString(m) } +func (*RangeLogRequest) ProtoMessage() {} +func (*RangeLogRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_admin_b98220ab75a6bb7c, []int{35} +} +func (m *RangeLogRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RangeLogRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RangeLogRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_RangeLogRequest.Merge(dst, src) +} +func (m *RangeLogRequest) XXX_Size() int { + return m.Size() +} +func (m *RangeLogRequest) XXX_DiscardUnknown() { + xxx_messageInfo_RangeLogRequest.DiscardUnknown(m) } -func (m *RangeLogRequest) Reset() { *m = RangeLogRequest{} } -func (m *RangeLogRequest) String() string { return proto.CompactTextString(m) } -func (*RangeLogRequest) ProtoMessage() {} -func (*RangeLogRequest) Descriptor() ([]byte, []int) { return fileDescriptorAdmin, []int{35} } +var xxx_messageInfo_RangeLogRequest proto.InternalMessageInfo // RangeLogResponse contains a list of entries from the range log table. type RangeLogResponse struct { - Events []RangeLogResponse_Event `protobuf:"bytes,2,rep,name=events" json:"events"` + Events []RangeLogResponse_Event `protobuf:"bytes,2,rep,name=events,proto3" json:"events"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *RangeLogResponse) Reset() { *m = RangeLogResponse{} } +func (m *RangeLogResponse) String() string { return proto.CompactTextString(m) } +func (*RangeLogResponse) ProtoMessage() {} +func (*RangeLogResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_admin_b98220ab75a6bb7c, []int{36} +} +func (m *RangeLogResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RangeLogResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RangeLogResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_RangeLogResponse.Merge(dst, src) +} +func (m *RangeLogResponse) XXX_Size() int { + return m.Size() +} +func (m *RangeLogResponse) XXX_DiscardUnknown() { + xxx_messageInfo_RangeLogResponse.DiscardUnknown(m) } -func (m *RangeLogResponse) Reset() { *m = RangeLogResponse{} } -func (m *RangeLogResponse) String() string { return proto.CompactTextString(m) } -func (*RangeLogResponse) ProtoMessage() {} -func (*RangeLogResponse) Descriptor() ([]byte, []int) { return fileDescriptorAdmin, []int{36} } +var xxx_messageInfo_RangeLogResponse proto.InternalMessageInfo // To avoid porting the pretty printing of keys and descriptors to // javascript, they will be precomputed on the serverside. type RangeLogResponse_PrettyInfo struct { - UpdatedDesc string `protobuf:"bytes,1,opt,name=updated_desc,json=updatedDesc,proto3" json:"updated_desc,omitempty"` - NewDesc string `protobuf:"bytes,2,opt,name=new_desc,json=newDesc,proto3" json:"new_desc,omitempty"` - AddedReplica string `protobuf:"bytes,3,opt,name=added_replica,json=addedReplica,proto3" json:"added_replica,omitempty"` - RemovedReplica string `protobuf:"bytes,4,opt,name=removed_replica,json=removedReplica,proto3" json:"removed_replica,omitempty"` - Reason string `protobuf:"bytes,5,opt,name=reason,proto3" json:"reason,omitempty"` - Details string `protobuf:"bytes,6,opt,name=details,proto3" json:"details,omitempty"` + UpdatedDesc string `protobuf:"bytes,1,opt,name=updated_desc,json=updatedDesc,proto3" json:"updated_desc,omitempty"` + NewDesc string `protobuf:"bytes,2,opt,name=new_desc,json=newDesc,proto3" json:"new_desc,omitempty"` + AddedReplica string `protobuf:"bytes,3,opt,name=added_replica,json=addedReplica,proto3" json:"added_replica,omitempty"` + RemovedReplica string `protobuf:"bytes,4,opt,name=removed_replica,json=removedReplica,proto3" json:"removed_replica,omitempty"` + Reason string `protobuf:"bytes,5,opt,name=reason,proto3" json:"reason,omitempty"` + Details string `protobuf:"bytes,6,opt,name=details,proto3" json:"details,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *RangeLogResponse_PrettyInfo) Reset() { *m = RangeLogResponse_PrettyInfo{} } func (m *RangeLogResponse_PrettyInfo) String() string { return proto.CompactTextString(m) } func (*RangeLogResponse_PrettyInfo) ProtoMessage() {} func (*RangeLogResponse_PrettyInfo) Descriptor() ([]byte, []int) { - return fileDescriptorAdmin, []int{36, 0} + return fileDescriptor_admin_b98220ab75a6bb7c, []int{36, 0} +} +func (m *RangeLogResponse_PrettyInfo) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RangeLogResponse_PrettyInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RangeLogResponse_PrettyInfo) XXX_Merge(src proto.Message) { + xxx_messageInfo_RangeLogResponse_PrettyInfo.Merge(dst, src) +} +func (m *RangeLogResponse_PrettyInfo) XXX_Size() int { + return m.Size() } +func (m *RangeLogResponse_PrettyInfo) XXX_DiscardUnknown() { + xxx_messageInfo_RangeLogResponse_PrettyInfo.DiscardUnknown(m) +} + +var xxx_messageInfo_RangeLogResponse_PrettyInfo proto.InternalMessageInfo type RangeLogResponse_Event struct { - Event cockroach_storage2.RangeLogEvent `protobuf:"bytes,1,opt,name=event" json:"event"` - PrettyInfo RangeLogResponse_PrettyInfo `protobuf:"bytes,2,opt,name=pretty_info,json=prettyInfo" json:"pretty_info"` + Event storagepb.RangeLogEvent `protobuf:"bytes,1,opt,name=event,proto3" json:"event"` + PrettyInfo RangeLogResponse_PrettyInfo `protobuf:"bytes,2,opt,name=pretty_info,json=prettyInfo,proto3" json:"pretty_info"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *RangeLogResponse_Event) Reset() { *m = RangeLogResponse_Event{} } +func (m *RangeLogResponse_Event) String() string { return proto.CompactTextString(m) } +func (*RangeLogResponse_Event) ProtoMessage() {} +func (*RangeLogResponse_Event) Descriptor() ([]byte, []int) { + return fileDescriptor_admin_b98220ab75a6bb7c, []int{36, 1} +} +func (m *RangeLogResponse_Event) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RangeLogResponse_Event) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RangeLogResponse_Event) XXX_Merge(src proto.Message) { + xxx_messageInfo_RangeLogResponse_Event.Merge(dst, src) +} +func (m *RangeLogResponse_Event) XXX_Size() int { + return m.Size() +} +func (m *RangeLogResponse_Event) XXX_DiscardUnknown() { + xxx_messageInfo_RangeLogResponse_Event.DiscardUnknown(m) } -func (m *RangeLogResponse_Event) Reset() { *m = RangeLogResponse_Event{} } -func (m *RangeLogResponse_Event) String() string { return proto.CompactTextString(m) } -func (*RangeLogResponse_Event) ProtoMessage() {} -func (*RangeLogResponse_Event) Descriptor() ([]byte, []int) { return fileDescriptorAdmin, []int{36, 1} } +var xxx_messageInfo_RangeLogResponse_Event proto.InternalMessageInfo // QueryPlanRequest requests the query plans for a SQL string. type QueryPlanRequest struct { // query is the SQL query string. - Query string `protobuf:"bytes,1,opt,name=query,proto3" json:"query,omitempty"` + Query string `protobuf:"bytes,1,opt,name=query,proto3" json:"query,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *QueryPlanRequest) Reset() { *m = QueryPlanRequest{} } +func (m *QueryPlanRequest) String() string { return proto.CompactTextString(m) } +func (*QueryPlanRequest) ProtoMessage() {} +func (*QueryPlanRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_admin_b98220ab75a6bb7c, []int{37} +} +func (m *QueryPlanRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *QueryPlanRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *QueryPlanRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_QueryPlanRequest.Merge(dst, src) +} +func (m *QueryPlanRequest) XXX_Size() int { + return m.Size() +} +func (m *QueryPlanRequest) XXX_DiscardUnknown() { + xxx_messageInfo_QueryPlanRequest.DiscardUnknown(m) } -func (m *QueryPlanRequest) Reset() { *m = QueryPlanRequest{} } -func (m *QueryPlanRequest) String() string { return proto.CompactTextString(m) } -func (*QueryPlanRequest) ProtoMessage() {} -func (*QueryPlanRequest) Descriptor() ([]byte, []int) { return fileDescriptorAdmin, []int{37} } +var xxx_messageInfo_QueryPlanRequest proto.InternalMessageInfo // QueryPlanResponse contains the query plans for a SQL string (currently only // the distsql physical query plan). type QueryPlanResponse struct { - DistSQLPhysicalQueryPlan string `protobuf:"bytes,1,opt,name=distsql_physical_query_plan,json=distsqlPhysicalQueryPlan,proto3" json:"distsql_physical_query_plan,omitempty"` + DistSQLPhysicalQueryPlan string `protobuf:"bytes,1,opt,name=distsql_physical_query_plan,json=distsqlPhysicalQueryPlan,proto3" json:"distsql_physical_query_plan,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *QueryPlanResponse) Reset() { *m = QueryPlanResponse{} } +func (m *QueryPlanResponse) String() string { return proto.CompactTextString(m) } +func (*QueryPlanResponse) ProtoMessage() {} +func (*QueryPlanResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_admin_b98220ab75a6bb7c, []int{38} +} +func (m *QueryPlanResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *QueryPlanResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *QueryPlanResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_QueryPlanResponse.Merge(dst, src) +} +func (m *QueryPlanResponse) XXX_Size() int { + return m.Size() +} +func (m *QueryPlanResponse) XXX_DiscardUnknown() { + xxx_messageInfo_QueryPlanResponse.DiscardUnknown(m) } -func (m *QueryPlanResponse) Reset() { *m = QueryPlanResponse{} } -func (m *QueryPlanResponse) String() string { return proto.CompactTextString(m) } -func (*QueryPlanResponse) ProtoMessage() {} -func (*QueryPlanResponse) Descriptor() ([]byte, []int) { return fileDescriptorAdmin, []int{38} } +var xxx_messageInfo_QueryPlanResponse proto.InternalMessageInfo type DataDistributionRequest struct { + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *DataDistributionRequest) Reset() { *m = DataDistributionRequest{} } +func (m *DataDistributionRequest) String() string { return proto.CompactTextString(m) } +func (*DataDistributionRequest) ProtoMessage() {} +func (*DataDistributionRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_admin_b98220ab75a6bb7c, []int{39} +} +func (m *DataDistributionRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *DataDistributionRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *DataDistributionRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_DataDistributionRequest.Merge(dst, src) +} +func (m *DataDistributionRequest) XXX_Size() int { + return m.Size() +} +func (m *DataDistributionRequest) XXX_DiscardUnknown() { + xxx_messageInfo_DataDistributionRequest.DiscardUnknown(m) } -func (m *DataDistributionRequest) Reset() { *m = DataDistributionRequest{} } -func (m *DataDistributionRequest) String() string { return proto.CompactTextString(m) } -func (*DataDistributionRequest) ProtoMessage() {} -func (*DataDistributionRequest) Descriptor() ([]byte, []int) { return fileDescriptorAdmin, []int{39} } +var xxx_messageInfo_DataDistributionRequest proto.InternalMessageInfo type DataDistributionResponse struct { // By database name. - DatabaseInfo map[string]DataDistributionResponse_DatabaseInfo `protobuf:"bytes,1,rep,name=database_info,json=databaseInfo" json:"database_info" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value"` + DatabaseInfo map[string]DataDistributionResponse_DatabaseInfo `protobuf:"bytes,1,rep,name=database_info,json=databaseInfo,proto3" json:"database_info" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` // By zone name. - ZoneConfigs map[string]DataDistributionResponse_ZoneConfig `protobuf:"bytes,3,rep,name=zone_configs,json=zoneConfigs" json:"zone_configs" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value"` + ZoneConfigs map[string]DataDistributionResponse_ZoneConfig `protobuf:"bytes,3,rep,name=zone_configs,json=zoneConfigs,proto3" json:"zone_configs" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *DataDistributionResponse) Reset() { *m = DataDistributionResponse{} } +func (m *DataDistributionResponse) String() string { return proto.CompactTextString(m) } +func (*DataDistributionResponse) ProtoMessage() {} +func (*DataDistributionResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_admin_b98220ab75a6bb7c, []int{40} +} +func (m *DataDistributionResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *DataDistributionResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *DataDistributionResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_DataDistributionResponse.Merge(dst, src) +} +func (m *DataDistributionResponse) XXX_Size() int { + return m.Size() +} +func (m *DataDistributionResponse) XXX_DiscardUnknown() { + xxx_messageInfo_DataDistributionResponse.DiscardUnknown(m) } -func (m *DataDistributionResponse) Reset() { *m = DataDistributionResponse{} } -func (m *DataDistributionResponse) String() string { return proto.CompactTextString(m) } -func (*DataDistributionResponse) ProtoMessage() {} -func (*DataDistributionResponse) Descriptor() ([]byte, []int) { return fileDescriptorAdmin, []int{40} } +var xxx_messageInfo_DataDistributionResponse proto.InternalMessageInfo type DataDistributionResponse_ZoneConfig struct { // zone_name is ".default" for the default zone config. // Otherwise, it's of the form [database.]table[@index[.partition]] - ZoneName string `protobuf:"bytes,1,opt,name=zone_name,json=zoneName,proto3" json:"zone_name,omitempty"` - Config cockroach_config.ZoneConfig `protobuf:"bytes,2,opt,name=config" json:"config"` + ZoneName string `protobuf:"bytes,1,opt,name=zone_name,json=zoneName,proto3" json:"zone_name,omitempty"` + Config config.ZoneConfig `protobuf:"bytes,2,opt,name=config,proto3" json:"config"` // config_sql is the SQL representation of config. - ConfigSQL string `protobuf:"bytes,4,opt,name=config_sql,json=configSql,proto3" json:"config_sql,omitempty"` + ConfigSQL string `protobuf:"bytes,4,opt,name=config_sql,json=configSql,proto3" json:"config_sql,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *DataDistributionResponse_ZoneConfig) Reset() { *m = DataDistributionResponse_ZoneConfig{} } func (m *DataDistributionResponse_ZoneConfig) String() string { return proto.CompactTextString(m) } func (*DataDistributionResponse_ZoneConfig) ProtoMessage() {} func (*DataDistributionResponse_ZoneConfig) Descriptor() ([]byte, []int) { - return fileDescriptorAdmin, []int{40, 0} + return fileDescriptor_admin_b98220ab75a6bb7c, []int{40, 0} +} +func (m *DataDistributionResponse_ZoneConfig) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *DataDistributionResponse_ZoneConfig) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *DataDistributionResponse_ZoneConfig) XXX_Merge(src proto.Message) { + xxx_messageInfo_DataDistributionResponse_ZoneConfig.Merge(dst, src) +} +func (m *DataDistributionResponse_ZoneConfig) XXX_Size() int { + return m.Size() +} +func (m *DataDistributionResponse_ZoneConfig) XXX_DiscardUnknown() { + xxx_messageInfo_DataDistributionResponse_ZoneConfig.DiscardUnknown(m) } +var xxx_messageInfo_DataDistributionResponse_ZoneConfig proto.InternalMessageInfo + type DataDistributionResponse_TableInfo struct { - ReplicaCountByNodeId map[github_com_cockroachdb_cockroach_pkg_roachpb.NodeID]int64 `protobuf:"bytes,1,rep,name=replica_count_by_node_id,json=replicaCountByNodeId,castkey=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"replica_count_by_node_id,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` + ReplicaCountByNodeId map[github_com_cockroachdb_cockroach_pkg_roachpb.NodeID]int64 `protobuf:"bytes,1,rep,name=replica_count_by_node_id,json=replicaCountByNodeId,proto3,castkey=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"replica_count_by_node_id,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` ZoneConfigId int64 `protobuf:"varint,2,opt,name=zone_config_id,json=zoneConfigId,proto3" json:"zone_config_id,omitempty"` - DroppedAt *time.Time `protobuf:"bytes,3,opt,name=dropped_at,json=droppedAt,stdtime" json:"dropped_at,omitempty"` + DroppedAt *time.Time `protobuf:"bytes,3,opt,name=dropped_at,json=droppedAt,proto3,stdtime" json:"dropped_at,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *DataDistributionResponse_TableInfo) Reset() { *m = DataDistributionResponse_TableInfo{} } func (m *DataDistributionResponse_TableInfo) String() string { return proto.CompactTextString(m) } func (*DataDistributionResponse_TableInfo) ProtoMessage() {} func (*DataDistributionResponse_TableInfo) Descriptor() ([]byte, []int) { - return fileDescriptorAdmin, []int{40, 1} + return fileDescriptor_admin_b98220ab75a6bb7c, []int{40, 1} +} +func (m *DataDistributionResponse_TableInfo) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *DataDistributionResponse_TableInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *DataDistributionResponse_TableInfo) XXX_Merge(src proto.Message) { + xxx_messageInfo_DataDistributionResponse_TableInfo.Merge(dst, src) +} +func (m *DataDistributionResponse_TableInfo) XXX_Size() int { + return m.Size() } +func (m *DataDistributionResponse_TableInfo) XXX_DiscardUnknown() { + xxx_messageInfo_DataDistributionResponse_TableInfo.DiscardUnknown(m) +} + +var xxx_messageInfo_DataDistributionResponse_TableInfo proto.InternalMessageInfo type DataDistributionResponse_DatabaseInfo struct { // By table name. - TableInfo map[string]DataDistributionResponse_TableInfo `protobuf:"bytes,1,rep,name=table_info,json=tableInfo" json:"table_info" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value"` + TableInfo map[string]DataDistributionResponse_TableInfo `protobuf:"bytes,1,rep,name=table_info,json=tableInfo,proto3" json:"table_info" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *DataDistributionResponse_DatabaseInfo) Reset() { *m = DataDistributionResponse_DatabaseInfo{} } func (m *DataDistributionResponse_DatabaseInfo) String() string { return proto.CompactTextString(m) } func (*DataDistributionResponse_DatabaseInfo) ProtoMessage() {} func (*DataDistributionResponse_DatabaseInfo) Descriptor() ([]byte, []int) { - return fileDescriptorAdmin, []int{40, 2} + return fileDescriptor_admin_b98220ab75a6bb7c, []int{40, 2} +} +func (m *DataDistributionResponse_DatabaseInfo) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *DataDistributionResponse_DatabaseInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *DataDistributionResponse_DatabaseInfo) XXX_Merge(src proto.Message) { + xxx_messageInfo_DataDistributionResponse_DatabaseInfo.Merge(dst, src) +} +func (m *DataDistributionResponse_DatabaseInfo) XXX_Size() int { + return m.Size() +} +func (m *DataDistributionResponse_DatabaseInfo) XXX_DiscardUnknown() { + xxx_messageInfo_DataDistributionResponse_DatabaseInfo.DiscardUnknown(m) } +var xxx_messageInfo_DataDistributionResponse_DatabaseInfo proto.InternalMessageInfo + // MetricMetadataRequest requests metadata for all metrics. type MetricMetadataRequest struct { + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *MetricMetadataRequest) Reset() { *m = MetricMetadataRequest{} } +func (m *MetricMetadataRequest) String() string { return proto.CompactTextString(m) } +func (*MetricMetadataRequest) ProtoMessage() {} +func (*MetricMetadataRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_admin_b98220ab75a6bb7c, []int{41} +} +func (m *MetricMetadataRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *MetricMetadataRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *MetricMetadataRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_MetricMetadataRequest.Merge(dst, src) +} +func (m *MetricMetadataRequest) XXX_Size() int { + return m.Size() +} +func (m *MetricMetadataRequest) XXX_DiscardUnknown() { + xxx_messageInfo_MetricMetadataRequest.DiscardUnknown(m) } -func (m *MetricMetadataRequest) Reset() { *m = MetricMetadataRequest{} } -func (m *MetricMetadataRequest) String() string { return proto.CompactTextString(m) } -func (*MetricMetadataRequest) ProtoMessage() {} -func (*MetricMetadataRequest) Descriptor() ([]byte, []int) { return fileDescriptorAdmin, []int{41} } +var xxx_messageInfo_MetricMetadataRequest proto.InternalMessageInfo // MetricMetadataResponse contains the metadata for all metics. type MetricMetadataResponse struct { - Metadata map[string]cockroach_util_metric.Metadata `protobuf:"bytes,1,rep,name=metadata" json:"metadata" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value"` + Metadata map[string]metric.Metadata `protobuf:"bytes,1,rep,name=metadata,proto3" json:"metadata" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *MetricMetadataResponse) Reset() { *m = MetricMetadataResponse{} } +func (m *MetricMetadataResponse) String() string { return proto.CompactTextString(m) } +func (*MetricMetadataResponse) ProtoMessage() {} +func (*MetricMetadataResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_admin_b98220ab75a6bb7c, []int{42} +} +func (m *MetricMetadataResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *MetricMetadataResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *MetricMetadataResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_MetricMetadataResponse.Merge(dst, src) +} +func (m *MetricMetadataResponse) XXX_Size() int { + return m.Size() +} +func (m *MetricMetadataResponse) XXX_DiscardUnknown() { + xxx_messageInfo_MetricMetadataResponse.DiscardUnknown(m) } -func (m *MetricMetadataResponse) Reset() { *m = MetricMetadataResponse{} } -func (m *MetricMetadataResponse) String() string { return proto.CompactTextString(m) } -func (*MetricMetadataResponse) ProtoMessage() {} -func (*MetricMetadataResponse) Descriptor() ([]byte, []int) { return fileDescriptorAdmin, []int{42} } +var xxx_messageInfo_MetricMetadataResponse proto.InternalMessageInfo type EnqueueRangeRequest struct { // The node on which the queue should process the range. If node_id is 0, @@ -1049,37 +2468,113 @@ type EnqueueRangeRequest struct { RangeID github_com_cockroachdb_cockroach_pkg_roachpb.RangeID `protobuf:"varint,3,opt,name=range_id,json=rangeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" json:"range_id,omitempty"` // If set, run the queue's process method without first checking whether the // replica should be processed by calling shouldQueue. - SkipShouldQueue bool `protobuf:"varint,4,opt,name=skip_should_queue,json=skipShouldQueue,proto3" json:"skip_should_queue,omitempty"` + SkipShouldQueue bool `protobuf:"varint,4,opt,name=skip_should_queue,json=skipShouldQueue,proto3" json:"skip_should_queue,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *EnqueueRangeRequest) Reset() { *m = EnqueueRangeRequest{} } +func (m *EnqueueRangeRequest) String() string { return proto.CompactTextString(m) } +func (*EnqueueRangeRequest) ProtoMessage() {} +func (*EnqueueRangeRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_admin_b98220ab75a6bb7c, []int{43} +} +func (m *EnqueueRangeRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *EnqueueRangeRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *EnqueueRangeRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_EnqueueRangeRequest.Merge(dst, src) +} +func (m *EnqueueRangeRequest) XXX_Size() int { + return m.Size() +} +func (m *EnqueueRangeRequest) XXX_DiscardUnknown() { + xxx_messageInfo_EnqueueRangeRequest.DiscardUnknown(m) } -func (m *EnqueueRangeRequest) Reset() { *m = EnqueueRangeRequest{} } -func (m *EnqueueRangeRequest) String() string { return proto.CompactTextString(m) } -func (*EnqueueRangeRequest) ProtoMessage() {} -func (*EnqueueRangeRequest) Descriptor() ([]byte, []int) { return fileDescriptorAdmin, []int{43} } +var xxx_messageInfo_EnqueueRangeRequest proto.InternalMessageInfo type EnqueueRangeResponse struct { - Details []*EnqueueRangeResponse_Details `protobuf:"bytes,1,rep,name=details" json:"details,omitempty"` + Details []*EnqueueRangeResponse_Details `protobuf:"bytes,1,rep,name=details,proto3" json:"details,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *EnqueueRangeResponse) Reset() { *m = EnqueueRangeResponse{} } +func (m *EnqueueRangeResponse) String() string { return proto.CompactTextString(m) } +func (*EnqueueRangeResponse) ProtoMessage() {} +func (*EnqueueRangeResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_admin_b98220ab75a6bb7c, []int{44} +} +func (m *EnqueueRangeResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *EnqueueRangeResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *EnqueueRangeResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_EnqueueRangeResponse.Merge(dst, src) +} +func (m *EnqueueRangeResponse) XXX_Size() int { + return m.Size() +} +func (m *EnqueueRangeResponse) XXX_DiscardUnknown() { + xxx_messageInfo_EnqueueRangeResponse.DiscardUnknown(m) } -func (m *EnqueueRangeResponse) Reset() { *m = EnqueueRangeResponse{} } -func (m *EnqueueRangeResponse) String() string { return proto.CompactTextString(m) } -func (*EnqueueRangeResponse) ProtoMessage() {} -func (*EnqueueRangeResponse) Descriptor() ([]byte, []int) { return fileDescriptorAdmin, []int{44} } +var xxx_messageInfo_EnqueueRangeResponse proto.InternalMessageInfo type EnqueueRangeResponse_Details struct { NodeID github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,1,opt,name=node_id,json=nodeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"node_id,omitempty"` // All trace events collected while processing the range in the queue. - Events []*TraceEvent `protobuf:"bytes,2,rep,name=events" json:"events,omitempty"` + Events []*TraceEvent `protobuf:"bytes,2,rep,name=events,proto3" json:"events,omitempty"` // The error message from the queue's processing, if any. - Error string `protobuf:"bytes,3,opt,name=error,proto3" json:"error,omitempty"` + Error string `protobuf:"bytes,3,opt,name=error,proto3" json:"error,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *EnqueueRangeResponse_Details) Reset() { *m = EnqueueRangeResponse_Details{} } func (m *EnqueueRangeResponse_Details) String() string { return proto.CompactTextString(m) } func (*EnqueueRangeResponse_Details) ProtoMessage() {} func (*EnqueueRangeResponse_Details) Descriptor() ([]byte, []int) { - return fileDescriptorAdmin, []int{44, 0} + return fileDescriptor_admin_b98220ab75a6bb7c, []int{44, 0} +} +func (m *EnqueueRangeResponse_Details) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *EnqueueRangeResponse_Details) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil } +func (dst *EnqueueRangeResponse_Details) XXX_Merge(src proto.Message) { + xxx_messageInfo_EnqueueRangeResponse_Details.Merge(dst, src) +} +func (m *EnqueueRangeResponse_Details) XXX_Size() int { + return m.Size() +} +func (m *EnqueueRangeResponse_Details) XXX_DiscardUnknown() { + xxx_messageInfo_EnqueueRangeResponse_Details.DiscardUnknown(m) +} + +var xxx_messageInfo_EnqueueRangeResponse_Details proto.InternalMessageInfo func init() { proto.RegisterType((*DatabasesRequest)(nil), "cockroach.server.serverpb.DatabasesRequest") @@ -1104,9 +2599,11 @@ func init() { proto.RegisterType((*EventsResponse)(nil), "cockroach.server.serverpb.EventsResponse") proto.RegisterType((*EventsResponse_Event)(nil), "cockroach.server.serverpb.EventsResponse.Event") proto.RegisterType((*SetUIDataRequest)(nil), "cockroach.server.serverpb.SetUIDataRequest") + proto.RegisterMapType((map[string][]byte)(nil), "cockroach.server.serverpb.SetUIDataRequest.KeyValuesEntry") proto.RegisterType((*SetUIDataResponse)(nil), "cockroach.server.serverpb.SetUIDataResponse") proto.RegisterType((*GetUIDataRequest)(nil), "cockroach.server.serverpb.GetUIDataRequest") proto.RegisterType((*GetUIDataResponse)(nil), "cockroach.server.serverpb.GetUIDataResponse") + proto.RegisterMapType((map[string]GetUIDataResponse_Value)(nil), "cockroach.server.serverpb.GetUIDataResponse.KeyValuesEntry") proto.RegisterType((*GetUIDataResponse_Value)(nil), "cockroach.server.serverpb.GetUIDataResponse.Value") proto.RegisterType((*ClusterRequest)(nil), "cockroach.server.serverpb.ClusterRequest") proto.RegisterType((*ClusterResponse)(nil), "cockroach.server.serverpb.ClusterResponse") @@ -1118,11 +2615,13 @@ func init() { proto.RegisterType((*DecommissionStatusResponse_Status)(nil), "cockroach.server.serverpb.DecommissionStatusResponse.Status") proto.RegisterType((*SettingsRequest)(nil), "cockroach.server.serverpb.SettingsRequest") proto.RegisterType((*SettingsResponse)(nil), "cockroach.server.serverpb.SettingsResponse") + proto.RegisterMapType((map[string]SettingsResponse_Value)(nil), "cockroach.server.serverpb.SettingsResponse.KeyValuesEntry") proto.RegisterType((*SettingsResponse_Value)(nil), "cockroach.server.serverpb.SettingsResponse.Value") proto.RegisterType((*HealthRequest)(nil), "cockroach.server.serverpb.HealthRequest") proto.RegisterType((*HealthResponse)(nil), "cockroach.server.serverpb.HealthResponse") proto.RegisterType((*LivenessRequest)(nil), "cockroach.server.serverpb.LivenessRequest") proto.RegisterType((*LivenessResponse)(nil), "cockroach.server.serverpb.LivenessResponse") + proto.RegisterMapType((map[github_com_cockroachdb_cockroach_pkg_roachpb.NodeID]storagepb.NodeLivenessStatus)(nil), "cockroach.server.serverpb.LivenessResponse.StatusesEntry") proto.RegisterType((*JobsRequest)(nil), "cockroach.server.serverpb.JobsRequest") proto.RegisterType((*JobsResponse)(nil), "cockroach.server.serverpb.JobsResponse") proto.RegisterType((*JobsResponse_Job)(nil), "cockroach.server.serverpb.JobsResponse.Job") @@ -1137,11 +2636,16 @@ func init() { proto.RegisterType((*QueryPlanResponse)(nil), "cockroach.server.serverpb.QueryPlanResponse") proto.RegisterType((*DataDistributionRequest)(nil), "cockroach.server.serverpb.DataDistributionRequest") proto.RegisterType((*DataDistributionResponse)(nil), "cockroach.server.serverpb.DataDistributionResponse") + proto.RegisterMapType((map[string]DataDistributionResponse_DatabaseInfo)(nil), "cockroach.server.serverpb.DataDistributionResponse.DatabaseInfoEntry") + proto.RegisterMapType((map[string]DataDistributionResponse_ZoneConfig)(nil), "cockroach.server.serverpb.DataDistributionResponse.ZoneConfigsEntry") proto.RegisterType((*DataDistributionResponse_ZoneConfig)(nil), "cockroach.server.serverpb.DataDistributionResponse.ZoneConfig") proto.RegisterType((*DataDistributionResponse_TableInfo)(nil), "cockroach.server.serverpb.DataDistributionResponse.TableInfo") + proto.RegisterMapType((map[github_com_cockroachdb_cockroach_pkg_roachpb.NodeID]int64)(nil), "cockroach.server.serverpb.DataDistributionResponse.TableInfo.ReplicaCountByNodeIdEntry") proto.RegisterType((*DataDistributionResponse_DatabaseInfo)(nil), "cockroach.server.serverpb.DataDistributionResponse.DatabaseInfo") + proto.RegisterMapType((map[string]DataDistributionResponse_TableInfo)(nil), "cockroach.server.serverpb.DataDistributionResponse.DatabaseInfo.TableInfoEntry") proto.RegisterType((*MetricMetadataRequest)(nil), "cockroach.server.serverpb.MetricMetadataRequest") proto.RegisterType((*MetricMetadataResponse)(nil), "cockroach.server.serverpb.MetricMetadataResponse") + proto.RegisterMapType((map[string]metric.Metadata)(nil), "cockroach.server.serverpb.MetricMetadataResponse.MetadataEntry") proto.RegisterType((*EnqueueRangeRequest)(nil), "cockroach.server.serverpb.EnqueueRangeRequest") proto.RegisterType((*EnqueueRangeResponse)(nil), "cockroach.server.serverpb.EnqueueRangeResponse") proto.RegisterType((*EnqueueRangeResponse_Details)(nil), "cockroach.server.serverpb.EnqueueRangeResponse.Details") @@ -1157,8 +2661,9 @@ var _ grpc.ClientConn // is compatible with the grpc package it is being compiled against. const _ = grpc.SupportPackageIsVersion4 -// Client API for Admin service - +// AdminClient is the client API for Admin service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream. type AdminClient interface { // URL: /_admin/v1/users Users(ctx context.Context, in *UsersRequest, opts ...grpc.CallOption) (*UsersResponse, error) @@ -1258,7 +2763,7 @@ func NewAdminClient(cc *grpc.ClientConn) AdminClient { func (c *adminClient) Users(ctx context.Context, in *UsersRequest, opts ...grpc.CallOption) (*UsersResponse, error) { out := new(UsersResponse) - err := grpc.Invoke(ctx, "/cockroach.server.serverpb.Admin/Users", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Admin/Users", in, out, opts...) if err != nil { return nil, err } @@ -1267,7 +2772,7 @@ func (c *adminClient) Users(ctx context.Context, in *UsersRequest, opts ...grpc. func (c *adminClient) Databases(ctx context.Context, in *DatabasesRequest, opts ...grpc.CallOption) (*DatabasesResponse, error) { out := new(DatabasesResponse) - err := grpc.Invoke(ctx, "/cockroach.server.serverpb.Admin/Databases", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Admin/Databases", in, out, opts...) if err != nil { return nil, err } @@ -1276,7 +2781,7 @@ func (c *adminClient) Databases(ctx context.Context, in *DatabasesRequest, opts func (c *adminClient) DatabaseDetails(ctx context.Context, in *DatabaseDetailsRequest, opts ...grpc.CallOption) (*DatabaseDetailsResponse, error) { out := new(DatabaseDetailsResponse) - err := grpc.Invoke(ctx, "/cockroach.server.serverpb.Admin/DatabaseDetails", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Admin/DatabaseDetails", in, out, opts...) if err != nil { return nil, err } @@ -1285,7 +2790,7 @@ func (c *adminClient) DatabaseDetails(ctx context.Context, in *DatabaseDetailsRe func (c *adminClient) TableDetails(ctx context.Context, in *TableDetailsRequest, opts ...grpc.CallOption) (*TableDetailsResponse, error) { out := new(TableDetailsResponse) - err := grpc.Invoke(ctx, "/cockroach.server.serverpb.Admin/TableDetails", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Admin/TableDetails", in, out, opts...) if err != nil { return nil, err } @@ -1294,7 +2799,7 @@ func (c *adminClient) TableDetails(ctx context.Context, in *TableDetailsRequest, func (c *adminClient) TableStats(ctx context.Context, in *TableStatsRequest, opts ...grpc.CallOption) (*TableStatsResponse, error) { out := new(TableStatsResponse) - err := grpc.Invoke(ctx, "/cockroach.server.serverpb.Admin/TableStats", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Admin/TableStats", in, out, opts...) if err != nil { return nil, err } @@ -1303,7 +2808,7 @@ func (c *adminClient) TableStats(ctx context.Context, in *TableStatsRequest, opt func (c *adminClient) NonTableStats(ctx context.Context, in *NonTableStatsRequest, opts ...grpc.CallOption) (*NonTableStatsResponse, error) { out := new(NonTableStatsResponse) - err := grpc.Invoke(ctx, "/cockroach.server.serverpb.Admin/NonTableStats", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Admin/NonTableStats", in, out, opts...) if err != nil { return nil, err } @@ -1312,7 +2817,7 @@ func (c *adminClient) NonTableStats(ctx context.Context, in *NonTableStatsReques func (c *adminClient) Events(ctx context.Context, in *EventsRequest, opts ...grpc.CallOption) (*EventsResponse, error) { out := new(EventsResponse) - err := grpc.Invoke(ctx, "/cockroach.server.serverpb.Admin/Events", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Admin/Events", in, out, opts...) if err != nil { return nil, err } @@ -1321,7 +2826,7 @@ func (c *adminClient) Events(ctx context.Context, in *EventsRequest, opts ...grp func (c *adminClient) SetUIData(ctx context.Context, in *SetUIDataRequest, opts ...grpc.CallOption) (*SetUIDataResponse, error) { out := new(SetUIDataResponse) - err := grpc.Invoke(ctx, "/cockroach.server.serverpb.Admin/SetUIData", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Admin/SetUIData", in, out, opts...) if err != nil { return nil, err } @@ -1330,7 +2835,7 @@ func (c *adminClient) SetUIData(ctx context.Context, in *SetUIDataRequest, opts func (c *adminClient) GetUIData(ctx context.Context, in *GetUIDataRequest, opts ...grpc.CallOption) (*GetUIDataResponse, error) { out := new(GetUIDataResponse) - err := grpc.Invoke(ctx, "/cockroach.server.serverpb.Admin/GetUIData", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Admin/GetUIData", in, out, opts...) if err != nil { return nil, err } @@ -1339,7 +2844,7 @@ func (c *adminClient) GetUIData(ctx context.Context, in *GetUIDataRequest, opts func (c *adminClient) Cluster(ctx context.Context, in *ClusterRequest, opts ...grpc.CallOption) (*ClusterResponse, error) { out := new(ClusterResponse) - err := grpc.Invoke(ctx, "/cockroach.server.serverpb.Admin/Cluster", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Admin/Cluster", in, out, opts...) if err != nil { return nil, err } @@ -1348,7 +2853,7 @@ func (c *adminClient) Cluster(ctx context.Context, in *ClusterRequest, opts ...g func (c *adminClient) Settings(ctx context.Context, in *SettingsRequest, opts ...grpc.CallOption) (*SettingsResponse, error) { out := new(SettingsResponse) - err := grpc.Invoke(ctx, "/cockroach.server.serverpb.Admin/Settings", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Admin/Settings", in, out, opts...) if err != nil { return nil, err } @@ -1357,7 +2862,7 @@ func (c *adminClient) Settings(ctx context.Context, in *SettingsRequest, opts .. func (c *adminClient) Health(ctx context.Context, in *HealthRequest, opts ...grpc.CallOption) (*HealthResponse, error) { out := new(HealthResponse) - err := grpc.Invoke(ctx, "/cockroach.server.serverpb.Admin/Health", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Admin/Health", in, out, opts...) if err != nil { return nil, err } @@ -1366,7 +2871,7 @@ func (c *adminClient) Health(ctx context.Context, in *HealthRequest, opts ...grp func (c *adminClient) Liveness(ctx context.Context, in *LivenessRequest, opts ...grpc.CallOption) (*LivenessResponse, error) { out := new(LivenessResponse) - err := grpc.Invoke(ctx, "/cockroach.server.serverpb.Admin/Liveness", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Admin/Liveness", in, out, opts...) if err != nil { return nil, err } @@ -1375,7 +2880,7 @@ func (c *adminClient) Liveness(ctx context.Context, in *LivenessRequest, opts .. func (c *adminClient) Jobs(ctx context.Context, in *JobsRequest, opts ...grpc.CallOption) (*JobsResponse, error) { out := new(JobsResponse) - err := grpc.Invoke(ctx, "/cockroach.server.serverpb.Admin/Jobs", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Admin/Jobs", in, out, opts...) if err != nil { return nil, err } @@ -1384,7 +2889,7 @@ func (c *adminClient) Jobs(ctx context.Context, in *JobsRequest, opts ...grpc.Ca func (c *adminClient) Locations(ctx context.Context, in *LocationsRequest, opts ...grpc.CallOption) (*LocationsResponse, error) { out := new(LocationsResponse) - err := grpc.Invoke(ctx, "/cockroach.server.serverpb.Admin/Locations", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Admin/Locations", in, out, opts...) if err != nil { return nil, err } @@ -1393,7 +2898,7 @@ func (c *adminClient) Locations(ctx context.Context, in *LocationsRequest, opts func (c *adminClient) QueryPlan(ctx context.Context, in *QueryPlanRequest, opts ...grpc.CallOption) (*QueryPlanResponse, error) { out := new(QueryPlanResponse) - err := grpc.Invoke(ctx, "/cockroach.server.serverpb.Admin/QueryPlan", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Admin/QueryPlan", in, out, opts...) if err != nil { return nil, err } @@ -1401,7 +2906,7 @@ func (c *adminClient) QueryPlan(ctx context.Context, in *QueryPlanRequest, opts } func (c *adminClient) Drain(ctx context.Context, in *DrainRequest, opts ...grpc.CallOption) (Admin_DrainClient, error) { - stream, err := grpc.NewClientStream(ctx, &_Admin_serviceDesc.Streams[0], c.cc, "/cockroach.server.serverpb.Admin/Drain", opts...) + stream, err := c.cc.NewStream(ctx, &_Admin_serviceDesc.Streams[0], "/cockroach.server.serverpb.Admin/Drain", opts...) if err != nil { return nil, err } @@ -1434,7 +2939,7 @@ func (x *adminDrainClient) Recv() (*DrainResponse, error) { func (c *adminClient) Decommission(ctx context.Context, in *DecommissionRequest, opts ...grpc.CallOption) (*DecommissionStatusResponse, error) { out := new(DecommissionStatusResponse) - err := grpc.Invoke(ctx, "/cockroach.server.serverpb.Admin/Decommission", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Admin/Decommission", in, out, opts...) if err != nil { return nil, err } @@ -1443,7 +2948,7 @@ func (c *adminClient) Decommission(ctx context.Context, in *DecommissionRequest, func (c *adminClient) DecommissionStatus(ctx context.Context, in *DecommissionStatusRequest, opts ...grpc.CallOption) (*DecommissionStatusResponse, error) { out := new(DecommissionStatusResponse) - err := grpc.Invoke(ctx, "/cockroach.server.serverpb.Admin/DecommissionStatus", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Admin/DecommissionStatus", in, out, opts...) if err != nil { return nil, err } @@ -1452,7 +2957,7 @@ func (c *adminClient) DecommissionStatus(ctx context.Context, in *DecommissionSt func (c *adminClient) RangeLog(ctx context.Context, in *RangeLogRequest, opts ...grpc.CallOption) (*RangeLogResponse, error) { out := new(RangeLogResponse) - err := grpc.Invoke(ctx, "/cockroach.server.serverpb.Admin/RangeLog", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Admin/RangeLog", in, out, opts...) if err != nil { return nil, err } @@ -1461,7 +2966,7 @@ func (c *adminClient) RangeLog(ctx context.Context, in *RangeLogRequest, opts .. func (c *adminClient) DataDistribution(ctx context.Context, in *DataDistributionRequest, opts ...grpc.CallOption) (*DataDistributionResponse, error) { out := new(DataDistributionResponse) - err := grpc.Invoke(ctx, "/cockroach.server.serverpb.Admin/DataDistribution", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Admin/DataDistribution", in, out, opts...) if err != nil { return nil, err } @@ -1470,7 +2975,7 @@ func (c *adminClient) DataDistribution(ctx context.Context, in *DataDistribution func (c *adminClient) AllMetricMetadata(ctx context.Context, in *MetricMetadataRequest, opts ...grpc.CallOption) (*MetricMetadataResponse, error) { out := new(MetricMetadataResponse) - err := grpc.Invoke(ctx, "/cockroach.server.serverpb.Admin/AllMetricMetadata", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Admin/AllMetricMetadata", in, out, opts...) if err != nil { return nil, err } @@ -1479,15 +2984,14 @@ func (c *adminClient) AllMetricMetadata(ctx context.Context, in *MetricMetadataR func (c *adminClient) EnqueueRange(ctx context.Context, in *EnqueueRangeRequest, opts ...grpc.CallOption) (*EnqueueRangeResponse, error) { out := new(EnqueueRangeResponse) - err := grpc.Invoke(ctx, "/cockroach.server.serverpb.Admin/EnqueueRange", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Admin/EnqueueRange", in, out, opts...) if err != nil { return nil, err } return out, nil } -// Server API for Admin service - +// AdminServer is the server API for Admin service. type AdminServer interface { // URL: /_admin/v1/users Users(context.Context, *UsersRequest) (*UsersResponse, error) @@ -2890,8 +4394,8 @@ func (m *EventsResponse_Event) MarshalTo(dAtA []byte) (int, error) { _ = l dAtA[i] = 0xa i++ - i = encodeVarintAdmin(dAtA, i, uint64(types.SizeOfStdTime(m.Timestamp))) - n6, err := types.StdTimeMarshalTo(m.Timestamp, dAtA[i:]) + i = encodeVarintAdmin(dAtA, i, uint64(github_com_gogo_protobuf_types.SizeOfStdTime(m.Timestamp))) + n6, err := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Timestamp, dAtA[i:]) if err != nil { return 0, err } @@ -2947,7 +4451,7 @@ func (m *SetUIDataRequest) MarshalTo(dAtA []byte) (int, error) { for k := range m.KeyValues { keysForKeyValues = append(keysForKeyValues, string(k)) } - sortkeys.Strings(keysForKeyValues) + github_com_gogo_protobuf_sortkeys.Strings(keysForKeyValues) for _, k := range keysForKeyValues { dAtA[i] = 0xa i++ @@ -3044,7 +4548,7 @@ func (m *GetUIDataResponse) MarshalTo(dAtA []byte) (int, error) { for k := range m.KeyValues { keysForKeyValues = append(keysForKeyValues, string(k)) } - sortkeys.Strings(keysForKeyValues) + github_com_gogo_protobuf_sortkeys.Strings(keysForKeyValues) for _, k := range keysForKeyValues { dAtA[i] = 0xa i++ @@ -3096,8 +4600,8 @@ func (m *GetUIDataResponse_Value) MarshalTo(dAtA []byte) (int, error) { } dAtA[i] = 0x12 i++ - i = encodeVarintAdmin(dAtA, i, uint64(types.SizeOfStdTime(m.LastUpdated))) - n8, err := types.StdTimeMarshalTo(m.LastUpdated, dAtA[i:]) + i = encodeVarintAdmin(dAtA, i, uint64(github_com_gogo_protobuf_types.SizeOfStdTime(m.LastUpdated))) + n8, err := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.LastUpdated, dAtA[i:]) if err != nil { return 0, err } @@ -3490,7 +4994,7 @@ func (m *SettingsResponse) MarshalTo(dAtA []byte) (int, error) { for k := range m.KeyValues { keysForKeyValues = append(keysForKeyValues, string(k)) } - sortkeys.Strings(keysForKeyValues) + github_com_gogo_protobuf_sortkeys.Strings(keysForKeyValues) for _, k := range keysForKeyValues { dAtA[i] = 0xa i++ @@ -3641,7 +5145,7 @@ func (m *LivenessResponse) MarshalTo(dAtA []byte) (int, error) { for k := range m.Statuses { keysForStatuses = append(keysForStatuses, int32(k)) } - sortkeys.Int32s(keysForStatuses) + github_com_gogo_protobuf_sortkeys.Int32s(keysForStatuses) for _, k := range keysForStatuses { dAtA[i] = 0x12 i++ @@ -3787,8 +5291,8 @@ func (m *JobsResponse_Job) MarshalTo(dAtA []byte) (int, error) { if m.Created != nil { dAtA[i] = 0x3a i++ - i = encodeVarintAdmin(dAtA, i, uint64(types.SizeOfStdTime(*m.Created))) - n22, err := types.StdTimeMarshalTo(*m.Created, dAtA[i:]) + i = encodeVarintAdmin(dAtA, i, uint64(github_com_gogo_protobuf_types.SizeOfStdTime(*m.Created))) + n22, err := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.Created, dAtA[i:]) if err != nil { return 0, err } @@ -3797,8 +5301,8 @@ func (m *JobsResponse_Job) MarshalTo(dAtA []byte) (int, error) { if m.Started != nil { dAtA[i] = 0x42 i++ - i = encodeVarintAdmin(dAtA, i, uint64(types.SizeOfStdTime(*m.Started))) - n23, err := types.StdTimeMarshalTo(*m.Started, dAtA[i:]) + i = encodeVarintAdmin(dAtA, i, uint64(github_com_gogo_protobuf_types.SizeOfStdTime(*m.Started))) + n23, err := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.Started, dAtA[i:]) if err != nil { return 0, err } @@ -3807,8 +5311,8 @@ func (m *JobsResponse_Job) MarshalTo(dAtA []byte) (int, error) { if m.Finished != nil { dAtA[i] = 0x4a i++ - i = encodeVarintAdmin(dAtA, i, uint64(types.SizeOfStdTime(*m.Finished))) - n24, err := types.StdTimeMarshalTo(*m.Finished, dAtA[i:]) + i = encodeVarintAdmin(dAtA, i, uint64(github_com_gogo_protobuf_types.SizeOfStdTime(*m.Finished))) + n24, err := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.Finished, dAtA[i:]) if err != nil { return 0, err } @@ -3817,8 +5321,8 @@ func (m *JobsResponse_Job) MarshalTo(dAtA []byte) (int, error) { if m.Modified != nil { dAtA[i] = 0x52 i++ - i = encodeVarintAdmin(dAtA, i, uint64(types.SizeOfStdTime(*m.Modified))) - n25, err := types.StdTimeMarshalTo(*m.Modified, dAtA[i:]) + i = encodeVarintAdmin(dAtA, i, uint64(github_com_gogo_protobuf_types.SizeOfStdTime(*m.Modified))) + n25, err := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.Modified, dAtA[i:]) if err != nil { return 0, err } @@ -3827,7 +5331,7 @@ func (m *JobsResponse_Job) MarshalTo(dAtA []byte) (int, error) { if m.FractionCompleted != 0 { dAtA[i] = 0x5d i++ - binary.LittleEndian.PutUint32(dAtA[i:], uint32(math.Float32bits(float32(m.FractionCompleted)))) + encoding_binary.LittleEndian.PutUint32(dAtA[i:], uint32(math.Float32bits(float32(m.FractionCompleted)))) i += 4 } if len(m.Error) > 0 { @@ -3839,8 +5343,8 @@ func (m *JobsResponse_Job) MarshalTo(dAtA []byte) (int, error) { if m.HighwaterTimestamp != nil { dAtA[i] = 0x6a i++ - i = encodeVarintAdmin(dAtA, i, uint64(types.SizeOfStdTime(*m.HighwaterTimestamp))) - n26, err := types.StdTimeMarshalTo(*m.HighwaterTimestamp, dAtA[i:]) + i = encodeVarintAdmin(dAtA, i, uint64(github_com_gogo_protobuf_types.SizeOfStdTime(*m.HighwaterTimestamp))) + n26, err := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.HighwaterTimestamp, dAtA[i:]) if err != nil { return 0, err } @@ -3939,13 +5443,13 @@ func (m *LocationsResponse_Location) MarshalTo(dAtA []byte) (int, error) { if m.Latitude != 0 { dAtA[i] = 0x19 i++ - binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.Latitude)))) + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.Latitude)))) i += 8 } if m.Longitude != 0 { dAtA[i] = 0x21 i++ - binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.Longitude)))) + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.Longitude)))) i += 8 } return i, nil @@ -4183,7 +5687,7 @@ func (m *DataDistributionResponse) MarshalTo(dAtA []byte) (int, error) { for k := range m.DatabaseInfo { keysForDatabaseInfo = append(keysForDatabaseInfo, string(k)) } - sortkeys.Strings(keysForDatabaseInfo) + github_com_gogo_protobuf_sortkeys.Strings(keysForDatabaseInfo) for _, k := range keysForDatabaseInfo { dAtA[i] = 0xa i++ @@ -4214,7 +5718,7 @@ func (m *DataDistributionResponse) MarshalTo(dAtA []byte) (int, error) { for k := range m.ZoneConfigs { keysForZoneConfigs = append(keysForZoneConfigs, string(k)) } - sortkeys.Strings(keysForZoneConfigs) + github_com_gogo_protobuf_sortkeys.Strings(keysForZoneConfigs) for _, k := range keysForZoneConfigs { dAtA[i] = 0x1a i++ @@ -4301,7 +5805,7 @@ func (m *DataDistributionResponse_TableInfo) MarshalTo(dAtA []byte) (int, error) for k := range m.ReplicaCountByNodeId { keysForReplicaCountByNodeId = append(keysForReplicaCountByNodeId, int32(k)) } - sortkeys.Int32s(keysForReplicaCountByNodeId) + github_com_gogo_protobuf_sortkeys.Int32s(keysForReplicaCountByNodeId) for _, k := range keysForReplicaCountByNodeId { dAtA[i] = 0xa i++ @@ -4324,8 +5828,8 @@ func (m *DataDistributionResponse_TableInfo) MarshalTo(dAtA []byte) (int, error) if m.DroppedAt != nil { dAtA[i] = 0x1a i++ - i = encodeVarintAdmin(dAtA, i, uint64(types.SizeOfStdTime(*m.DroppedAt))) - n32, err := types.StdTimeMarshalTo(*m.DroppedAt, dAtA[i:]) + i = encodeVarintAdmin(dAtA, i, uint64(github_com_gogo_protobuf_types.SizeOfStdTime(*m.DroppedAt))) + n32, err := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.DroppedAt, dAtA[i:]) if err != nil { return 0, err } @@ -4354,7 +5858,7 @@ func (m *DataDistributionResponse_DatabaseInfo) MarshalTo(dAtA []byte) (int, err for k := range m.TableInfo { keysForTableInfo = append(keysForTableInfo, string(k)) } - sortkeys.Strings(keysForTableInfo) + github_com_gogo_protobuf_sortkeys.Strings(keysForTableInfo) for _, k := range keysForTableInfo { dAtA[i] = 0xa i++ @@ -4421,7 +5925,7 @@ func (m *MetricMetadataResponse) MarshalTo(dAtA []byte) (int, error) { for k := range m.Metadata { keysForMetadata = append(keysForMetadata, string(k)) } - sortkeys.Strings(keysForMetadata) + github_com_gogo_protobuf_sortkeys.Strings(keysForMetadata) for _, k := range keysForMetadata { dAtA[i] = 0xa i++ @@ -4575,12 +6079,18 @@ func encodeVarintAdmin(dAtA []byte, offset int, v uint64) int { return offset + 1 } func (m *DatabasesRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l return n } func (m *DatabasesResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.Databases) > 0 { @@ -4593,6 +6103,9 @@ func (m *DatabasesResponse) Size() (n int) { } func (m *DatabaseDetailsRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Database) @@ -4603,6 +6116,9 @@ func (m *DatabaseDetailsRequest) Size() (n int) { } func (m *DatabaseDetailsResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.Grants) > 0 { @@ -4629,6 +6145,9 @@ func (m *DatabaseDetailsResponse) Size() (n int) { } func (m *DatabaseDetailsResponse_Grant) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.User) @@ -4645,6 +6164,9 @@ func (m *DatabaseDetailsResponse_Grant) Size() (n int) { } func (m *TableDetailsRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Database) @@ -4659,6 +6181,9 @@ func (m *TableDetailsRequest) Size() (n int) { } func (m *TableDetailsResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.Grants) > 0 { @@ -4698,6 +6223,9 @@ func (m *TableDetailsResponse) Size() (n int) { } func (m *TableDetailsResponse_Grant) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.User) @@ -4714,6 +6242,9 @@ func (m *TableDetailsResponse_Grant) Size() (n int) { } func (m *TableDetailsResponse_Column) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Name) @@ -4742,6 +6273,9 @@ func (m *TableDetailsResponse_Column) Size() (n int) { } func (m *TableDetailsResponse_Index) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Name) @@ -4772,6 +6306,9 @@ func (m *TableDetailsResponse_Index) Size() (n int) { } func (m *TableStatsRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Database) @@ -4786,6 +6323,9 @@ func (m *TableStatsRequest) Size() (n int) { } func (m *TableStatsResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.RangeCount != 0 { @@ -4812,6 +6352,9 @@ func (m *TableStatsResponse) Size() (n int) { } func (m *TableStatsResponse_MissingNode) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.NodeID) @@ -4826,12 +6369,18 @@ func (m *TableStatsResponse_MissingNode) Size() (n int) { } func (m *NonTableStatsRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l return n } func (m *NonTableStatsResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.TimeSeriesStats != nil { @@ -4846,12 +6395,18 @@ func (m *NonTableStatsResponse) Size() (n int) { } func (m *UsersRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l return n } func (m *UsersResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.Users) > 0 { @@ -4864,6 +6419,9 @@ func (m *UsersResponse) Size() (n int) { } func (m *UsersResponse_User) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Username) @@ -4874,6 +6432,9 @@ func (m *UsersResponse_User) Size() (n int) { } func (m *EventsRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Type) @@ -4890,6 +6451,9 @@ func (m *EventsRequest) Size() (n int) { } func (m *EventsResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.Events) > 0 { @@ -4902,9 +6466,12 @@ func (m *EventsResponse) Size() (n int) { } func (m *EventsResponse_Event) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l - l = types.SizeOfStdTime(m.Timestamp) + l = github_com_gogo_protobuf_types.SizeOfStdTime(m.Timestamp) n += 1 + l + sovAdmin(uint64(l)) l = len(m.EventType) if l > 0 { @@ -4928,6 +6495,9 @@ func (m *EventsResponse_Event) Size() (n int) { } func (m *SetUIDataRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.KeyValues) > 0 { @@ -4946,12 +6516,18 @@ func (m *SetUIDataRequest) Size() (n int) { } func (m *SetUIDataResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l return n } func (m *GetUIDataRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.Keys) > 0 { @@ -4964,6 +6540,9 @@ func (m *GetUIDataRequest) Size() (n int) { } func (m *GetUIDataResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.KeyValues) > 0 { @@ -4979,24 +6558,33 @@ func (m *GetUIDataResponse) Size() (n int) { } func (m *GetUIDataResponse_Value) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Value) if l > 0 { n += 1 + l + sovAdmin(uint64(l)) } - l = types.SizeOfStdTime(m.LastUpdated) + l = github_com_gogo_protobuf_types.SizeOfStdTime(m.LastUpdated) n += 1 + l + sovAdmin(uint64(l)) return n } func (m *ClusterRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l return n } func (m *ClusterResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.ClusterID) @@ -5013,6 +6601,9 @@ func (m *ClusterResponse) Size() (n int) { } func (m *DrainRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.On) > 0 { @@ -5036,6 +6627,9 @@ func (m *DrainRequest) Size() (n int) { } func (m *DrainResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.On) > 0 { @@ -5049,6 +6643,9 @@ func (m *DrainResponse) Size() (n int) { } func (m *DecommissionStatusRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.NodeIDs) > 0 { @@ -5062,6 +6659,9 @@ func (m *DecommissionStatusRequest) Size() (n int) { } func (m *DecommissionRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.NodeIDs) > 0 { @@ -5078,6 +6678,9 @@ func (m *DecommissionRequest) Size() (n int) { } func (m *DecommissionStatusResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.Status) > 0 { @@ -5090,6 +6693,9 @@ func (m *DecommissionStatusResponse) Size() (n int) { } func (m *DecommissionStatusResponse_Status) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.NodeID != 0 { @@ -5111,6 +6717,9 @@ func (m *DecommissionStatusResponse_Status) Size() (n int) { } func (m *SettingsRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.Keys) > 0 { @@ -5123,6 +6732,9 @@ func (m *SettingsRequest) Size() (n int) { } func (m *SettingsResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.KeyValues) > 0 { @@ -5138,6 +6750,9 @@ func (m *SettingsResponse) Size() (n int) { } func (m *SettingsResponse_Value) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Value) @@ -5156,24 +6771,36 @@ func (m *SettingsResponse_Value) Size() (n int) { } func (m *HealthRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l return n } func (m *HealthResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l return n } func (m *LivenessRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l return n } func (m *LivenessResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.Livenesses) > 0 { @@ -5194,6 +6821,9 @@ func (m *LivenessResponse) Size() (n int) { } func (m *JobsRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Limit != 0 { @@ -5210,6 +6840,9 @@ func (m *JobsRequest) Size() (n int) { } func (m *JobsResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.Jobs) > 0 { @@ -5222,6 +6855,9 @@ func (m *JobsResponse) Size() (n int) { } func (m *JobsResponse_Job) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.ID != 0 { @@ -5251,19 +6887,19 @@ func (m *JobsResponse_Job) Size() (n int) { n += 1 + l + sovAdmin(uint64(l)) } if m.Created != nil { - l = types.SizeOfStdTime(*m.Created) + l = github_com_gogo_protobuf_types.SizeOfStdTime(*m.Created) n += 1 + l + sovAdmin(uint64(l)) } if m.Started != nil { - l = types.SizeOfStdTime(*m.Started) + l = github_com_gogo_protobuf_types.SizeOfStdTime(*m.Started) n += 1 + l + sovAdmin(uint64(l)) } if m.Finished != nil { - l = types.SizeOfStdTime(*m.Finished) + l = github_com_gogo_protobuf_types.SizeOfStdTime(*m.Finished) n += 1 + l + sovAdmin(uint64(l)) } if m.Modified != nil { - l = types.SizeOfStdTime(*m.Modified) + l = github_com_gogo_protobuf_types.SizeOfStdTime(*m.Modified) n += 1 + l + sovAdmin(uint64(l)) } if m.FractionCompleted != 0 { @@ -5274,7 +6910,7 @@ func (m *JobsResponse_Job) Size() (n int) { n += 1 + l + sovAdmin(uint64(l)) } if m.HighwaterTimestamp != nil { - l = types.SizeOfStdTime(*m.HighwaterTimestamp) + l = github_com_gogo_protobuf_types.SizeOfStdTime(*m.HighwaterTimestamp) n += 1 + l + sovAdmin(uint64(l)) } l = len(m.HighwaterDecimal) @@ -5289,12 +6925,18 @@ func (m *JobsResponse_Job) Size() (n int) { } func (m *LocationsRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l return n } func (m *LocationsResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.Locations) > 0 { @@ -5307,6 +6949,9 @@ func (m *LocationsResponse) Size() (n int) { } func (m *LocationsResponse_Location) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.LocalityKey) @@ -5327,6 +6972,9 @@ func (m *LocationsResponse_Location) Size() (n int) { } func (m *RangeLogRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.RangeId != 0 { @@ -5339,6 +6987,9 @@ func (m *RangeLogRequest) Size() (n int) { } func (m *RangeLogResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.Events) > 0 { @@ -5351,6 +7002,9 @@ func (m *RangeLogResponse) Size() (n int) { } func (m *RangeLogResponse_PrettyInfo) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.UpdatedDesc) @@ -5381,6 +7035,9 @@ func (m *RangeLogResponse_PrettyInfo) Size() (n int) { } func (m *RangeLogResponse_Event) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.Event.Size() @@ -5391,6 +7048,9 @@ func (m *RangeLogResponse_Event) Size() (n int) { } func (m *QueryPlanRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Query) @@ -5401,6 +7061,9 @@ func (m *QueryPlanRequest) Size() (n int) { } func (m *QueryPlanResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.DistSQLPhysicalQueryPlan) @@ -5411,12 +7074,18 @@ func (m *QueryPlanResponse) Size() (n int) { } func (m *DataDistributionRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l return n } func (m *DataDistributionResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.DatabaseInfo) > 0 { @@ -5441,6 +7110,9 @@ func (m *DataDistributionResponse) Size() (n int) { } func (m *DataDistributionResponse_ZoneConfig) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.ZoneName) @@ -5457,6 +7129,9 @@ func (m *DataDistributionResponse_ZoneConfig) Size() (n int) { } func (m *DataDistributionResponse_TableInfo) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.ReplicaCountByNodeId) > 0 { @@ -5471,13 +7146,16 @@ func (m *DataDistributionResponse_TableInfo) Size() (n int) { n += 1 + sovAdmin(uint64(m.ZoneConfigId)) } if m.DroppedAt != nil { - l = types.SizeOfStdTime(*m.DroppedAt) + l = github_com_gogo_protobuf_types.SizeOfStdTime(*m.DroppedAt) n += 1 + l + sovAdmin(uint64(l)) } return n } func (m *DataDistributionResponse_DatabaseInfo) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.TableInfo) > 0 { @@ -5493,12 +7171,18 @@ func (m *DataDistributionResponse_DatabaseInfo) Size() (n int) { } func (m *MetricMetadataRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l return n } func (m *MetricMetadataResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.Metadata) > 0 { @@ -5514,6 +7198,9 @@ func (m *MetricMetadataResponse) Size() (n int) { } func (m *EnqueueRangeRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.NodeID != 0 { @@ -5533,6 +7220,9 @@ func (m *EnqueueRangeRequest) Size() (n int) { } func (m *EnqueueRangeResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.Details) > 0 { @@ -5545,6 +7235,9 @@ func (m *EnqueueRangeResponse) Size() (n int) { } func (m *EnqueueRangeResponse_Details) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.NodeID != 0 { @@ -7999,7 +9692,7 @@ func (m *EventsResponse_Event) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if err := types.StdTimeUnmarshal(&m.Timestamp, dAtA[iNdEx:postIndex]); err != nil { + if err := github_com_gogo_protobuf_types.StdTimeUnmarshal(&m.Timestamp, dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -8708,7 +10401,7 @@ func (m *GetUIDataResponse_Value) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if err := types.StdTimeUnmarshal(&m.LastUpdated, dAtA[iNdEx:postIndex]); err != nil { + if err := github_com_gogo_protobuf_types.StdTimeUnmarshal(&m.LastUpdated, dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -8972,6 +10665,17 @@ func (m *DrainRequest) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + var count int + for _, integer := range dAtA { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.On) == 0 { + m.On = make([]int32, 0, elementCount) + } for iNdEx < postIndex { var v int32 for shift := uint(0); ; shift += 7 { @@ -9034,6 +10738,17 @@ func (m *DrainRequest) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + var count int + for _, integer := range dAtA { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.Off) == 0 { + m.Off = make([]int32, 0, elementCount) + } for iNdEx < postIndex { var v int32 for shift := uint(0); ; shift += 7 { @@ -9166,6 +10881,17 @@ func (m *DrainResponse) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + var count int + for _, integer := range dAtA { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.On) == 0 { + m.On = make([]int32, 0, elementCount) + } for iNdEx < postIndex { var v int32 for shift := uint(0); ; shift += 7 { @@ -9278,6 +11004,17 @@ func (m *DecommissionStatusRequest) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + var count int + for _, integer := range dAtA { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.NodeIDs) == 0 { + m.NodeIDs = make([]github_com_cockroachdb_cockroach_pkg_roachpb.NodeID, 0, elementCount) + } for iNdEx < postIndex { var v github_com_cockroachdb_cockroach_pkg_roachpb.NodeID for shift := uint(0); ; shift += 7 { @@ -9390,6 +11127,17 @@ func (m *DecommissionRequest) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + var count int + for _, integer := range dAtA { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.NodeIDs) == 0 { + m.NodeIDs = make([]github_com_cockroachdb_cockroach_pkg_roachpb.NodeID, 0, elementCount) + } for iNdEx < postIndex { var v github_com_cockroachdb_cockroach_pkg_roachpb.NodeID for shift := uint(0); ; shift += 7 { @@ -10275,7 +12023,7 @@ func (m *LivenessResponse) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Livenesses = append(m.Livenesses, cockroach_storage.Liveness{}) + m.Livenesses = append(m.Livenesses, storagepb.Liveness{}) if err := m.Livenesses[len(m.Livenesses)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -10307,10 +12055,10 @@ func (m *LivenessResponse) Unmarshal(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Statuses == nil { - m.Statuses = make(map[github_com_cockroachdb_cockroach_pkg_roachpb.NodeID]cockroach_storage.NodeLivenessStatus) + m.Statuses = make(map[github_com_cockroachdb_cockroach_pkg_roachpb.NodeID]storagepb.NodeLivenessStatus) } var mapkey int32 - var mapvalue cockroach_storage.NodeLivenessStatus + var mapvalue storagepb.NodeLivenessStatus for iNdEx < postIndex { entryPreIndex := iNdEx var wire uint64 @@ -10354,7 +12102,7 @@ func (m *LivenessResponse) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - mapvalue |= (cockroach_storage.NodeLivenessStatus(b) & 0x7F) << shift + mapvalue |= (storagepb.NodeLivenessStatus(b) & 0x7F) << shift if b < 0x80 { break } @@ -10488,7 +12236,7 @@ func (m *JobsRequest) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Type |= (cockroach_sql_jobs_jobspb.Type(b) & 0x7F) << shift + m.Type |= (jobspb.Type(b) & 0x7F) << shift if b < 0x80 { break } @@ -10771,6 +12519,17 @@ func (m *JobsResponse_Job) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + var count int + for _, integer := range dAtA { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.DescriptorIDs) == 0 { + m.DescriptorIDs = make([]github_com_cockroachdb_cockroach_pkg_sql_sqlbase.ID, 0, elementCount) + } for iNdEx < postIndex { var v github_com_cockroachdb_cockroach_pkg_sql_sqlbase.ID for shift := uint(0); ; shift += 7 { @@ -10850,7 +12609,7 @@ func (m *JobsResponse_Job) Unmarshal(dAtA []byte) error { if m.Created == nil { m.Created = new(time.Time) } - if err := types.StdTimeUnmarshal(m.Created, dAtA[iNdEx:postIndex]); err != nil { + if err := github_com_gogo_protobuf_types.StdTimeUnmarshal(m.Created, dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -10883,7 +12642,7 @@ func (m *JobsResponse_Job) Unmarshal(dAtA []byte) error { if m.Started == nil { m.Started = new(time.Time) } - if err := types.StdTimeUnmarshal(m.Started, dAtA[iNdEx:postIndex]); err != nil { + if err := github_com_gogo_protobuf_types.StdTimeUnmarshal(m.Started, dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -10916,7 +12675,7 @@ func (m *JobsResponse_Job) Unmarshal(dAtA []byte) error { if m.Finished == nil { m.Finished = new(time.Time) } - if err := types.StdTimeUnmarshal(m.Finished, dAtA[iNdEx:postIndex]); err != nil { + if err := github_com_gogo_protobuf_types.StdTimeUnmarshal(m.Finished, dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -10949,7 +12708,7 @@ func (m *JobsResponse_Job) Unmarshal(dAtA []byte) error { if m.Modified == nil { m.Modified = new(time.Time) } - if err := types.StdTimeUnmarshal(m.Modified, dAtA[iNdEx:postIndex]); err != nil { + if err := github_com_gogo_protobuf_types.StdTimeUnmarshal(m.Modified, dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -10961,7 +12720,7 @@ func (m *JobsResponse_Job) Unmarshal(dAtA []byte) error { if (iNdEx + 4) > l { return io.ErrUnexpectedEOF } - v = uint32(binary.LittleEndian.Uint32(dAtA[iNdEx:])) + v = uint32(encoding_binary.LittleEndian.Uint32(dAtA[iNdEx:])) iNdEx += 4 m.FractionCompleted = float32(math.Float32frombits(v)) case 12: @@ -11022,7 +12781,7 @@ func (m *JobsResponse_Job) Unmarshal(dAtA []byte) error { if m.HighwaterTimestamp == nil { m.HighwaterTimestamp = new(time.Time) } - if err := types.StdTimeUnmarshal(m.HighwaterTimestamp, dAtA[iNdEx:postIndex]); err != nil { + if err := github_com_gogo_protobuf_types.StdTimeUnmarshal(m.HighwaterTimestamp, dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -11331,7 +13090,7 @@ func (m *LocationsResponse_Location) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - v = uint64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 m.Latitude = float64(math.Float64frombits(v)) case 4: @@ -11342,7 +13101,7 @@ func (m *LocationsResponse_Location) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - v = uint64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 m.Longitude = float64(math.Float64frombits(v)) default: @@ -12684,7 +14443,7 @@ func (m *DataDistributionResponse_TableInfo) Unmarshal(dAtA []byte) error { if m.DroppedAt == nil { m.DroppedAt = new(time.Time) } - if err := types.StdTimeUnmarshal(m.DroppedAt, dAtA[iNdEx:postIndex]); err != nil { + if err := github_com_gogo_protobuf_types.StdTimeUnmarshal(m.DroppedAt, dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -12988,10 +14747,10 @@ func (m *MetricMetadataResponse) Unmarshal(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Metadata == nil { - m.Metadata = make(map[string]cockroach_util_metric.Metadata) + m.Metadata = make(map[string]metric.Metadata) } var mapkey string - mapvalue := &cockroach_util_metric.Metadata{} + mapvalue := &metric.Metadata{} for iNdEx < postIndex { entryPreIndex := iNdEx var wire uint64 @@ -13062,7 +14821,7 @@ func (m *MetricMetadataResponse) Unmarshal(dAtA []byte) error { if postmsgIndex > l { return io.ErrUnexpectedEOF } - mapvalue = &cockroach_util_metric.Metadata{} + mapvalue = &metric.Metadata{} if err := mapvalue.Unmarshal(dAtA[iNdEx:postmsgIndex]); err != nil { return err } @@ -13557,9 +15316,9 @@ var ( ErrIntOverflowAdmin = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("server/serverpb/admin.proto", fileDescriptorAdmin) } +func init() { proto.RegisterFile("server/serverpb/admin.proto", fileDescriptor_admin_b98220ab75a6bb7c) } -var fileDescriptorAdmin = []byte{ +var fileDescriptor_admin_b98220ab75a6bb7c = []byte{ // 3912 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x3a, 0x5d, 0x6f, 0x1b, 0xd9, 0x75, 0x1e, 0x52, 0xa4, 0xc8, 0x43, 0x52, 0x22, 0xaf, 0xb5, 0x5a, 0x9a, 0x76, 0x44, 0xed, 0x38, diff --git a/pkg/server/serverpb/authentication.pb.go b/pkg/server/serverpb/authentication.pb.go index d532e1bf87ee..733d76ae8eea 100644 --- a/pkg/server/serverpb/authentication.pb.go +++ b/pkg/server/serverpb/authentication.pb.go @@ -7,8 +7,10 @@ import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" -import context "context" -import grpc "google.golang.org/grpc" +import ( + context "context" + grpc "google.golang.org/grpc" +) import io "io" @@ -17,48 +19,158 @@ var _ = proto.Marshal var _ = fmt.Errorf var _ = math.Inf +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package + // UserLoginRequest contains credentials a user must provide to log in. type UserLoginRequest struct { // A username which must correspond to a database user on the cluster. Username string `protobuf:"bytes,1,opt,name=username,proto3" json:"username,omitempty"` // A password for the provided username. - Password string `protobuf:"bytes,2,opt,name=password,proto3" json:"password,omitempty"` + Password string `protobuf:"bytes,2,opt,name=password,proto3" json:"password,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *UserLoginRequest) Reset() { *m = UserLoginRequest{} } +func (m *UserLoginRequest) String() string { return proto.CompactTextString(m) } +func (*UserLoginRequest) ProtoMessage() {} +func (*UserLoginRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_authentication_a7ea30ecf9a74aa6, []int{0} +} +func (m *UserLoginRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *UserLoginRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *UserLoginRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_UserLoginRequest.Merge(dst, src) +} +func (m *UserLoginRequest) XXX_Size() int { + return m.Size() +} +func (m *UserLoginRequest) XXX_DiscardUnknown() { + xxx_messageInfo_UserLoginRequest.DiscardUnknown(m) } -func (m *UserLoginRequest) Reset() { *m = UserLoginRequest{} } -func (m *UserLoginRequest) String() string { return proto.CompactTextString(m) } -func (*UserLoginRequest) ProtoMessage() {} -func (*UserLoginRequest) Descriptor() ([]byte, []int) { return fileDescriptorAuthentication, []int{0} } +var xxx_messageInfo_UserLoginRequest proto.InternalMessageInfo // UserLoginResponse is currently empty. If a login is successful, an HTTP // Set-Cookie header will be added to the response with a session // cookie identifying the created session. type UserLoginResponse struct { + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *UserLoginResponse) Reset() { *m = UserLoginResponse{} } +func (m *UserLoginResponse) String() string { return proto.CompactTextString(m) } +func (*UserLoginResponse) ProtoMessage() {} +func (*UserLoginResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_authentication_a7ea30ecf9a74aa6, []int{1} +} +func (m *UserLoginResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *UserLoginResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *UserLoginResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_UserLoginResponse.Merge(dst, src) +} +func (m *UserLoginResponse) XXX_Size() int { + return m.Size() +} +func (m *UserLoginResponse) XXX_DiscardUnknown() { + xxx_messageInfo_UserLoginResponse.DiscardUnknown(m) } -func (m *UserLoginResponse) Reset() { *m = UserLoginResponse{} } -func (m *UserLoginResponse) String() string { return proto.CompactTextString(m) } -func (*UserLoginResponse) ProtoMessage() {} -func (*UserLoginResponse) Descriptor() ([]byte, []int) { return fileDescriptorAuthentication, []int{1} } +var xxx_messageInfo_UserLoginResponse proto.InternalMessageInfo // UserLogoutRequest will terminate the current session in use. The request // is empty because the current session is identified by an HTTP cookie on the // incoming request. type UserLogoutRequest struct { + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *UserLogoutRequest) Reset() { *m = UserLogoutRequest{} } +func (m *UserLogoutRequest) String() string { return proto.CompactTextString(m) } +func (*UserLogoutRequest) ProtoMessage() {} +func (*UserLogoutRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_authentication_a7ea30ecf9a74aa6, []int{2} +} +func (m *UserLogoutRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *UserLogoutRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *UserLogoutRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_UserLogoutRequest.Merge(dst, src) +} +func (m *UserLogoutRequest) XXX_Size() int { + return m.Size() +} +func (m *UserLogoutRequest) XXX_DiscardUnknown() { + xxx_messageInfo_UserLogoutRequest.DiscardUnknown(m) } -func (m *UserLogoutRequest) Reset() { *m = UserLogoutRequest{} } -func (m *UserLogoutRequest) String() string { return proto.CompactTextString(m) } -func (*UserLogoutRequest) ProtoMessage() {} -func (*UserLogoutRequest) Descriptor() ([]byte, []int) { return fileDescriptorAuthentication, []int{2} } +var xxx_messageInfo_UserLogoutRequest proto.InternalMessageInfo type UserLogoutResponse struct { + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *UserLogoutResponse) Reset() { *m = UserLogoutResponse{} } +func (m *UserLogoutResponse) String() string { return proto.CompactTextString(m) } +func (*UserLogoutResponse) ProtoMessage() {} +func (*UserLogoutResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_authentication_a7ea30ecf9a74aa6, []int{3} +} +func (m *UserLogoutResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *UserLogoutResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *UserLogoutResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_UserLogoutResponse.Merge(dst, src) +} +func (m *UserLogoutResponse) XXX_Size() int { + return m.Size() +} +func (m *UserLogoutResponse) XXX_DiscardUnknown() { + xxx_messageInfo_UserLogoutResponse.DiscardUnknown(m) } -func (m *UserLogoutResponse) Reset() { *m = UserLogoutResponse{} } -func (m *UserLogoutResponse) String() string { return proto.CompactTextString(m) } -func (*UserLogoutResponse) ProtoMessage() {} -func (*UserLogoutResponse) Descriptor() ([]byte, []int) { return fileDescriptorAuthentication, []int{3} } +var xxx_messageInfo_UserLogoutResponse proto.InternalMessageInfo // SessionCookie is a message used to encode the authentication cookie returned // from successful login requests. @@ -66,13 +178,39 @@ type SessionCookie struct { // The unique ID of the session. ID int64 `protobuf:"varint,1,opt,name=id,proto3" json:"id,omitempty"` // The secret needed to verify ownership of a session. - Secret []byte `protobuf:"bytes,2,opt,name=secret,proto3" json:"secret,omitempty"` + Secret []byte `protobuf:"bytes,2,opt,name=secret,proto3" json:"secret,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *SessionCookie) Reset() { *m = SessionCookie{} } +func (m *SessionCookie) String() string { return proto.CompactTextString(m) } +func (*SessionCookie) ProtoMessage() {} +func (*SessionCookie) Descriptor() ([]byte, []int) { + return fileDescriptor_authentication_a7ea30ecf9a74aa6, []int{4} +} +func (m *SessionCookie) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SessionCookie) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *SessionCookie) XXX_Merge(src proto.Message) { + xxx_messageInfo_SessionCookie.Merge(dst, src) +} +func (m *SessionCookie) XXX_Size() int { + return m.Size() +} +func (m *SessionCookie) XXX_DiscardUnknown() { + xxx_messageInfo_SessionCookie.DiscardUnknown(m) } -func (m *SessionCookie) Reset() { *m = SessionCookie{} } -func (m *SessionCookie) String() string { return proto.CompactTextString(m) } -func (*SessionCookie) ProtoMessage() {} -func (*SessionCookie) Descriptor() ([]byte, []int) { return fileDescriptorAuthentication, []int{4} } +var xxx_messageInfo_SessionCookie proto.InternalMessageInfo func init() { proto.RegisterType((*UserLoginRequest)(nil), "cockroach.server.serverpb.UserLoginRequest") @@ -90,8 +228,9 @@ var _ grpc.ClientConn // is compatible with the grpc package it is being compiled against. const _ = grpc.SupportPackageIsVersion4 -// Client API for LogIn service - +// LogInClient is the client API for LogIn service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream. type LogInClient interface { // UserLogin is used to create a web authentication session. UserLogin(ctx context.Context, in *UserLoginRequest, opts ...grpc.CallOption) (*UserLoginResponse, error) @@ -107,15 +246,14 @@ func NewLogInClient(cc *grpc.ClientConn) LogInClient { func (c *logInClient) UserLogin(ctx context.Context, in *UserLoginRequest, opts ...grpc.CallOption) (*UserLoginResponse, error) { out := new(UserLoginResponse) - err := grpc.Invoke(ctx, "/cockroach.server.serverpb.LogIn/UserLogin", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.LogIn/UserLogin", in, out, opts...) if err != nil { return nil, err } return out, nil } -// Server API for LogIn service - +// LogInServer is the server API for LogIn service. type LogInServer interface { // UserLogin is used to create a web authentication session. UserLogin(context.Context, *UserLoginRequest) (*UserLoginResponse, error) @@ -156,8 +294,9 @@ var _LogIn_serviceDesc = grpc.ServiceDesc{ Metadata: "server/serverpb/authentication.proto", } -// Client API for LogOut service - +// LogOutClient is the client API for LogOut service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream. type LogOutClient interface { // UserLogout terminates an active authentication session. UserLogout(ctx context.Context, in *UserLogoutRequest, opts ...grpc.CallOption) (*UserLogoutResponse, error) @@ -173,15 +312,14 @@ func NewLogOutClient(cc *grpc.ClientConn) LogOutClient { func (c *logOutClient) UserLogout(ctx context.Context, in *UserLogoutRequest, opts ...grpc.CallOption) (*UserLogoutResponse, error) { out := new(UserLogoutResponse) - err := grpc.Invoke(ctx, "/cockroach.server.serverpb.LogOut/UserLogout", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.LogOut/UserLogout", in, out, opts...) if err != nil { return nil, err } return out, nil } -// Server API for LogOut service - +// LogOutServer is the server API for LogOut service. type LogOutServer interface { // UserLogout terminates an active authentication session. UserLogout(context.Context, *UserLogoutRequest) (*UserLogoutResponse, error) @@ -345,6 +483,9 @@ func encodeVarintAuthentication(dAtA []byte, offset int, v uint64) int { return offset + 1 } func (m *UserLoginRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Username) @@ -359,24 +500,36 @@ func (m *UserLoginRequest) Size() (n int) { } func (m *UserLoginResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l return n } func (m *UserLogoutRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l return n } func (m *UserLogoutResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l return n } func (m *SessionCookie) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.ID != 0 { @@ -865,9 +1018,11 @@ var ( ErrIntOverflowAuthentication = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("server/serverpb/authentication.proto", fileDescriptorAuthentication) } +func init() { + proto.RegisterFile("server/serverpb/authentication.proto", fileDescriptor_authentication_a7ea30ecf9a74aa6) +} -var fileDescriptorAuthentication = []byte{ +var fileDescriptor_authentication_a7ea30ecf9a74aa6 = []byte{ // 368 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x92, 0x4d, 0x4b, 0xeb, 0x40, 0x14, 0x86, 0x3b, 0xb9, 0xdc, 0xdc, 0x76, 0xee, 0xbd, 0x68, 0xc7, 0x52, 0x6a, 0x90, 0x28, 0xc1, diff --git a/pkg/server/serverpb/init.pb.go b/pkg/server/serverpb/init.pb.go index 97ea9b6e6adb..a7c49e255246 100644 --- a/pkg/server/serverpb/init.pb.go +++ b/pkg/server/serverpb/init.pb.go @@ -7,8 +7,10 @@ import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" -import context "context" -import grpc "google.golang.org/grpc" +import ( + context "context" + grpc "google.golang.org/grpc" +) import io "io" @@ -17,21 +19,79 @@ var _ = proto.Marshal var _ = fmt.Errorf var _ = math.Inf +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package + type BootstrapRequest struct { + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *BootstrapRequest) Reset() { *m = BootstrapRequest{} } +func (m *BootstrapRequest) String() string { return proto.CompactTextString(m) } +func (*BootstrapRequest) ProtoMessage() {} +func (*BootstrapRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_init_a2f92a9ec99703a0, []int{0} +} +func (m *BootstrapRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *BootstrapRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *BootstrapRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_BootstrapRequest.Merge(dst, src) +} +func (m *BootstrapRequest) XXX_Size() int { + return m.Size() +} +func (m *BootstrapRequest) XXX_DiscardUnknown() { + xxx_messageInfo_BootstrapRequest.DiscardUnknown(m) } -func (m *BootstrapRequest) Reset() { *m = BootstrapRequest{} } -func (m *BootstrapRequest) String() string { return proto.CompactTextString(m) } -func (*BootstrapRequest) ProtoMessage() {} -func (*BootstrapRequest) Descriptor() ([]byte, []int) { return fileDescriptorInit, []int{0} } +var xxx_messageInfo_BootstrapRequest proto.InternalMessageInfo type BootstrapResponse struct { + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *BootstrapResponse) Reset() { *m = BootstrapResponse{} } -func (m *BootstrapResponse) String() string { return proto.CompactTextString(m) } -func (*BootstrapResponse) ProtoMessage() {} -func (*BootstrapResponse) Descriptor() ([]byte, []int) { return fileDescriptorInit, []int{1} } +func (m *BootstrapResponse) Reset() { *m = BootstrapResponse{} } +func (m *BootstrapResponse) String() string { return proto.CompactTextString(m) } +func (*BootstrapResponse) ProtoMessage() {} +func (*BootstrapResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_init_a2f92a9ec99703a0, []int{1} +} +func (m *BootstrapResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *BootstrapResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *BootstrapResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_BootstrapResponse.Merge(dst, src) +} +func (m *BootstrapResponse) XXX_Size() int { + return m.Size() +} +func (m *BootstrapResponse) XXX_DiscardUnknown() { + xxx_messageInfo_BootstrapResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_BootstrapResponse proto.InternalMessageInfo func init() { proto.RegisterType((*BootstrapRequest)(nil), "cockroach.server.serverpb.BootstrapRequest") @@ -46,8 +106,9 @@ var _ grpc.ClientConn // is compatible with the grpc package it is being compiled against. const _ = grpc.SupportPackageIsVersion4 -// Client API for Init service - +// InitClient is the client API for Init service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream. type InitClient interface { // Bootstrap an uninitialized cluster. Bootstrap(ctx context.Context, in *BootstrapRequest, opts ...grpc.CallOption) (*BootstrapResponse, error) @@ -63,15 +124,14 @@ func NewInitClient(cc *grpc.ClientConn) InitClient { func (c *initClient) Bootstrap(ctx context.Context, in *BootstrapRequest, opts ...grpc.CallOption) (*BootstrapResponse, error) { out := new(BootstrapResponse) - err := grpc.Invoke(ctx, "/cockroach.server.serverpb.Init/Bootstrap", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Init/Bootstrap", in, out, opts...) if err != nil { return nil, err } return out, nil } -// Server API for Init service - +// InitServer is the server API for Init service. type InitServer interface { // Bootstrap an uninitialized cluster. Bootstrap(context.Context, *BootstrapRequest) (*BootstrapResponse, error) @@ -158,12 +218,18 @@ func encodeVarintInit(dAtA []byte, offset int, v uint64) int { return offset + 1 } func (m *BootstrapRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l return n } func (m *BootstrapResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l return n @@ -387,9 +453,9 @@ var ( ErrIntOverflowInit = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("server/serverpb/init.proto", fileDescriptorInit) } +func init() { proto.RegisterFile("server/serverpb/init.proto", fileDescriptor_init_a2f92a9ec99703a0) } -var fileDescriptorInit = []byte{ +var fileDescriptor_init_a2f92a9ec99703a0 = []byte{ // 164 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x92, 0x2a, 0x4e, 0x2d, 0x2a, 0x4b, 0x2d, 0xd2, 0x87, 0x50, 0x05, 0x49, 0xfa, 0x99, 0x79, 0x99, 0x25, 0x7a, 0x05, 0x45, 0xf9, diff --git a/pkg/server/serverpb/status.pb.go b/pkg/server/serverpb/status.pb.go index 3b8f4b122eed..2e3088558173 100644 --- a/pkg/server/serverpb/status.pb.go +++ b/pkg/server/serverpb/status.pb.go @@ -6,29 +6,29 @@ package serverpb import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" -import cockroach_build "github.com/cockroachdb/cockroach/pkg/build" -import cockroach_gossip "github.com/cockroachdb/cockroach/pkg/gossip" -import cockroach_sql "github.com/cockroachdb/cockroach/pkg/roachpb" -import cockroach_roachpb1 "github.com/cockroachdb/cockroach/pkg/roachpb" -import cockroach_server_diagnosticspb "github.com/cockroachdb/cockroach/pkg/server/diagnosticspb" -import cockroach_server_status_statuspb "github.com/cockroachdb/cockroach/pkg/server/status/statuspb" -import cockroach_storage_engine_enginepb1 "github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb" -import cockroach_storage1 "github.com/cockroachdb/cockroach/pkg/storage/storagepb" -import cockroach_storage_storagepb "github.com/cockroachdb/cockroach/pkg/storage/storagepb" -import cockroach_util_log "github.com/cockroachdb/cockroach/pkg/util/log" -import cockroach_util "github.com/cockroachdb/cockroach/pkg/util" +import build "github.com/cockroachdb/cockroach/pkg/build" import raftpb "go.etcd.io/etcd/raft/raftpb" +import gossip "github.com/cockroachdb/cockroach/pkg/gossip" +import roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" +import diagnosticspb "github.com/cockroachdb/cockroach/pkg/server/diagnosticspb" +import statuspb "github.com/cockroachdb/cockroach/pkg/server/status/statuspb" +import enginepb "github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb" +import storagepb "github.com/cockroachdb/cockroach/pkg/storage/storagepb" +import util "github.com/cockroachdb/cockroach/pkg/util" +import log "github.com/cockroachdb/cockroach/pkg/util/log" import github_com_cockroachdb_cockroach_pkg_roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" import time "time" import github_com_cockroachdb_cockroach_pkg_util_uuid "github.com/cockroachdb/cockroach/pkg/util/uuid" -import context "context" -import grpc "google.golang.org/grpc" +import ( + context "context" + grpc "google.golang.org/grpc" +) -import sortkeys "github.com/gogo/protobuf/sortkeys" -import binary "encoding/binary" -import types "github.com/gogo/protobuf/types" +import github_com_gogo_protobuf_sortkeys "github.com/gogo/protobuf/sortkeys" +import encoding_binary "encoding/binary" +import github_com_gogo_protobuf_types "github.com/gogo/protobuf/types" import io "io" @@ -38,6 +38,12 @@ var _ = fmt.Errorf var _ = math.Inf var _ = time.Kitchen +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package + // Represents the type of file. // TODO(ridwanmsharif): Add support for log files. They're currently served // by an endpoint that parses the log messages, which is not what the @@ -58,7 +64,9 @@ var FileType_value = map[string]int32{ func (x FileType) String() string { return proto.EnumName(FileType_name, int32(x)) } -func (FileType) EnumDescriptor() ([]byte, []int) { return fileDescriptorStatus, []int{0} } +func (FileType) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{0} +} // We use an enum to allow reporting of client certs and potential others (eg: // UI). @@ -94,7 +102,7 @@ func (x CertificateDetails_CertificateType) String() string { return proto.EnumName(CertificateDetails_CertificateType_name, int32(x)) } func (CertificateDetails_CertificateType) EnumDescriptor() ([]byte, []int) { - return fileDescriptorStatus, []int{1, 0} + return fileDescriptor_status_0de29a344bd79652, []int{1, 0} } type ProfileRequest_Type int32 @@ -113,7 +121,9 @@ var ProfileRequest_Type_value = map[string]int32{ func (x ProfileRequest_Type) String() string { return proto.EnumName(ProfileRequest_Type_name, int32(x)) } -func (ProfileRequest_Type) EnumDescriptor() ([]byte, []int) { return fileDescriptorStatus, []int{33, 0} } +func (ProfileRequest_Type) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{33, 0} +} // Enum for phase of execution. type ActiveQuery_Phase int32 @@ -135,18 +145,46 @@ var ActiveQuery_Phase_value = map[string]int32{ func (x ActiveQuery_Phase) String() string { return proto.EnumName(ActiveQuery_Phase_name, int32(x)) } -func (ActiveQuery_Phase) EnumDescriptor() ([]byte, []int) { return fileDescriptorStatus, []int{40, 0} } +func (ActiveQuery_Phase) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{40, 0} +} type CertificatesRequest struct { // node_id is a string so that "local" can be used to specify that no // forwarding is necessary. - NodeId string `protobuf:"bytes,1,opt,name=node_id,json=nodeId,proto3" json:"node_id,omitempty"` + NodeId string `protobuf:"bytes,1,opt,name=node_id,json=nodeId,proto3" json:"node_id,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *CertificatesRequest) Reset() { *m = CertificatesRequest{} } +func (m *CertificatesRequest) String() string { return proto.CompactTextString(m) } +func (*CertificatesRequest) ProtoMessage() {} +func (*CertificatesRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{0} +} +func (m *CertificatesRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *CertificatesRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *CertificatesRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_CertificatesRequest.Merge(dst, src) +} +func (m *CertificatesRequest) XXX_Size() int { + return m.Size() +} +func (m *CertificatesRequest) XXX_DiscardUnknown() { + xxx_messageInfo_CertificatesRequest.DiscardUnknown(m) } -func (m *CertificatesRequest) Reset() { *m = CertificatesRequest{} } -func (m *CertificatesRequest) String() string { return proto.CompactTextString(m) } -func (*CertificatesRequest) ProtoMessage() {} -func (*CertificatesRequest) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{0} } +var xxx_messageInfo_CertificatesRequest proto.InternalMessageInfo type CertificateDetails struct { Type CertificateDetails_CertificateType `protobuf:"varint,1,opt,name=type,proto3,enum=cockroach.server.serverpb.CertificateDetails_CertificateType" json:"type,omitempty"` @@ -154,42 +192,118 @@ type CertificateDetails struct { ErrorMessage string `protobuf:"bytes,2,opt,name=error_message,json=errorMessage,proto3" json:"error_message,omitempty"` // data is the raw file contents of the certificate. This means PEM-encoded // DER data. - Data []byte `protobuf:"bytes,3,opt,name=data,proto3" json:"data,omitempty"` - Fields []CertificateDetails_Fields `protobuf:"bytes,4,rep,name=fields" json:"fields"` + Data []byte `protobuf:"bytes,3,opt,name=data,proto3" json:"data,omitempty"` + Fields []CertificateDetails_Fields `protobuf:"bytes,4,rep,name=fields,proto3" json:"fields"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *CertificateDetails) Reset() { *m = CertificateDetails{} } +func (m *CertificateDetails) String() string { return proto.CompactTextString(m) } +func (*CertificateDetails) ProtoMessage() {} +func (*CertificateDetails) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{1} +} +func (m *CertificateDetails) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *CertificateDetails) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *CertificateDetails) XXX_Merge(src proto.Message) { + xxx_messageInfo_CertificateDetails.Merge(dst, src) +} +func (m *CertificateDetails) XXX_Size() int { + return m.Size() +} +func (m *CertificateDetails) XXX_DiscardUnknown() { + xxx_messageInfo_CertificateDetails.DiscardUnknown(m) } -func (m *CertificateDetails) Reset() { *m = CertificateDetails{} } -func (m *CertificateDetails) String() string { return proto.CompactTextString(m) } -func (*CertificateDetails) ProtoMessage() {} -func (*CertificateDetails) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{1} } +var xxx_messageInfo_CertificateDetails proto.InternalMessageInfo type CertificateDetails_Fields struct { - Issuer string `protobuf:"bytes,1,opt,name=issuer,proto3" json:"issuer,omitempty"` - Subject string `protobuf:"bytes,2,opt,name=subject,proto3" json:"subject,omitempty"` - ValidFrom int64 `protobuf:"varint,3,opt,name=valid_from,json=validFrom,proto3" json:"valid_from,omitempty"` - ValidUntil int64 `protobuf:"varint,4,opt,name=valid_until,json=validUntil,proto3" json:"valid_until,omitempty"` - Addresses []string `protobuf:"bytes,5,rep,name=addresses" json:"addresses,omitempty"` - SignatureAlgorithm string `protobuf:"bytes,6,opt,name=signature_algorithm,json=signatureAlgorithm,proto3" json:"signature_algorithm,omitempty"` - PublicKey string `protobuf:"bytes,7,opt,name=public_key,json=publicKey,proto3" json:"public_key,omitempty"` - KeyUsage []string `protobuf:"bytes,8,rep,name=key_usage,json=keyUsage" json:"key_usage,omitempty"` - ExtendedKeyUsage []string `protobuf:"bytes,9,rep,name=extended_key_usage,json=extendedKeyUsage" json:"extended_key_usage,omitempty"` + Issuer string `protobuf:"bytes,1,opt,name=issuer,proto3" json:"issuer,omitempty"` + Subject string `protobuf:"bytes,2,opt,name=subject,proto3" json:"subject,omitempty"` + ValidFrom int64 `protobuf:"varint,3,opt,name=valid_from,json=validFrom,proto3" json:"valid_from,omitempty"` + ValidUntil int64 `protobuf:"varint,4,opt,name=valid_until,json=validUntil,proto3" json:"valid_until,omitempty"` + Addresses []string `protobuf:"bytes,5,rep,name=addresses,proto3" json:"addresses,omitempty"` + SignatureAlgorithm string `protobuf:"bytes,6,opt,name=signature_algorithm,json=signatureAlgorithm,proto3" json:"signature_algorithm,omitempty"` + PublicKey string `protobuf:"bytes,7,opt,name=public_key,json=publicKey,proto3" json:"public_key,omitempty"` + KeyUsage []string `protobuf:"bytes,8,rep,name=key_usage,json=keyUsage,proto3" json:"key_usage,omitempty"` + ExtendedKeyUsage []string `protobuf:"bytes,9,rep,name=extended_key_usage,json=extendedKeyUsage,proto3" json:"extended_key_usage,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *CertificateDetails_Fields) Reset() { *m = CertificateDetails_Fields{} } func (m *CertificateDetails_Fields) String() string { return proto.CompactTextString(m) } func (*CertificateDetails_Fields) ProtoMessage() {} func (*CertificateDetails_Fields) Descriptor() ([]byte, []int) { - return fileDescriptorStatus, []int{1, 0} + return fileDescriptor_status_0de29a344bd79652, []int{1, 0} +} +func (m *CertificateDetails_Fields) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *CertificateDetails_Fields) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil } +func (dst *CertificateDetails_Fields) XXX_Merge(src proto.Message) { + xxx_messageInfo_CertificateDetails_Fields.Merge(dst, src) +} +func (m *CertificateDetails_Fields) XXX_Size() int { + return m.Size() +} +func (m *CertificateDetails_Fields) XXX_DiscardUnknown() { + xxx_messageInfo_CertificateDetails_Fields.DiscardUnknown(m) +} + +var xxx_messageInfo_CertificateDetails_Fields proto.InternalMessageInfo type CertificatesResponse struct { - Certificates []CertificateDetails `protobuf:"bytes,1,rep,name=certificates" json:"certificates"` + Certificates []CertificateDetails `protobuf:"bytes,1,rep,name=certificates,proto3" json:"certificates"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *CertificatesResponse) Reset() { *m = CertificatesResponse{} } +func (m *CertificatesResponse) String() string { return proto.CompactTextString(m) } +func (*CertificatesResponse) ProtoMessage() {} +func (*CertificatesResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{2} +} +func (m *CertificatesResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *CertificatesResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *CertificatesResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_CertificatesResponse.Merge(dst, src) +} +func (m *CertificatesResponse) XXX_Size() int { + return m.Size() +} +func (m *CertificatesResponse) XXX_DiscardUnknown() { + xxx_messageInfo_CertificatesResponse.DiscardUnknown(m) } -func (m *CertificatesResponse) Reset() { *m = CertificatesResponse{} } -func (m *CertificatesResponse) String() string { return proto.CompactTextString(m) } -func (*CertificatesResponse) ProtoMessage() {} -func (*CertificatesResponse) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{2} } +var xxx_messageInfo_CertificatesResponse proto.InternalMessageInfo // DetailsRequest requests a nodes details. type DetailsRequest struct { @@ -198,100 +312,308 @@ type DetailsRequest struct { NodeId string `protobuf:"bytes,1,opt,name=node_id,json=nodeId,proto3" json:"node_id,omitempty"` // ready specifies whether the client wants to know whether a node is ready to // receive traffic. If a node is unready, an error will be returned. - Ready bool `protobuf:"varint,2,opt,name=ready,proto3" json:"ready,omitempty"` + Ready bool `protobuf:"varint,2,opt,name=ready,proto3" json:"ready,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *DetailsRequest) Reset() { *m = DetailsRequest{} } +func (m *DetailsRequest) String() string { return proto.CompactTextString(m) } +func (*DetailsRequest) ProtoMessage() {} +func (*DetailsRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{3} +} +func (m *DetailsRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *DetailsRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *DetailsRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_DetailsRequest.Merge(dst, src) +} +func (m *DetailsRequest) XXX_Size() int { + return m.Size() +} +func (m *DetailsRequest) XXX_DiscardUnknown() { + xxx_messageInfo_DetailsRequest.DiscardUnknown(m) } -func (m *DetailsRequest) Reset() { *m = DetailsRequest{} } -func (m *DetailsRequest) String() string { return proto.CompactTextString(m) } -func (*DetailsRequest) ProtoMessage() {} -func (*DetailsRequest) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{3} } +var xxx_messageInfo_DetailsRequest proto.InternalMessageInfo // SystemInfo contains information about the host system. type SystemInfo struct { // system_info is the output from `uname -a` SystemInfo string `protobuf:"bytes,1,opt,name=system_info,json=systemInfo,proto3" json:"system_info,omitempty"` // kernel_info is the output from `uname -r`. - KernelInfo string `protobuf:"bytes,2,opt,name=kernel_info,json=kernelInfo,proto3" json:"kernel_info,omitempty"` + KernelInfo string `protobuf:"bytes,2,opt,name=kernel_info,json=kernelInfo,proto3" json:"kernel_info,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *SystemInfo) Reset() { *m = SystemInfo{} } +func (m *SystemInfo) String() string { return proto.CompactTextString(m) } +func (*SystemInfo) ProtoMessage() {} +func (*SystemInfo) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{4} +} +func (m *SystemInfo) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SystemInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *SystemInfo) XXX_Merge(src proto.Message) { + xxx_messageInfo_SystemInfo.Merge(dst, src) +} +func (m *SystemInfo) XXX_Size() int { + return m.Size() +} +func (m *SystemInfo) XXX_DiscardUnknown() { + xxx_messageInfo_SystemInfo.DiscardUnknown(m) } -func (m *SystemInfo) Reset() { *m = SystemInfo{} } -func (m *SystemInfo) String() string { return proto.CompactTextString(m) } -func (*SystemInfo) ProtoMessage() {} -func (*SystemInfo) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{4} } +var xxx_messageInfo_SystemInfo proto.InternalMessageInfo type DetailsResponse struct { - NodeID github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,1,opt,name=node_id,json=nodeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"node_id,omitempty"` - Address cockroach_util.UnresolvedAddr `protobuf:"bytes,2,opt,name=address" json:"address"` - BuildInfo cockroach_build.Info `protobuf:"bytes,3,opt,name=build_info,json=buildInfo" json:"build_info"` - SystemInfo SystemInfo `protobuf:"bytes,4,opt,name=system_info,json=systemInfo" json:"system_info"` + NodeID github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,1,opt,name=node_id,json=nodeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"node_id,omitempty"` + Address util.UnresolvedAddr `protobuf:"bytes,2,opt,name=address,proto3" json:"address"` + BuildInfo build.Info `protobuf:"bytes,3,opt,name=build_info,json=buildInfo,proto3" json:"build_info"` + SystemInfo SystemInfo `protobuf:"bytes,4,opt,name=system_info,json=systemInfo,proto3" json:"system_info"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *DetailsResponse) Reset() { *m = DetailsResponse{} } +func (m *DetailsResponse) String() string { return proto.CompactTextString(m) } +func (*DetailsResponse) ProtoMessage() {} +func (*DetailsResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{5} +} +func (m *DetailsResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *DetailsResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *DetailsResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_DetailsResponse.Merge(dst, src) +} +func (m *DetailsResponse) XXX_Size() int { + return m.Size() +} +func (m *DetailsResponse) XXX_DiscardUnknown() { + xxx_messageInfo_DetailsResponse.DiscardUnknown(m) } -func (m *DetailsResponse) Reset() { *m = DetailsResponse{} } -func (m *DetailsResponse) String() string { return proto.CompactTextString(m) } -func (*DetailsResponse) ProtoMessage() {} -func (*DetailsResponse) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{5} } +var xxx_messageInfo_DetailsResponse proto.InternalMessageInfo type NodesRequest struct { + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *NodesRequest) Reset() { *m = NodesRequest{} } +func (m *NodesRequest) String() string { return proto.CompactTextString(m) } +func (*NodesRequest) ProtoMessage() {} +func (*NodesRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{6} +} +func (m *NodesRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *NodesRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *NodesRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_NodesRequest.Merge(dst, src) +} +func (m *NodesRequest) XXX_Size() int { + return m.Size() +} +func (m *NodesRequest) XXX_DiscardUnknown() { + xxx_messageInfo_NodesRequest.DiscardUnknown(m) } -func (m *NodesRequest) Reset() { *m = NodesRequest{} } -func (m *NodesRequest) String() string { return proto.CompactTextString(m) } -func (*NodesRequest) ProtoMessage() {} -func (*NodesRequest) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{6} } +var xxx_messageInfo_NodesRequest proto.InternalMessageInfo type NodesResponse struct { - Nodes []cockroach_server_status_statuspb.NodeStatus `protobuf:"bytes,1,rep,name=nodes" json:"nodes"` + Nodes []statuspb.NodeStatus `protobuf:"bytes,1,rep,name=nodes,proto3" json:"nodes"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *NodesResponse) Reset() { *m = NodesResponse{} } +func (m *NodesResponse) String() string { return proto.CompactTextString(m) } +func (*NodesResponse) ProtoMessage() {} +func (*NodesResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{7} +} +func (m *NodesResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *NodesResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *NodesResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_NodesResponse.Merge(dst, src) +} +func (m *NodesResponse) XXX_Size() int { + return m.Size() +} +func (m *NodesResponse) XXX_DiscardUnknown() { + xxx_messageInfo_NodesResponse.DiscardUnknown(m) } -func (m *NodesResponse) Reset() { *m = NodesResponse{} } -func (m *NodesResponse) String() string { return proto.CompactTextString(m) } -func (*NodesResponse) ProtoMessage() {} -func (*NodesResponse) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{7} } +var xxx_messageInfo_NodesResponse proto.InternalMessageInfo type NodeRequest struct { // node_id is a string so that "local" can be used to specify that no // forwarding is necessary. - NodeId string `protobuf:"bytes,1,opt,name=node_id,json=nodeId,proto3" json:"node_id,omitempty"` + NodeId string `protobuf:"bytes,1,opt,name=node_id,json=nodeId,proto3" json:"node_id,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *NodeRequest) Reset() { *m = NodeRequest{} } +func (m *NodeRequest) String() string { return proto.CompactTextString(m) } +func (*NodeRequest) ProtoMessage() {} +func (*NodeRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{8} +} +func (m *NodeRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *NodeRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *NodeRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_NodeRequest.Merge(dst, src) +} +func (m *NodeRequest) XXX_Size() int { + return m.Size() +} +func (m *NodeRequest) XXX_DiscardUnknown() { + xxx_messageInfo_NodeRequest.DiscardUnknown(m) } -func (m *NodeRequest) Reset() { *m = NodeRequest{} } -func (m *NodeRequest) String() string { return proto.CompactTextString(m) } -func (*NodeRequest) ProtoMessage() {} -func (*NodeRequest) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{8} } +var xxx_messageInfo_NodeRequest proto.InternalMessageInfo // RaftState gives internal details about a Raft group's state. // Closely mirrors the upstream definitions in github.com/etcd-io/etcd/raft. type RaftState struct { ReplicaID uint64 `protobuf:"varint,1,opt,name=replica_id,json=replicaId,proto3" json:"replica_id,omitempty"` - HardState raftpb.HardState `protobuf:"bytes,2,opt,name=hard_state,json=hardState" json:"hard_state"` + HardState raftpb.HardState `protobuf:"bytes,2,opt,name=hard_state,json=hardState,proto3" json:"hard_state"` // Lead is part of Raft's SoftState. Lead uint64 `protobuf:"varint,3,opt,name=lead,proto3" json:"lead,omitempty"` // State is part of Raft's SoftState. // It's not an enum because this is primarily for ui consumption and there // are issues associated with them. - State string `protobuf:"bytes,4,opt,name=state,proto3" json:"state,omitempty"` - Applied uint64 `protobuf:"varint,5,opt,name=applied,proto3" json:"applied,omitempty"` - Progress map[uint64]RaftState_Progress `protobuf:"bytes,6,rep,name=progress" json:"progress" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value"` - LeadTransferee uint64 `protobuf:"varint,7,opt,name=lead_transferee,json=leadTransferee,proto3" json:"lead_transferee,omitempty"` + State string `protobuf:"bytes,4,opt,name=state,proto3" json:"state,omitempty"` + Applied uint64 `protobuf:"varint,5,opt,name=applied,proto3" json:"applied,omitempty"` + Progress map[uint64]RaftState_Progress `protobuf:"bytes,6,rep,name=progress,proto3" json:"progress" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + LeadTransferee uint64 `protobuf:"varint,7,opt,name=lead_transferee,json=leadTransferee,proto3" json:"lead_transferee,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *RaftState) Reset() { *m = RaftState{} } +func (m *RaftState) String() string { return proto.CompactTextString(m) } +func (*RaftState) ProtoMessage() {} +func (*RaftState) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{9} +} +func (m *RaftState) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RaftState) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RaftState) XXX_Merge(src proto.Message) { + xxx_messageInfo_RaftState.Merge(dst, src) +} +func (m *RaftState) XXX_Size() int { + return m.Size() +} +func (m *RaftState) XXX_DiscardUnknown() { + xxx_messageInfo_RaftState.DiscardUnknown(m) } -func (m *RaftState) Reset() { *m = RaftState{} } -func (m *RaftState) String() string { return proto.CompactTextString(m) } -func (*RaftState) ProtoMessage() {} -func (*RaftState) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{9} } +var xxx_messageInfo_RaftState proto.InternalMessageInfo type RaftState_Progress struct { - Match uint64 `protobuf:"varint,1,opt,name=match,proto3" json:"match,omitempty"` - Next uint64 `protobuf:"varint,2,opt,name=next,proto3" json:"next,omitempty"` - State string `protobuf:"bytes,3,opt,name=state,proto3" json:"state,omitempty"` - Paused bool `protobuf:"varint,4,opt,name=paused,proto3" json:"paused,omitempty"` - PendingSnapshot uint64 `protobuf:"varint,5,opt,name=pending_snapshot,json=pendingSnapshot,proto3" json:"pending_snapshot,omitempty"` + Match uint64 `protobuf:"varint,1,opt,name=match,proto3" json:"match,omitempty"` + Next uint64 `protobuf:"varint,2,opt,name=next,proto3" json:"next,omitempty"` + State string `protobuf:"bytes,3,opt,name=state,proto3" json:"state,omitempty"` + Paused bool `protobuf:"varint,4,opt,name=paused,proto3" json:"paused,omitempty"` + PendingSnapshot uint64 `protobuf:"varint,5,opt,name=pending_snapshot,json=pendingSnapshot,proto3" json:"pending_snapshot,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *RaftState_Progress) Reset() { *m = RaftState_Progress{} } +func (m *RaftState_Progress) String() string { return proto.CompactTextString(m) } +func (*RaftState_Progress) ProtoMessage() {} +func (*RaftState_Progress) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{9, 0} +} +func (m *RaftState_Progress) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RaftState_Progress) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RaftState_Progress) XXX_Merge(src proto.Message) { + xxx_messageInfo_RaftState_Progress.Merge(dst, src) +} +func (m *RaftState_Progress) XXX_Size() int { + return m.Size() +} +func (m *RaftState_Progress) XXX_DiscardUnknown() { + xxx_messageInfo_RaftState_Progress.DiscardUnknown(m) } -func (m *RaftState_Progress) Reset() { *m = RaftState_Progress{} } -func (m *RaftState_Progress) String() string { return proto.CompactTextString(m) } -func (*RaftState_Progress) ProtoMessage() {} -func (*RaftState_Progress) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{9, 0} } +var xxx_messageInfo_RaftState_Progress proto.InternalMessageInfo type RangeProblems struct { Unavailable bool `protobuf:"varint,1,opt,name=unavailable,proto3" json:"unavailable,omitempty"` @@ -305,238 +627,784 @@ type RangeProblems struct { // list of replicas that are ticking, warn about it. QuiescentEqualsTicking bool `protobuf:"varint,6,opt,name=quiescent_equals_ticking,json=quiescentEqualsTicking,proto3" json:"quiescent_equals_ticking,omitempty"` // When the raft log is too large, it can be a symptom of other issues. - RaftLogTooLarge bool `protobuf:"varint,7,opt,name=raft_log_too_large,json=raftLogTooLarge,proto3" json:"raft_log_too_large,omitempty"` + RaftLogTooLarge bool `protobuf:"varint,7,opt,name=raft_log_too_large,json=raftLogTooLarge,proto3" json:"raft_log_too_large,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *RangeProblems) Reset() { *m = RangeProblems{} } +func (m *RangeProblems) String() string { return proto.CompactTextString(m) } +func (*RangeProblems) ProtoMessage() {} +func (*RangeProblems) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{10} +} +func (m *RangeProblems) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RangeProblems) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RangeProblems) XXX_Merge(src proto.Message) { + xxx_messageInfo_RangeProblems.Merge(dst, src) +} +func (m *RangeProblems) XXX_Size() int { + return m.Size() +} +func (m *RangeProblems) XXX_DiscardUnknown() { + xxx_messageInfo_RangeProblems.DiscardUnknown(m) } -func (m *RangeProblems) Reset() { *m = RangeProblems{} } -func (m *RangeProblems) String() string { return proto.CompactTextString(m) } -func (*RangeProblems) ProtoMessage() {} -func (*RangeProblems) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{10} } +var xxx_messageInfo_RangeProblems proto.InternalMessageInfo type RangeStatistics struct { // Note that queries per second will only be known by the leaseholder. // All other replicas will report it as 0. - QueriesPerSecond float64 `protobuf:"fixed64,1,opt,name=queries_per_second,json=queriesPerSecond,proto3" json:"queries_per_second,omitempty"` - WritesPerSecond float64 `protobuf:"fixed64,2,opt,name=writes_per_second,json=writesPerSecond,proto3" json:"writes_per_second,omitempty"` + QueriesPerSecond float64 `protobuf:"fixed64,1,opt,name=queries_per_second,json=queriesPerSecond,proto3" json:"queries_per_second,omitempty"` + WritesPerSecond float64 `protobuf:"fixed64,2,opt,name=writes_per_second,json=writesPerSecond,proto3" json:"writes_per_second,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *RangeStatistics) Reset() { *m = RangeStatistics{} } +func (m *RangeStatistics) String() string { return proto.CompactTextString(m) } +func (*RangeStatistics) ProtoMessage() {} +func (*RangeStatistics) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{11} +} +func (m *RangeStatistics) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RangeStatistics) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RangeStatistics) XXX_Merge(src proto.Message) { + xxx_messageInfo_RangeStatistics.Merge(dst, src) +} +func (m *RangeStatistics) XXX_Size() int { + return m.Size() +} +func (m *RangeStatistics) XXX_DiscardUnknown() { + xxx_messageInfo_RangeStatistics.DiscardUnknown(m) } -func (m *RangeStatistics) Reset() { *m = RangeStatistics{} } -func (m *RangeStatistics) String() string { return proto.CompactTextString(m) } -func (*RangeStatistics) ProtoMessage() {} -func (*RangeStatistics) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{11} } +var xxx_messageInfo_RangeStatistics proto.InternalMessageInfo type PrettySpan struct { - StartKey string `protobuf:"bytes,1,opt,name=start_key,json=startKey,proto3" json:"start_key,omitempty"` - EndKey string `protobuf:"bytes,2,opt,name=end_key,json=endKey,proto3" json:"end_key,omitempty"` + StartKey string `protobuf:"bytes,1,opt,name=start_key,json=startKey,proto3" json:"start_key,omitempty"` + EndKey string `protobuf:"bytes,2,opt,name=end_key,json=endKey,proto3" json:"end_key,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *PrettySpan) Reset() { *m = PrettySpan{} } +func (m *PrettySpan) String() string { return proto.CompactTextString(m) } +func (*PrettySpan) ProtoMessage() {} +func (*PrettySpan) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{12} +} +func (m *PrettySpan) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *PrettySpan) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *PrettySpan) XXX_Merge(src proto.Message) { + xxx_messageInfo_PrettySpan.Merge(dst, src) +} +func (m *PrettySpan) XXX_Size() int { + return m.Size() +} +func (m *PrettySpan) XXX_DiscardUnknown() { + xxx_messageInfo_PrettySpan.DiscardUnknown(m) } -func (m *PrettySpan) Reset() { *m = PrettySpan{} } -func (m *PrettySpan) String() string { return proto.CompactTextString(m) } -func (*PrettySpan) ProtoMessage() {} -func (*PrettySpan) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{12} } +var xxx_messageInfo_PrettySpan proto.InternalMessageInfo type RangeInfo struct { - Span PrettySpan `protobuf:"bytes,1,opt,name=span" json:"span"` - RaftState RaftState `protobuf:"bytes,2,opt,name=raft_state,json=raftState" json:"raft_state"` - State cockroach_storage_storagepb.RangeInfo `protobuf:"bytes,4,opt,name=state" json:"state"` - SourceNodeID github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,5,opt,name=source_node_id,json=sourceNodeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"source_node_id,omitempty"` - SourceStoreID github_com_cockroachdb_cockroach_pkg_roachpb.StoreID `protobuf:"varint,6,opt,name=source_store_id,json=sourceStoreId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.StoreID" json:"source_store_id,omitempty"` - ErrorMessage string `protobuf:"bytes,7,opt,name=error_message,json=errorMessage,proto3" json:"error_message,omitempty"` - LeaseHistory []cockroach_roachpb1.Lease `protobuf:"bytes,8,rep,name=lease_history,json=leaseHistory" json:"lease_history"` - Problems RangeProblems `protobuf:"bytes,9,opt,name=problems" json:"problems"` - Stats RangeStatistics `protobuf:"bytes,10,opt,name=stats" json:"stats"` - LatchesLocal cockroach_storage_storagepb.LatchManagerInfo `protobuf:"bytes,11,opt,name=latches_local,json=latchesLocal" json:"latches_local"` - LatchesGlobal cockroach_storage_storagepb.LatchManagerInfo `protobuf:"bytes,12,opt,name=latches_global,json=latchesGlobal" json:"latches_global"` - LeaseStatus cockroach_storage1.LeaseStatus `protobuf:"bytes,13,opt,name=lease_status,json=leaseStatus" json:"lease_status"` - Quiescent bool `protobuf:"varint,14,opt,name=quiescent,proto3" json:"quiescent,omitempty"` - Ticking bool `protobuf:"varint,15,opt,name=ticking,proto3" json:"ticking,omitempty"` -} - -func (m *RangeInfo) Reset() { *m = RangeInfo{} } -func (m *RangeInfo) String() string { return proto.CompactTextString(m) } -func (*RangeInfo) ProtoMessage() {} -func (*RangeInfo) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{13} } + Span PrettySpan `protobuf:"bytes,1,opt,name=span,proto3" json:"span"` + RaftState RaftState `protobuf:"bytes,2,opt,name=raft_state,json=raftState,proto3" json:"raft_state"` + State storagepb.RangeInfo `protobuf:"bytes,4,opt,name=state,proto3" json:"state"` + SourceNodeID github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,5,opt,name=source_node_id,json=sourceNodeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"source_node_id,omitempty"` + SourceStoreID github_com_cockroachdb_cockroach_pkg_roachpb.StoreID `protobuf:"varint,6,opt,name=source_store_id,json=sourceStoreId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.StoreID" json:"source_store_id,omitempty"` + ErrorMessage string `protobuf:"bytes,7,opt,name=error_message,json=errorMessage,proto3" json:"error_message,omitempty"` + LeaseHistory []roachpb.Lease `protobuf:"bytes,8,rep,name=lease_history,json=leaseHistory,proto3" json:"lease_history"` + Problems RangeProblems `protobuf:"bytes,9,opt,name=problems,proto3" json:"problems"` + Stats RangeStatistics `protobuf:"bytes,10,opt,name=stats,proto3" json:"stats"` + LatchesLocal storagepb.LatchManagerInfo `protobuf:"bytes,11,opt,name=latches_local,json=latchesLocal,proto3" json:"latches_local"` + LatchesGlobal storagepb.LatchManagerInfo `protobuf:"bytes,12,opt,name=latches_global,json=latchesGlobal,proto3" json:"latches_global"` + LeaseStatus storagepb.LeaseStatus `protobuf:"bytes,13,opt,name=lease_status,json=leaseStatus,proto3" json:"lease_status"` + Quiescent bool `protobuf:"varint,14,opt,name=quiescent,proto3" json:"quiescent,omitempty"` + Ticking bool `protobuf:"varint,15,opt,name=ticking,proto3" json:"ticking,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_status_0de29a344bd79652, []int{13} +} +func (m *RangeInfo) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RangeInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RangeInfo) XXX_Merge(src proto.Message) { + xxx_messageInfo_RangeInfo.Merge(dst, src) +} +func (m *RangeInfo) XXX_Size() int { + return m.Size() +} +func (m *RangeInfo) XXX_DiscardUnknown() { + xxx_messageInfo_RangeInfo.DiscardUnknown(m) +} + +var xxx_messageInfo_RangeInfo proto.InternalMessageInfo type RangesRequest struct { // node_id is a string so that "local" can be used to specify that no // forwarding is necessary. - NodeId string `protobuf:"bytes,1,opt,name=node_id,json=nodeId,proto3" json:"node_id,omitempty"` - RangeIDs []github_com_cockroachdb_cockroach_pkg_roachpb.RangeID `protobuf:"varint,2,rep,packed,name=range_ids,json=rangeIds,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" json:"range_ids,omitempty"` + NodeId string `protobuf:"bytes,1,opt,name=node_id,json=nodeId,proto3" json:"node_id,omitempty"` + RangeIDs []github_com_cockroachdb_cockroach_pkg_roachpb.RangeID `protobuf:"varint,2,rep,packed,name=range_ids,json=rangeIds,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" json:"range_ids,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *RangesRequest) Reset() { *m = RangesRequest{} } +func (m *RangesRequest) String() string { return proto.CompactTextString(m) } +func (*RangesRequest) ProtoMessage() {} +func (*RangesRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{14} +} +func (m *RangesRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RangesRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RangesRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_RangesRequest.Merge(dst, src) +} +func (m *RangesRequest) XXX_Size() int { + return m.Size() +} +func (m *RangesRequest) XXX_DiscardUnknown() { + xxx_messageInfo_RangesRequest.DiscardUnknown(m) } -func (m *RangesRequest) Reset() { *m = RangesRequest{} } -func (m *RangesRequest) String() string { return proto.CompactTextString(m) } -func (*RangesRequest) ProtoMessage() {} -func (*RangesRequest) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{14} } +var xxx_messageInfo_RangesRequest proto.InternalMessageInfo type RangesResponse struct { - Ranges []RangeInfo `protobuf:"bytes,1,rep,name=ranges" json:"ranges"` + Ranges []RangeInfo `protobuf:"bytes,1,rep,name=ranges,proto3" json:"ranges"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *RangesResponse) Reset() { *m = RangesResponse{} } +func (m *RangesResponse) String() string { return proto.CompactTextString(m) } +func (*RangesResponse) ProtoMessage() {} +func (*RangesResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{15} +} +func (m *RangesResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RangesResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RangesResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_RangesResponse.Merge(dst, src) +} +func (m *RangesResponse) XXX_Size() int { + return m.Size() +} +func (m *RangesResponse) XXX_DiscardUnknown() { + xxx_messageInfo_RangesResponse.DiscardUnknown(m) } -func (m *RangesResponse) Reset() { *m = RangesResponse{} } -func (m *RangesResponse) String() string { return proto.CompactTextString(m) } -func (*RangesResponse) ProtoMessage() {} -func (*RangesResponse) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{15} } +var xxx_messageInfo_RangesResponse proto.InternalMessageInfo type GossipRequest struct { // node_id is a string so that "local" can be used to specify that no // forwarding is necessary. - NodeId string `protobuf:"bytes,1,opt,name=node_id,json=nodeId,proto3" json:"node_id,omitempty"` + NodeId string `protobuf:"bytes,1,opt,name=node_id,json=nodeId,proto3" json:"node_id,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *GossipRequest) Reset() { *m = GossipRequest{} } +func (m *GossipRequest) String() string { return proto.CompactTextString(m) } +func (*GossipRequest) ProtoMessage() {} +func (*GossipRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{16} +} +func (m *GossipRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *GossipRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *GossipRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_GossipRequest.Merge(dst, src) +} +func (m *GossipRequest) XXX_Size() int { + return m.Size() +} +func (m *GossipRequest) XXX_DiscardUnknown() { + xxx_messageInfo_GossipRequest.DiscardUnknown(m) } -func (m *GossipRequest) Reset() { *m = GossipRequest{} } -func (m *GossipRequest) String() string { return proto.CompactTextString(m) } -func (*GossipRequest) ProtoMessage() {} -func (*GossipRequest) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{16} } +var xxx_messageInfo_GossipRequest proto.InternalMessageInfo type TraceEvent struct { - Time time.Time `protobuf:"bytes,1,opt,name=time,stdtime" json:"time"` - Message string `protobuf:"bytes,2,opt,name=message,proto3" json:"message,omitempty"` + Time time.Time `protobuf:"bytes,1,opt,name=time,proto3,stdtime" json:"time"` + Message string `protobuf:"bytes,2,opt,name=message,proto3" json:"message,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *TraceEvent) Reset() { *m = TraceEvent{} } +func (m *TraceEvent) String() string { return proto.CompactTextString(m) } +func (*TraceEvent) ProtoMessage() {} +func (*TraceEvent) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{17} +} +func (m *TraceEvent) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *TraceEvent) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *TraceEvent) XXX_Merge(src proto.Message) { + xxx_messageInfo_TraceEvent.Merge(dst, src) +} +func (m *TraceEvent) XXX_Size() int { + return m.Size() +} +func (m *TraceEvent) XXX_DiscardUnknown() { + xxx_messageInfo_TraceEvent.DiscardUnknown(m) } -func (m *TraceEvent) Reset() { *m = TraceEvent{} } -func (m *TraceEvent) String() string { return proto.CompactTextString(m) } -func (*TraceEvent) ProtoMessage() {} -func (*TraceEvent) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{17} } +var xxx_messageInfo_TraceEvent proto.InternalMessageInfo type AllocatorDryRun struct { - RangeID github_com_cockroachdb_cockroach_pkg_roachpb.RangeID `protobuf:"varint,1,opt,name=range_id,json=rangeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" json:"range_id,omitempty"` - Events []*TraceEvent `protobuf:"bytes,2,rep,name=events" json:"events,omitempty"` + RangeID github_com_cockroachdb_cockroach_pkg_roachpb.RangeID `protobuf:"varint,1,opt,name=range_id,json=rangeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" json:"range_id,omitempty"` + Events []*TraceEvent `protobuf:"bytes,2,rep,name=events,proto3" json:"events,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *AllocatorDryRun) Reset() { *m = AllocatorDryRun{} } +func (m *AllocatorDryRun) String() string { return proto.CompactTextString(m) } +func (*AllocatorDryRun) ProtoMessage() {} +func (*AllocatorDryRun) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{18} +} +func (m *AllocatorDryRun) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *AllocatorDryRun) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *AllocatorDryRun) XXX_Merge(src proto.Message) { + xxx_messageInfo_AllocatorDryRun.Merge(dst, src) +} +func (m *AllocatorDryRun) XXX_Size() int { + return m.Size() +} +func (m *AllocatorDryRun) XXX_DiscardUnknown() { + xxx_messageInfo_AllocatorDryRun.DiscardUnknown(m) } -func (m *AllocatorDryRun) Reset() { *m = AllocatorDryRun{} } -func (m *AllocatorDryRun) String() string { return proto.CompactTextString(m) } -func (*AllocatorDryRun) ProtoMessage() {} -func (*AllocatorDryRun) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{18} } +var xxx_messageInfo_AllocatorDryRun proto.InternalMessageInfo type AllocatorRangeRequest struct { - RangeId int64 `protobuf:"varint,1,opt,name=range_id,json=rangeId,proto3" json:"range_id,omitempty"` + RangeId int64 `protobuf:"varint,1,opt,name=range_id,json=rangeId,proto3" json:"range_id,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *AllocatorRangeRequest) Reset() { *m = AllocatorRangeRequest{} } +func (m *AllocatorRangeRequest) String() string { return proto.CompactTextString(m) } +func (*AllocatorRangeRequest) ProtoMessage() {} +func (*AllocatorRangeRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{19} +} +func (m *AllocatorRangeRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *AllocatorRangeRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *AllocatorRangeRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_AllocatorRangeRequest.Merge(dst, src) +} +func (m *AllocatorRangeRequest) XXX_Size() int { + return m.Size() +} +func (m *AllocatorRangeRequest) XXX_DiscardUnknown() { + xxx_messageInfo_AllocatorRangeRequest.DiscardUnknown(m) } -func (m *AllocatorRangeRequest) Reset() { *m = AllocatorRangeRequest{} } -func (m *AllocatorRangeRequest) String() string { return proto.CompactTextString(m) } -func (*AllocatorRangeRequest) ProtoMessage() {} -func (*AllocatorRangeRequest) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{19} } +var xxx_messageInfo_AllocatorRangeRequest proto.InternalMessageInfo type AllocatorRangeResponse struct { // The NodeID of the store whose dry run is returned. Only the leaseholder // for a given range will do an allocator dry run for it. - NodeID github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,1,opt,name=node_id,json=nodeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"node_id,omitempty"` - DryRun *AllocatorDryRun `protobuf:"bytes,2,opt,name=dry_run,json=dryRun" json:"dry_run,omitempty"` + NodeID github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,1,opt,name=node_id,json=nodeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"node_id,omitempty"` + DryRun *AllocatorDryRun `protobuf:"bytes,2,opt,name=dry_run,json=dryRun,proto3" json:"dry_run,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *AllocatorRangeResponse) Reset() { *m = AllocatorRangeResponse{} } +func (m *AllocatorRangeResponse) String() string { return proto.CompactTextString(m) } +func (*AllocatorRangeResponse) ProtoMessage() {} +func (*AllocatorRangeResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{20} +} +func (m *AllocatorRangeResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *AllocatorRangeResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *AllocatorRangeResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_AllocatorRangeResponse.Merge(dst, src) +} +func (m *AllocatorRangeResponse) XXX_Size() int { + return m.Size() +} +func (m *AllocatorRangeResponse) XXX_DiscardUnknown() { + xxx_messageInfo_AllocatorRangeResponse.DiscardUnknown(m) } -func (m *AllocatorRangeResponse) Reset() { *m = AllocatorRangeResponse{} } -func (m *AllocatorRangeResponse) String() string { return proto.CompactTextString(m) } -func (*AllocatorRangeResponse) ProtoMessage() {} -func (*AllocatorRangeResponse) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{20} } +var xxx_messageInfo_AllocatorRangeResponse proto.InternalMessageInfo type AllocatorRequest struct { - NodeId string `protobuf:"bytes,1,opt,name=node_id,json=nodeId,proto3" json:"node_id,omitempty"` - RangeIDs []github_com_cockroachdb_cockroach_pkg_roachpb.RangeID `protobuf:"varint,2,rep,packed,name=range_ids,json=rangeIds,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" json:"range_ids,omitempty"` + NodeId string `protobuf:"bytes,1,opt,name=node_id,json=nodeId,proto3" json:"node_id,omitempty"` + RangeIDs []github_com_cockroachdb_cockroach_pkg_roachpb.RangeID `protobuf:"varint,2,rep,packed,name=range_ids,json=rangeIds,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" json:"range_ids,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *AllocatorRequest) Reset() { *m = AllocatorRequest{} } +func (m *AllocatorRequest) String() string { return proto.CompactTextString(m) } +func (*AllocatorRequest) ProtoMessage() {} +func (*AllocatorRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{21} +} +func (m *AllocatorRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *AllocatorRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *AllocatorRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_AllocatorRequest.Merge(dst, src) +} +func (m *AllocatorRequest) XXX_Size() int { + return m.Size() +} +func (m *AllocatorRequest) XXX_DiscardUnknown() { + xxx_messageInfo_AllocatorRequest.DiscardUnknown(m) } -func (m *AllocatorRequest) Reset() { *m = AllocatorRequest{} } -func (m *AllocatorRequest) String() string { return proto.CompactTextString(m) } -func (*AllocatorRequest) ProtoMessage() {} -func (*AllocatorRequest) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{21} } +var xxx_messageInfo_AllocatorRequest proto.InternalMessageInfo type AllocatorResponse struct { - DryRuns []*AllocatorDryRun `protobuf:"bytes,1,rep,name=dry_runs,json=dryRuns" json:"dry_runs,omitempty"` + DryRuns []*AllocatorDryRun `protobuf:"bytes,1,rep,name=dry_runs,json=dryRuns,proto3" json:"dry_runs,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *AllocatorResponse) Reset() { *m = AllocatorResponse{} } +func (m *AllocatorResponse) String() string { return proto.CompactTextString(m) } +func (*AllocatorResponse) ProtoMessage() {} +func (*AllocatorResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{22} +} +func (m *AllocatorResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *AllocatorResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *AllocatorResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_AllocatorResponse.Merge(dst, src) +} +func (m *AllocatorResponse) XXX_Size() int { + return m.Size() +} +func (m *AllocatorResponse) XXX_DiscardUnknown() { + xxx_messageInfo_AllocatorResponse.DiscardUnknown(m) } -func (m *AllocatorResponse) Reset() { *m = AllocatorResponse{} } -func (m *AllocatorResponse) String() string { return proto.CompactTextString(m) } -func (*AllocatorResponse) ProtoMessage() {} -func (*AllocatorResponse) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{22} } +var xxx_messageInfo_AllocatorResponse proto.InternalMessageInfo type JSONResponse struct { - Data []byte `protobuf:"bytes,1,opt,name=data,proto3" json:"data,omitempty"` + Data []byte `protobuf:"bytes,1,opt,name=data,proto3" json:"data,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *JSONResponse) Reset() { *m = JSONResponse{} } +func (m *JSONResponse) String() string { return proto.CompactTextString(m) } +func (*JSONResponse) ProtoMessage() {} +func (*JSONResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{23} +} +func (m *JSONResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *JSONResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *JSONResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_JSONResponse.Merge(dst, src) +} +func (m *JSONResponse) XXX_Size() int { + return m.Size() +} +func (m *JSONResponse) XXX_DiscardUnknown() { + xxx_messageInfo_JSONResponse.DiscardUnknown(m) } -func (m *JSONResponse) Reset() { *m = JSONResponse{} } -func (m *JSONResponse) String() string { return proto.CompactTextString(m) } -func (*JSONResponse) ProtoMessage() {} -func (*JSONResponse) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{23} } +var xxx_messageInfo_JSONResponse proto.InternalMessageInfo type LogsRequest struct { // node_id is a string so that "local" can be used to specify that no // forwarding is necessary. - NodeId string `protobuf:"bytes,1,opt,name=node_id,json=nodeId,proto3" json:"node_id,omitempty"` - Level string `protobuf:"bytes,2,opt,name=level,proto3" json:"level,omitempty"` - StartTime string `protobuf:"bytes,3,opt,name=start_time,json=startTime,proto3" json:"start_time,omitempty"` - EndTime string `protobuf:"bytes,4,opt,name=end_time,json=endTime,proto3" json:"end_time,omitempty"` - Max string `protobuf:"bytes,5,opt,name=max,proto3" json:"max,omitempty"` - Pattern string `protobuf:"bytes,6,opt,name=pattern,proto3" json:"pattern,omitempty"` -} + NodeId string `protobuf:"bytes,1,opt,name=node_id,json=nodeId,proto3" json:"node_id,omitempty"` + Level string `protobuf:"bytes,2,opt,name=level,proto3" json:"level,omitempty"` + StartTime string `protobuf:"bytes,3,opt,name=start_time,json=startTime,proto3" json:"start_time,omitempty"` + EndTime string `protobuf:"bytes,4,opt,name=end_time,json=endTime,proto3" json:"end_time,omitempty"` + Max string `protobuf:"bytes,5,opt,name=max,proto3" json:"max,omitempty"` + Pattern string `protobuf:"bytes,6,opt,name=pattern,proto3" json:"pattern,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *LogsRequest) Reset() { *m = LogsRequest{} } +func (m *LogsRequest) String() string { return proto.CompactTextString(m) } +func (*LogsRequest) ProtoMessage() {} +func (*LogsRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{24} +} +func (m *LogsRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *LogsRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *LogsRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_LogsRequest.Merge(dst, src) +} +func (m *LogsRequest) XXX_Size() int { + return m.Size() +} +func (m *LogsRequest) XXX_DiscardUnknown() { + xxx_messageInfo_LogsRequest.DiscardUnknown(m) +} -func (m *LogsRequest) Reset() { *m = LogsRequest{} } -func (m *LogsRequest) String() string { return proto.CompactTextString(m) } -func (*LogsRequest) ProtoMessage() {} -func (*LogsRequest) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{24} } +var xxx_messageInfo_LogsRequest proto.InternalMessageInfo type LogEntriesResponse struct { - Entries []cockroach_util_log.Entry `protobuf:"bytes,1,rep,name=entries" json:"entries"` + Entries []log.Entry `protobuf:"bytes,1,rep,name=entries,proto3" json:"entries"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *LogEntriesResponse) Reset() { *m = LogEntriesResponse{} } +func (m *LogEntriesResponse) String() string { return proto.CompactTextString(m) } +func (*LogEntriesResponse) ProtoMessage() {} +func (*LogEntriesResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{25} +} +func (m *LogEntriesResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *LogEntriesResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *LogEntriesResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_LogEntriesResponse.Merge(dst, src) +} +func (m *LogEntriesResponse) XXX_Size() int { + return m.Size() +} +func (m *LogEntriesResponse) XXX_DiscardUnknown() { + xxx_messageInfo_LogEntriesResponse.DiscardUnknown(m) } -func (m *LogEntriesResponse) Reset() { *m = LogEntriesResponse{} } -func (m *LogEntriesResponse) String() string { return proto.CompactTextString(m) } -func (*LogEntriesResponse) ProtoMessage() {} -func (*LogEntriesResponse) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{25} } +var xxx_messageInfo_LogEntriesResponse proto.InternalMessageInfo type LogFilesListRequest struct { // node_id is a string so that "local" can be used to specify that no // forwarding is necessary. - NodeId string `protobuf:"bytes,1,opt,name=node_id,json=nodeId,proto3" json:"node_id,omitempty"` + NodeId string `protobuf:"bytes,1,opt,name=node_id,json=nodeId,proto3" json:"node_id,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *LogFilesListRequest) Reset() { *m = LogFilesListRequest{} } -func (m *LogFilesListRequest) String() string { return proto.CompactTextString(m) } -func (*LogFilesListRequest) ProtoMessage() {} -func (*LogFilesListRequest) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{26} } +func (m *LogFilesListRequest) Reset() { *m = LogFilesListRequest{} } +func (m *LogFilesListRequest) String() string { return proto.CompactTextString(m) } +func (*LogFilesListRequest) ProtoMessage() {} +func (*LogFilesListRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{26} +} +func (m *LogFilesListRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *LogFilesListRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *LogFilesListRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_LogFilesListRequest.Merge(dst, src) +} +func (m *LogFilesListRequest) XXX_Size() int { + return m.Size() +} +func (m *LogFilesListRequest) XXX_DiscardUnknown() { + xxx_messageInfo_LogFilesListRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_LogFilesListRequest proto.InternalMessageInfo type LogFilesListResponse struct { - Files []cockroach_util_log.FileInfo `protobuf:"bytes,1,rep,name=files" json:"files"` + Files []log.FileInfo `protobuf:"bytes,1,rep,name=files,proto3" json:"files"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *LogFilesListResponse) Reset() { *m = LogFilesListResponse{} } -func (m *LogFilesListResponse) String() string { return proto.CompactTextString(m) } -func (*LogFilesListResponse) ProtoMessage() {} -func (*LogFilesListResponse) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{27} } +func (m *LogFilesListResponse) Reset() { *m = LogFilesListResponse{} } +func (m *LogFilesListResponse) String() string { return proto.CompactTextString(m) } +func (*LogFilesListResponse) ProtoMessage() {} +func (*LogFilesListResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{27} +} +func (m *LogFilesListResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *LogFilesListResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *LogFilesListResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_LogFilesListResponse.Merge(dst, src) +} +func (m *LogFilesListResponse) XXX_Size() int { + return m.Size() +} +func (m *LogFilesListResponse) XXX_DiscardUnknown() { + xxx_messageInfo_LogFilesListResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_LogFilesListResponse proto.InternalMessageInfo type LogFileRequest struct { // node_id is a string so that "local" can be used to specify that no // forwarding is necessary. - NodeId string `protobuf:"bytes,1,opt,name=node_id,json=nodeId,proto3" json:"node_id,omitempty"` - File string `protobuf:"bytes,2,opt,name=file,proto3" json:"file,omitempty"` + NodeId string `protobuf:"bytes,1,opt,name=node_id,json=nodeId,proto3" json:"node_id,omitempty"` + File string `protobuf:"bytes,2,opt,name=file,proto3" json:"file,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *LogFileRequest) Reset() { *m = LogFileRequest{} } -func (m *LogFileRequest) String() string { return proto.CompactTextString(m) } -func (*LogFileRequest) ProtoMessage() {} -func (*LogFileRequest) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{28} } +func (m *LogFileRequest) Reset() { *m = LogFileRequest{} } +func (m *LogFileRequest) String() string { return proto.CompactTextString(m) } +func (*LogFileRequest) ProtoMessage() {} +func (*LogFileRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{28} +} +func (m *LogFileRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *LogFileRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *LogFileRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_LogFileRequest.Merge(dst, src) +} +func (m *LogFileRequest) XXX_Size() int { + return m.Size() +} +func (m *LogFileRequest) XXX_DiscardUnknown() { + xxx_messageInfo_LogFileRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_LogFileRequest proto.InternalMessageInfo type StacksRequest struct { // node_id is a string so that "local" can be used to specify that no // forwarding is necessary. - NodeId string `protobuf:"bytes,1,opt,name=node_id,json=nodeId,proto3" json:"node_id,omitempty"` + NodeId string `protobuf:"bytes,1,opt,name=node_id,json=nodeId,proto3" json:"node_id,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *StacksRequest) Reset() { *m = StacksRequest{} } -func (m *StacksRequest) String() string { return proto.CompactTextString(m) } -func (*StacksRequest) ProtoMessage() {} -func (*StacksRequest) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{29} } +func (m *StacksRequest) Reset() { *m = StacksRequest{} } +func (m *StacksRequest) String() string { return proto.CompactTextString(m) } +func (*StacksRequest) ProtoMessage() {} +func (*StacksRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{29} +} +func (m *StacksRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *StacksRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *StacksRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_StacksRequest.Merge(dst, src) +} +func (m *StacksRequest) XXX_Size() int { + return m.Size() +} +func (m *StacksRequest) XXX_DiscardUnknown() { + xxx_messageInfo_StacksRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_StacksRequest proto.InternalMessageInfo type File struct { Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` FileSize int64 `protobuf:"varint,2,opt,name=file_size,json=fileSize,proto3" json:"file_size,omitempty"` // Contents may not be populated if only a list of Files are requested. - Contents []byte `protobuf:"bytes,3,opt,name=contents,proto3" json:"contents,omitempty"` + Contents []byte `protobuf:"bytes,3,opt,name=contents,proto3" json:"contents,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *File) Reset() { *m = File{} } +func (m *File) String() string { return proto.CompactTextString(m) } +func (*File) ProtoMessage() {} +func (*File) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{30} +} +func (m *File) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *File) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *File) XXX_Merge(src proto.Message) { + xxx_messageInfo_File.Merge(dst, src) +} +func (m *File) XXX_Size() int { + return m.Size() +} +func (m *File) XXX_DiscardUnknown() { + xxx_messageInfo_File.DiscardUnknown(m) } -func (m *File) Reset() { *m = File{} } -func (m *File) String() string { return proto.CompactTextString(m) } -func (*File) ProtoMessage() {} -func (*File) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{30} } +var xxx_messageInfo_File proto.InternalMessageInfo type GetFilesRequest struct { // node_id is a string so that "local" can be used to specify that no @@ -550,95 +1418,329 @@ type GetFilesRequest struct { // using filepath.Glob(). The patterns only match to filenames and so path // separators cannot be used. // Example: * will match all files of requested type. - Patterns []string `protobuf:"bytes,6,rep,name=patterns" json:"patterns,omitempty"` + Patterns []string `protobuf:"bytes,6,rep,name=patterns,proto3" json:"patterns,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *GetFilesRequest) Reset() { *m = GetFilesRequest{} } -func (m *GetFilesRequest) String() string { return proto.CompactTextString(m) } -func (*GetFilesRequest) ProtoMessage() {} -func (*GetFilesRequest) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{31} } +func (m *GetFilesRequest) Reset() { *m = GetFilesRequest{} } +func (m *GetFilesRequest) String() string { return proto.CompactTextString(m) } +func (*GetFilesRequest) ProtoMessage() {} +func (*GetFilesRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{31} +} +func (m *GetFilesRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *GetFilesRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *GetFilesRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_GetFilesRequest.Merge(dst, src) +} +func (m *GetFilesRequest) XXX_Size() int { + return m.Size() +} +func (m *GetFilesRequest) XXX_DiscardUnknown() { + xxx_messageInfo_GetFilesRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_GetFilesRequest proto.InternalMessageInfo type GetFilesResponse struct { - Files []*File `protobuf:"bytes,1,rep,name=files" json:"files,omitempty"` + Files []*File `protobuf:"bytes,1,rep,name=files,proto3" json:"files,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *GetFilesResponse) Reset() { *m = GetFilesResponse{} } +func (m *GetFilesResponse) String() string { return proto.CompactTextString(m) } +func (*GetFilesResponse) ProtoMessage() {} +func (*GetFilesResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{32} +} +func (m *GetFilesResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *GetFilesResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *GetFilesResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_GetFilesResponse.Merge(dst, src) +} +func (m *GetFilesResponse) XXX_Size() int { + return m.Size() +} +func (m *GetFilesResponse) XXX_DiscardUnknown() { + xxx_messageInfo_GetFilesResponse.DiscardUnknown(m) } -func (m *GetFilesResponse) Reset() { *m = GetFilesResponse{} } -func (m *GetFilesResponse) String() string { return proto.CompactTextString(m) } -func (*GetFilesResponse) ProtoMessage() {} -func (*GetFilesResponse) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{32} } +var xxx_messageInfo_GetFilesResponse proto.InternalMessageInfo type ProfileRequest struct { // node_id is a string so that "local" can be used to specify that no // forwarding is necessary. NodeId string `protobuf:"bytes,1,opt,name=node_id,json=nodeId,proto3" json:"node_id,omitempty"` // The type of profile to retrieve. - Type ProfileRequest_Type `protobuf:"varint,5,opt,name=type,proto3,enum=cockroach.server.serverpb.ProfileRequest_Type" json:"type,omitempty"` + Type ProfileRequest_Type `protobuf:"varint,5,opt,name=type,proto3,enum=cockroach.server.serverpb.ProfileRequest_Type" json:"type,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *ProfileRequest) Reset() { *m = ProfileRequest{} } -func (m *ProfileRequest) String() string { return proto.CompactTextString(m) } -func (*ProfileRequest) ProtoMessage() {} -func (*ProfileRequest) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{33} } +func (m *ProfileRequest) Reset() { *m = ProfileRequest{} } +func (m *ProfileRequest) String() string { return proto.CompactTextString(m) } +func (*ProfileRequest) ProtoMessage() {} +func (*ProfileRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{33} +} +func (m *ProfileRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ProfileRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ProfileRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_ProfileRequest.Merge(dst, src) +} +func (m *ProfileRequest) XXX_Size() int { + return m.Size() +} +func (m *ProfileRequest) XXX_DiscardUnknown() { + xxx_messageInfo_ProfileRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_ProfileRequest proto.InternalMessageInfo type MetricsRequest struct { // node_id is a string so that "local" can be used to specify that no // forwarding is necessary. - NodeId string `protobuf:"bytes,1,opt,name=node_id,json=nodeId,proto3" json:"node_id,omitempty"` + NodeId string `protobuf:"bytes,1,opt,name=node_id,json=nodeId,proto3" json:"node_id,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *MetricsRequest) Reset() { *m = MetricsRequest{} } -func (m *MetricsRequest) String() string { return proto.CompactTextString(m) } -func (*MetricsRequest) ProtoMessage() {} -func (*MetricsRequest) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{34} } +func (m *MetricsRequest) Reset() { *m = MetricsRequest{} } +func (m *MetricsRequest) String() string { return proto.CompactTextString(m) } +func (*MetricsRequest) ProtoMessage() {} +func (*MetricsRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{34} +} +func (m *MetricsRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *MetricsRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *MetricsRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_MetricsRequest.Merge(dst, src) +} +func (m *MetricsRequest) XXX_Size() int { + return m.Size() +} +func (m *MetricsRequest) XXX_DiscardUnknown() { + xxx_messageInfo_MetricsRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_MetricsRequest proto.InternalMessageInfo type RaftRangeNode struct { - NodeID github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,1,opt,name=node_id,json=nodeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"node_id,omitempty"` - Range RangeInfo `protobuf:"bytes,2,opt,name=range" json:"range"` + NodeID github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,1,opt,name=node_id,json=nodeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"node_id,omitempty"` + Range RangeInfo `protobuf:"bytes,2,opt,name=range,proto3" json:"range"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *RaftRangeNode) Reset() { *m = RaftRangeNode{} } -func (m *RaftRangeNode) String() string { return proto.CompactTextString(m) } -func (*RaftRangeNode) ProtoMessage() {} -func (*RaftRangeNode) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{35} } +func (m *RaftRangeNode) Reset() { *m = RaftRangeNode{} } +func (m *RaftRangeNode) String() string { return proto.CompactTextString(m) } +func (*RaftRangeNode) ProtoMessage() {} +func (*RaftRangeNode) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{35} +} +func (m *RaftRangeNode) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RaftRangeNode) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RaftRangeNode) XXX_Merge(src proto.Message) { + xxx_messageInfo_RaftRangeNode.Merge(dst, src) +} +func (m *RaftRangeNode) XXX_Size() int { + return m.Size() +} +func (m *RaftRangeNode) XXX_DiscardUnknown() { + xxx_messageInfo_RaftRangeNode.DiscardUnknown(m) +} + +var xxx_messageInfo_RaftRangeNode proto.InternalMessageInfo type RaftRangeError struct { - Message string `protobuf:"bytes,1,opt,name=message,proto3" json:"message,omitempty"` + Message string `protobuf:"bytes,1,opt,name=message,proto3" json:"message,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *RaftRangeError) Reset() { *m = RaftRangeError{} } +func (m *RaftRangeError) String() string { return proto.CompactTextString(m) } +func (*RaftRangeError) ProtoMessage() {} +func (*RaftRangeError) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{36} +} +func (m *RaftRangeError) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RaftRangeError) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RaftRangeError) XXX_Merge(src proto.Message) { + xxx_messageInfo_RaftRangeError.Merge(dst, src) +} +func (m *RaftRangeError) XXX_Size() int { + return m.Size() +} +func (m *RaftRangeError) XXX_DiscardUnknown() { + xxx_messageInfo_RaftRangeError.DiscardUnknown(m) } -func (m *RaftRangeError) Reset() { *m = RaftRangeError{} } -func (m *RaftRangeError) String() string { return proto.CompactTextString(m) } -func (*RaftRangeError) ProtoMessage() {} -func (*RaftRangeError) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{36} } +var xxx_messageInfo_RaftRangeError proto.InternalMessageInfo type RaftRangeStatus struct { - RangeID github_com_cockroachdb_cockroach_pkg_roachpb.RangeID `protobuf:"varint,1,opt,name=range_id,json=rangeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" json:"range_id,omitempty"` - Errors []RaftRangeError `protobuf:"bytes,2,rep,name=errors" json:"errors"` - Nodes []RaftRangeNode `protobuf:"bytes,3,rep,name=nodes" json:"nodes"` + RangeID github_com_cockroachdb_cockroach_pkg_roachpb.RangeID `protobuf:"varint,1,opt,name=range_id,json=rangeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" json:"range_id,omitempty"` + Errors []RaftRangeError `protobuf:"bytes,2,rep,name=errors,proto3" json:"errors"` + Nodes []RaftRangeNode `protobuf:"bytes,3,rep,name=nodes,proto3" json:"nodes"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *RaftRangeStatus) Reset() { *m = RaftRangeStatus{} } -func (m *RaftRangeStatus) String() string { return proto.CompactTextString(m) } -func (*RaftRangeStatus) ProtoMessage() {} -func (*RaftRangeStatus) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{37} } +func (m *RaftRangeStatus) Reset() { *m = RaftRangeStatus{} } +func (m *RaftRangeStatus) String() string { return proto.CompactTextString(m) } +func (*RaftRangeStatus) ProtoMessage() {} +func (*RaftRangeStatus) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{37} +} +func (m *RaftRangeStatus) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RaftRangeStatus) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RaftRangeStatus) XXX_Merge(src proto.Message) { + xxx_messageInfo_RaftRangeStatus.Merge(dst, src) +} +func (m *RaftRangeStatus) XXX_Size() int { + return m.Size() +} +func (m *RaftRangeStatus) XXX_DiscardUnknown() { + xxx_messageInfo_RaftRangeStatus.DiscardUnknown(m) +} + +var xxx_messageInfo_RaftRangeStatus proto.InternalMessageInfo type RaftDebugRequest struct { - RangeIDs []github_com_cockroachdb_cockroach_pkg_roachpb.RangeID `protobuf:"varint,1,rep,packed,name=range_ids,json=rangeIds,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" json:"range_ids,omitempty"` + RangeIDs []github_com_cockroachdb_cockroach_pkg_roachpb.RangeID `protobuf:"varint,1,rep,packed,name=range_ids,json=rangeIds,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" json:"range_ids,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *RaftDebugRequest) Reset() { *m = RaftDebugRequest{} } +func (m *RaftDebugRequest) String() string { return proto.CompactTextString(m) } +func (*RaftDebugRequest) ProtoMessage() {} +func (*RaftDebugRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{38} +} +func (m *RaftDebugRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RaftDebugRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RaftDebugRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_RaftDebugRequest.Merge(dst, src) +} +func (m *RaftDebugRequest) XXX_Size() int { + return m.Size() +} +func (m *RaftDebugRequest) XXX_DiscardUnknown() { + xxx_messageInfo_RaftDebugRequest.DiscardUnknown(m) } -func (m *RaftDebugRequest) Reset() { *m = RaftDebugRequest{} } -func (m *RaftDebugRequest) String() string { return proto.CompactTextString(m) } -func (*RaftDebugRequest) ProtoMessage() {} -func (*RaftDebugRequest) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{38} } +var xxx_messageInfo_RaftDebugRequest proto.InternalMessageInfo type RaftDebugResponse struct { - Ranges map[github_com_cockroachdb_cockroach_pkg_roachpb.RangeID]RaftRangeStatus `protobuf:"bytes,1,rep,name=ranges,castkey=github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" json:"ranges" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value"` - Errors []RaftRangeError `protobuf:"bytes,2,rep,name=errors" json:"errors"` + Ranges map[github_com_cockroachdb_cockroach_pkg_roachpb.RangeID]RaftRangeStatus `protobuf:"bytes,1,rep,name=ranges,proto3,castkey=github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" json:"ranges" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Errors []RaftRangeError `protobuf:"bytes,2,rep,name=errors,proto3" json:"errors"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *RaftDebugResponse) Reset() { *m = RaftDebugResponse{} } -func (m *RaftDebugResponse) String() string { return proto.CompactTextString(m) } -func (*RaftDebugResponse) ProtoMessage() {} -func (*RaftDebugResponse) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{39} } +func (m *RaftDebugResponse) Reset() { *m = RaftDebugResponse{} } +func (m *RaftDebugResponse) String() string { return proto.CompactTextString(m) } +func (*RaftDebugResponse) ProtoMessage() {} +func (*RaftDebugResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{39} +} +func (m *RaftDebugResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RaftDebugResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RaftDebugResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_RaftDebugResponse.Merge(dst, src) +} +func (m *RaftDebugResponse) XXX_Size() int { + return m.Size() +} +func (m *RaftDebugResponse) XXX_DiscardUnknown() { + xxx_messageInfo_RaftDebugResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_RaftDebugResponse proto.InternalMessageInfo // ActiveQuery represents a query in flight on some Session. type ActiveQuery struct { @@ -647,28 +1749,80 @@ type ActiveQuery struct { // SQL query string specified by the user. Sql string `protobuf:"bytes,2,opt,name=sql,proto3" json:"sql,omitempty"` // Start timestamp of this query. - Start time.Time `protobuf:"bytes,3,opt,name=start,stdtime" json:"start"` + Start time.Time `protobuf:"bytes,3,opt,name=start,proto3,stdtime" json:"start"` // True if this query is distributed. IsDistributed bool `protobuf:"varint,4,opt,name=is_distributed,json=isDistributed,proto3" json:"is_distributed,omitempty"` // phase stores the current phase of execution for this query. - Phase ActiveQuery_Phase `protobuf:"varint,5,opt,name=phase,proto3,enum=cockroach.server.serverpb.ActiveQuery_Phase" json:"phase,omitempty"` + Phase ActiveQuery_Phase `protobuf:"varint,5,opt,name=phase,proto3,enum=cockroach.server.serverpb.ActiveQuery_Phase" json:"phase,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *ActiveQuery) Reset() { *m = ActiveQuery{} } -func (m *ActiveQuery) String() string { return proto.CompactTextString(m) } -func (*ActiveQuery) ProtoMessage() {} -func (*ActiveQuery) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{40} } +func (m *ActiveQuery) Reset() { *m = ActiveQuery{} } +func (m *ActiveQuery) String() string { return proto.CompactTextString(m) } +func (*ActiveQuery) ProtoMessage() {} +func (*ActiveQuery) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{40} +} +func (m *ActiveQuery) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ActiveQuery) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ActiveQuery) XXX_Merge(src proto.Message) { + xxx_messageInfo_ActiveQuery.Merge(dst, src) +} +func (m *ActiveQuery) XXX_Size() int { + return m.Size() +} +func (m *ActiveQuery) XXX_DiscardUnknown() { + xxx_messageInfo_ActiveQuery.DiscardUnknown(m) +} + +var xxx_messageInfo_ActiveQuery proto.InternalMessageInfo // Request object for ListSessions and ListLocalSessions. type ListSessionsRequest struct { // Username of the user making this request. - Username string `protobuf:"bytes,1,opt,name=username,proto3" json:"username,omitempty"` + Username string `protobuf:"bytes,1,opt,name=username,proto3" json:"username,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *ListSessionsRequest) Reset() { *m = ListSessionsRequest{} } -func (m *ListSessionsRequest) String() string { return proto.CompactTextString(m) } -func (*ListSessionsRequest) ProtoMessage() {} -func (*ListSessionsRequest) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{41} } +func (m *ListSessionsRequest) Reset() { *m = ListSessionsRequest{} } +func (m *ListSessionsRequest) String() string { return proto.CompactTextString(m) } +func (*ListSessionsRequest) ProtoMessage() {} +func (*ListSessionsRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{41} +} +func (m *ListSessionsRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ListSessionsRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ListSessionsRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_ListSessionsRequest.Merge(dst, src) +} +func (m *ListSessionsRequest) XXX_Size() int { + return m.Size() +} +func (m *ListSessionsRequest) XXX_DiscardUnknown() { + xxx_messageInfo_ListSessionsRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_ListSessionsRequest proto.InternalMessageInfo // Session represents one SQL session. type Session struct { @@ -681,9 +1835,9 @@ type Session struct { // Application name specified by the client. ApplicationName string `protobuf:"bytes,4,opt,name=application_name,json=applicationName,proto3" json:"application_name,omitempty"` // Queries in progress on this session. - ActiveQueries []ActiveQuery `protobuf:"bytes,5,rep,name=active_queries,json=activeQueries" json:"active_queries"` + ActiveQueries []ActiveQuery `protobuf:"bytes,5,rep,name=active_queries,json=activeQueries,proto3" json:"active_queries"` // Timestamp of session's start. - Start time.Time `protobuf:"bytes,6,opt,name=start,stdtime" json:"start"` + Start time.Time `protobuf:"bytes,6,opt,name=start,proto3,stdtime" json:"start"` // ID of the current KV transaction for this session. Nil if the session // doesn't currently have a transaction. KvTxnID *github_com_cockroachdb_cockroach_pkg_util_uuid.UUID `protobuf:"bytes,7,opt,name=kv_txn_id,json=kvTxnId,proto3,customtype=github.com/cockroachdb/cockroach/pkg/util/uuid.UUID" json:"kv_txn_id,omitempty"` @@ -694,39 +1848,117 @@ type Session struct { // Number of currently allocated bytes in the session memory monitor. AllocBytes int64 `protobuf:"varint,10,opt,name=alloc_bytes,json=allocBytes,proto3" json:"alloc_bytes,omitempty"` // High water mark of allocated bytes in the session memory monitor. - MaxAllocBytes int64 `protobuf:"varint,11,opt,name=max_alloc_bytes,json=maxAllocBytes,proto3" json:"max_alloc_bytes,omitempty"` + MaxAllocBytes int64 `protobuf:"varint,11,opt,name=max_alloc_bytes,json=maxAllocBytes,proto3" json:"max_alloc_bytes,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *Session) Reset() { *m = Session{} } -func (m *Session) String() string { return proto.CompactTextString(m) } -func (*Session) ProtoMessage() {} -func (*Session) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{42} } +func (m *Session) Reset() { *m = Session{} } +func (m *Session) String() string { return proto.CompactTextString(m) } +func (*Session) ProtoMessage() {} +func (*Session) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{42} +} +func (m *Session) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Session) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *Session) XXX_Merge(src proto.Message) { + xxx_messageInfo_Session.Merge(dst, src) +} +func (m *Session) XXX_Size() int { + return m.Size() +} +func (m *Session) XXX_DiscardUnknown() { + xxx_messageInfo_Session.DiscardUnknown(m) +} + +var xxx_messageInfo_Session proto.InternalMessageInfo // An error wrapper object for ListSessionsResponse. type ListSessionsError struct { // ID of node that was being contacted when this error occurred NodeID github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,1,opt,name=node_id,json=nodeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"node_id,omitempty"` // Error message. - Message string `protobuf:"bytes,2,opt,name=message,proto3" json:"message,omitempty"` + Message string `protobuf:"bytes,2,opt,name=message,proto3" json:"message,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *ListSessionsError) Reset() { *m = ListSessionsError{} } -func (m *ListSessionsError) String() string { return proto.CompactTextString(m) } -func (*ListSessionsError) ProtoMessage() {} -func (*ListSessionsError) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{43} } +func (m *ListSessionsError) Reset() { *m = ListSessionsError{} } +func (m *ListSessionsError) String() string { return proto.CompactTextString(m) } +func (*ListSessionsError) ProtoMessage() {} +func (*ListSessionsError) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{43} +} +func (m *ListSessionsError) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ListSessionsError) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ListSessionsError) XXX_Merge(src proto.Message) { + xxx_messageInfo_ListSessionsError.Merge(dst, src) +} +func (m *ListSessionsError) XXX_Size() int { + return m.Size() +} +func (m *ListSessionsError) XXX_DiscardUnknown() { + xxx_messageInfo_ListSessionsError.DiscardUnknown(m) +} + +var xxx_messageInfo_ListSessionsError proto.InternalMessageInfo // Response object for ListSessions and ListLocalSessions. type ListSessionsResponse struct { // A list of sessions on this node or cluster. - Sessions []Session `protobuf:"bytes,1,rep,name=sessions" json:"sessions"` + Sessions []Session `protobuf:"bytes,1,rep,name=sessions,proto3" json:"sessions"` // Any errors that occurred during fan-out calls to other nodes. - Errors []ListSessionsError `protobuf:"bytes,2,rep,name=errors" json:"errors"` + Errors []ListSessionsError `protobuf:"bytes,2,rep,name=errors,proto3" json:"errors"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *ListSessionsResponse) Reset() { *m = ListSessionsResponse{} } -func (m *ListSessionsResponse) String() string { return proto.CompactTextString(m) } -func (*ListSessionsResponse) ProtoMessage() {} -func (*ListSessionsResponse) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{44} } +func (m *ListSessionsResponse) Reset() { *m = ListSessionsResponse{} } +func (m *ListSessionsResponse) String() string { return proto.CompactTextString(m) } +func (*ListSessionsResponse) ProtoMessage() {} +func (*ListSessionsResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{44} +} +func (m *ListSessionsResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ListSessionsResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ListSessionsResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_ListSessionsResponse.Merge(dst, src) +} +func (m *ListSessionsResponse) XXX_Size() int { + return m.Size() +} +func (m *ListSessionsResponse) XXX_DiscardUnknown() { + xxx_messageInfo_ListSessionsResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_ListSessionsResponse proto.InternalMessageInfo // Request object for issing a query cancel request. type CancelQueryRequest struct { @@ -741,26 +1973,78 @@ type CancelQueryRequest struct { // ID of query to be canceled (converted to string). QueryID string `protobuf:"bytes,2,opt,name=query_id,json=queryId,proto3" json:"query_id,omitempty"` // Username of the user making this cancellation request. - Username string `protobuf:"bytes,3,opt,name=username,proto3" json:"username,omitempty"` + Username string `protobuf:"bytes,3,opt,name=username,proto3" json:"username,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *CancelQueryRequest) Reset() { *m = CancelQueryRequest{} } -func (m *CancelQueryRequest) String() string { return proto.CompactTextString(m) } -func (*CancelQueryRequest) ProtoMessage() {} -func (*CancelQueryRequest) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{45} } +func (m *CancelQueryRequest) Reset() { *m = CancelQueryRequest{} } +func (m *CancelQueryRequest) String() string { return proto.CompactTextString(m) } +func (*CancelQueryRequest) ProtoMessage() {} +func (*CancelQueryRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{45} +} +func (m *CancelQueryRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *CancelQueryRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *CancelQueryRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_CancelQueryRequest.Merge(dst, src) +} +func (m *CancelQueryRequest) XXX_Size() int { + return m.Size() +} +func (m *CancelQueryRequest) XXX_DiscardUnknown() { + xxx_messageInfo_CancelQueryRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_CancelQueryRequest proto.InternalMessageInfo // Response returned by target query's gateway node. type CancelQueryResponse struct { // Whether the cancellation request succeeded and the query was canceled. Canceled bool `protobuf:"varint,1,opt,name=canceled,proto3" json:"canceled,omitempty"` // Error message (accompanied with canceled = false). - Error string `protobuf:"bytes,2,opt,name=error,proto3" json:"error,omitempty"` + Error string `protobuf:"bytes,2,opt,name=error,proto3" json:"error,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *CancelQueryResponse) Reset() { *m = CancelQueryResponse{} } +func (m *CancelQueryResponse) String() string { return proto.CompactTextString(m) } +func (*CancelQueryResponse) ProtoMessage() {} +func (*CancelQueryResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{46} +} +func (m *CancelQueryResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *CancelQueryResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *CancelQueryResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_CancelQueryResponse.Merge(dst, src) +} +func (m *CancelQueryResponse) XXX_Size() int { + return m.Size() +} +func (m *CancelQueryResponse) XXX_DiscardUnknown() { + xxx_messageInfo_CancelQueryResponse.DiscardUnknown(m) } -func (m *CancelQueryResponse) Reset() { *m = CancelQueryResponse{} } -func (m *CancelQueryResponse) String() string { return proto.CompactTextString(m) } -func (*CancelQueryResponse) ProtoMessage() {} -func (*CancelQueryResponse) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{46} } +var xxx_messageInfo_CancelQueryResponse proto.InternalMessageInfo type CancelSessionRequest struct { // TODO(abhimadan): use [(gogoproto.customname) = "NodeID"] below. Need to @@ -768,142 +2052,450 @@ type CancelSessionRequest struct { // // node_id is a string so that "local" can be used to specify that no // forwarding is necessary. - NodeId string `protobuf:"bytes,1,opt,name=node_id,json=nodeId,proto3" json:"node_id,omitempty"` - SessionID []byte `protobuf:"bytes,2,opt,name=session_id,json=sessionId,proto3" json:"session_id,omitempty"` - Username string `protobuf:"bytes,3,opt,name=username,proto3" json:"username,omitempty"` + NodeId string `protobuf:"bytes,1,opt,name=node_id,json=nodeId,proto3" json:"node_id,omitempty"` + SessionID []byte `protobuf:"bytes,2,opt,name=session_id,json=sessionId,proto3" json:"session_id,omitempty"` + Username string `protobuf:"bytes,3,opt,name=username,proto3" json:"username,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *CancelSessionRequest) Reset() { *m = CancelSessionRequest{} } +func (m *CancelSessionRequest) String() string { return proto.CompactTextString(m) } +func (*CancelSessionRequest) ProtoMessage() {} +func (*CancelSessionRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{47} +} +func (m *CancelSessionRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *CancelSessionRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *CancelSessionRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_CancelSessionRequest.Merge(dst, src) +} +func (m *CancelSessionRequest) XXX_Size() int { + return m.Size() +} +func (m *CancelSessionRequest) XXX_DiscardUnknown() { + xxx_messageInfo_CancelSessionRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_CancelSessionRequest proto.InternalMessageInfo + +type CancelSessionResponse struct { + Canceled bool `protobuf:"varint,1,opt,name=canceled,proto3" json:"canceled,omitempty"` + Error string `protobuf:"bytes,2,opt,name=error,proto3" json:"error,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *CancelSessionResponse) Reset() { *m = CancelSessionResponse{} } +func (m *CancelSessionResponse) String() string { return proto.CompactTextString(m) } +func (*CancelSessionResponse) ProtoMessage() {} +func (*CancelSessionResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{48} +} +func (m *CancelSessionResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *CancelSessionResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *CancelSessionResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_CancelSessionResponse.Merge(dst, src) +} +func (m *CancelSessionResponse) XXX_Size() int { + return m.Size() +} +func (m *CancelSessionResponse) XXX_DiscardUnknown() { + xxx_messageInfo_CancelSessionResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_CancelSessionResponse proto.InternalMessageInfo + +type SpanStatsRequest struct { + NodeID string `protobuf:"bytes,1,opt,name=node_id,json=nodeId,proto3" json:"node_id,omitempty"` + StartKey github_com_cockroachdb_cockroach_pkg_roachpb.RKey `protobuf:"bytes,2,opt,name=start_key,json=startKey,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.RKey" json:"start_key,omitempty"` + EndKey github_com_cockroachdb_cockroach_pkg_roachpb.RKey `protobuf:"bytes,3,opt,name=end_key,json=endKey,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.RKey" json:"end_key,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *SpanStatsRequest) Reset() { *m = SpanStatsRequest{} } +func (m *SpanStatsRequest) String() string { return proto.CompactTextString(m) } +func (*SpanStatsRequest) ProtoMessage() {} +func (*SpanStatsRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{49} +} +func (m *SpanStatsRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SpanStatsRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil } - -func (m *CancelSessionRequest) Reset() { *m = CancelSessionRequest{} } -func (m *CancelSessionRequest) String() string { return proto.CompactTextString(m) } -func (*CancelSessionRequest) ProtoMessage() {} -func (*CancelSessionRequest) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{47} } - -type CancelSessionResponse struct { - Canceled bool `protobuf:"varint,1,opt,name=canceled,proto3" json:"canceled,omitempty"` - Error string `protobuf:"bytes,2,opt,name=error,proto3" json:"error,omitempty"` +func (dst *SpanStatsRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_SpanStatsRequest.Merge(dst, src) } - -func (m *CancelSessionResponse) Reset() { *m = CancelSessionResponse{} } -func (m *CancelSessionResponse) String() string { return proto.CompactTextString(m) } -func (*CancelSessionResponse) ProtoMessage() {} -func (*CancelSessionResponse) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{48} } - -type SpanStatsRequest struct { - NodeID string `protobuf:"bytes,1,opt,name=node_id,json=nodeId,proto3" json:"node_id,omitempty"` - StartKey github_com_cockroachdb_cockroach_pkg_roachpb.RKey `protobuf:"bytes,2,opt,name=start_key,json=startKey,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.RKey" json:"start_key,omitempty"` - EndKey github_com_cockroachdb_cockroach_pkg_roachpb.RKey `protobuf:"bytes,3,opt,name=end_key,json=endKey,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.RKey" json:"end_key,omitempty"` +func (m *SpanStatsRequest) XXX_Size() int { + return m.Size() +} +func (m *SpanStatsRequest) XXX_DiscardUnknown() { + xxx_messageInfo_SpanStatsRequest.DiscardUnknown(m) } -func (m *SpanStatsRequest) Reset() { *m = SpanStatsRequest{} } -func (m *SpanStatsRequest) String() string { return proto.CompactTextString(m) } -func (*SpanStatsRequest) ProtoMessage() {} -func (*SpanStatsRequest) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{49} } +var xxx_messageInfo_SpanStatsRequest proto.InternalMessageInfo type SpanStatsResponse struct { - RangeCount int32 `protobuf:"varint,2,opt,name=range_count,json=rangeCount,proto3" json:"range_count,omitempty"` - ApproximateDiskBytes uint64 `protobuf:"varint,3,opt,name=approximate_disk_bytes,json=approximateDiskBytes,proto3" json:"approximate_disk_bytes,omitempty"` - TotalStats cockroach_storage_engine_enginepb1.MVCCStats `protobuf:"bytes,1,opt,name=total_stats,json=totalStats" json:"total_stats"` + RangeCount int32 `protobuf:"varint,2,opt,name=range_count,json=rangeCount,proto3" json:"range_count,omitempty"` + ApproximateDiskBytes uint64 `protobuf:"varint,3,opt,name=approximate_disk_bytes,json=approximateDiskBytes,proto3" json:"approximate_disk_bytes,omitempty"` + TotalStats enginepb.MVCCStats `protobuf:"bytes,1,opt,name=total_stats,json=totalStats,proto3" json:"total_stats"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *SpanStatsResponse) Reset() { *m = SpanStatsResponse{} } +func (m *SpanStatsResponse) String() string { return proto.CompactTextString(m) } +func (*SpanStatsResponse) ProtoMessage() {} +func (*SpanStatsResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{50} +} +func (m *SpanStatsResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SpanStatsResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *SpanStatsResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_SpanStatsResponse.Merge(dst, src) +} +func (m *SpanStatsResponse) XXX_Size() int { + return m.Size() +} +func (m *SpanStatsResponse) XXX_DiscardUnknown() { + xxx_messageInfo_SpanStatsResponse.DiscardUnknown(m) } -func (m *SpanStatsResponse) Reset() { *m = SpanStatsResponse{} } -func (m *SpanStatsResponse) String() string { return proto.CompactTextString(m) } -func (*SpanStatsResponse) ProtoMessage() {} -func (*SpanStatsResponse) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{50} } +var xxx_messageInfo_SpanStatsResponse proto.InternalMessageInfo type ProblemRangesRequest struct { - NodeID string `protobuf:"bytes,1,opt,name=node_id,json=nodeId,proto3" json:"node_id,omitempty"` + NodeID string `protobuf:"bytes,1,opt,name=node_id,json=nodeId,proto3" json:"node_id,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ProblemRangesRequest) Reset() { *m = ProblemRangesRequest{} } +func (m *ProblemRangesRequest) String() string { return proto.CompactTextString(m) } +func (*ProblemRangesRequest) ProtoMessage() {} +func (*ProblemRangesRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{51} +} +func (m *ProblemRangesRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ProblemRangesRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ProblemRangesRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_ProblemRangesRequest.Merge(dst, src) +} +func (m *ProblemRangesRequest) XXX_Size() int { + return m.Size() +} +func (m *ProblemRangesRequest) XXX_DiscardUnknown() { + xxx_messageInfo_ProblemRangesRequest.DiscardUnknown(m) } -func (m *ProblemRangesRequest) Reset() { *m = ProblemRangesRequest{} } -func (m *ProblemRangesRequest) String() string { return proto.CompactTextString(m) } -func (*ProblemRangesRequest) ProtoMessage() {} -func (*ProblemRangesRequest) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{51} } +var xxx_messageInfo_ProblemRangesRequest proto.InternalMessageInfo type ProblemRangesResponse struct { // NodeID is the node that submitted all the requests. - NodeID github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,8,opt,name=node_id,json=nodeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"node_id,omitempty"` - ProblemsByNodeID map[github_com_cockroachdb_cockroach_pkg_roachpb.NodeID]ProblemRangesResponse_NodeProblems `protobuf:"bytes,9,rep,name=problems_by_node_id,json=problemsByNodeId,castkey=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"problems_by_node_id" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value"` + NodeID github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,8,opt,name=node_id,json=nodeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"node_id,omitempty"` + ProblemsByNodeID map[github_com_cockroachdb_cockroach_pkg_roachpb.NodeID]ProblemRangesResponse_NodeProblems `protobuf:"bytes,9,rep,name=problems_by_node_id,json=problemsByNodeId,proto3,castkey=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"problems_by_node_id" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ProblemRangesResponse) Reset() { *m = ProblemRangesResponse{} } +func (m *ProblemRangesResponse) String() string { return proto.CompactTextString(m) } +func (*ProblemRangesResponse) ProtoMessage() {} +func (*ProblemRangesResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{52} +} +func (m *ProblemRangesResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ProblemRangesResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ProblemRangesResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_ProblemRangesResponse.Merge(dst, src) +} +func (m *ProblemRangesResponse) XXX_Size() int { + return m.Size() +} +func (m *ProblemRangesResponse) XXX_DiscardUnknown() { + xxx_messageInfo_ProblemRangesResponse.DiscardUnknown(m) } -func (m *ProblemRangesResponse) Reset() { *m = ProblemRangesResponse{} } -func (m *ProblemRangesResponse) String() string { return proto.CompactTextString(m) } -func (*ProblemRangesResponse) ProtoMessage() {} -func (*ProblemRangesResponse) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{52} } +var xxx_messageInfo_ProblemRangesResponse proto.InternalMessageInfo type ProblemRangesResponse_NodeProblems struct { ErrorMessage string `protobuf:"bytes,1,opt,name=error_message,json=errorMessage,proto3" json:"error_message,omitempty"` - UnavailableRangeIDs []github_com_cockroachdb_cockroach_pkg_roachpb.RangeID `protobuf:"varint,2,rep,packed,name=unavailable_range_ids,json=unavailableRangeIds,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" json:"unavailable_range_ids,omitempty"` - RaftLeaderNotLeaseHolderRangeIDs []github_com_cockroachdb_cockroach_pkg_roachpb.RangeID `protobuf:"varint,3,rep,packed,name=raft_leader_not_lease_holder_range_ids,json=raftLeaderNotLeaseHolderRangeIds,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" json:"raft_leader_not_lease_holder_range_ids,omitempty"` - NoRaftLeaderRangeIDs []github_com_cockroachdb_cockroach_pkg_roachpb.RangeID `protobuf:"varint,4,rep,packed,name=no_raft_leader_range_ids,json=noRaftLeaderRangeIds,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" json:"no_raft_leader_range_ids,omitempty"` - NoLeaseRangeIDs []github_com_cockroachdb_cockroach_pkg_roachpb.RangeID `protobuf:"varint,5,rep,packed,name=no_lease_range_ids,json=noLeaseRangeIds,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" json:"no_lease_range_ids,omitempty"` - UnderreplicatedRangeIDs []github_com_cockroachdb_cockroach_pkg_roachpb.RangeID `protobuf:"varint,6,rep,packed,name=underreplicated_range_ids,json=underreplicatedRangeIds,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" json:"underreplicated_range_ids,omitempty"` - QuiescentEqualsTickingRangeIDs []github_com_cockroachdb_cockroach_pkg_roachpb.RangeID `protobuf:"varint,7,rep,packed,name=quiescent_equals_ticking_range_ids,json=quiescentEqualsTickingRangeIds,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" json:"quiescent_equals_ticking_range_ids,omitempty"` - RaftLogTooLargeRangeIDs []github_com_cockroachdb_cockroach_pkg_roachpb.RangeID `protobuf:"varint,8,rep,packed,name=raft_log_too_large_range_ids,json=raftLogTooLargeRangeIds,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" json:"raft_log_too_large_range_ids,omitempty"` + UnavailableRangeIDs []github_com_cockroachdb_cockroach_pkg_roachpb.RangeID `protobuf:"varint,2,rep,packed,name=unavailable_range_ids,json=unavailableRangeIds,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" json:"unavailable_range_ids,omitempty"` + RaftLeaderNotLeaseHolderRangeIDs []github_com_cockroachdb_cockroach_pkg_roachpb.RangeID `protobuf:"varint,3,rep,packed,name=raft_leader_not_lease_holder_range_ids,json=raftLeaderNotLeaseHolderRangeIds,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" json:"raft_leader_not_lease_holder_range_ids,omitempty"` + NoRaftLeaderRangeIDs []github_com_cockroachdb_cockroach_pkg_roachpb.RangeID `protobuf:"varint,4,rep,packed,name=no_raft_leader_range_ids,json=noRaftLeaderRangeIds,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" json:"no_raft_leader_range_ids,omitempty"` + NoLeaseRangeIDs []github_com_cockroachdb_cockroach_pkg_roachpb.RangeID `protobuf:"varint,5,rep,packed,name=no_lease_range_ids,json=noLeaseRangeIds,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" json:"no_lease_range_ids,omitempty"` + UnderreplicatedRangeIDs []github_com_cockroachdb_cockroach_pkg_roachpb.RangeID `protobuf:"varint,6,rep,packed,name=underreplicated_range_ids,json=underreplicatedRangeIds,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" json:"underreplicated_range_ids,omitempty"` + QuiescentEqualsTickingRangeIDs []github_com_cockroachdb_cockroach_pkg_roachpb.RangeID `protobuf:"varint,7,rep,packed,name=quiescent_equals_ticking_range_ids,json=quiescentEqualsTickingRangeIds,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" json:"quiescent_equals_ticking_range_ids,omitempty"` + RaftLogTooLargeRangeIDs []github_com_cockroachdb_cockroach_pkg_roachpb.RangeID `protobuf:"varint,8,rep,packed,name=raft_log_too_large_range_ids,json=raftLogTooLargeRangeIds,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" json:"raft_log_too_large_range_ids,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *ProblemRangesResponse_NodeProblems) Reset() { *m = ProblemRangesResponse_NodeProblems{} } func (m *ProblemRangesResponse_NodeProblems) String() string { return proto.CompactTextString(m) } func (*ProblemRangesResponse_NodeProblems) ProtoMessage() {} func (*ProblemRangesResponse_NodeProblems) Descriptor() ([]byte, []int) { - return fileDescriptorStatus, []int{52, 0} + return fileDescriptor_status_0de29a344bd79652, []int{52, 0} +} +func (m *ProblemRangesResponse_NodeProblems) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ProblemRangesResponse_NodeProblems) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ProblemRangesResponse_NodeProblems) XXX_Merge(src proto.Message) { + xxx_messageInfo_ProblemRangesResponse_NodeProblems.Merge(dst, src) +} +func (m *ProblemRangesResponse_NodeProblems) XXX_Size() int { + return m.Size() } +func (m *ProblemRangesResponse_NodeProblems) XXX_DiscardUnknown() { + xxx_messageInfo_ProblemRangesResponse_NodeProblems.DiscardUnknown(m) +} + +var xxx_messageInfo_ProblemRangesResponse_NodeProblems proto.InternalMessageInfo type RangeRequest struct { - RangeId int64 `protobuf:"varint,1,opt,name=range_id,json=rangeId,proto3" json:"range_id,omitempty"` + RangeId int64 `protobuf:"varint,1,opt,name=range_id,json=rangeId,proto3" json:"range_id,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *RangeRequest) Reset() { *m = RangeRequest{} } +func (m *RangeRequest) String() string { return proto.CompactTextString(m) } +func (*RangeRequest) ProtoMessage() {} +func (*RangeRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{53} +} +func (m *RangeRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RangeRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RangeRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_RangeRequest.Merge(dst, src) +} +func (m *RangeRequest) XXX_Size() int { + return m.Size() +} +func (m *RangeRequest) XXX_DiscardUnknown() { + xxx_messageInfo_RangeRequest.DiscardUnknown(m) } -func (m *RangeRequest) Reset() { *m = RangeRequest{} } -func (m *RangeRequest) String() string { return proto.CompactTextString(m) } -func (*RangeRequest) ProtoMessage() {} -func (*RangeRequest) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{53} } +var xxx_messageInfo_RangeRequest proto.InternalMessageInfo type RangeResponse struct { // NodeID is the node that submitted all the requests. - NodeID github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,1,opt,name=node_id,json=nodeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"node_id,omitempty"` - RangeID github_com_cockroachdb_cockroach_pkg_roachpb.RangeID `protobuf:"varint,2,opt,name=range_id,json=rangeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" json:"range_id,omitempty"` - ResponsesByNodeID map[github_com_cockroachdb_cockroach_pkg_roachpb.NodeID]RangeResponse_NodeResponse `protobuf:"bytes,3,rep,name=responses_by_node_id,json=responsesByNodeId,castkey=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"responses_by_node_id" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value"` + NodeID github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,1,opt,name=node_id,json=nodeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"node_id,omitempty"` + RangeID github_com_cockroachdb_cockroach_pkg_roachpb.RangeID `protobuf:"varint,2,opt,name=range_id,json=rangeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" json:"range_id,omitempty"` + ResponsesByNodeID map[github_com_cockroachdb_cockroach_pkg_roachpb.NodeID]RangeResponse_NodeResponse `protobuf:"bytes,3,rep,name=responses_by_node_id,json=responsesByNodeId,proto3,castkey=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"responses_by_node_id" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *RangeResponse) Reset() { *m = RangeResponse{} } +func (m *RangeResponse) String() string { return proto.CompactTextString(m) } +func (*RangeResponse) ProtoMessage() {} +func (*RangeResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{54} +} +func (m *RangeResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RangeResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RangeResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_RangeResponse.Merge(dst, src) +} +func (m *RangeResponse) XXX_Size() int { + return m.Size() +} +func (m *RangeResponse) XXX_DiscardUnknown() { + xxx_messageInfo_RangeResponse.DiscardUnknown(m) } -func (m *RangeResponse) Reset() { *m = RangeResponse{} } -func (m *RangeResponse) String() string { return proto.CompactTextString(m) } -func (*RangeResponse) ProtoMessage() {} -func (*RangeResponse) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{54} } +var xxx_messageInfo_RangeResponse proto.InternalMessageInfo type RangeResponse_NodeResponse struct { - Response bool `protobuf:"varint,1,opt,name=response,proto3" json:"response,omitempty"` - ErrorMessage string `protobuf:"bytes,2,opt,name=error_message,json=errorMessage,proto3" json:"error_message,omitempty"` - Infos []RangeInfo `protobuf:"bytes,3,rep,name=infos" json:"infos"` + Response bool `protobuf:"varint,1,opt,name=response,proto3" json:"response,omitempty"` + ErrorMessage string `protobuf:"bytes,2,opt,name=error_message,json=errorMessage,proto3" json:"error_message,omitempty"` + Infos []RangeInfo `protobuf:"bytes,3,rep,name=infos,proto3" json:"infos"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *RangeResponse_NodeResponse) Reset() { *m = RangeResponse_NodeResponse{} } func (m *RangeResponse_NodeResponse) String() string { return proto.CompactTextString(m) } func (*RangeResponse_NodeResponse) ProtoMessage() {} func (*RangeResponse_NodeResponse) Descriptor() ([]byte, []int) { - return fileDescriptorStatus, []int{54, 0} + return fileDescriptor_status_0de29a344bd79652, []int{54, 0} +} +func (m *RangeResponse_NodeResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RangeResponse_NodeResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RangeResponse_NodeResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_RangeResponse_NodeResponse.Merge(dst, src) +} +func (m *RangeResponse_NodeResponse) XXX_Size() int { + return m.Size() +} +func (m *RangeResponse_NodeResponse) XXX_DiscardUnknown() { + xxx_messageInfo_RangeResponse_NodeResponse.DiscardUnknown(m) } +var xxx_messageInfo_RangeResponse_NodeResponse proto.InternalMessageInfo + // DiagnosticsRequest requests a diagnostics report. type DiagnosticsRequest struct { // node_id is a string so that "local" can be used to specify that no // forwarding is necessary. - NodeId string `protobuf:"bytes,1,opt,name=node_id,json=nodeId,proto3" json:"node_id,omitempty"` + NodeId string `protobuf:"bytes,1,opt,name=node_id,json=nodeId,proto3" json:"node_id,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *DiagnosticsRequest) Reset() { *m = DiagnosticsRequest{} } +func (m *DiagnosticsRequest) String() string { return proto.CompactTextString(m) } +func (*DiagnosticsRequest) ProtoMessage() {} +func (*DiagnosticsRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{55} +} +func (m *DiagnosticsRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *DiagnosticsRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *DiagnosticsRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_DiagnosticsRequest.Merge(dst, src) +} +func (m *DiagnosticsRequest) XXX_Size() int { + return m.Size() +} +func (m *DiagnosticsRequest) XXX_DiscardUnknown() { + xxx_messageInfo_DiagnosticsRequest.DiscardUnknown(m) } -func (m *DiagnosticsRequest) Reset() { *m = DiagnosticsRequest{} } -func (m *DiagnosticsRequest) String() string { return proto.CompactTextString(m) } -func (*DiagnosticsRequest) ProtoMessage() {} -func (*DiagnosticsRequest) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{55} } +var xxx_messageInfo_DiagnosticsRequest proto.InternalMessageInfo type StoresRequest struct { // node_id is a string so that "local" can be used to specify that no // forwarding is necessary. - NodeId string `protobuf:"bytes,1,opt,name=node_id,json=nodeId,proto3" json:"node_id,omitempty"` + NodeId string `protobuf:"bytes,1,opt,name=node_id,json=nodeId,proto3" json:"node_id,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *StoresRequest) Reset() { *m = StoresRequest{} } +func (m *StoresRequest) String() string { return proto.CompactTextString(m) } +func (*StoresRequest) ProtoMessage() {} +func (*StoresRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{56} +} +func (m *StoresRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *StoresRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *StoresRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_StoresRequest.Merge(dst, src) +} +func (m *StoresRequest) XXX_Size() int { + return m.Size() +} +func (m *StoresRequest) XXX_DiscardUnknown() { + xxx_messageInfo_StoresRequest.DiscardUnknown(m) } -func (m *StoresRequest) Reset() { *m = StoresRequest{} } -func (m *StoresRequest) String() string { return proto.CompactTextString(m) } -func (*StoresRequest) ProtoMessage() {} -func (*StoresRequest) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{56} } +var xxx_messageInfo_StoresRequest proto.InternalMessageInfo type StoreDetails struct { StoreID github_com_cockroachdb_cockroach_pkg_roachpb.StoreID `protobuf:"varint,1,opt,name=store_id,json=storeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.StoreID" json:"store_id,omitempty"` @@ -915,47 +2507,153 @@ type StoreDetails struct { TotalFiles uint64 `protobuf:"varint,3,opt,name=total_files,json=totalFiles,proto3" json:"total_files,omitempty"` TotalBytes uint64 `protobuf:"varint,4,opt,name=total_bytes,json=totalBytes,proto3" json:"total_bytes,omitempty"` // Files/bytes using the active data key. - ActiveKeyFiles uint64 `protobuf:"varint,5,opt,name=active_key_files,json=activeKeyFiles,proto3" json:"active_key_files,omitempty"` - ActiveKeyBytes uint64 `protobuf:"varint,6,opt,name=active_key_bytes,json=activeKeyBytes,proto3" json:"active_key_bytes,omitempty"` + ActiveKeyFiles uint64 `protobuf:"varint,5,opt,name=active_key_files,json=activeKeyFiles,proto3" json:"active_key_files,omitempty"` + ActiveKeyBytes uint64 `protobuf:"varint,6,opt,name=active_key_bytes,json=activeKeyBytes,proto3" json:"active_key_bytes,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *StoreDetails) Reset() { *m = StoreDetails{} } +func (m *StoreDetails) String() string { return proto.CompactTextString(m) } +func (*StoreDetails) ProtoMessage() {} +func (*StoreDetails) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{57} +} +func (m *StoreDetails) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *StoreDetails) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *StoreDetails) XXX_Merge(src proto.Message) { + xxx_messageInfo_StoreDetails.Merge(dst, src) +} +func (m *StoreDetails) XXX_Size() int { + return m.Size() +} +func (m *StoreDetails) XXX_DiscardUnknown() { + xxx_messageInfo_StoreDetails.DiscardUnknown(m) } -func (m *StoreDetails) Reset() { *m = StoreDetails{} } -func (m *StoreDetails) String() string { return proto.CompactTextString(m) } -func (*StoreDetails) ProtoMessage() {} -func (*StoreDetails) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{57} } +var xxx_messageInfo_StoreDetails proto.InternalMessageInfo type StoresResponse struct { - Stores []StoreDetails `protobuf:"bytes,1,rep,name=stores" json:"stores"` + Stores []StoreDetails `protobuf:"bytes,1,rep,name=stores,proto3" json:"stores"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *StoresResponse) Reset() { *m = StoresResponse{} } +func (m *StoresResponse) String() string { return proto.CompactTextString(m) } +func (*StoresResponse) ProtoMessage() {} +func (*StoresResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{58} +} +func (m *StoresResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *StoresResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *StoresResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_StoresResponse.Merge(dst, src) +} +func (m *StoresResponse) XXX_Size() int { + return m.Size() +} +func (m *StoresResponse) XXX_DiscardUnknown() { + xxx_messageInfo_StoresResponse.DiscardUnknown(m) } -func (m *StoresResponse) Reset() { *m = StoresResponse{} } -func (m *StoresResponse) String() string { return proto.CompactTextString(m) } -func (*StoresResponse) ProtoMessage() {} -func (*StoresResponse) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{58} } +var xxx_messageInfo_StoresResponse proto.InternalMessageInfo type StatementsRequest struct { - NodeID string `protobuf:"bytes,1,opt,name=node_id,json=nodeId,proto3" json:"node_id,omitempty"` + NodeID string `protobuf:"bytes,1,opt,name=node_id,json=nodeId,proto3" json:"node_id,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *StatementsRequest) Reset() { *m = StatementsRequest{} } +func (m *StatementsRequest) String() string { return proto.CompactTextString(m) } +func (*StatementsRequest) ProtoMessage() {} +func (*StatementsRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{59} +} +func (m *StatementsRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *StatementsRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *StatementsRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_StatementsRequest.Merge(dst, src) +} +func (m *StatementsRequest) XXX_Size() int { + return m.Size() +} +func (m *StatementsRequest) XXX_DiscardUnknown() { + xxx_messageInfo_StatementsRequest.DiscardUnknown(m) } -func (m *StatementsRequest) Reset() { *m = StatementsRequest{} } -func (m *StatementsRequest) String() string { return proto.CompactTextString(m) } -func (*StatementsRequest) ProtoMessage() {} -func (*StatementsRequest) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{59} } +var xxx_messageInfo_StatementsRequest proto.InternalMessageInfo type StatementsResponse struct { - Statements []StatementsResponse_CollectedStatementStatistics `protobuf:"bytes,1,rep,name=statements" json:"statements"` + Statements []StatementsResponse_CollectedStatementStatistics `protobuf:"bytes,1,rep,name=statements,proto3" json:"statements"` // Timestamp of the last stats reset. - LastReset time.Time `protobuf:"bytes,3,opt,name=last_reset,json=lastReset,stdtime" json:"last_reset"` + LastReset time.Time `protobuf:"bytes,3,opt,name=last_reset,json=lastReset,proto3,stdtime" json:"last_reset"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *StatementsResponse) Reset() { *m = StatementsResponse{} } +func (m *StatementsResponse) String() string { return proto.CompactTextString(m) } +func (*StatementsResponse) ProtoMessage() {} +func (*StatementsResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_status_0de29a344bd79652, []int{60} +} +func (m *StatementsResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *StatementsResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *StatementsResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_StatementsResponse.Merge(dst, src) +} +func (m *StatementsResponse) XXX_Size() int { + return m.Size() +} +func (m *StatementsResponse) XXX_DiscardUnknown() { + xxx_messageInfo_StatementsResponse.DiscardUnknown(m) } -func (m *StatementsResponse) Reset() { *m = StatementsResponse{} } -func (m *StatementsResponse) String() string { return proto.CompactTextString(m) } -func (*StatementsResponse) ProtoMessage() {} -func (*StatementsResponse) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{60} } +var xxx_messageInfo_StatementsResponse proto.InternalMessageInfo type StatementsResponse_ExtendedStatementStatisticsKey struct { - KeyData cockroach_sql.StatementStatisticsKey `protobuf:"bytes,1,opt,name=key_data,json=keyData" json:"key_data"` - NodeID github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,2,opt,name=node_id,json=nodeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"node_id,omitempty"` + KeyData roachpb.StatementStatisticsKey `protobuf:"bytes,1,opt,name=key_data,json=keyData,proto3" json:"key_data"` + NodeID github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,2,opt,name=node_id,json=nodeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"node_id,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *StatementsResponse_ExtendedStatementStatisticsKey) Reset() { @@ -966,12 +2664,36 @@ func (m *StatementsResponse_ExtendedStatementStatisticsKey) String() string { } func (*StatementsResponse_ExtendedStatementStatisticsKey) ProtoMessage() {} func (*StatementsResponse_ExtendedStatementStatisticsKey) Descriptor() ([]byte, []int) { - return fileDescriptorStatus, []int{60, 0} + return fileDescriptor_status_0de29a344bd79652, []int{60, 0} +} +func (m *StatementsResponse_ExtendedStatementStatisticsKey) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *StatementsResponse_ExtendedStatementStatisticsKey) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *StatementsResponse_ExtendedStatementStatisticsKey) XXX_Merge(src proto.Message) { + xxx_messageInfo_StatementsResponse_ExtendedStatementStatisticsKey.Merge(dst, src) +} +func (m *StatementsResponse_ExtendedStatementStatisticsKey) XXX_Size() int { + return m.Size() } +func (m *StatementsResponse_ExtendedStatementStatisticsKey) XXX_DiscardUnknown() { + xxx_messageInfo_StatementsResponse_ExtendedStatementStatisticsKey.DiscardUnknown(m) +} + +var xxx_messageInfo_StatementsResponse_ExtendedStatementStatisticsKey proto.InternalMessageInfo type StatementsResponse_CollectedStatementStatistics struct { - Key StatementsResponse_ExtendedStatementStatisticsKey `protobuf:"bytes,1,opt,name=key" json:"key"` - Stats cockroach_sql.StatementStatistics `protobuf:"bytes,2,opt,name=stats" json:"stats"` + Key StatementsResponse_ExtendedStatementStatisticsKey `protobuf:"bytes,1,opt,name=key,proto3" json:"key"` + Stats roachpb.StatementStatistics `protobuf:"bytes,2,opt,name=stats,proto3" json:"stats"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *StatementsResponse_CollectedStatementStatistics) Reset() { @@ -982,9 +2704,31 @@ func (m *StatementsResponse_CollectedStatementStatistics) String() string { } func (*StatementsResponse_CollectedStatementStatistics) ProtoMessage() {} func (*StatementsResponse_CollectedStatementStatistics) Descriptor() ([]byte, []int) { - return fileDescriptorStatus, []int{60, 1} + return fileDescriptor_status_0de29a344bd79652, []int{60, 1} +} +func (m *StatementsResponse_CollectedStatementStatistics) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *StatementsResponse_CollectedStatementStatistics) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *StatementsResponse_CollectedStatementStatistics) XXX_Merge(src proto.Message) { + xxx_messageInfo_StatementsResponse_CollectedStatementStatistics.Merge(dst, src) +} +func (m *StatementsResponse_CollectedStatementStatistics) XXX_Size() int { + return m.Size() +} +func (m *StatementsResponse_CollectedStatementStatistics) XXX_DiscardUnknown() { + xxx_messageInfo_StatementsResponse_CollectedStatementStatistics.DiscardUnknown(m) } +var xxx_messageInfo_StatementsResponse_CollectedStatementStatistics proto.InternalMessageInfo + func init() { proto.RegisterType((*CertificatesRequest)(nil), "cockroach.server.serverpb.CertificatesRequest") proto.RegisterType((*CertificateDetails)(nil), "cockroach.server.serverpb.CertificateDetails") @@ -997,6 +2741,7 @@ func init() { proto.RegisterType((*NodesResponse)(nil), "cockroach.server.serverpb.NodesResponse") proto.RegisterType((*NodeRequest)(nil), "cockroach.server.serverpb.NodeRequest") proto.RegisterType((*RaftState)(nil), "cockroach.server.serverpb.RaftState") + proto.RegisterMapType((map[uint64]RaftState_Progress)(nil), "cockroach.server.serverpb.RaftState.ProgressEntry") proto.RegisterType((*RaftState_Progress)(nil), "cockroach.server.serverpb.RaftState.Progress") proto.RegisterType((*RangeProblems)(nil), "cockroach.server.serverpb.RangeProblems") proto.RegisterType((*RangeStatistics)(nil), "cockroach.server.serverpb.RangeStatistics") @@ -1028,6 +2773,7 @@ func init() { proto.RegisterType((*RaftRangeStatus)(nil), "cockroach.server.serverpb.RaftRangeStatus") proto.RegisterType((*RaftDebugRequest)(nil), "cockroach.server.serverpb.RaftDebugRequest") proto.RegisterType((*RaftDebugResponse)(nil), "cockroach.server.serverpb.RaftDebugResponse") + proto.RegisterMapType((map[github_com_cockroachdb_cockroach_pkg_roachpb.RangeID]RaftRangeStatus)(nil), "cockroach.server.serverpb.RaftDebugResponse.RangesEntry") proto.RegisterType((*ActiveQuery)(nil), "cockroach.server.serverpb.ActiveQuery") proto.RegisterType((*ListSessionsRequest)(nil), "cockroach.server.serverpb.ListSessionsRequest") proto.RegisterType((*Session)(nil), "cockroach.server.serverpb.Session") @@ -1041,9 +2787,11 @@ func init() { proto.RegisterType((*SpanStatsResponse)(nil), "cockroach.server.serverpb.SpanStatsResponse") proto.RegisterType((*ProblemRangesRequest)(nil), "cockroach.server.serverpb.ProblemRangesRequest") proto.RegisterType((*ProblemRangesResponse)(nil), "cockroach.server.serverpb.ProblemRangesResponse") + proto.RegisterMapType((map[github_com_cockroachdb_cockroach_pkg_roachpb.NodeID]ProblemRangesResponse_NodeProblems)(nil), "cockroach.server.serverpb.ProblemRangesResponse.ProblemsByNodeIdEntry") proto.RegisterType((*ProblemRangesResponse_NodeProblems)(nil), "cockroach.server.serverpb.ProblemRangesResponse.NodeProblems") proto.RegisterType((*RangeRequest)(nil), "cockroach.server.serverpb.RangeRequest") proto.RegisterType((*RangeResponse)(nil), "cockroach.server.serverpb.RangeResponse") + proto.RegisterMapType((map[github_com_cockroachdb_cockroach_pkg_roachpb.NodeID]RangeResponse_NodeResponse)(nil), "cockroach.server.serverpb.RangeResponse.ResponsesByNodeIdEntry") proto.RegisterType((*RangeResponse_NodeResponse)(nil), "cockroach.server.serverpb.RangeResponse.NodeResponse") proto.RegisterType((*DiagnosticsRequest)(nil), "cockroach.server.serverpb.DiagnosticsRequest") proto.RegisterType((*StoresRequest)(nil), "cockroach.server.serverpb.StoresRequest") @@ -1094,16 +2842,17 @@ var _ grpc.ClientConn // is compatible with the grpc package it is being compiled against. const _ = grpc.SupportPackageIsVersion4 -// Client API for Status service - +// StatusClient is the client API for Status service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream. type StatusClient interface { Certificates(ctx context.Context, in *CertificatesRequest, opts ...grpc.CallOption) (*CertificatesResponse, error) Details(ctx context.Context, in *DetailsRequest, opts ...grpc.CallOption) (*DetailsResponse, error) Nodes(ctx context.Context, in *NodesRequest, opts ...grpc.CallOption) (*NodesResponse, error) - Node(ctx context.Context, in *NodeRequest, opts ...grpc.CallOption) (*cockroach_server_status_statuspb.NodeStatus, error) + Node(ctx context.Context, in *NodeRequest, opts ...grpc.CallOption) (*statuspb.NodeStatus, error) RaftDebug(ctx context.Context, in *RaftDebugRequest, opts ...grpc.CallOption) (*RaftDebugResponse, error) Ranges(ctx context.Context, in *RangesRequest, opts ...grpc.CallOption) (*RangesResponse, error) - Gossip(ctx context.Context, in *GossipRequest, opts ...grpc.CallOption) (*cockroach_gossip.InfoStatus, error) + Gossip(ctx context.Context, in *GossipRequest, opts ...grpc.CallOption) (*gossip.InfoStatus, error) Allocator(ctx context.Context, in *AllocatorRequest, opts ...grpc.CallOption) (*AllocatorResponse, error) AllocatorRange(ctx context.Context, in *AllocatorRangeRequest, opts ...grpc.CallOption) (*AllocatorRangeResponse, error) ListSessions(ctx context.Context, in *ListSessionsRequest, opts ...grpc.CallOption) (*ListSessionsResponse, error) @@ -1125,7 +2874,7 @@ type StatusClient interface { Logs(ctx context.Context, in *LogsRequest, opts ...grpc.CallOption) (*LogEntriesResponse, error) ProblemRanges(ctx context.Context, in *ProblemRangesRequest, opts ...grpc.CallOption) (*ProblemRangesResponse, error) Range(ctx context.Context, in *RangeRequest, opts ...grpc.CallOption) (*RangeResponse, error) - Diagnostics(ctx context.Context, in *DiagnosticsRequest, opts ...grpc.CallOption) (*cockroach_server_diagnosticspb.DiagnosticReport, error) + Diagnostics(ctx context.Context, in *DiagnosticsRequest, opts ...grpc.CallOption) (*diagnosticspb.DiagnosticReport, error) Stores(ctx context.Context, in *StoresRequest, opts ...grpc.CallOption) (*StoresResponse, error) Statements(ctx context.Context, in *StatementsRequest, opts ...grpc.CallOption) (*StatementsResponse, error) } @@ -1140,7 +2889,7 @@ func NewStatusClient(cc *grpc.ClientConn) StatusClient { func (c *statusClient) Certificates(ctx context.Context, in *CertificatesRequest, opts ...grpc.CallOption) (*CertificatesResponse, error) { out := new(CertificatesResponse) - err := grpc.Invoke(ctx, "/cockroach.server.serverpb.Status/Certificates", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Status/Certificates", in, out, opts...) if err != nil { return nil, err } @@ -1149,7 +2898,7 @@ func (c *statusClient) Certificates(ctx context.Context, in *CertificatesRequest func (c *statusClient) Details(ctx context.Context, in *DetailsRequest, opts ...grpc.CallOption) (*DetailsResponse, error) { out := new(DetailsResponse) - err := grpc.Invoke(ctx, "/cockroach.server.serverpb.Status/Details", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Status/Details", in, out, opts...) if err != nil { return nil, err } @@ -1158,16 +2907,16 @@ func (c *statusClient) Details(ctx context.Context, in *DetailsRequest, opts ... func (c *statusClient) Nodes(ctx context.Context, in *NodesRequest, opts ...grpc.CallOption) (*NodesResponse, error) { out := new(NodesResponse) - err := grpc.Invoke(ctx, "/cockroach.server.serverpb.Status/Nodes", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Status/Nodes", in, out, opts...) if err != nil { return nil, err } return out, nil } -func (c *statusClient) Node(ctx context.Context, in *NodeRequest, opts ...grpc.CallOption) (*cockroach_server_status_statuspb.NodeStatus, error) { - out := new(cockroach_server_status_statuspb.NodeStatus) - err := grpc.Invoke(ctx, "/cockroach.server.serverpb.Status/Node", in, out, c.cc, opts...) +func (c *statusClient) Node(ctx context.Context, in *NodeRequest, opts ...grpc.CallOption) (*statuspb.NodeStatus, error) { + out := new(statuspb.NodeStatus) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Status/Node", in, out, opts...) if err != nil { return nil, err } @@ -1176,7 +2925,7 @@ func (c *statusClient) Node(ctx context.Context, in *NodeRequest, opts ...grpc.C func (c *statusClient) RaftDebug(ctx context.Context, in *RaftDebugRequest, opts ...grpc.CallOption) (*RaftDebugResponse, error) { out := new(RaftDebugResponse) - err := grpc.Invoke(ctx, "/cockroach.server.serverpb.Status/RaftDebug", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Status/RaftDebug", in, out, opts...) if err != nil { return nil, err } @@ -1185,16 +2934,16 @@ func (c *statusClient) RaftDebug(ctx context.Context, in *RaftDebugRequest, opts func (c *statusClient) Ranges(ctx context.Context, in *RangesRequest, opts ...grpc.CallOption) (*RangesResponse, error) { out := new(RangesResponse) - err := grpc.Invoke(ctx, "/cockroach.server.serverpb.Status/Ranges", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Status/Ranges", in, out, opts...) if err != nil { return nil, err } return out, nil } -func (c *statusClient) Gossip(ctx context.Context, in *GossipRequest, opts ...grpc.CallOption) (*cockroach_gossip.InfoStatus, error) { - out := new(cockroach_gossip.InfoStatus) - err := grpc.Invoke(ctx, "/cockroach.server.serverpb.Status/Gossip", in, out, c.cc, opts...) +func (c *statusClient) Gossip(ctx context.Context, in *GossipRequest, opts ...grpc.CallOption) (*gossip.InfoStatus, error) { + out := new(gossip.InfoStatus) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Status/Gossip", in, out, opts...) if err != nil { return nil, err } @@ -1203,7 +2952,7 @@ func (c *statusClient) Gossip(ctx context.Context, in *GossipRequest, opts ...gr func (c *statusClient) Allocator(ctx context.Context, in *AllocatorRequest, opts ...grpc.CallOption) (*AllocatorResponse, error) { out := new(AllocatorResponse) - err := grpc.Invoke(ctx, "/cockroach.server.serverpb.Status/Allocator", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Status/Allocator", in, out, opts...) if err != nil { return nil, err } @@ -1212,7 +2961,7 @@ func (c *statusClient) Allocator(ctx context.Context, in *AllocatorRequest, opts func (c *statusClient) AllocatorRange(ctx context.Context, in *AllocatorRangeRequest, opts ...grpc.CallOption) (*AllocatorRangeResponse, error) { out := new(AllocatorRangeResponse) - err := grpc.Invoke(ctx, "/cockroach.server.serverpb.Status/AllocatorRange", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Status/AllocatorRange", in, out, opts...) if err != nil { return nil, err } @@ -1221,7 +2970,7 @@ func (c *statusClient) AllocatorRange(ctx context.Context, in *AllocatorRangeReq func (c *statusClient) ListSessions(ctx context.Context, in *ListSessionsRequest, opts ...grpc.CallOption) (*ListSessionsResponse, error) { out := new(ListSessionsResponse) - err := grpc.Invoke(ctx, "/cockroach.server.serverpb.Status/ListSessions", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Status/ListSessions", in, out, opts...) if err != nil { return nil, err } @@ -1230,7 +2979,7 @@ func (c *statusClient) ListSessions(ctx context.Context, in *ListSessionsRequest func (c *statusClient) ListLocalSessions(ctx context.Context, in *ListSessionsRequest, opts ...grpc.CallOption) (*ListSessionsResponse, error) { out := new(ListSessionsResponse) - err := grpc.Invoke(ctx, "/cockroach.server.serverpb.Status/ListLocalSessions", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Status/ListLocalSessions", in, out, opts...) if err != nil { return nil, err } @@ -1239,7 +2988,7 @@ func (c *statusClient) ListLocalSessions(ctx context.Context, in *ListSessionsRe func (c *statusClient) CancelQuery(ctx context.Context, in *CancelQueryRequest, opts ...grpc.CallOption) (*CancelQueryResponse, error) { out := new(CancelQueryResponse) - err := grpc.Invoke(ctx, "/cockroach.server.serverpb.Status/CancelQuery", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Status/CancelQuery", in, out, opts...) if err != nil { return nil, err } @@ -1248,7 +2997,7 @@ func (c *statusClient) CancelQuery(ctx context.Context, in *CancelQueryRequest, func (c *statusClient) CancelSession(ctx context.Context, in *CancelSessionRequest, opts ...grpc.CallOption) (*CancelSessionResponse, error) { out := new(CancelSessionResponse) - err := grpc.Invoke(ctx, "/cockroach.server.serverpb.Status/CancelSession", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Status/CancelSession", in, out, opts...) if err != nil { return nil, err } @@ -1257,7 +3006,7 @@ func (c *statusClient) CancelSession(ctx context.Context, in *CancelSessionReque func (c *statusClient) SpanStats(ctx context.Context, in *SpanStatsRequest, opts ...grpc.CallOption) (*SpanStatsResponse, error) { out := new(SpanStatsResponse) - err := grpc.Invoke(ctx, "/cockroach.server.serverpb.Status/SpanStats", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Status/SpanStats", in, out, opts...) if err != nil { return nil, err } @@ -1266,7 +3015,7 @@ func (c *statusClient) SpanStats(ctx context.Context, in *SpanStatsRequest, opts func (c *statusClient) Stacks(ctx context.Context, in *StacksRequest, opts ...grpc.CallOption) (*JSONResponse, error) { out := new(JSONResponse) - err := grpc.Invoke(ctx, "/cockroach.server.serverpb.Status/Stacks", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Status/Stacks", in, out, opts...) if err != nil { return nil, err } @@ -1275,7 +3024,7 @@ func (c *statusClient) Stacks(ctx context.Context, in *StacksRequest, opts ...gr func (c *statusClient) Profile(ctx context.Context, in *ProfileRequest, opts ...grpc.CallOption) (*JSONResponse, error) { out := new(JSONResponse) - err := grpc.Invoke(ctx, "/cockroach.server.serverpb.Status/Profile", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Status/Profile", in, out, opts...) if err != nil { return nil, err } @@ -1284,7 +3033,7 @@ func (c *statusClient) Profile(ctx context.Context, in *ProfileRequest, opts ... func (c *statusClient) Metrics(ctx context.Context, in *MetricsRequest, opts ...grpc.CallOption) (*JSONResponse, error) { out := new(JSONResponse) - err := grpc.Invoke(ctx, "/cockroach.server.serverpb.Status/Metrics", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Status/Metrics", in, out, opts...) if err != nil { return nil, err } @@ -1293,7 +3042,7 @@ func (c *statusClient) Metrics(ctx context.Context, in *MetricsRequest, opts ... func (c *statusClient) GetFiles(ctx context.Context, in *GetFilesRequest, opts ...grpc.CallOption) (*GetFilesResponse, error) { out := new(GetFilesResponse) - err := grpc.Invoke(ctx, "/cockroach.server.serverpb.Status/GetFiles", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Status/GetFiles", in, out, opts...) if err != nil { return nil, err } @@ -1302,7 +3051,7 @@ func (c *statusClient) GetFiles(ctx context.Context, in *GetFilesRequest, opts . func (c *statusClient) LogFilesList(ctx context.Context, in *LogFilesListRequest, opts ...grpc.CallOption) (*LogFilesListResponse, error) { out := new(LogFilesListResponse) - err := grpc.Invoke(ctx, "/cockroach.server.serverpb.Status/LogFilesList", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Status/LogFilesList", in, out, opts...) if err != nil { return nil, err } @@ -1311,7 +3060,7 @@ func (c *statusClient) LogFilesList(ctx context.Context, in *LogFilesListRequest func (c *statusClient) LogFile(ctx context.Context, in *LogFileRequest, opts ...grpc.CallOption) (*LogEntriesResponse, error) { out := new(LogEntriesResponse) - err := grpc.Invoke(ctx, "/cockroach.server.serverpb.Status/LogFile", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Status/LogFile", in, out, opts...) if err != nil { return nil, err } @@ -1320,7 +3069,7 @@ func (c *statusClient) LogFile(ctx context.Context, in *LogFileRequest, opts ... func (c *statusClient) Logs(ctx context.Context, in *LogsRequest, opts ...grpc.CallOption) (*LogEntriesResponse, error) { out := new(LogEntriesResponse) - err := grpc.Invoke(ctx, "/cockroach.server.serverpb.Status/Logs", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Status/Logs", in, out, opts...) if err != nil { return nil, err } @@ -1329,7 +3078,7 @@ func (c *statusClient) Logs(ctx context.Context, in *LogsRequest, opts ...grpc.C func (c *statusClient) ProblemRanges(ctx context.Context, in *ProblemRangesRequest, opts ...grpc.CallOption) (*ProblemRangesResponse, error) { out := new(ProblemRangesResponse) - err := grpc.Invoke(ctx, "/cockroach.server.serverpb.Status/ProblemRanges", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Status/ProblemRanges", in, out, opts...) if err != nil { return nil, err } @@ -1338,16 +3087,16 @@ func (c *statusClient) ProblemRanges(ctx context.Context, in *ProblemRangesReque func (c *statusClient) Range(ctx context.Context, in *RangeRequest, opts ...grpc.CallOption) (*RangeResponse, error) { out := new(RangeResponse) - err := grpc.Invoke(ctx, "/cockroach.server.serverpb.Status/Range", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Status/Range", in, out, opts...) if err != nil { return nil, err } return out, nil } -func (c *statusClient) Diagnostics(ctx context.Context, in *DiagnosticsRequest, opts ...grpc.CallOption) (*cockroach_server_diagnosticspb.DiagnosticReport, error) { - out := new(cockroach_server_diagnosticspb.DiagnosticReport) - err := grpc.Invoke(ctx, "/cockroach.server.serverpb.Status/Diagnostics", in, out, c.cc, opts...) +func (c *statusClient) Diagnostics(ctx context.Context, in *DiagnosticsRequest, opts ...grpc.CallOption) (*diagnosticspb.DiagnosticReport, error) { + out := new(diagnosticspb.DiagnosticReport) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Status/Diagnostics", in, out, opts...) if err != nil { return nil, err } @@ -1356,7 +3105,7 @@ func (c *statusClient) Diagnostics(ctx context.Context, in *DiagnosticsRequest, func (c *statusClient) Stores(ctx context.Context, in *StoresRequest, opts ...grpc.CallOption) (*StoresResponse, error) { out := new(StoresResponse) - err := grpc.Invoke(ctx, "/cockroach.server.serverpb.Status/Stores", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Status/Stores", in, out, opts...) if err != nil { return nil, err } @@ -1365,23 +3114,22 @@ func (c *statusClient) Stores(ctx context.Context, in *StoresRequest, opts ...gr func (c *statusClient) Statements(ctx context.Context, in *StatementsRequest, opts ...grpc.CallOption) (*StatementsResponse, error) { out := new(StatementsResponse) - err := grpc.Invoke(ctx, "/cockroach.server.serverpb.Status/Statements", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/cockroach.server.serverpb.Status/Statements", in, out, opts...) if err != nil { return nil, err } return out, nil } -// Server API for Status service - +// StatusServer is the server API for Status service. type StatusServer interface { Certificates(context.Context, *CertificatesRequest) (*CertificatesResponse, error) Details(context.Context, *DetailsRequest) (*DetailsResponse, error) Nodes(context.Context, *NodesRequest) (*NodesResponse, error) - Node(context.Context, *NodeRequest) (*cockroach_server_status_statuspb.NodeStatus, error) + Node(context.Context, *NodeRequest) (*statuspb.NodeStatus, error) RaftDebug(context.Context, *RaftDebugRequest) (*RaftDebugResponse, error) Ranges(context.Context, *RangesRequest) (*RangesResponse, error) - Gossip(context.Context, *GossipRequest) (*cockroach_gossip.InfoStatus, error) + Gossip(context.Context, *GossipRequest) (*gossip.InfoStatus, error) Allocator(context.Context, *AllocatorRequest) (*AllocatorResponse, error) AllocatorRange(context.Context, *AllocatorRangeRequest) (*AllocatorRangeResponse, error) ListSessions(context.Context, *ListSessionsRequest) (*ListSessionsResponse, error) @@ -1403,7 +3151,7 @@ type StatusServer interface { Logs(context.Context, *LogsRequest) (*LogEntriesResponse, error) ProblemRanges(context.Context, *ProblemRangesRequest) (*ProblemRangesResponse, error) Range(context.Context, *RangeRequest) (*RangeResponse, error) - Diagnostics(context.Context, *DiagnosticsRequest) (*cockroach_server_diagnosticspb.DiagnosticReport, error) + Diagnostics(context.Context, *DiagnosticsRequest) (*diagnosticspb.DiagnosticReport, error) Stores(context.Context, *StoresRequest) (*StoresResponse, error) Statements(context.Context, *StatementsRequest) (*StatementsResponse, error) } @@ -2423,7 +4171,7 @@ func (m *RaftState) MarshalTo(dAtA []byte) (int, error) { for k := range m.Progress { keysForProgress = append(keysForProgress, uint64(k)) } - sortkeys.Uint64s(keysForProgress) + github_com_gogo_protobuf_sortkeys.Uint64s(keysForProgress) for _, k := range keysForProgress { dAtA[i] = 0x32 i++ @@ -2611,13 +4359,13 @@ func (m *RangeStatistics) MarshalTo(dAtA []byte) (int, error) { if m.QueriesPerSecond != 0 { dAtA[i] = 0x9 i++ - binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.QueriesPerSecond)))) + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.QueriesPerSecond)))) i += 8 } if m.WritesPerSecond != 0 { dAtA[i] = 0x11 i++ - binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.WritesPerSecond)))) + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.WritesPerSecond)))) i += 8 } return i, nil @@ -2896,8 +4644,8 @@ func (m *TraceEvent) MarshalTo(dAtA []byte) (int, error) { _ = l dAtA[i] = 0xa i++ - i = encodeVarintStatus(dAtA, i, uint64(types.SizeOfStdTime(m.Time))) - n16, err := types.StdTimeMarshalTo(m.Time, dAtA[i:]) + i = encodeVarintStatus(dAtA, i, uint64(github_com_gogo_protobuf_types.SizeOfStdTime(m.Time))) + n16, err := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Time, dAtA[i:]) if err != nil { return 0, err } @@ -3620,7 +5368,7 @@ func (m *RaftDebugResponse) MarshalTo(dAtA []byte) (int, error) { for k := range m.Ranges { keysForRanges = append(keysForRanges, int64(k)) } - sortkeys.Int64s(keysForRanges) + github_com_gogo_protobuf_sortkeys.Int64s(keysForRanges) for _, k := range keysForRanges { dAtA[i] = 0xa i++ @@ -3689,8 +5437,8 @@ func (m *ActiveQuery) MarshalTo(dAtA []byte) (int, error) { } dAtA[i] = 0x1a i++ - i = encodeVarintStatus(dAtA, i, uint64(types.SizeOfStdTime(m.Start))) - n24, err := types.StdTimeMarshalTo(m.Start, dAtA[i:]) + i = encodeVarintStatus(dAtA, i, uint64(github_com_gogo_protobuf_types.SizeOfStdTime(m.Start))) + n24, err := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Start, dAtA[i:]) if err != nil { return 0, err } @@ -3789,8 +5537,8 @@ func (m *Session) MarshalTo(dAtA []byte) (int, error) { } dAtA[i] = 0x32 i++ - i = encodeVarintStatus(dAtA, i, uint64(types.SizeOfStdTime(m.Start))) - n25, err := types.StdTimeMarshalTo(m.Start, dAtA[i:]) + i = encodeVarintStatus(dAtA, i, uint64(github_com_gogo_protobuf_types.SizeOfStdTime(m.Start))) + n25, err := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Start, dAtA[i:]) if err != nil { return 0, err } @@ -4162,7 +5910,7 @@ func (m *ProblemRangesResponse) MarshalTo(dAtA []byte) (int, error) { for k := range m.ProblemsByNodeID { keysForProblemsByNodeID = append(keysForProblemsByNodeID, int32(k)) } - sortkeys.Int32s(keysForProblemsByNodeID) + github_com_gogo_protobuf_sortkeys.Int32s(keysForProblemsByNodeID) for _, k := range keysForProblemsByNodeID { dAtA[i] = 0x4a i++ @@ -4393,7 +6141,7 @@ func (m *RangeResponse) MarshalTo(dAtA []byte) (int, error) { for k := range m.ResponsesByNodeID { keysForResponsesByNodeID = append(keysForResponsesByNodeID, int32(k)) } - sortkeys.Int32s(keysForResponsesByNodeID) + github_com_gogo_protobuf_sortkeys.Int32s(keysForResponsesByNodeID) for _, k := range keysForResponsesByNodeID { dAtA[i] = 0x1a i++ @@ -4647,8 +6395,8 @@ func (m *StatementsResponse) MarshalTo(dAtA []byte) (int, error) { } dAtA[i] = 0x1a i++ - i = encodeVarintStatus(dAtA, i, uint64(types.SizeOfStdTime(m.LastReset))) - n44, err := types.StdTimeMarshalTo(m.LastReset, dAtA[i:]) + i = encodeVarintStatus(dAtA, i, uint64(github_com_gogo_protobuf_types.SizeOfStdTime(m.LastReset))) + n44, err := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.LastReset, dAtA[i:]) if err != nil { return 0, err } @@ -4731,6 +6479,9 @@ func encodeVarintStatus(dAtA []byte, offset int, v uint64) int { return offset + 1 } func (m *CertificatesRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.NodeId) @@ -4741,6 +6492,9 @@ func (m *CertificatesRequest) Size() (n int) { } func (m *CertificateDetails) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Type != 0 { @@ -4764,6 +6518,9 @@ func (m *CertificateDetails) Size() (n int) { } func (m *CertificateDetails_Fields) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Issuer) @@ -4810,6 +6567,9 @@ func (m *CertificateDetails_Fields) Size() (n int) { } func (m *CertificatesResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.Certificates) > 0 { @@ -4822,6 +6582,9 @@ func (m *CertificatesResponse) Size() (n int) { } func (m *DetailsRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.NodeId) @@ -4835,6 +6598,9 @@ func (m *DetailsRequest) Size() (n int) { } func (m *SystemInfo) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.SystemInfo) @@ -4849,6 +6615,9 @@ func (m *SystemInfo) Size() (n int) { } func (m *DetailsResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.NodeID != 0 { @@ -4864,12 +6633,18 @@ func (m *DetailsResponse) Size() (n int) { } func (m *NodesRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l return n } func (m *NodesResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.Nodes) > 0 { @@ -4882,6 +6657,9 @@ func (m *NodesResponse) Size() (n int) { } func (m *NodeRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.NodeId) @@ -4892,6 +6670,9 @@ func (m *NodeRequest) Size() (n int) { } func (m *RaftState) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.ReplicaID != 0 { @@ -4925,6 +6706,9 @@ func (m *RaftState) Size() (n int) { } func (m *RaftState_Progress) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Match != 0 { @@ -4947,6 +6731,9 @@ func (m *RaftState_Progress) Size() (n int) { } func (m *RangeProblems) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Unavailable { @@ -4974,6 +6761,9 @@ func (m *RangeProblems) Size() (n int) { } func (m *RangeStatistics) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.QueriesPerSecond != 0 { @@ -4986,6 +6776,9 @@ func (m *RangeStatistics) Size() (n int) { } func (m *PrettySpan) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.StartKey) @@ -5000,6 +6793,9 @@ func (m *PrettySpan) Size() (n int) { } func (m *RangeInfo) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.Span.Size() @@ -5044,6 +6840,9 @@ func (m *RangeInfo) Size() (n int) { } func (m *RangesRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.NodeId) @@ -5061,6 +6860,9 @@ func (m *RangesRequest) Size() (n int) { } func (m *RangesResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.Ranges) > 0 { @@ -5073,6 +6875,9 @@ func (m *RangesResponse) Size() (n int) { } func (m *GossipRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.NodeId) @@ -5083,9 +6888,12 @@ func (m *GossipRequest) Size() (n int) { } func (m *TraceEvent) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l - l = types.SizeOfStdTime(m.Time) + l = github_com_gogo_protobuf_types.SizeOfStdTime(m.Time) n += 1 + l + sovStatus(uint64(l)) l = len(m.Message) if l > 0 { @@ -5095,6 +6903,9 @@ func (m *TraceEvent) Size() (n int) { } func (m *AllocatorDryRun) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.RangeID != 0 { @@ -5110,6 +6921,9 @@ func (m *AllocatorDryRun) Size() (n int) { } func (m *AllocatorRangeRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.RangeId != 0 { @@ -5119,6 +6933,9 @@ func (m *AllocatorRangeRequest) Size() (n int) { } func (m *AllocatorRangeResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.NodeID != 0 { @@ -5132,6 +6949,9 @@ func (m *AllocatorRangeResponse) Size() (n int) { } func (m *AllocatorRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.NodeId) @@ -5149,6 +6969,9 @@ func (m *AllocatorRequest) Size() (n int) { } func (m *AllocatorResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.DryRuns) > 0 { @@ -5161,6 +6984,9 @@ func (m *AllocatorResponse) Size() (n int) { } func (m *JSONResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Data) @@ -5171,6 +6997,9 @@ func (m *JSONResponse) Size() (n int) { } func (m *LogsRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.NodeId) @@ -5201,6 +7030,9 @@ func (m *LogsRequest) Size() (n int) { } func (m *LogEntriesResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.Entries) > 0 { @@ -5213,6 +7045,9 @@ func (m *LogEntriesResponse) Size() (n int) { } func (m *LogFilesListRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.NodeId) @@ -5223,6 +7058,9 @@ func (m *LogFilesListRequest) Size() (n int) { } func (m *LogFilesListResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.Files) > 0 { @@ -5235,6 +7073,9 @@ func (m *LogFilesListResponse) Size() (n int) { } func (m *LogFileRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.NodeId) @@ -5249,6 +7090,9 @@ func (m *LogFileRequest) Size() (n int) { } func (m *StacksRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.NodeId) @@ -5259,6 +7103,9 @@ func (m *StacksRequest) Size() (n int) { } func (m *File) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Name) @@ -5276,6 +7123,9 @@ func (m *File) Size() (n int) { } func (m *GetFilesRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.NodeId) @@ -5298,6 +7148,9 @@ func (m *GetFilesRequest) Size() (n int) { } func (m *GetFilesResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.Files) > 0 { @@ -5310,6 +7163,9 @@ func (m *GetFilesResponse) Size() (n int) { } func (m *ProfileRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.NodeId) @@ -5323,6 +7179,9 @@ func (m *ProfileRequest) Size() (n int) { } func (m *MetricsRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.NodeId) @@ -5333,6 +7192,9 @@ func (m *MetricsRequest) Size() (n int) { } func (m *RaftRangeNode) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.NodeID != 0 { @@ -5344,6 +7206,9 @@ func (m *RaftRangeNode) Size() (n int) { } func (m *RaftRangeError) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Message) @@ -5354,6 +7219,9 @@ func (m *RaftRangeError) Size() (n int) { } func (m *RaftRangeStatus) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.RangeID != 0 { @@ -5375,6 +7243,9 @@ func (m *RaftRangeStatus) Size() (n int) { } func (m *RaftDebugRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.RangeIDs) > 0 { @@ -5388,6 +7259,9 @@ func (m *RaftDebugRequest) Size() (n int) { } func (m *RaftDebugResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.Ranges) > 0 { @@ -5409,6 +7283,9 @@ func (m *RaftDebugResponse) Size() (n int) { } func (m *ActiveQuery) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.ID) @@ -5419,7 +7296,7 @@ func (m *ActiveQuery) Size() (n int) { if l > 0 { n += 1 + l + sovStatus(uint64(l)) } - l = types.SizeOfStdTime(m.Start) + l = github_com_gogo_protobuf_types.SizeOfStdTime(m.Start) n += 1 + l + sovStatus(uint64(l)) if m.IsDistributed { n += 2 @@ -5431,6 +7308,9 @@ func (m *ActiveQuery) Size() (n int) { } func (m *ListSessionsRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Username) @@ -5441,6 +7321,9 @@ func (m *ListSessionsRequest) Size() (n int) { } func (m *Session) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.NodeID != 0 { @@ -5464,7 +7347,7 @@ func (m *Session) Size() (n int) { n += 1 + l + sovStatus(uint64(l)) } } - l = types.SizeOfStdTime(m.Start) + l = github_com_gogo_protobuf_types.SizeOfStdTime(m.Start) n += 1 + l + sovStatus(uint64(l)) if m.KvTxnID != nil { l = m.KvTxnID.Size() @@ -5488,6 +7371,9 @@ func (m *Session) Size() (n int) { } func (m *ListSessionsError) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.NodeID != 0 { @@ -5501,6 +7387,9 @@ func (m *ListSessionsError) Size() (n int) { } func (m *ListSessionsResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.Sessions) > 0 { @@ -5519,6 +7408,9 @@ func (m *ListSessionsResponse) Size() (n int) { } func (m *CancelQueryRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.NodeId) @@ -5537,6 +7429,9 @@ func (m *CancelQueryRequest) Size() (n int) { } func (m *CancelQueryResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Canceled { @@ -5550,6 +7445,9 @@ func (m *CancelQueryResponse) Size() (n int) { } func (m *CancelSessionRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.NodeId) @@ -5568,6 +7466,9 @@ func (m *CancelSessionRequest) Size() (n int) { } func (m *CancelSessionResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Canceled { @@ -5581,6 +7482,9 @@ func (m *CancelSessionResponse) Size() (n int) { } func (m *SpanStatsRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.NodeID) @@ -5599,6 +7503,9 @@ func (m *SpanStatsRequest) Size() (n int) { } func (m *SpanStatsResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.TotalStats.Size() @@ -5613,6 +7520,9 @@ func (m *SpanStatsResponse) Size() (n int) { } func (m *ProblemRangesRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.NodeID) @@ -5623,6 +7533,9 @@ func (m *ProblemRangesRequest) Size() (n int) { } func (m *ProblemRangesResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.NodeID != 0 { @@ -5641,6 +7554,9 @@ func (m *ProblemRangesResponse) Size() (n int) { } func (m *ProblemRangesResponse_NodeProblems) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.ErrorMessage) @@ -5700,6 +7616,9 @@ func (m *ProblemRangesResponse_NodeProblems) Size() (n int) { } func (m *RangeRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.RangeId != 0 { @@ -5709,6 +7628,9 @@ func (m *RangeRequest) Size() (n int) { } func (m *RangeResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.NodeID != 0 { @@ -5730,6 +7652,9 @@ func (m *RangeResponse) Size() (n int) { } func (m *RangeResponse_NodeResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Response { @@ -5749,6 +7674,9 @@ func (m *RangeResponse_NodeResponse) Size() (n int) { } func (m *DiagnosticsRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.NodeId) @@ -5759,6 +7687,9 @@ func (m *DiagnosticsRequest) Size() (n int) { } func (m *StoresRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.NodeId) @@ -5769,6 +7700,9 @@ func (m *StoresRequest) Size() (n int) { } func (m *StoreDetails) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.StoreID != 0 { @@ -5794,6 +7728,9 @@ func (m *StoreDetails) Size() (n int) { } func (m *StoresResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.Stores) > 0 { @@ -5806,6 +7743,9 @@ func (m *StoresResponse) Size() (n int) { } func (m *StatementsRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.NodeID) @@ -5816,6 +7756,9 @@ func (m *StatementsRequest) Size() (n int) { } func (m *StatementsResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.Statements) > 0 { @@ -5824,12 +7767,15 @@ func (m *StatementsResponse) Size() (n int) { n += 1 + l + sovStatus(uint64(l)) } } - l = types.SizeOfStdTime(m.LastReset) + l = github_com_gogo_protobuf_types.SizeOfStdTime(m.LastReset) n += 1 + l + sovStatus(uint64(l)) return n } func (m *StatementsResponse_ExtendedStatementStatisticsKey) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.KeyData.Size() @@ -5841,6 +7787,9 @@ func (m *StatementsResponse_ExtendedStatementStatisticsKey) Size() (n int) { } func (m *StatementsResponse_CollectedStatementStatistics) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.Key.Size() @@ -6945,7 +8894,7 @@ func (m *NodesResponse) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Nodes = append(m.Nodes, cockroach_server_status_statuspb.NodeStatus{}) + m.Nodes = append(m.Nodes, statuspb.NodeStatus{}) if err := m.Nodes[len(m.Nodes)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -7730,7 +9679,7 @@ func (m *RangeStatistics) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - v = uint64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 m.QueriesPerSecond = float64(math.Float64frombits(v)) case 2: @@ -7741,7 +9690,7 @@ func (m *RangeStatistics) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - v = uint64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 m.WritesPerSecond = float64(math.Float64frombits(v)) default: @@ -8085,7 +10034,7 @@ func (m *RangeInfo) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.LeaseHistory = append(m.LeaseHistory, cockroach_roachpb1.Lease{}) + m.LeaseHistory = append(m.LeaseHistory, roachpb.Lease{}) if err := m.LeaseHistory[len(m.LeaseHistory)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -8400,6 +10349,17 @@ func (m *RangesRequest) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + var count int + for _, integer := range dAtA { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.RangeIDs) == 0 { + m.RangeIDs = make([]github_com_cockroachdb_cockroach_pkg_roachpb.RangeID, 0, elementCount) + } for iNdEx < postIndex { var v github_com_cockroachdb_cockroach_pkg_roachpb.RangeID for shift := uint(0); ; shift += 7 { @@ -8657,7 +10617,7 @@ func (m *TraceEvent) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if err := types.StdTimeUnmarshal(&m.Time, dAtA[iNdEx:postIndex]); err != nil { + if err := github_com_gogo_protobuf_types.StdTimeUnmarshal(&m.Time, dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -9081,6 +11041,17 @@ func (m *AllocatorRequest) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + var count int + for _, integer := range dAtA { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.RangeIDs) == 0 { + m.RangeIDs = make([]github_com_cockroachdb_cockroach_pkg_roachpb.RangeID, 0, elementCount) + } for iNdEx < postIndex { var v github_com_cockroachdb_cockroach_pkg_roachpb.RangeID for shift := uint(0); ; shift += 7 { @@ -9564,7 +11535,7 @@ func (m *LogEntriesResponse) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Entries = append(m.Entries, cockroach_util_log.Entry{}) + m.Entries = append(m.Entries, log.Entry{}) if err := m.Entries[len(m.Entries)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -9724,7 +11695,7 @@ func (m *LogFilesListResponse) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Files = append(m.Files, cockroach_util_log.FileInfo{}) + m.Files = append(m.Files, log.FileInfo{}) if err := m.Files[len(m.Files)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -10850,6 +12821,17 @@ func (m *RaftDebugRequest) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + var count int + for _, integer := range dAtA { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.RangeIDs) == 0 { + m.RangeIDs = make([]github_com_cockroachdb_cockroach_pkg_roachpb.RangeID, 0, elementCount) + } for iNdEx < postIndex { var v github_com_cockroachdb_cockroach_pkg_roachpb.RangeID for shift := uint(0); ; shift += 7 { @@ -11198,7 +13180,7 @@ func (m *ActiveQuery) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if err := types.StdTimeUnmarshal(&m.Start, dAtA[iNdEx:postIndex]); err != nil { + if err := github_com_gogo_protobuf_types.StdTimeUnmarshal(&m.Start, dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -11533,7 +13515,7 @@ func (m *Session) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if err := types.StdTimeUnmarshal(&m.Start, dAtA[iNdEx:postIndex]); err != nil { + if err := github_com_gogo_protobuf_types.StdTimeUnmarshal(&m.Start, dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -12990,6 +14972,17 @@ func (m *ProblemRangesResponse_NodeProblems) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + var count int + for _, integer := range dAtA { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.UnavailableRangeIDs) == 0 { + m.UnavailableRangeIDs = make([]github_com_cockroachdb_cockroach_pkg_roachpb.RangeID, 0, elementCount) + } for iNdEx < postIndex { var v github_com_cockroachdb_cockroach_pkg_roachpb.RangeID for shift := uint(0); ; shift += 7 { @@ -13052,6 +15045,17 @@ func (m *ProblemRangesResponse_NodeProblems) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + var count int + for _, integer := range dAtA { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.RaftLeaderNotLeaseHolderRangeIDs) == 0 { + m.RaftLeaderNotLeaseHolderRangeIDs = make([]github_com_cockroachdb_cockroach_pkg_roachpb.RangeID, 0, elementCount) + } for iNdEx < postIndex { var v github_com_cockroachdb_cockroach_pkg_roachpb.RangeID for shift := uint(0); ; shift += 7 { @@ -13114,6 +15118,17 @@ func (m *ProblemRangesResponse_NodeProblems) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + var count int + for _, integer := range dAtA { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.NoRaftLeaderRangeIDs) == 0 { + m.NoRaftLeaderRangeIDs = make([]github_com_cockroachdb_cockroach_pkg_roachpb.RangeID, 0, elementCount) + } for iNdEx < postIndex { var v github_com_cockroachdb_cockroach_pkg_roachpb.RangeID for shift := uint(0); ; shift += 7 { @@ -13176,6 +15191,17 @@ func (m *ProblemRangesResponse_NodeProblems) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + var count int + for _, integer := range dAtA { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.NoLeaseRangeIDs) == 0 { + m.NoLeaseRangeIDs = make([]github_com_cockroachdb_cockroach_pkg_roachpb.RangeID, 0, elementCount) + } for iNdEx < postIndex { var v github_com_cockroachdb_cockroach_pkg_roachpb.RangeID for shift := uint(0); ; shift += 7 { @@ -13238,6 +15264,17 @@ func (m *ProblemRangesResponse_NodeProblems) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + var count int + for _, integer := range dAtA { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.UnderreplicatedRangeIDs) == 0 { + m.UnderreplicatedRangeIDs = make([]github_com_cockroachdb_cockroach_pkg_roachpb.RangeID, 0, elementCount) + } for iNdEx < postIndex { var v github_com_cockroachdb_cockroach_pkg_roachpb.RangeID for shift := uint(0); ; shift += 7 { @@ -13300,6 +15337,17 @@ func (m *ProblemRangesResponse_NodeProblems) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + var count int + for _, integer := range dAtA { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.QuiescentEqualsTickingRangeIDs) == 0 { + m.QuiescentEqualsTickingRangeIDs = make([]github_com_cockroachdb_cockroach_pkg_roachpb.RangeID, 0, elementCount) + } for iNdEx < postIndex { var v github_com_cockroachdb_cockroach_pkg_roachpb.RangeID for shift := uint(0); ; shift += 7 { @@ -13362,6 +15410,17 @@ func (m *ProblemRangesResponse_NodeProblems) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + var count int + for _, integer := range dAtA { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.RaftLogTooLargeRangeIDs) == 0 { + m.RaftLogTooLargeRangeIDs = make([]github_com_cockroachdb_cockroach_pkg_roachpb.RangeID, 0, elementCount) + } for iNdEx < postIndex { var v github_com_cockroachdb_cockroach_pkg_roachpb.RangeID for shift := uint(0); ; shift += 7 { @@ -14383,7 +16442,7 @@ func (m *StatementsResponse) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if err := types.StdTimeUnmarshal(&m.LastReset, dAtA[iNdEx:postIndex]); err != nil { + if err := github_com_gogo_protobuf_types.StdTimeUnmarshal(&m.LastReset, dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -14722,9 +16781,11 @@ var ( ErrIntOverflowStatus = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("server/serverpb/status.proto", fileDescriptorStatus) } +func init() { + proto.RegisterFile("server/serverpb/status.proto", fileDescriptor_status_0de29a344bd79652) +} -var fileDescriptorStatus = []byte{ +var fileDescriptor_status_0de29a344bd79652 = []byte{ // 4627 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xcc, 0x5b, 0x5f, 0x6c, 0x1c, 0x49, 0x5a, 0x4f, 0xdb, 0x33, 0xe3, 0x99, 0xcf, 0xff, 0xc6, 0x65, 0x27, 0x99, 0x4c, 0xb2, 0x9e, 0x6c, diff --git a/pkg/server/status/statuspb/status.pb.go b/pkg/server/status/statuspb/status.pb.go index 873c6042097b..3cd9c0227215 100644 --- a/pkg/server/status/statuspb/status.pb.go +++ b/pkg/server/status/statuspb/status.pb.go @@ -1,30 +1,18 @@ // Code generated by protoc-gen-gogo. DO NOT EDIT. // source: server/status/statuspb/status.proto -/* - Package statuspb is a generated protocol buffer package. - - It is generated from these files: - server/status/statuspb/status.proto - - It has these top-level messages: - StoreStatus - NodeStatus - HealthAlert - HealthCheckResult -*/ package statuspb import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" -import cockroach_roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" -import cockroach_build "github.com/cockroachdb/cockroach/pkg/build" +import build "github.com/cockroachdb/cockroach/pkg/build" +import roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" import github_com_cockroachdb_cockroach_pkg_roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" -import sortkeys "github.com/gogo/protobuf/sortkeys" -import binary "encoding/binary" +import github_com_gogo_protobuf_sortkeys "github.com/gogo/protobuf/sortkeys" +import encoding_binary "encoding/binary" import io "io" @@ -58,92 +46,226 @@ var HealthAlert_Category_value = map[string]int32{ func (x HealthAlert_Category) String() string { return proto.EnumName(HealthAlert_Category_name, int32(x)) } -func (HealthAlert_Category) EnumDescriptor() ([]byte, []int) { return fileDescriptorStatus, []int{2, 0} } +func (HealthAlert_Category) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_status_39af85395986a59c, []int{2, 0} +} // StoreStatus records the most recent values of metrics for a store. type StoreStatus struct { - Desc cockroach_roachpb.StoreDescriptor `protobuf:"bytes,1,opt,name=desc" json:"desc"` - Metrics map[string]float64 `protobuf:"bytes,2,rep,name=metrics" json:"metrics,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"fixed64,2,opt,name=value,proto3"` + Desc roachpb.StoreDescriptor `protobuf:"bytes,1,opt,name=desc,proto3" json:"desc"` + Metrics map[string]float64 `protobuf:"bytes,2,rep,name=metrics,proto3" json:"metrics,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"fixed64,2,opt,name=value,proto3"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *StoreStatus) Reset() { *m = StoreStatus{} } -func (m *StoreStatus) String() string { return proto.CompactTextString(m) } -func (*StoreStatus) ProtoMessage() {} -func (*StoreStatus) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{0} } +func (m *StoreStatus) Reset() { *m = StoreStatus{} } +func (m *StoreStatus) String() string { return proto.CompactTextString(m) } +func (*StoreStatus) ProtoMessage() {} +func (*StoreStatus) Descriptor() ([]byte, []int) { + return fileDescriptor_status_39af85395986a59c, []int{0} +} +func (m *StoreStatus) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *StoreStatus) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *StoreStatus) XXX_Merge(src proto.Message) { + xxx_messageInfo_StoreStatus.Merge(dst, src) +} +func (m *StoreStatus) XXX_Size() int { + return m.Size() +} +func (m *StoreStatus) XXX_DiscardUnknown() { + xxx_messageInfo_StoreStatus.DiscardUnknown(m) +} + +var xxx_messageInfo_StoreStatus proto.InternalMessageInfo // NodeStatus records the most recent values of metrics for a node. type NodeStatus struct { - Desc cockroach_roachpb.NodeDescriptor `protobuf:"bytes,1,opt,name=desc" json:"desc"` - BuildInfo cockroach_build.Info `protobuf:"bytes,2,opt,name=build_info,json=buildInfo" json:"build_info"` - StartedAt int64 `protobuf:"varint,3,opt,name=started_at,json=startedAt,proto3" json:"started_at,omitempty"` - UpdatedAt int64 `protobuf:"varint,4,opt,name=updated_at,json=updatedAt,proto3" json:"updated_at,omitempty"` - Metrics map[string]float64 `protobuf:"bytes,5,rep,name=metrics" json:"metrics,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"fixed64,2,opt,name=value,proto3"` - StoreStatuses []StoreStatus `protobuf:"bytes,6,rep,name=store_statuses,json=storeStatuses" json:"store_statuses"` - Args []string `protobuf:"bytes,7,rep,name=args" json:"args,omitempty"` - Env []string `protobuf:"bytes,8,rep,name=env" json:"env,omitempty"` + Desc roachpb.NodeDescriptor `protobuf:"bytes,1,opt,name=desc,proto3" json:"desc"` + BuildInfo build.Info `protobuf:"bytes,2,opt,name=build_info,json=buildInfo,proto3" json:"build_info"` + StartedAt int64 `protobuf:"varint,3,opt,name=started_at,json=startedAt,proto3" json:"started_at,omitempty"` + UpdatedAt int64 `protobuf:"varint,4,opt,name=updated_at,json=updatedAt,proto3" json:"updated_at,omitempty"` + Metrics map[string]float64 `protobuf:"bytes,5,rep,name=metrics,proto3" json:"metrics,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"fixed64,2,opt,name=value,proto3"` + StoreStatuses []StoreStatus `protobuf:"bytes,6,rep,name=store_statuses,json=storeStatuses,proto3" json:"store_statuses"` + Args []string `protobuf:"bytes,7,rep,name=args,proto3" json:"args,omitempty"` + Env []string `protobuf:"bytes,8,rep,name=env,proto3" json:"env,omitempty"` // latencies is a map of nodeIDs to nanoseconds which is the latency // between this node and the other node. // // NOTE: this is deprecated and is only set if the min supported // cluster version is >= VersionRPCNetworkStats. - Latencies map[github_com_cockroachdb_cockroach_pkg_roachpb.NodeID]int64 `protobuf:"bytes,9,rep,name=latencies,castkey=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"latencies" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` + Latencies map[github_com_cockroachdb_cockroach_pkg_roachpb.NodeID]int64 `protobuf:"bytes,9,rep,name=latencies,proto3,castkey=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"latencies" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` // activity is a map of nodeIDs to network statistics from this node // to other nodes. - Activity map[github_com_cockroachdb_cockroach_pkg_roachpb.NodeID]NodeStatus_NetworkActivity `protobuf:"bytes,10,rep,name=activity,castkey=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"activity" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value"` + Activity map[github_com_cockroachdb_cockroach_pkg_roachpb.NodeID]NodeStatus_NetworkActivity `protobuf:"bytes,10,rep,name=activity,proto3,castkey=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"activity" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` // total_system_memory is the total RAM available to the system // (or, if possible, the memory available to the cgroup this process is in) // in bytes. TotalSystemMemory int64 `protobuf:"varint,11,opt,name=total_system_memory,json=totalSystemMemory,proto3" json:"total_system_memory,omitempty"` // num_cpus is the number of logical CPUs on this machine. - NumCpus int32 `protobuf:"varint,12,opt,name=num_cpus,json=numCpus,proto3" json:"num_cpus,omitempty"` + NumCpus int32 `protobuf:"varint,12,opt,name=num_cpus,json=numCpus,proto3" json:"num_cpus,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *NodeStatus) Reset() { *m = NodeStatus{} } +func (m *NodeStatus) String() string { return proto.CompactTextString(m) } +func (*NodeStatus) ProtoMessage() {} +func (*NodeStatus) Descriptor() ([]byte, []int) { + return fileDescriptor_status_39af85395986a59c, []int{1} +} +func (m *NodeStatus) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *NodeStatus) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *NodeStatus) XXX_Merge(src proto.Message) { + xxx_messageInfo_NodeStatus.Merge(dst, src) +} +func (m *NodeStatus) XXX_Size() int { + return m.Size() +} +func (m *NodeStatus) XXX_DiscardUnknown() { + xxx_messageInfo_NodeStatus.DiscardUnknown(m) } -func (m *NodeStatus) Reset() { *m = NodeStatus{} } -func (m *NodeStatus) String() string { return proto.CompactTextString(m) } -func (*NodeStatus) ProtoMessage() {} -func (*NodeStatus) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{1} } +var xxx_messageInfo_NodeStatus proto.InternalMessageInfo type NodeStatus_NetworkActivity struct { - Incoming int64 `protobuf:"varint,1,opt,name=incoming,proto3" json:"incoming,omitempty"` - Outgoing int64 `protobuf:"varint,2,opt,name=outgoing,proto3" json:"outgoing,omitempty"` - Latency int64 `protobuf:"varint,3,opt,name=latency,proto3" json:"latency,omitempty"` + Incoming int64 `protobuf:"varint,1,opt,name=incoming,proto3" json:"incoming,omitempty"` + Outgoing int64 `protobuf:"varint,2,opt,name=outgoing,proto3" json:"outgoing,omitempty"` + Latency int64 `protobuf:"varint,3,opt,name=latency,proto3" json:"latency,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *NodeStatus_NetworkActivity) Reset() { *m = NodeStatus_NetworkActivity{} } func (m *NodeStatus_NetworkActivity) String() string { return proto.CompactTextString(m) } func (*NodeStatus_NetworkActivity) ProtoMessage() {} func (*NodeStatus_NetworkActivity) Descriptor() ([]byte, []int) { - return fileDescriptorStatus, []int{1, 2} + return fileDescriptor_status_39af85395986a59c, []int{1, 2} +} +func (m *NodeStatus_NetworkActivity) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *NodeStatus_NetworkActivity) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *NodeStatus_NetworkActivity) XXX_Merge(src proto.Message) { + xxx_messageInfo_NodeStatus_NetworkActivity.Merge(dst, src) +} +func (m *NodeStatus_NetworkActivity) XXX_Size() int { + return m.Size() } +func (m *NodeStatus_NetworkActivity) XXX_DiscardUnknown() { + xxx_messageInfo_NodeStatus_NetworkActivity.DiscardUnknown(m) +} + +var xxx_messageInfo_NodeStatus_NetworkActivity proto.InternalMessageInfo // A HealthAlert is an undesired condition detected by a server which should be // exposed to the operators. type HealthAlert struct { // store_id is zero for alerts not specific to a store (i.e. apply at the node level). - StoreID github_com_cockroachdb_cockroach_pkg_roachpb.StoreID `protobuf:"varint,1,opt,name=store_id,json=storeId,proto3,customtype=github.com/cockroachdb/cockroach/pkg/roachpb.StoreID" json:"store_id"` - Category HealthAlert_Category `protobuf:"varint,2,opt,name=category,proto3,enum=cockroach.server.status.statuspb.HealthAlert_Category" json:"category,omitempty"` - Description string `protobuf:"bytes,3,opt,name=description,proto3" json:"description,omitempty"` - Value float64 `protobuf:"fixed64,4,opt,name=value,proto3" json:"value,omitempty"` + StoreID github_com_cockroachdb_cockroach_pkg_roachpb.StoreID `protobuf:"varint,1,opt,name=store_id,json=storeId,proto3,customtype=github.com/cockroachdb/cockroach/pkg/roachpb.StoreID" json:"store_id"` + Category HealthAlert_Category `protobuf:"varint,2,opt,name=category,proto3,enum=cockroach.server.status.statuspb.HealthAlert_Category" json:"category,omitempty"` + Description string `protobuf:"bytes,3,opt,name=description,proto3" json:"description,omitempty"` + Value float64 `protobuf:"fixed64,4,opt,name=value,proto3" json:"value,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *HealthAlert) Reset() { *m = HealthAlert{} } -func (m *HealthAlert) String() string { return proto.CompactTextString(m) } -func (*HealthAlert) ProtoMessage() {} -func (*HealthAlert) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{2} } +func (m *HealthAlert) Reset() { *m = HealthAlert{} } +func (m *HealthAlert) String() string { return proto.CompactTextString(m) } +func (*HealthAlert) ProtoMessage() {} +func (*HealthAlert) Descriptor() ([]byte, []int) { + return fileDescriptor_status_39af85395986a59c, []int{2} +} +func (m *HealthAlert) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *HealthAlert) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *HealthAlert) XXX_Merge(src proto.Message) { + xxx_messageInfo_HealthAlert.Merge(dst, src) +} +func (m *HealthAlert) XXX_Size() int { + return m.Size() +} +func (m *HealthAlert) XXX_DiscardUnknown() { + xxx_messageInfo_HealthAlert.DiscardUnknown(m) +} + +var xxx_messageInfo_HealthAlert proto.InternalMessageInfo // HealthCheckResult holds a number of HealthAlerts. type HealthCheckResult struct { - Alerts []HealthAlert `protobuf:"bytes,1,rep,name=alerts" json:"alerts"` + Alerts []HealthAlert `protobuf:"bytes,1,rep,name=alerts,proto3" json:"alerts"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *HealthCheckResult) Reset() { *m = HealthCheckResult{} } -func (m *HealthCheckResult) String() string { return proto.CompactTextString(m) } -func (*HealthCheckResult) ProtoMessage() {} -func (*HealthCheckResult) Descriptor() ([]byte, []int) { return fileDescriptorStatus, []int{3} } +func (m *HealthCheckResult) Reset() { *m = HealthCheckResult{} } +func (m *HealthCheckResult) String() string { return proto.CompactTextString(m) } +func (*HealthCheckResult) ProtoMessage() {} +func (*HealthCheckResult) Descriptor() ([]byte, []int) { + return fileDescriptor_status_39af85395986a59c, []int{3} +} +func (m *HealthCheckResult) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *HealthCheckResult) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *HealthCheckResult) XXX_Merge(src proto.Message) { + xxx_messageInfo_HealthCheckResult.Merge(dst, src) +} +func (m *HealthCheckResult) XXX_Size() int { + return m.Size() +} +func (m *HealthCheckResult) XXX_DiscardUnknown() { + xxx_messageInfo_HealthCheckResult.DiscardUnknown(m) +} + +var xxx_messageInfo_HealthCheckResult proto.InternalMessageInfo func init() { proto.RegisterType((*StoreStatus)(nil), "cockroach.server.status.statuspb.StoreStatus") + proto.RegisterMapType((map[string]float64)(nil), "cockroach.server.status.statuspb.StoreStatus.MetricsEntry") proto.RegisterType((*NodeStatus)(nil), "cockroach.server.status.statuspb.NodeStatus") + proto.RegisterMapType((map[github_com_cockroachdb_cockroach_pkg_roachpb.NodeID]NodeStatus_NetworkActivity)(nil), "cockroach.server.status.statuspb.NodeStatus.ActivityEntry") + proto.RegisterMapType((map[github_com_cockroachdb_cockroach_pkg_roachpb.NodeID]int64)(nil), "cockroach.server.status.statuspb.NodeStatus.LatenciesEntry") + proto.RegisterMapType((map[string]float64)(nil), "cockroach.server.status.statuspb.NodeStatus.MetricsEntry") proto.RegisterType((*NodeStatus_NetworkActivity)(nil), "cockroach.server.status.statuspb.NodeStatus.NetworkActivity") proto.RegisterType((*HealthAlert)(nil), "cockroach.server.status.statuspb.HealthAlert") proto.RegisterType((*HealthCheckResult)(nil), "cockroach.server.status.statuspb.HealthCheckResult") @@ -177,7 +299,7 @@ func (m *StoreStatus) MarshalTo(dAtA []byte) (int, error) { for k := range m.Metrics { keysForMetrics = append(keysForMetrics, string(k)) } - sortkeys.Strings(keysForMetrics) + github_com_gogo_protobuf_sortkeys.Strings(keysForMetrics) for _, k := range keysForMetrics { dAtA[i] = 0x12 i++ @@ -190,7 +312,7 @@ func (m *StoreStatus) MarshalTo(dAtA []byte) (int, error) { i += copy(dAtA[i:], k) dAtA[i] = 0x11 i++ - binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(v)))) + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(v)))) i += 8 } } @@ -243,7 +365,7 @@ func (m *NodeStatus) MarshalTo(dAtA []byte) (int, error) { for k := range m.Metrics { keysForMetrics = append(keysForMetrics, string(k)) } - sortkeys.Strings(keysForMetrics) + github_com_gogo_protobuf_sortkeys.Strings(keysForMetrics) for _, k := range keysForMetrics { dAtA[i] = 0x2a i++ @@ -256,7 +378,7 @@ func (m *NodeStatus) MarshalTo(dAtA []byte) (int, error) { i += copy(dAtA[i:], k) dAtA[i] = 0x11 i++ - binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(v)))) + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(v)))) i += 8 } } @@ -307,7 +429,7 @@ func (m *NodeStatus) MarshalTo(dAtA []byte) (int, error) { for k := range m.Latencies { keysForLatencies = append(keysForLatencies, int32(k)) } - sortkeys.Int32s(keysForLatencies) + github_com_gogo_protobuf_sortkeys.Int32s(keysForLatencies) for _, k := range keysForLatencies { dAtA[i] = 0x4a i++ @@ -327,7 +449,7 @@ func (m *NodeStatus) MarshalTo(dAtA []byte) (int, error) { for k := range m.Activity { keysForActivity = append(keysForActivity, int32(k)) } - sortkeys.Int32s(keysForActivity) + github_com_gogo_protobuf_sortkeys.Int32s(keysForActivity) for _, k := range keysForActivity { dAtA[i] = 0x52 i++ @@ -432,7 +554,7 @@ func (m *HealthAlert) MarshalTo(dAtA []byte) (int, error) { if m.Value != 0 { dAtA[i] = 0x21 i++ - binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.Value)))) + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.Value)))) i += 8 } return i, nil @@ -478,6 +600,9 @@ func encodeVarintStatus(dAtA []byte, offset int, v uint64) int { return offset + 1 } func (m *StoreStatus) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.Desc.Size() @@ -494,6 +619,9 @@ func (m *StoreStatus) Size() (n int) { } func (m *NodeStatus) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.Desc.Size() @@ -559,6 +687,9 @@ func (m *NodeStatus) Size() (n int) { } func (m *NodeStatus_NetworkActivity) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Incoming != 0 { @@ -574,6 +705,9 @@ func (m *NodeStatus_NetworkActivity) Size() (n int) { } func (m *HealthAlert) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.StoreID != 0 { @@ -593,6 +727,9 @@ func (m *HealthAlert) Size() (n int) { } func (m *HealthCheckResult) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.Alerts) > 0 { @@ -756,7 +893,7 @@ func (m *StoreStatus) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - mapvaluetemp = uint64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + mapvaluetemp = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 mapvalue = math.Float64frombits(mapvaluetemp) } else { @@ -1004,7 +1141,7 @@ func (m *NodeStatus) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - mapvaluetemp = uint64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + mapvaluetemp = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 mapvalue = math.Float64frombits(mapvaluetemp) } else { @@ -1591,7 +1728,7 @@ func (m *HealthAlert) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - v = uint64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 m.Value = float64(math.Float64frombits(v)) default: @@ -1801,9 +1938,11 @@ var ( ErrIntOverflowStatus = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("server/status/statuspb/status.proto", fileDescriptorStatus) } +func init() { + proto.RegisterFile("server/status/statuspb/status.proto", fileDescriptor_status_39af85395986a59c) +} -var fileDescriptorStatus = []byte{ +var fileDescriptor_status_39af85395986a59c = []byte{ // 808 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x55, 0x4f, 0x6f, 0xeb, 0x44, 0x10, 0xef, 0x36, 0x69, 0xe3, 0x8c, 0xdf, 0x2b, 0x7d, 0xcb, 0x03, 0x99, 0x48, 0xa4, 0x26, 0x70, diff --git a/pkg/settings/cluster/cluster_version.pb.go b/pkg/settings/cluster/cluster_version.pb.go index 3501db391a8f..c4054ae69c40 100644 --- a/pkg/settings/cluster/cluster_version.pb.go +++ b/pkg/settings/cluster/cluster_version.pb.go @@ -1,21 +1,12 @@ // Code generated by protoc-gen-gogo. DO NOT EDIT. // source: settings/cluster/cluster_version.proto -/* - Package cluster is a generated protocol buffer package. - - It is generated from these files: - settings/cluster/cluster_version.proto - - It has these top-level messages: - ClusterVersion -*/ package cluster import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" -import cockroach_roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" +import roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" import io "io" @@ -33,20 +24,46 @@ const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package type ClusterVersion struct { // The minimum_version required for any node to support. This // value must monotonically increase. - MinimumVersion cockroach_roachpb.Version `protobuf:"bytes,1,opt,name=minimum_version,json=minimumVersion" json:"minimum_version"` + MinimumVersion roachpb.Version `protobuf:"bytes,1,opt,name=minimum_version,json=minimumVersion,proto3" json:"minimum_version"` // The version of functionality in use in the cluster. Unlike // minimum_version, use_version may be downgraded, which will // disable functionality requiring a higher version. However, // some functionality, once in use, can not be discontinued. // Support for that functionality is guaranteed by the ratchet // of minimum_version. - UseVersion cockroach_roachpb.Version `protobuf:"bytes,2,opt,name=use_version,json=useVersion" json:"use_version"` + UseVersion roachpb.Version `protobuf:"bytes,2,opt,name=use_version,json=useVersion,proto3" json:"use_version"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *ClusterVersion) Reset() { *m = ClusterVersion{} } -func (m *ClusterVersion) String() string { return proto.CompactTextString(m) } -func (*ClusterVersion) ProtoMessage() {} -func (*ClusterVersion) Descriptor() ([]byte, []int) { return fileDescriptorClusterVersion, []int{0} } +func (m *ClusterVersion) Reset() { *m = ClusterVersion{} } +func (m *ClusterVersion) String() string { return proto.CompactTextString(m) } +func (*ClusterVersion) ProtoMessage() {} +func (*ClusterVersion) Descriptor() ([]byte, []int) { + return fileDescriptor_cluster_version_0ae72bf8da5cffa7, []int{0} +} +func (m *ClusterVersion) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ClusterVersion) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ClusterVersion) XXX_Merge(src proto.Message) { + xxx_messageInfo_ClusterVersion.Merge(dst, src) +} +func (m *ClusterVersion) XXX_Size() int { + return m.Size() +} +func (m *ClusterVersion) XXX_DiscardUnknown() { + xxx_messageInfo_ClusterVersion.DiscardUnknown(m) +} + +var xxx_messageInfo_ClusterVersion proto.InternalMessageInfo func init() { proto.RegisterType((*ClusterVersion)(nil), "cockroach.base.ClusterVersion") @@ -95,6 +112,9 @@ func encodeVarintClusterVersion(dAtA []byte, offset int, v uint64) int { return offset + 1 } func (m *ClusterVersion) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.MinimumVersion.Size() @@ -333,10 +353,10 @@ var ( ) func init() { - proto.RegisterFile("settings/cluster/cluster_version.proto", fileDescriptorClusterVersion) + proto.RegisterFile("settings/cluster/cluster_version.proto", fileDescriptor_cluster_version_0ae72bf8da5cffa7) } -var fileDescriptorClusterVersion = []byte{ +var fileDescriptor_cluster_version_0ae72bf8da5cffa7 = []byte{ // 210 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x52, 0x2b, 0x4e, 0x2d, 0x29, 0xc9, 0xcc, 0x4b, 0x2f, 0xd6, 0x4f, 0xce, 0x29, 0x2d, 0x2e, 0x49, 0x2d, 0x82, 0xd1, 0xf1, 0x65, diff --git a/pkg/sql/distsqlpb/api.pb.go b/pkg/sql/distsqlpb/api.pb.go index dddd41cbc777..6d5ddc419045 100644 --- a/pkg/sql/distsqlpb/api.pb.go +++ b/pkg/sql/distsqlpb/api.pb.go @@ -1,81 +1,21 @@ // Code generated by protoc-gen-gogo. DO NOT EDIT. // source: sql/distsqlpb/api.proto -/* - Package distsqlpb is a generated protocol buffer package. - - It is generated from these files: - sql/distsqlpb/api.proto - sql/distsqlpb/data.proto - sql/distsqlpb/processors.proto - - It has these top-level messages: - SetupFlowRequest - EvalContext - SequenceState - SimpleResponse - ConsumerSignal - DrainRequest - ConsumerHandshake - Error - Expression - Ordering - StreamEndpointSpec - InputSyncSpec - OutputRouterSpec - DatumInfo - ProducerHeader - ProducerData - ProducerMessage - RemoteProducerMetadata - DistSQLVersionGossipInfo - DistSQLDrainingInfo - ProcessorSpec - PostProcessSpec - ProcessorCoreUnion - NoopCoreSpec - MetadataTestSenderSpec - MetadataTestReceiverSpec - ValuesCoreSpec - TableReaderSpan - TableReaderSpec - JoinReaderSpec - SorterSpec - DistinctSpec - ZigzagJoinerSpec - LocalPlanNodeSpec - Columns - MergeJoinerSpec - HashJoinerSpec - AggregatorSpec - BackfillerSpec - FlowSpec - JobProgress - ReadImportDataSpec - SSTWriterSpec - CSVWriterSpec - SketchSpec - SamplerSpec - SampleAggregatorSpec - InterleavedReaderJoinerSpec - ProjectSetSpec - WindowerSpec - ChangeAggregatorSpec - ChangeFrontierSpec -*/ package distsqlpb import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" -import cockroach_roachpb1 "github.com/cockroachdb/cockroach/pkg/roachpb" +import roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" import time "time" -import context "context" -import grpc "google.golang.org/grpc" +import ( + context "context" + grpc "google.golang.org/grpc" +) -import types "github.com/gogo/protobuf/types" +import github_com_gogo_protobuf_types "github.com/gogo/protobuf/types" import io "io" @@ -127,29 +67,57 @@ func (x *BytesEncodeFormat) UnmarshalJSON(data []byte) error { *x = BytesEncodeFormat(value) return nil } -func (BytesEncodeFormat) EnumDescriptor() ([]byte, []int) { return fileDescriptorApi, []int{0} } +func (BytesEncodeFormat) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_api_f73c3942e4fe4344, []int{0} +} type SetupFlowRequest struct { // TxnCoordMeta is the TxnCoordMeta for the transaction in which the flow // will run. If nil, the flow will not run in a higher-level transaction // (i.e. it is responsible for managing its own transactions, if any). Most // flows expect to run in a txn, but some, like backfills, don't. - TxnCoordMeta *cockroach_roachpb1.TxnCoordMeta `protobuf:"bytes,7,opt,name=txn_coord_meta,json=txnCoordMeta" json:"txn_coord_meta,omitempty"` + TxnCoordMeta *roachpb.TxnCoordMeta `protobuf:"bytes,7,opt,name=txn_coord_meta,json=txnCoordMeta" json:"txn_coord_meta,omitempty"` // deprecated_txn used to play the role that TxnCoordMeta now plays. It // can be removed in v2.2. - DeprecatedTxn *cockroach_roachpb1.Transaction `protobuf:"bytes,1,opt,name=deprecated_txn,json=deprecatedTxn" json:"deprecated_txn,omitempty"` + DeprecatedTxn *roachpb.Transaction `protobuf:"bytes,1,opt,name=deprecated_txn,json=deprecatedTxn" json:"deprecated_txn,omitempty"` // Version of distsqlrun protocol; a server accepts a certain range of // versions, up to its own version. See server.go for more details. - Version DistSQLVersion `protobuf:"varint,5,opt,name=version,casttype=DistSQLVersion" json:"version"` - Flow FlowSpec `protobuf:"bytes,3,opt,name=flow" json:"flow"` - EvalContext EvalContext `protobuf:"bytes,6,opt,name=evalContext" json:"evalContext"` - TraceKV bool `protobuf:"varint,8,opt,name=TraceKV" json:"TraceKV"` + Version DistSQLVersion `protobuf:"varint,5,opt,name=version,casttype=DistSQLVersion" json:"version"` + Flow FlowSpec `protobuf:"bytes,3,opt,name=flow" json:"flow"` + EvalContext EvalContext `protobuf:"bytes,6,opt,name=evalContext" json:"evalContext"` + TraceKV bool `protobuf:"varint,8,opt,name=TraceKV" json:"TraceKV"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *SetupFlowRequest) Reset() { *m = SetupFlowRequest{} } +func (m *SetupFlowRequest) String() string { return proto.CompactTextString(m) } +func (*SetupFlowRequest) ProtoMessage() {} +func (*SetupFlowRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_api_f73c3942e4fe4344, []int{0} +} +func (m *SetupFlowRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SetupFlowRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *SetupFlowRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_SetupFlowRequest.Merge(dst, src) +} +func (m *SetupFlowRequest) XXX_Size() int { + return m.Size() +} +func (m *SetupFlowRequest) XXX_DiscardUnknown() { + xxx_messageInfo_SetupFlowRequest.DiscardUnknown(m) } -func (m *SetupFlowRequest) Reset() { *m = SetupFlowRequest{} } -func (m *SetupFlowRequest) String() string { return proto.CompactTextString(m) } -func (*SetupFlowRequest) ProtoMessage() {} -func (*SetupFlowRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{0} } +var xxx_messageInfo_SetupFlowRequest proto.InternalMessageInfo // EvalContext is used to marshall some planner.EvalContext members. type EvalContext struct { @@ -157,53 +125,157 @@ type EvalContext struct { TxnTimestampNanos int64 `protobuf:"varint,2,opt,name=txnTimestampNanos" json:"txnTimestampNanos"` // The name of the location according to whose current timezone we're going to // parse timestamps. Used to init EvalContext.Location. - Location string `protobuf:"bytes,4,opt,name=location" json:"location"` - Database string `protobuf:"bytes,5,opt,name=database" json:"database"` - SearchPath []string `protobuf:"bytes,6,rep,name=searchPath" json:"searchPath,omitempty"` - User string `protobuf:"bytes,7,opt,name=user" json:"user"` - SeqState SequenceState `protobuf:"bytes,8,opt,name=seq_state,json=seqState" json:"seq_state"` - ApplicationName string `protobuf:"bytes,9,opt,name=application_name,json=applicationName" json:"application_name"` - BytesEncodeFormat BytesEncodeFormat `protobuf:"varint,10,opt,name=bytes_encode_format,json=bytesEncodeFormat,enum=cockroach.sql.distsqlpb.BytesEncodeFormat" json:"bytes_encode_format"` - ExtraFloatDigits int32 `protobuf:"varint,11,opt,name=extra_float_digits,json=extraFloatDigits" json:"extra_float_digits"` + Location string `protobuf:"bytes,4,opt,name=location" json:"location"` + Database string `protobuf:"bytes,5,opt,name=database" json:"database"` + SearchPath []string `protobuf:"bytes,6,rep,name=searchPath" json:"searchPath,omitempty"` + User string `protobuf:"bytes,7,opt,name=user" json:"user"` + SeqState SequenceState `protobuf:"bytes,8,opt,name=seq_state,json=seqState" json:"seq_state"` + ApplicationName string `protobuf:"bytes,9,opt,name=application_name,json=applicationName" json:"application_name"` + BytesEncodeFormat BytesEncodeFormat `protobuf:"varint,10,opt,name=bytes_encode_format,json=bytesEncodeFormat,enum=cockroach.sql.distsqlpb.BytesEncodeFormat" json:"bytes_encode_format"` + ExtraFloatDigits int32 `protobuf:"varint,11,opt,name=extra_float_digits,json=extraFloatDigits" json:"extra_float_digits"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *EvalContext) Reset() { *m = EvalContext{} } +func (m *EvalContext) String() string { return proto.CompactTextString(m) } +func (*EvalContext) ProtoMessage() {} +func (*EvalContext) Descriptor() ([]byte, []int) { + return fileDescriptor_api_f73c3942e4fe4344, []int{1} +} +func (m *EvalContext) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *EvalContext) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *EvalContext) XXX_Merge(src proto.Message) { + xxx_messageInfo_EvalContext.Merge(dst, src) +} +func (m *EvalContext) XXX_Size() int { + return m.Size() +} +func (m *EvalContext) XXX_DiscardUnknown() { + xxx_messageInfo_EvalContext.DiscardUnknown(m) } -func (m *EvalContext) Reset() { *m = EvalContext{} } -func (m *EvalContext) String() string { return proto.CompactTextString(m) } -func (*EvalContext) ProtoMessage() {} -func (*EvalContext) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{1} } +var xxx_messageInfo_EvalContext proto.InternalMessageInfo // SequenceState is used to marshall the sessiondata.SequenceState struct. type SequenceState struct { Seqs []*SequenceState_Seq `protobuf:"bytes,1,rep,name=seqs" json:"seqs,omitempty"` // last_seq_incremented is the id of the last sequence incremented by the // session. This field is filled in iff seqs is not empty. - LastSeqIncremented *uint32 `protobuf:"varint,2,opt,name=last_seq_incremented,json=lastSeqIncremented" json:"last_seq_incremented,omitempty"` + LastSeqIncremented *uint32 `protobuf:"varint,2,opt,name=last_seq_incremented,json=lastSeqIncremented" json:"last_seq_incremented,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *SequenceState) Reset() { *m = SequenceState{} } +func (m *SequenceState) String() string { return proto.CompactTextString(m) } +func (*SequenceState) ProtoMessage() {} +func (*SequenceState) Descriptor() ([]byte, []int) { + return fileDescriptor_api_f73c3942e4fe4344, []int{2} +} +func (m *SequenceState) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SequenceState) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *SequenceState) XXX_Merge(src proto.Message) { + xxx_messageInfo_SequenceState.Merge(dst, src) +} +func (m *SequenceState) XXX_Size() int { + return m.Size() +} +func (m *SequenceState) XXX_DiscardUnknown() { + xxx_messageInfo_SequenceState.DiscardUnknown(m) } -func (m *SequenceState) Reset() { *m = SequenceState{} } -func (m *SequenceState) String() string { return proto.CompactTextString(m) } -func (*SequenceState) ProtoMessage() {} -func (*SequenceState) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{2} } +var xxx_messageInfo_SequenceState proto.InternalMessageInfo // Seq represents the last value of one sequence modified by the session. type SequenceState_Seq struct { - SeqID uint32 `protobuf:"varint,1,opt,name=seq_id,json=seqId" json:"seq_id"` - LatestVal int64 `protobuf:"varint,2,opt,name=latest_val,json=latestVal" json:"latest_val"` + SeqID uint32 `protobuf:"varint,1,opt,name=seq_id,json=seqId" json:"seq_id"` + LatestVal int64 `protobuf:"varint,2,opt,name=latest_val,json=latestVal" json:"latest_val"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *SequenceState_Seq) Reset() { *m = SequenceState_Seq{} } -func (m *SequenceState_Seq) String() string { return proto.CompactTextString(m) } -func (*SequenceState_Seq) ProtoMessage() {} -func (*SequenceState_Seq) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{2, 0} } +func (m *SequenceState_Seq) Reset() { *m = SequenceState_Seq{} } +func (m *SequenceState_Seq) String() string { return proto.CompactTextString(m) } +func (*SequenceState_Seq) ProtoMessage() {} +func (*SequenceState_Seq) Descriptor() ([]byte, []int) { + return fileDescriptor_api_f73c3942e4fe4344, []int{2, 0} +} +func (m *SequenceState_Seq) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SequenceState_Seq) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *SequenceState_Seq) XXX_Merge(src proto.Message) { + xxx_messageInfo_SequenceState_Seq.Merge(dst, src) +} +func (m *SequenceState_Seq) XXX_Size() int { + return m.Size() +} +func (m *SequenceState_Seq) XXX_DiscardUnknown() { + xxx_messageInfo_SequenceState_Seq.DiscardUnknown(m) +} + +var xxx_messageInfo_SequenceState_Seq proto.InternalMessageInfo type SimpleResponse struct { - Error *Error `protobuf:"bytes,1,opt,name=error" json:"error,omitempty"` + Error *Error `protobuf:"bytes,1,opt,name=error" json:"error,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *SimpleResponse) Reset() { *m = SimpleResponse{} } +func (m *SimpleResponse) String() string { return proto.CompactTextString(m) } +func (*SimpleResponse) ProtoMessage() {} +func (*SimpleResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_api_f73c3942e4fe4344, []int{3} +} +func (m *SimpleResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SimpleResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *SimpleResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_SimpleResponse.Merge(dst, src) +} +func (m *SimpleResponse) XXX_Size() int { + return m.Size() +} +func (m *SimpleResponse) XXX_DiscardUnknown() { + xxx_messageInfo_SimpleResponse.DiscardUnknown(m) } -func (m *SimpleResponse) Reset() { *m = SimpleResponse{} } -func (m *SimpleResponse) String() string { return proto.CompactTextString(m) } -func (*SimpleResponse) ProtoMessage() {} -func (*SimpleResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{3} } +var xxx_messageInfo_SimpleResponse proto.InternalMessageInfo // ConsumerSignal are messages flowing from consumer to producer (so, from RPC // server to client) for the FlowStream RPC. @@ -216,21 +288,73 @@ type ConsumerSignal struct { // contain this message. SetupFlowRequest *SetupFlowRequest `protobuf:"bytes,2,opt,name=setup_flow_request,json=setupFlowRequest" json:"setup_flow_request,omitempty"` // Consumer->Producer handshake messages. See message definition. - Handshake *ConsumerHandshake `protobuf:"bytes,3,opt,name=handshake" json:"handshake,omitempty"` + Handshake *ConsumerHandshake `protobuf:"bytes,3,opt,name=handshake" json:"handshake,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *ConsumerSignal) Reset() { *m = ConsumerSignal{} } -func (m *ConsumerSignal) String() string { return proto.CompactTextString(m) } -func (*ConsumerSignal) ProtoMessage() {} -func (*ConsumerSignal) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{4} } +func (m *ConsumerSignal) Reset() { *m = ConsumerSignal{} } +func (m *ConsumerSignal) String() string { return proto.CompactTextString(m) } +func (*ConsumerSignal) ProtoMessage() {} +func (*ConsumerSignal) Descriptor() ([]byte, []int) { + return fileDescriptor_api_f73c3942e4fe4344, []int{4} +} +func (m *ConsumerSignal) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ConsumerSignal) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ConsumerSignal) XXX_Merge(src proto.Message) { + xxx_messageInfo_ConsumerSignal.Merge(dst, src) +} +func (m *ConsumerSignal) XXX_Size() int { + return m.Size() +} +func (m *ConsumerSignal) XXX_DiscardUnknown() { + xxx_messageInfo_ConsumerSignal.DiscardUnknown(m) +} + +var xxx_messageInfo_ConsumerSignal proto.InternalMessageInfo type DrainRequest struct { + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *DrainRequest) Reset() { *m = DrainRequest{} } +func (m *DrainRequest) String() string { return proto.CompactTextString(m) } +func (*DrainRequest) ProtoMessage() {} +func (*DrainRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_api_f73c3942e4fe4344, []int{5} +} +func (m *DrainRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *DrainRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *DrainRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_DrainRequest.Merge(dst, src) +} +func (m *DrainRequest) XXX_Size() int { + return m.Size() +} +func (m *DrainRequest) XXX_DiscardUnknown() { + xxx_messageInfo_DrainRequest.DiscardUnknown(m) } -func (m *DrainRequest) Reset() { *m = DrainRequest{} } -func (m *DrainRequest) String() string { return proto.CompactTextString(m) } -func (*DrainRequest) ProtoMessage() {} -func (*DrainRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{5} } +var xxx_messageInfo_DrainRequest proto.InternalMessageInfo // ConsumerHandshake is the first one or two message sent in the // consumer->producer direction on a stream. It informs the producer about the @@ -250,14 +374,40 @@ type ConsumerHandshake struct { // dealine, this stream will be disconnected by the server-side. ConsumerScheduleDeadline *time.Time `protobuf:"bytes,2,opt,name=consumer_schedule_deadline,json=consumerScheduleDeadline,stdtime" json:"consumer_schedule_deadline,omitempty"` // The server's DistSQL version range. - Version DistSQLVersion `protobuf:"varint,3,opt,name=version,casttype=DistSQLVersion" json:"version"` - MinAcceptedVersion DistSQLVersion `protobuf:"varint,4,opt,name=min_accepted_version,json=minAcceptedVersion,casttype=DistSQLVersion" json:"min_accepted_version"` + Version DistSQLVersion `protobuf:"varint,3,opt,name=version,casttype=DistSQLVersion" json:"version"` + MinAcceptedVersion DistSQLVersion `protobuf:"varint,4,opt,name=min_accepted_version,json=minAcceptedVersion,casttype=DistSQLVersion" json:"min_accepted_version"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ConsumerHandshake) Reset() { *m = ConsumerHandshake{} } +func (m *ConsumerHandshake) String() string { return proto.CompactTextString(m) } +func (*ConsumerHandshake) ProtoMessage() {} +func (*ConsumerHandshake) Descriptor() ([]byte, []int) { + return fileDescriptor_api_f73c3942e4fe4344, []int{6} +} +func (m *ConsumerHandshake) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ConsumerHandshake) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ConsumerHandshake) XXX_Merge(src proto.Message) { + xxx_messageInfo_ConsumerHandshake.Merge(dst, src) +} +func (m *ConsumerHandshake) XXX_Size() int { + return m.Size() +} +func (m *ConsumerHandshake) XXX_DiscardUnknown() { + xxx_messageInfo_ConsumerHandshake.DiscardUnknown(m) } -func (m *ConsumerHandshake) Reset() { *m = ConsumerHandshake{} } -func (m *ConsumerHandshake) String() string { return proto.CompactTextString(m) } -func (*ConsumerHandshake) ProtoMessage() {} -func (*ConsumerHandshake) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{6} } +var xxx_messageInfo_ConsumerHandshake proto.InternalMessageInfo func init() { proto.RegisterType((*SetupFlowRequest)(nil), "cockroach.sql.distsqlpb.SetupFlowRequest") @@ -279,8 +429,9 @@ var _ grpc.ClientConn // is compatible with the grpc package it is being compiled against. const _ = grpc.SupportPackageIsVersion4 -// Client API for DistSQL service - +// DistSQLClient is the client API for DistSQL service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream. type DistSQLClient interface { // RunSyncFlow instantiates a flow and streams back results of that flow. // The request must contain one flow, and that flow must have a single mailbox @@ -311,7 +462,7 @@ func NewDistSQLClient(cc *grpc.ClientConn) DistSQLClient { } func (c *distSQLClient) RunSyncFlow(ctx context.Context, opts ...grpc.CallOption) (DistSQL_RunSyncFlowClient, error) { - stream, err := grpc.NewClientStream(ctx, &_DistSQL_serviceDesc.Streams[0], c.cc, "/cockroach.sql.distsqlpb.DistSQL/RunSyncFlow", opts...) + stream, err := c.cc.NewStream(ctx, &_DistSQL_serviceDesc.Streams[0], "/cockroach.sql.distsqlpb.DistSQL/RunSyncFlow", opts...) if err != nil { return nil, err } @@ -343,7 +494,7 @@ func (x *distSQLRunSyncFlowClient) Recv() (*ProducerMessage, error) { func (c *distSQLClient) SetupFlow(ctx context.Context, in *SetupFlowRequest, opts ...grpc.CallOption) (*SimpleResponse, error) { out := new(SimpleResponse) - err := grpc.Invoke(ctx, "/cockroach.sql.distsqlpb.DistSQL/SetupFlow", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/cockroach.sql.distsqlpb.DistSQL/SetupFlow", in, out, opts...) if err != nil { return nil, err } @@ -351,7 +502,7 @@ func (c *distSQLClient) SetupFlow(ctx context.Context, in *SetupFlowRequest, opt } func (c *distSQLClient) FlowStream(ctx context.Context, opts ...grpc.CallOption) (DistSQL_FlowStreamClient, error) { - stream, err := grpc.NewClientStream(ctx, &_DistSQL_serviceDesc.Streams[1], c.cc, "/cockroach.sql.distsqlpb.DistSQL/FlowStream", opts...) + stream, err := c.cc.NewStream(ctx, &_DistSQL_serviceDesc.Streams[1], "/cockroach.sql.distsqlpb.DistSQL/FlowStream", opts...) if err != nil { return nil, err } @@ -381,8 +532,7 @@ func (x *distSQLFlowStreamClient) Recv() (*ConsumerSignal, error) { return m, nil } -// Server API for DistSQL service - +// DistSQLServer is the server API for DistSQL service. type DistSQLServer interface { // RunSyncFlow instantiates a flow and streams back results of that flow. // The request must contain one flow, and that flow must have a single mailbox @@ -817,8 +967,8 @@ func (m *ConsumerHandshake) MarshalTo(dAtA []byte) (int, error) { if m.ConsumerScheduleDeadline != nil { dAtA[i] = 0x12 i++ - i = encodeVarintApi(dAtA, i, uint64(types.SizeOfStdTime(*m.ConsumerScheduleDeadline))) - n10, err := types.StdTimeMarshalTo(*m.ConsumerScheduleDeadline, dAtA[i:]) + i = encodeVarintApi(dAtA, i, uint64(github_com_gogo_protobuf_types.SizeOfStdTime(*m.ConsumerScheduleDeadline))) + n10, err := github_com_gogo_protobuf_types.StdTimeMarshalTo(*m.ConsumerScheduleDeadline, dAtA[i:]) if err != nil { return 0, err } @@ -843,6 +993,9 @@ func encodeVarintApi(dAtA []byte, offset int, v uint64) int { return offset + 1 } func (m *SetupFlowRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.DeprecatedTxn != nil { @@ -863,6 +1016,9 @@ func (m *SetupFlowRequest) Size() (n int) { } func (m *EvalContext) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovApi(uint64(m.StmtTimestampNanos)) @@ -889,6 +1045,9 @@ func (m *EvalContext) Size() (n int) { } func (m *SequenceState) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.Seqs) > 0 { @@ -904,6 +1063,9 @@ func (m *SequenceState) Size() (n int) { } func (m *SequenceState_Seq) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovApi(uint64(m.SeqID)) @@ -912,6 +1074,9 @@ func (m *SequenceState_Seq) Size() (n int) { } func (m *SimpleResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Error != nil { @@ -922,6 +1087,9 @@ func (m *SimpleResponse) Size() (n int) { } func (m *ConsumerSignal) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.DrainRequest != nil { @@ -940,17 +1108,23 @@ func (m *ConsumerSignal) Size() (n int) { } func (m *DrainRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l return n } func (m *ConsumerHandshake) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 2 if m.ConsumerScheduleDeadline != nil { - l = types.SizeOfStdTime(*m.ConsumerScheduleDeadline) + l = github_com_gogo_protobuf_types.SizeOfStdTime(*m.ConsumerScheduleDeadline) n += 1 + l + sovApi(uint64(l)) } n += 1 + sovApi(uint64(m.Version)) @@ -1053,7 +1227,7 @@ func (m *SetupFlowRequest) Unmarshal(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.DeprecatedTxn == nil { - m.DeprecatedTxn = &cockroach_roachpb1.Transaction{} + m.DeprecatedTxn = &roachpb.Transaction{} } if err := m.DeprecatedTxn.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err @@ -1165,7 +1339,7 @@ func (m *SetupFlowRequest) Unmarshal(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.TxnCoordMeta == nil { - m.TxnCoordMeta = &cockroach_roachpb1.TxnCoordMeta{} + m.TxnCoordMeta = &roachpb.TxnCoordMeta{} } if err := m.TxnCoordMeta.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err @@ -2062,7 +2236,7 @@ func (m *ConsumerHandshake) Unmarshal(dAtA []byte) error { if m.ConsumerScheduleDeadline == nil { m.ConsumerScheduleDeadline = new(time.Time) } - if err := types.StdTimeUnmarshal(m.ConsumerScheduleDeadline, dAtA[iNdEx:postIndex]); err != nil { + if err := github_com_gogo_protobuf_types.StdTimeUnmarshal(m.ConsumerScheduleDeadline, dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -2230,9 +2404,9 @@ var ( ErrIntOverflowApi = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("sql/distsqlpb/api.proto", fileDescriptorApi) } +func init() { proto.RegisterFile("sql/distsqlpb/api.proto", fileDescriptor_api_f73c3942e4fe4344) } -var fileDescriptorApi = []byte{ +var fileDescriptor_api_f73c3942e4fe4344 = []byte{ // 1068 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x55, 0x4f, 0x6f, 0xe3, 0x44, 0x14, 0x8f, 0x13, 0xf7, 0x4f, 0x5e, 0x36, 0x21, 0x1d, 0x56, 0xac, 0x95, 0x43, 0x12, 0xb2, 0x0b, diff --git a/pkg/sql/distsqlpb/data.pb.go b/pkg/sql/distsqlpb/data.pb.go index b6f3bf477d8e..ea3a4f260e17 100644 --- a/pkg/sql/distsqlpb/data.pb.go +++ b/pkg/sql/distsqlpb/data.pb.go @@ -6,13 +6,10 @@ package distsqlpb import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" -import cockroach_roachpb3 "github.com/cockroachdb/cockroach/pkg/roachpb" -import cockroach_roachpb1 "github.com/cockroachdb/cockroach/pkg/roachpb" -import cockroach_roachpb2 "github.com/cockroachdb/cockroach/pkg/roachpb" -import cockroach_pgerror "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" -import cockroach_sql_sqlbase1 "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" -import cockroach_sql_sqlbase2 "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" -import cockroach_util_tracing "github.com/cockroachdb/cockroach/pkg/util/tracing" +import roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" +import pgerror "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" +import sqlbase "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" +import tracing "github.com/cockroachdb/cockroach/pkg/util/tracing" import github_com_cockroachdb_cockroach_pkg_roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -23,6 +20,12 @@ var _ = proto.Marshal var _ = fmt.Errorf var _ = math.Inf +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package + // The direction of the desired ordering for a column. type Ordering_Column_Direction int32 @@ -57,7 +60,7 @@ func (x *Ordering_Column_Direction) UnmarshalJSON(data []byte) error { return nil } func (Ordering_Column_Direction) EnumDescriptor() ([]byte, []int) { - return fileDescriptorData, []int{2, 0, 0} + return fileDescriptor_data_f33d9406823eb329, []int{2, 0, 0} } type StreamEndpointSpec_Type int32 @@ -102,7 +105,7 @@ func (x *StreamEndpointSpec_Type) UnmarshalJSON(data []byte) error { return nil } func (StreamEndpointSpec_Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptorData, []int{3, 0} + return fileDescriptor_data_f33d9406823eb329, []int{3, 0} } type InputSyncSpec_Type int32 @@ -141,7 +144,9 @@ func (x *InputSyncSpec_Type) UnmarshalJSON(data []byte) error { *x = InputSyncSpec_Type(value) return nil } -func (InputSyncSpec_Type) EnumDescriptor() ([]byte, []int) { return fileDescriptorData, []int{4, 0} } +func (InputSyncSpec_Type) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_data_f33d9406823eb329, []int{4, 0} +} type OutputRouterSpec_Type int32 @@ -187,19 +192,47 @@ func (x *OutputRouterSpec_Type) UnmarshalJSON(data []byte) error { *x = OutputRouterSpec_Type(value) return nil } -func (OutputRouterSpec_Type) EnumDescriptor() ([]byte, []int) { return fileDescriptorData, []int{5, 0} } +func (OutputRouterSpec_Type) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_data_f33d9406823eb329, []int{5, 0} +} // Error is a generic representation including a string message. type Error struct { // Types that are valid to be assigned to Detail: // *Error_PGError // *Error_RetryableTxnError - Detail isError_Detail `protobuf_oneof:"detail"` + Detail isError_Detail `protobuf_oneof:"detail"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *Error) Reset() { *m = Error{} } -func (*Error) ProtoMessage() {} -func (*Error) Descriptor() ([]byte, []int) { return fileDescriptorData, []int{0} } +func (m *Error) Reset() { *m = Error{} } +func (*Error) ProtoMessage() {} +func (*Error) Descriptor() ([]byte, []int) { + return fileDescriptor_data_f33d9406823eb329, []int{0} +} +func (m *Error) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Error) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *Error) XXX_Merge(src proto.Message) { + xxx_messageInfo_Error.Merge(dst, src) +} +func (m *Error) XXX_Size() int { + return m.Size() +} +func (m *Error) XXX_DiscardUnknown() { + xxx_messageInfo_Error.DiscardUnknown(m) +} + +var xxx_messageInfo_Error proto.InternalMessageInfo type isError_Detail interface { isError_Detail() @@ -208,10 +241,10 @@ type isError_Detail interface { } type Error_PGError struct { - PGError *cockroach_pgerror.Error `protobuf:"bytes,1,opt,name=pg_error,json=pgError,oneof"` + PGError *pgerror.Error `protobuf:"bytes,1,opt,name=pg_error,json=pgError,oneof"` } type Error_RetryableTxnError struct { - RetryableTxnError *cockroach_roachpb2.UnhandledRetryableError `protobuf:"bytes,2,opt,name=retryableTxnError,oneof"` + RetryableTxnError *roachpb.UnhandledRetryableError `protobuf:"bytes,2,opt,name=retryableTxnError,oneof"` } func (*Error_PGError) isError_Detail() {} @@ -224,14 +257,14 @@ func (m *Error) GetDetail() isError_Detail { return nil } -func (m *Error) GetPGError() *cockroach_pgerror.Error { +func (m *Error) GetPGError() *pgerror.Error { if x, ok := m.GetDetail().(*Error_PGError); ok { return x.PGError } return nil } -func (m *Error) GetRetryableTxnError() *cockroach_roachpb2.UnhandledRetryableError { +func (m *Error) GetRetryableTxnError() *roachpb.UnhandledRetryableError { if x, ok := m.GetDetail().(*Error_RetryableTxnError); ok { return x.RetryableTxnError } @@ -274,7 +307,7 @@ func _Error_OneofUnmarshaler(msg proto.Message, tag, wire int, b *proto.Buffer) if wire != proto.WireBytes { return true, proto.ErrInternalBadWireType } - msg := new(cockroach_pgerror.Error) + msg := new(pgerror.Error) err := b.DecodeMessage(msg) m.Detail = &Error_PGError{msg} return true, err @@ -282,7 +315,7 @@ func _Error_OneofUnmarshaler(msg proto.Message, tag, wire int, b *proto.Buffer) if wire != proto.WireBytes { return true, proto.ErrInternalBadWireType } - msg := new(cockroach_roachpb2.UnhandledRetryableError) + msg := new(roachpb.UnhandledRetryableError) err := b.DecodeMessage(msg) m.Detail = &Error_RetryableTxnError{msg} return true, err @@ -297,12 +330,12 @@ func _Error_OneofSizer(msg proto.Message) (n int) { switch x := m.Detail.(type) { case *Error_PGError: s := proto.Size(x.PGError) - n += proto.SizeVarint(1<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *Error_RetryableTxnError: s := proto.Size(x.RetryableTxnError) - n += proto.SizeVarint(2<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case nil: @@ -312,30 +345,106 @@ func _Error_OneofSizer(msg proto.Message) (n int) { return n } -func (m *Expression) Reset() { *m = Expression{} } -func (*Expression) ProtoMessage() {} -func (*Expression) Descriptor() ([]byte, []int) { return fileDescriptorData, []int{1} } +func (m *Expression) Reset() { *m = Expression{} } +func (*Expression) ProtoMessage() {} +func (*Expression) Descriptor() ([]byte, []int) { + return fileDescriptor_data_f33d9406823eb329, []int{1} +} +func (m *Expression) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Expression) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *Expression) XXX_Merge(src proto.Message) { + xxx_messageInfo_Expression.Merge(dst, src) +} +func (m *Expression) XXX_Size() int { + return m.Size() +} +func (m *Expression) XXX_DiscardUnknown() { + xxx_messageInfo_Expression.DiscardUnknown(m) +} + +var xxx_messageInfo_Expression proto.InternalMessageInfo // Ordering defines an order - specifically a list of column indices and // directions. See sqlbase.ColumnOrdering. type Ordering struct { - Columns []Ordering_Column `protobuf:"bytes,1,rep,name=columns" json:"columns"` + Columns []Ordering_Column `protobuf:"bytes,1,rep,name=columns" json:"columns"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *Ordering) Reset() { *m = Ordering{} } -func (m *Ordering) String() string { return proto.CompactTextString(m) } -func (*Ordering) ProtoMessage() {} -func (*Ordering) Descriptor() ([]byte, []int) { return fileDescriptorData, []int{2} } +func (m *Ordering) Reset() { *m = Ordering{} } +func (m *Ordering) String() string { return proto.CompactTextString(m) } +func (*Ordering) ProtoMessage() {} +func (*Ordering) Descriptor() ([]byte, []int) { + return fileDescriptor_data_f33d9406823eb329, []int{2} +} +func (m *Ordering) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Ordering) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *Ordering) XXX_Merge(src proto.Message) { + xxx_messageInfo_Ordering.Merge(dst, src) +} +func (m *Ordering) XXX_Size() int { + return m.Size() +} +func (m *Ordering) XXX_DiscardUnknown() { + xxx_messageInfo_Ordering.DiscardUnknown(m) +} + +var xxx_messageInfo_Ordering proto.InternalMessageInfo type Ordering_Column struct { - ColIdx uint32 `protobuf:"varint,1,opt,name=col_idx,json=colIdx" json:"col_idx"` - Direction Ordering_Column_Direction `protobuf:"varint,2,opt,name=direction,enum=cockroach.sql.distsqlpb.Ordering_Column_Direction" json:"direction"` + ColIdx uint32 `protobuf:"varint,1,opt,name=col_idx,json=colIdx" json:"col_idx"` + Direction Ordering_Column_Direction `protobuf:"varint,2,opt,name=direction,enum=cockroach.sql.distsqlpb.Ordering_Column_Direction" json:"direction"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Ordering_Column) Reset() { *m = Ordering_Column{} } +func (m *Ordering_Column) String() string { return proto.CompactTextString(m) } +func (*Ordering_Column) ProtoMessage() {} +func (*Ordering_Column) Descriptor() ([]byte, []int) { + return fileDescriptor_data_f33d9406823eb329, []int{2, 0} +} +func (m *Ordering_Column) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Ordering_Column) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *Ordering_Column) XXX_Merge(src proto.Message) { + xxx_messageInfo_Ordering_Column.Merge(dst, src) +} +func (m *Ordering_Column) XXX_Size() int { + return m.Size() +} +func (m *Ordering_Column) XXX_DiscardUnknown() { + xxx_messageInfo_Ordering_Column.DiscardUnknown(m) } -func (m *Ordering_Column) Reset() { *m = Ordering_Column{} } -func (m *Ordering_Column) String() string { return proto.CompactTextString(m) } -func (*Ordering_Column) ProtoMessage() {} -func (*Ordering_Column) Descriptor() ([]byte, []int) { return fileDescriptorData, []int{2, 0} } +var xxx_messageInfo_Ordering_Column proto.InternalMessageInfo // StreamEndpointSpec describes one of the endpoints (input or output) of a physical // stream. @@ -355,13 +464,39 @@ type StreamEndpointSpec struct { TargetNodeID github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,4,opt,name=target_node_id,json=targetNodeId,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"target_node_id"` // Serving address for the target host. Used by 2.0 nodes instead of target_node_id. // TODO(bdarnell): Remove after 2.1 - DeprecatedTargetAddr string `protobuf:"bytes,3,opt,name=deprecated_target_addr,json=deprecatedTargetAddr" json:"deprecated_target_addr"` + DeprecatedTargetAddr string `protobuf:"bytes,3,opt,name=deprecated_target_addr,json=deprecatedTargetAddr" json:"deprecated_target_addr"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *StreamEndpointSpec) Reset() { *m = StreamEndpointSpec{} } -func (m *StreamEndpointSpec) String() string { return proto.CompactTextString(m) } -func (*StreamEndpointSpec) ProtoMessage() {} -func (*StreamEndpointSpec) Descriptor() ([]byte, []int) { return fileDescriptorData, []int{3} } +func (m *StreamEndpointSpec) Reset() { *m = StreamEndpointSpec{} } +func (m *StreamEndpointSpec) String() string { return proto.CompactTextString(m) } +func (*StreamEndpointSpec) ProtoMessage() {} +func (*StreamEndpointSpec) Descriptor() ([]byte, []int) { + return fileDescriptor_data_f33d9406823eb329, []int{3} +} +func (m *StreamEndpointSpec) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *StreamEndpointSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *StreamEndpointSpec) XXX_Merge(src proto.Message) { + xxx_messageInfo_StreamEndpointSpec.Merge(dst, src) +} +func (m *StreamEndpointSpec) XXX_Size() int { + return m.Size() +} +func (m *StreamEndpointSpec) XXX_DiscardUnknown() { + xxx_messageInfo_StreamEndpointSpec.DiscardUnknown(m) +} + +var xxx_messageInfo_StreamEndpointSpec proto.InternalMessageInfo // InputSyncSpec is the specification for an input synchronizer; it decides how // to interleave rows from multiple input streams. @@ -370,13 +505,39 @@ type InputSyncSpec struct { Ordering Ordering `protobuf:"bytes,2,opt,name=ordering" json:"ordering"` Streams []StreamEndpointSpec `protobuf:"bytes,3,rep,name=streams" json:"streams"` // Schema for the streams entering this synchronizer. - ColumnTypes []cockroach_sql_sqlbase1.ColumnType `protobuf:"bytes,4,rep,name=column_types,json=columnTypes" json:"column_types"` + ColumnTypes []sqlbase.ColumnType `protobuf:"bytes,4,rep,name=column_types,json=columnTypes" json:"column_types"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *InputSyncSpec) Reset() { *m = InputSyncSpec{} } +func (m *InputSyncSpec) String() string { return proto.CompactTextString(m) } +func (*InputSyncSpec) ProtoMessage() {} +func (*InputSyncSpec) Descriptor() ([]byte, []int) { + return fileDescriptor_data_f33d9406823eb329, []int{4} +} +func (m *InputSyncSpec) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *InputSyncSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *InputSyncSpec) XXX_Merge(src proto.Message) { + xxx_messageInfo_InputSyncSpec.Merge(dst, src) +} +func (m *InputSyncSpec) XXX_Size() int { + return m.Size() +} +func (m *InputSyncSpec) XXX_DiscardUnknown() { + xxx_messageInfo_InputSyncSpec.DiscardUnknown(m) } -func (m *InputSyncSpec) Reset() { *m = InputSyncSpec{} } -func (m *InputSyncSpec) String() string { return proto.CompactTextString(m) } -func (*InputSyncSpec) ProtoMessage() {} -func (*InputSyncSpec) Descriptor() ([]byte, []int) { return fileDescriptorData, []int{4} } +var xxx_messageInfo_InputSyncSpec proto.InternalMessageInfo // OutputRouterSpec is the specification for the output router of a processor; // it decides how to send results to multiple output streams. @@ -390,13 +551,39 @@ type OutputRouterSpec struct { // disable_buffering disables output buffering. Generally buffering should be // enabled to prevent deadlocks. However some plans are known not to deadlock, // and so can set this flag to prevent unbounded buffering causing OOMs. - DisableBuffering bool `protobuf:"varint,5,opt,name=disable_buffering,json=disableBuffering" json:"disable_buffering"` + DisableBuffering bool `protobuf:"varint,5,opt,name=disable_buffering,json=disableBuffering" json:"disable_buffering"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *OutputRouterSpec) Reset() { *m = OutputRouterSpec{} } -func (m *OutputRouterSpec) String() string { return proto.CompactTextString(m) } -func (*OutputRouterSpec) ProtoMessage() {} -func (*OutputRouterSpec) Descriptor() ([]byte, []int) { return fileDescriptorData, []int{5} } +func (m *OutputRouterSpec) Reset() { *m = OutputRouterSpec{} } +func (m *OutputRouterSpec) String() string { return proto.CompactTextString(m) } +func (*OutputRouterSpec) ProtoMessage() {} +func (*OutputRouterSpec) Descriptor() ([]byte, []int) { + return fileDescriptor_data_f33d9406823eb329, []int{5} +} +func (m *OutputRouterSpec) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *OutputRouterSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *OutputRouterSpec) XXX_Merge(src proto.Message) { + xxx_messageInfo_OutputRouterSpec.Merge(dst, src) +} +func (m *OutputRouterSpec) XXX_Size() int { + return m.Size() +} +func (m *OutputRouterSpec) XXX_DiscardUnknown() { + xxx_messageInfo_OutputRouterSpec.DiscardUnknown(m) +} + +var xxx_messageInfo_OutputRouterSpec proto.InternalMessageInfo type OutputRouterSpec_RangeRouterSpec struct { // spans is a slice of Span. Input matching a span will be routed to its @@ -409,23 +596,49 @@ type OutputRouterSpec_RangeRouterSpec struct { // encodings is a slice of columns and encodings. Each will be appended to a // []byte, which is used as input to the spans. Columns from the input rows // potentially need to be recoded to match the encoding used for the spans. - Encodings []OutputRouterSpec_RangeRouterSpec_ColumnEncoding `protobuf:"bytes,3,rep,name=encodings" json:"encodings"` + Encodings []OutputRouterSpec_RangeRouterSpec_ColumnEncoding `protobuf:"bytes,3,rep,name=encodings" json:"encodings"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *OutputRouterSpec_RangeRouterSpec) Reset() { *m = OutputRouterSpec_RangeRouterSpec{} } func (m *OutputRouterSpec_RangeRouterSpec) String() string { return proto.CompactTextString(m) } func (*OutputRouterSpec_RangeRouterSpec) ProtoMessage() {} func (*OutputRouterSpec_RangeRouterSpec) Descriptor() ([]byte, []int) { - return fileDescriptorData, []int{5, 0} + return fileDescriptor_data_f33d9406823eb329, []int{5, 0} +} +func (m *OutputRouterSpec_RangeRouterSpec) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *OutputRouterSpec_RangeRouterSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *OutputRouterSpec_RangeRouterSpec) XXX_Merge(src proto.Message) { + xxx_messageInfo_OutputRouterSpec_RangeRouterSpec.Merge(dst, src) +} +func (m *OutputRouterSpec_RangeRouterSpec) XXX_Size() int { + return m.Size() +} +func (m *OutputRouterSpec_RangeRouterSpec) XXX_DiscardUnknown() { + xxx_messageInfo_OutputRouterSpec_RangeRouterSpec.DiscardUnknown(m) } +var xxx_messageInfo_OutputRouterSpec_RangeRouterSpec proto.InternalMessageInfo + type OutputRouterSpec_RangeRouterSpec_ColumnEncoding struct { // column is the index of a column to encode. Column uint32 `protobuf:"varint,1,opt,name=column" json:"column"` // encoding specifies how a particular column is to be encoded for // generating the sort key for a row. This needs to correspond to the way // the Span.{start,end} keys have been generated. - Encoding cockroach_sql_sqlbase2.DatumEncoding `protobuf:"varint,2,opt,name=encoding,enum=cockroach.sql.sqlbase.DatumEncoding" json:"encoding"` + Encoding sqlbase.DatumEncoding `protobuf:"varint,2,opt,name=encoding,enum=cockroach.sql.sqlbase.DatumEncoding" json:"encoding"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *OutputRouterSpec_RangeRouterSpec_ColumnEncoding) Reset() { @@ -436,44 +649,142 @@ func (m *OutputRouterSpec_RangeRouterSpec_ColumnEncoding) String() string { } func (*OutputRouterSpec_RangeRouterSpec_ColumnEncoding) ProtoMessage() {} func (*OutputRouterSpec_RangeRouterSpec_ColumnEncoding) Descriptor() ([]byte, []int) { - return fileDescriptorData, []int{5, 0, 0} + return fileDescriptor_data_f33d9406823eb329, []int{5, 0, 0} +} +func (m *OutputRouterSpec_RangeRouterSpec_ColumnEncoding) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *OutputRouterSpec_RangeRouterSpec_ColumnEncoding) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *OutputRouterSpec_RangeRouterSpec_ColumnEncoding) XXX_Merge(src proto.Message) { + xxx_messageInfo_OutputRouterSpec_RangeRouterSpec_ColumnEncoding.Merge(dst, src) +} +func (m *OutputRouterSpec_RangeRouterSpec_ColumnEncoding) XXX_Size() int { + return m.Size() +} +func (m *OutputRouterSpec_RangeRouterSpec_ColumnEncoding) XXX_DiscardUnknown() { + xxx_messageInfo_OutputRouterSpec_RangeRouterSpec_ColumnEncoding.DiscardUnknown(m) } +var xxx_messageInfo_OutputRouterSpec_RangeRouterSpec_ColumnEncoding proto.InternalMessageInfo + // Span matches bytes in [start, end). type OutputRouterSpec_RangeRouterSpec_Span struct { Start []byte `protobuf:"bytes,1,opt,name=start" json:"start,omitempty"` End []byte `protobuf:"bytes,2,opt,name=end" json:"end,omitempty"` // stream is the index of the destination stream. - Stream int32 `protobuf:"varint,3,opt,name=stream" json:"stream"` + Stream int32 `protobuf:"varint,3,opt,name=stream" json:"stream"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *OutputRouterSpec_RangeRouterSpec_Span) Reset() { *m = OutputRouterSpec_RangeRouterSpec_Span{} } func (m *OutputRouterSpec_RangeRouterSpec_Span) String() string { return proto.CompactTextString(m) } func (*OutputRouterSpec_RangeRouterSpec_Span) ProtoMessage() {} func (*OutputRouterSpec_RangeRouterSpec_Span) Descriptor() ([]byte, []int) { - return fileDescriptorData, []int{5, 0, 1} + return fileDescriptor_data_f33d9406823eb329, []int{5, 0, 1} +} +func (m *OutputRouterSpec_RangeRouterSpec_Span) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *OutputRouterSpec_RangeRouterSpec_Span) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *OutputRouterSpec_RangeRouterSpec_Span) XXX_Merge(src proto.Message) { + xxx_messageInfo_OutputRouterSpec_RangeRouterSpec_Span.Merge(dst, src) } +func (m *OutputRouterSpec_RangeRouterSpec_Span) XXX_Size() int { + return m.Size() +} +func (m *OutputRouterSpec_RangeRouterSpec_Span) XXX_DiscardUnknown() { + xxx_messageInfo_OutputRouterSpec_RangeRouterSpec_Span.DiscardUnknown(m) +} + +var xxx_messageInfo_OutputRouterSpec_RangeRouterSpec_Span proto.InternalMessageInfo type DatumInfo struct { - Encoding cockroach_sql_sqlbase2.DatumEncoding `protobuf:"varint,1,opt,name=encoding,enum=cockroach.sql.sqlbase.DatumEncoding" json:"encoding"` - Type cockroach_sql_sqlbase1.ColumnType `protobuf:"bytes,2,opt,name=type" json:"type"` + Encoding sqlbase.DatumEncoding `protobuf:"varint,1,opt,name=encoding,enum=cockroach.sql.sqlbase.DatumEncoding" json:"encoding"` + Type sqlbase.ColumnType `protobuf:"bytes,2,opt,name=type" json:"type"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *DatumInfo) Reset() { *m = DatumInfo{} } -func (m *DatumInfo) String() string { return proto.CompactTextString(m) } -func (*DatumInfo) ProtoMessage() {} -func (*DatumInfo) Descriptor() ([]byte, []int) { return fileDescriptorData, []int{6} } +func (m *DatumInfo) Reset() { *m = DatumInfo{} } +func (m *DatumInfo) String() string { return proto.CompactTextString(m) } +func (*DatumInfo) ProtoMessage() {} +func (*DatumInfo) Descriptor() ([]byte, []int) { + return fileDescriptor_data_f33d9406823eb329, []int{6} +} +func (m *DatumInfo) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *DatumInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *DatumInfo) XXX_Merge(src proto.Message) { + xxx_messageInfo_DatumInfo.Merge(dst, src) +} +func (m *DatumInfo) XXX_Size() int { + return m.Size() +} +func (m *DatumInfo) XXX_DiscardUnknown() { + xxx_messageInfo_DatumInfo.DiscardUnknown(m) +} + +var xxx_messageInfo_DatumInfo proto.InternalMessageInfo // ProducerHeader is a message that is sent once at the beginning of a stream. type ProducerHeader struct { - FlowID FlowID `protobuf:"bytes,1,opt,name=flow_id,json=flowId,customtype=FlowID" json:"flow_id"` - StreamID StreamID `protobuf:"varint,2,opt,name=stream_id,json=streamId,casttype=StreamID" json:"stream_id"` + FlowID FlowID `protobuf:"bytes,1,opt,name=flow_id,json=flowId,customtype=FlowID" json:"flow_id"` + StreamID StreamID `protobuf:"varint,2,opt,name=stream_id,json=streamId,casttype=StreamID" json:"stream_id"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ProducerHeader) Reset() { *m = ProducerHeader{} } +func (m *ProducerHeader) String() string { return proto.CompactTextString(m) } +func (*ProducerHeader) ProtoMessage() {} +func (*ProducerHeader) Descriptor() ([]byte, []int) { + return fileDescriptor_data_f33d9406823eb329, []int{7} +} +func (m *ProducerHeader) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ProducerHeader) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ProducerHeader) XXX_Merge(src proto.Message) { + xxx_messageInfo_ProducerHeader.Merge(dst, src) +} +func (m *ProducerHeader) XXX_Size() int { + return m.Size() +} +func (m *ProducerHeader) XXX_DiscardUnknown() { + xxx_messageInfo_ProducerHeader.DiscardUnknown(m) } -func (m *ProducerHeader) Reset() { *m = ProducerHeader{} } -func (m *ProducerHeader) String() string { return proto.CompactTextString(m) } -func (*ProducerHeader) ProtoMessage() {} -func (*ProducerHeader) Descriptor() ([]byte, []int) { return fileDescriptorData, []int{7} } +var xxx_messageInfo_ProducerHeader proto.InternalMessageInfo // ProducerData is a message that can be sent multiple times as part of a stream // from a producer to a consumer. It contains 0 or more rows and/or 0 or more @@ -486,13 +797,39 @@ type ProducerData struct { // passed instead. NumEmptyRows int32 `protobuf:"varint,3,opt,name=num_empty_rows,json=numEmptyRows" json:"num_empty_rows"` // A bunch of metadata messages. - Metadata []RemoteProducerMetadata `protobuf:"bytes,2,rep,name=metadata" json:"metadata"` + Metadata []RemoteProducerMetadata `protobuf:"bytes,2,rep,name=metadata" json:"metadata"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *ProducerData) Reset() { *m = ProducerData{} } -func (m *ProducerData) String() string { return proto.CompactTextString(m) } -func (*ProducerData) ProtoMessage() {} -func (*ProducerData) Descriptor() ([]byte, []int) { return fileDescriptorData, []int{8} } +func (m *ProducerData) Reset() { *m = ProducerData{} } +func (m *ProducerData) String() string { return proto.CompactTextString(m) } +func (*ProducerData) ProtoMessage() {} +func (*ProducerData) Descriptor() ([]byte, []int) { + return fileDescriptor_data_f33d9406823eb329, []int{8} +} +func (m *ProducerData) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ProducerData) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ProducerData) XXX_Merge(src proto.Message) { + xxx_messageInfo_ProducerData.Merge(dst, src) +} +func (m *ProducerData) XXX_Size() int { + return m.Size() +} +func (m *ProducerData) XXX_DiscardUnknown() { + xxx_messageInfo_ProducerData.DiscardUnknown(m) +} + +var xxx_messageInfo_ProducerData proto.InternalMessageInfo type ProducerMessage struct { Header *ProducerHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"` @@ -503,14 +840,40 @@ type ProducerMessage struct { // TODO(andrei): It'd be nice if the typing information for streams would be // configured statically at plan creation time, instead of being discovered // dynamically through the first rows that flow. - Typing []DatumInfo `protobuf:"bytes,2,rep,name=typing" json:"typing"` - Data ProducerData `protobuf:"bytes,3,opt,name=data" json:"data"` + Typing []DatumInfo `protobuf:"bytes,2,rep,name=typing" json:"typing"` + Data ProducerData `protobuf:"bytes,3,opt,name=data" json:"data"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ProducerMessage) Reset() { *m = ProducerMessage{} } +func (m *ProducerMessage) String() string { return proto.CompactTextString(m) } +func (*ProducerMessage) ProtoMessage() {} +func (*ProducerMessage) Descriptor() ([]byte, []int) { + return fileDescriptor_data_f33d9406823eb329, []int{9} +} +func (m *ProducerMessage) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ProducerMessage) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ProducerMessage) XXX_Merge(src proto.Message) { + xxx_messageInfo_ProducerMessage.Merge(dst, src) +} +func (m *ProducerMessage) XXX_Size() int { + return m.Size() +} +func (m *ProducerMessage) XXX_DiscardUnknown() { + xxx_messageInfo_ProducerMessage.DiscardUnknown(m) } -func (m *ProducerMessage) Reset() { *m = ProducerMessage{} } -func (m *ProducerMessage) String() string { return proto.CompactTextString(m) } -func (*ProducerMessage) ProtoMessage() {} -func (*ProducerMessage) Descriptor() ([]byte, []int) { return fileDescriptorData, []int{9} } +var xxx_messageInfo_ProducerMessage proto.InternalMessageInfo // RemoteProducerMetadata represents records that a producer wants to pass to // a consumer, other than data rows. It's named RemoteProducerMetadata to not @@ -523,13 +886,39 @@ type RemoteProducerMetadata struct { // *RemoteProducerMetadata_TraceData_ // *RemoteProducerMetadata_TxnCoordMeta // *RemoteProducerMetadata_RowNum_ - Value isRemoteProducerMetadata_Value `protobuf_oneof:"value"` + Value isRemoteProducerMetadata_Value `protobuf_oneof:"value"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *RemoteProducerMetadata) Reset() { *m = RemoteProducerMetadata{} } -func (m *RemoteProducerMetadata) String() string { return proto.CompactTextString(m) } -func (*RemoteProducerMetadata) ProtoMessage() {} -func (*RemoteProducerMetadata) Descriptor() ([]byte, []int) { return fileDescriptorData, []int{10} } +func (m *RemoteProducerMetadata) Reset() { *m = RemoteProducerMetadata{} } +func (m *RemoteProducerMetadata) String() string { return proto.CompactTextString(m) } +func (*RemoteProducerMetadata) ProtoMessage() {} +func (*RemoteProducerMetadata) Descriptor() ([]byte, []int) { + return fileDescriptor_data_f33d9406823eb329, []int{10} +} +func (m *RemoteProducerMetadata) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RemoteProducerMetadata) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RemoteProducerMetadata) XXX_Merge(src proto.Message) { + xxx_messageInfo_RemoteProducerMetadata.Merge(dst, src) +} +func (m *RemoteProducerMetadata) XXX_Size() int { + return m.Size() +} +func (m *RemoteProducerMetadata) XXX_DiscardUnknown() { + xxx_messageInfo_RemoteProducerMetadata.DiscardUnknown(m) +} + +var xxx_messageInfo_RemoteProducerMetadata proto.InternalMessageInfo type isRemoteProducerMetadata_Value interface { isRemoteProducerMetadata_Value() @@ -547,7 +936,7 @@ type RemoteProducerMetadata_TraceData_ struct { TraceData *RemoteProducerMetadata_TraceData `protobuf:"bytes,3,opt,name=trace_data,json=traceData,oneof"` } type RemoteProducerMetadata_TxnCoordMeta struct { - TxnCoordMeta *cockroach_roachpb1.TxnCoordMeta `protobuf:"bytes,4,opt,name=txn_coord_meta,json=txnCoordMeta,oneof"` + TxnCoordMeta *roachpb.TxnCoordMeta `protobuf:"bytes,4,opt,name=txn_coord_meta,json=txnCoordMeta,oneof"` } type RemoteProducerMetadata_RowNum_ struct { RowNum *RemoteProducerMetadata_RowNum `protobuf:"bytes,5,opt,name=row_num,json=rowNum,oneof"` @@ -587,7 +976,7 @@ func (m *RemoteProducerMetadata) GetTraceData() *RemoteProducerMetadata_TraceDat return nil } -func (m *RemoteProducerMetadata) GetTxnCoordMeta() *cockroach_roachpb1.TxnCoordMeta { +func (m *RemoteProducerMetadata) GetTxnCoordMeta() *roachpb.TxnCoordMeta { if x, ok := m.GetValue().(*RemoteProducerMetadata_TxnCoordMeta); ok { return x.TxnCoordMeta } @@ -679,7 +1068,7 @@ func _RemoteProducerMetadata_OneofUnmarshaler(msg proto.Message, tag, wire int, if wire != proto.WireBytes { return true, proto.ErrInternalBadWireType } - msg := new(cockroach_roachpb1.TxnCoordMeta) + msg := new(roachpb.TxnCoordMeta) err := b.DecodeMessage(msg) m.Value = &RemoteProducerMetadata_TxnCoordMeta{msg} return true, err @@ -702,27 +1091,27 @@ func _RemoteProducerMetadata_OneofSizer(msg proto.Message) (n int) { switch x := m.Value.(type) { case *RemoteProducerMetadata_RangeInfo: s := proto.Size(x.RangeInfo) - n += proto.SizeVarint(1<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *RemoteProducerMetadata_Error: s := proto.Size(x.Error) - n += proto.SizeVarint(2<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *RemoteProducerMetadata_TraceData_: s := proto.Size(x.TraceData) - n += proto.SizeVarint(3<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *RemoteProducerMetadata_TxnCoordMeta: s := proto.Size(x.TxnCoordMeta) - n += proto.SizeVarint(4<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *RemoteProducerMetadata_RowNum_: s := proto.Size(x.RowNum) - n += proto.SizeVarint(5<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case nil: @@ -733,27 +1122,75 @@ func _RemoteProducerMetadata_OneofSizer(msg proto.Message) (n int) { } type RemoteProducerMetadata_RangeInfos struct { - RangeInfo []cockroach_roachpb3.RangeInfo `protobuf:"bytes,1,rep,name=range_info,json=rangeInfo" json:"range_info"` + RangeInfo []roachpb.RangeInfo `protobuf:"bytes,1,rep,name=range_info,json=rangeInfo" json:"range_info"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *RemoteProducerMetadata_RangeInfos) Reset() { *m = RemoteProducerMetadata_RangeInfos{} } func (m *RemoteProducerMetadata_RangeInfos) String() string { return proto.CompactTextString(m) } func (*RemoteProducerMetadata_RangeInfos) ProtoMessage() {} func (*RemoteProducerMetadata_RangeInfos) Descriptor() ([]byte, []int) { - return fileDescriptorData, []int{10, 0} + return fileDescriptor_data_f33d9406823eb329, []int{10, 0} +} +func (m *RemoteProducerMetadata_RangeInfos) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RemoteProducerMetadata_RangeInfos) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RemoteProducerMetadata_RangeInfos) XXX_Merge(src proto.Message) { + xxx_messageInfo_RemoteProducerMetadata_RangeInfos.Merge(dst, src) +} +func (m *RemoteProducerMetadata_RangeInfos) XXX_Size() int { + return m.Size() +} +func (m *RemoteProducerMetadata_RangeInfos) XXX_DiscardUnknown() { + xxx_messageInfo_RemoteProducerMetadata_RangeInfos.DiscardUnknown(m) } +var xxx_messageInfo_RemoteProducerMetadata_RangeInfos proto.InternalMessageInfo + type RemoteProducerMetadata_TraceData struct { - CollectedSpans []cockroach_util_tracing.RecordedSpan `protobuf:"bytes,1,rep,name=collected_spans,json=collectedSpans" json:"collected_spans"` + CollectedSpans []tracing.RecordedSpan `protobuf:"bytes,1,rep,name=collected_spans,json=collectedSpans" json:"collected_spans"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *RemoteProducerMetadata_TraceData) Reset() { *m = RemoteProducerMetadata_TraceData{} } func (m *RemoteProducerMetadata_TraceData) String() string { return proto.CompactTextString(m) } func (*RemoteProducerMetadata_TraceData) ProtoMessage() {} func (*RemoteProducerMetadata_TraceData) Descriptor() ([]byte, []int) { - return fileDescriptorData, []int{10, 1} + return fileDescriptor_data_f33d9406823eb329, []int{10, 1} +} +func (m *RemoteProducerMetadata_TraceData) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RemoteProducerMetadata_TraceData) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RemoteProducerMetadata_TraceData) XXX_Merge(src proto.Message) { + xxx_messageInfo_RemoteProducerMetadata_TraceData.Merge(dst, src) +} +func (m *RemoteProducerMetadata_TraceData) XXX_Size() int { + return m.Size() +} +func (m *RemoteProducerMetadata_TraceData) XXX_DiscardUnknown() { + xxx_messageInfo_RemoteProducerMetadata_TraceData.DiscardUnknown(m) } +var xxx_messageInfo_RemoteProducerMetadata_TraceData proto.InternalMessageInfo + // RowNum is used to count the rows sent from a processor. It is used in tests // to check that metadata is propagated correctly. type RemoteProducerMetadata_RowNum struct { @@ -763,16 +1200,40 @@ type RemoteProducerMetadata_RowNum struct { RowNum int32 `protobuf:"varint,2,opt,name=row_num,json=rowNum" json:"row_num"` // When set, indicates that the row count contains the expected number of // RowNum messages with this ID. - LastMsg bool `protobuf:"varint,3,opt,name=last_msg,json=lastMsg" json:"last_msg"` + LastMsg bool `protobuf:"varint,3,opt,name=last_msg,json=lastMsg" json:"last_msg"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *RemoteProducerMetadata_RowNum) Reset() { *m = RemoteProducerMetadata_RowNum{} } func (m *RemoteProducerMetadata_RowNum) String() string { return proto.CompactTextString(m) } func (*RemoteProducerMetadata_RowNum) ProtoMessage() {} func (*RemoteProducerMetadata_RowNum) Descriptor() ([]byte, []int) { - return fileDescriptorData, []int{10, 2} + return fileDescriptor_data_f33d9406823eb329, []int{10, 2} +} +func (m *RemoteProducerMetadata_RowNum) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RemoteProducerMetadata_RowNum) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RemoteProducerMetadata_RowNum) XXX_Merge(src proto.Message) { + xxx_messageInfo_RemoteProducerMetadata_RowNum.Merge(dst, src) +} +func (m *RemoteProducerMetadata_RowNum) XXX_Size() int { + return m.Size() +} +func (m *RemoteProducerMetadata_RowNum) XXX_DiscardUnknown() { + xxx_messageInfo_RemoteProducerMetadata_RowNum.DiscardUnknown(m) } +var xxx_messageInfo_RemoteProducerMetadata_RowNum proto.InternalMessageInfo + // DistSQLVersionGossipInfo represents the DistSQL server version information // that gets gossiped for each node. This is used by planners to avoid planning // on nodes with incompatible version during rolling cluster updates. @@ -780,26 +1241,78 @@ func (*RemoteProducerMetadata_RowNum) Descriptor() ([]byte, []int) { // For the meaning of the fields, see the corresponding constants in // distsqlrun/server.go. type DistSQLVersionGossipInfo struct { - Version DistSQLVersion `protobuf:"varint,1,opt,name=version,casttype=DistSQLVersion" json:"version"` - MinAcceptedVersion DistSQLVersion `protobuf:"varint,2,opt,name=min_accepted_version,json=minAcceptedVersion,casttype=DistSQLVersion" json:"min_accepted_version"` + Version DistSQLVersion `protobuf:"varint,1,opt,name=version,casttype=DistSQLVersion" json:"version"` + MinAcceptedVersion DistSQLVersion `protobuf:"varint,2,opt,name=min_accepted_version,json=minAcceptedVersion,casttype=DistSQLVersion" json:"min_accepted_version"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *DistSQLVersionGossipInfo) Reset() { *m = DistSQLVersionGossipInfo{} } -func (m *DistSQLVersionGossipInfo) String() string { return proto.CompactTextString(m) } -func (*DistSQLVersionGossipInfo) ProtoMessage() {} -func (*DistSQLVersionGossipInfo) Descriptor() ([]byte, []int) { return fileDescriptorData, []int{11} } +func (m *DistSQLVersionGossipInfo) Reset() { *m = DistSQLVersionGossipInfo{} } +func (m *DistSQLVersionGossipInfo) String() string { return proto.CompactTextString(m) } +func (*DistSQLVersionGossipInfo) ProtoMessage() {} +func (*DistSQLVersionGossipInfo) Descriptor() ([]byte, []int) { + return fileDescriptor_data_f33d9406823eb329, []int{11} +} +func (m *DistSQLVersionGossipInfo) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *DistSQLVersionGossipInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *DistSQLVersionGossipInfo) XXX_Merge(src proto.Message) { + xxx_messageInfo_DistSQLVersionGossipInfo.Merge(dst, src) +} +func (m *DistSQLVersionGossipInfo) XXX_Size() int { + return m.Size() +} +func (m *DistSQLVersionGossipInfo) XXX_DiscardUnknown() { + xxx_messageInfo_DistSQLVersionGossipInfo.DiscardUnknown(m) +} + +var xxx_messageInfo_DistSQLVersionGossipInfo proto.InternalMessageInfo // DistSQLDrainingInfo represents the DistSQL draining state that gets gossiped // for each node. This is used by planners to avoid planning on nodes that are // known to be draining. type DistSQLDrainingInfo struct { - Draining bool `protobuf:"varint,1,opt,name=draining" json:"draining"` + Draining bool `protobuf:"varint,1,opt,name=draining" json:"draining"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *DistSQLDrainingInfo) Reset() { *m = DistSQLDrainingInfo{} } +func (m *DistSQLDrainingInfo) String() string { return proto.CompactTextString(m) } +func (*DistSQLDrainingInfo) ProtoMessage() {} +func (*DistSQLDrainingInfo) Descriptor() ([]byte, []int) { + return fileDescriptor_data_f33d9406823eb329, []int{12} +} +func (m *DistSQLDrainingInfo) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *DistSQLDrainingInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *DistSQLDrainingInfo) XXX_Merge(src proto.Message) { + xxx_messageInfo_DistSQLDrainingInfo.Merge(dst, src) +} +func (m *DistSQLDrainingInfo) XXX_Size() int { + return m.Size() +} +func (m *DistSQLDrainingInfo) XXX_DiscardUnknown() { + xxx_messageInfo_DistSQLDrainingInfo.DiscardUnknown(m) } -func (m *DistSQLDrainingInfo) Reset() { *m = DistSQLDrainingInfo{} } -func (m *DistSQLDrainingInfo) String() string { return proto.CompactTextString(m) } -func (*DistSQLDrainingInfo) ProtoMessage() {} -func (*DistSQLDrainingInfo) Descriptor() ([]byte, []int) { return fileDescriptorData, []int{12} } +var xxx_messageInfo_DistSQLDrainingInfo proto.InternalMessageInfo func init() { proto.RegisterType((*Error)(nil), "cockroach.sql.distsqlpb.Error") @@ -1653,6 +2166,9 @@ func encodeVarintData(dAtA []byte, offset int, v uint64) int { return offset + 1 } func (m *Error) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Detail != nil { @@ -1662,6 +2178,9 @@ func (m *Error) Size() (n int) { } func (m *Error_PGError) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.PGError != nil { @@ -1671,6 +2190,9 @@ func (m *Error_PGError) Size() (n int) { return n } func (m *Error_RetryableTxnError) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.RetryableTxnError != nil { @@ -1680,6 +2202,9 @@ func (m *Error_RetryableTxnError) Size() (n int) { return n } func (m *Expression) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Version) @@ -1690,6 +2215,9 @@ func (m *Expression) Size() (n int) { } func (m *Ordering) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.Columns) > 0 { @@ -1702,6 +2230,9 @@ func (m *Ordering) Size() (n int) { } func (m *Ordering_Column) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovData(uint64(m.ColIdx)) @@ -1710,6 +2241,9 @@ func (m *Ordering_Column) Size() (n int) { } func (m *StreamEndpointSpec) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovData(uint64(m.Type)) @@ -1721,6 +2255,9 @@ func (m *StreamEndpointSpec) Size() (n int) { } func (m *InputSyncSpec) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovData(uint64(m.Type)) @@ -1742,6 +2279,9 @@ func (m *InputSyncSpec) Size() (n int) { } func (m *OutputRouterSpec) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovData(uint64(m.Type)) @@ -1763,6 +2303,9 @@ func (m *OutputRouterSpec) Size() (n int) { } func (m *OutputRouterSpec_RangeRouterSpec) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.Spans) > 0 { @@ -1784,6 +2327,9 @@ func (m *OutputRouterSpec_RangeRouterSpec) Size() (n int) { } func (m *OutputRouterSpec_RangeRouterSpec_ColumnEncoding) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovData(uint64(m.Column)) @@ -1792,6 +2338,9 @@ func (m *OutputRouterSpec_RangeRouterSpec_ColumnEncoding) Size() (n int) { } func (m *OutputRouterSpec_RangeRouterSpec_Span) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Start != nil { @@ -1807,6 +2356,9 @@ func (m *OutputRouterSpec_RangeRouterSpec_Span) Size() (n int) { } func (m *DatumInfo) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovData(uint64(m.Encoding)) @@ -1816,6 +2368,9 @@ func (m *DatumInfo) Size() (n int) { } func (m *ProducerHeader) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.FlowID.Size() @@ -1825,6 +2380,9 @@ func (m *ProducerHeader) Size() (n int) { } func (m *ProducerData) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.RawBytes != nil { @@ -1842,6 +2400,9 @@ func (m *ProducerData) Size() (n int) { } func (m *ProducerMessage) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Header != nil { @@ -1860,6 +2421,9 @@ func (m *ProducerMessage) Size() (n int) { } func (m *RemoteProducerMetadata) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Value != nil { @@ -1869,6 +2433,9 @@ func (m *RemoteProducerMetadata) Size() (n int) { } func (m *RemoteProducerMetadata_RangeInfo) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.RangeInfo != nil { @@ -1878,6 +2445,9 @@ func (m *RemoteProducerMetadata_RangeInfo) Size() (n int) { return n } func (m *RemoteProducerMetadata_Error) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Error != nil { @@ -1887,6 +2457,9 @@ func (m *RemoteProducerMetadata_Error) Size() (n int) { return n } func (m *RemoteProducerMetadata_TraceData_) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.TraceData != nil { @@ -1896,6 +2469,9 @@ func (m *RemoteProducerMetadata_TraceData_) Size() (n int) { return n } func (m *RemoteProducerMetadata_TxnCoordMeta) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.TxnCoordMeta != nil { @@ -1905,6 +2481,9 @@ func (m *RemoteProducerMetadata_TxnCoordMeta) Size() (n int) { return n } func (m *RemoteProducerMetadata_RowNum_) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.RowNum != nil { @@ -1914,6 +2493,9 @@ func (m *RemoteProducerMetadata_RowNum_) Size() (n int) { return n } func (m *RemoteProducerMetadata_RangeInfos) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.RangeInfo) > 0 { @@ -1926,6 +2508,9 @@ func (m *RemoteProducerMetadata_RangeInfos) Size() (n int) { } func (m *RemoteProducerMetadata_TraceData) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.CollectedSpans) > 0 { @@ -1938,6 +2523,9 @@ func (m *RemoteProducerMetadata_TraceData) Size() (n int) { } func (m *RemoteProducerMetadata_RowNum) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.SenderID) @@ -1948,6 +2536,9 @@ func (m *RemoteProducerMetadata_RowNum) Size() (n int) { } func (m *DistSQLVersionGossipInfo) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovData(uint64(m.Version)) @@ -1956,6 +2547,9 @@ func (m *DistSQLVersionGossipInfo) Size() (n int) { } func (m *DistSQLDrainingInfo) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 2 @@ -2030,7 +2624,7 @@ func (m *Error) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - v := &cockroach_pgerror.Error{} + v := &pgerror.Error{} if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -2062,7 +2656,7 @@ func (m *Error) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - v := &cockroach_roachpb2.UnhandledRetryableError{} + v := &roachpb.UnhandledRetryableError{} if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -2637,7 +3231,7 @@ func (m *InputSyncSpec) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.ColumnTypes = append(m.ColumnTypes, cockroach_sql_sqlbase1.ColumnType{}) + m.ColumnTypes = append(m.ColumnTypes, sqlbase.ColumnType{}) if err := m.ColumnTypes[len(m.ColumnTypes)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -2783,6 +3377,17 @@ func (m *OutputRouterSpec) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + var count int + for _, integer := range dAtA { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.HashColumns) == 0 { + m.HashColumns = make([]uint32, 0, elementCount) + } for iNdEx < postIndex { var v uint32 for shift := uint(0); ; shift += 7 { @@ -3069,7 +3674,7 @@ func (m *OutputRouterSpec_RangeRouterSpec_ColumnEncoding) Unmarshal(dAtA []byte) } b := dAtA[iNdEx] iNdEx++ - m.Encoding |= (cockroach_sql_sqlbase2.DatumEncoding(b) & 0x7F) << shift + m.Encoding |= (sqlbase.DatumEncoding(b) & 0x7F) << shift if b < 0x80 { break } @@ -3269,7 +3874,7 @@ func (m *DatumInfo) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Encoding |= (cockroach_sql_sqlbase2.DatumEncoding(b) & 0x7F) << shift + m.Encoding |= (sqlbase.DatumEncoding(b) & 0x7F) << shift if b < 0x80 { break } @@ -3850,7 +4455,7 @@ func (m *RemoteProducerMetadata) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - v := &cockroach_roachpb1.TxnCoordMeta{} + v := &roachpb.TxnCoordMeta{} if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -3964,7 +4569,7 @@ func (m *RemoteProducerMetadata_RangeInfos) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.RangeInfo = append(m.RangeInfo, cockroach_roachpb3.RangeInfo{}) + m.RangeInfo = append(m.RangeInfo, roachpb.RangeInfo{}) if err := m.RangeInfo[len(m.RangeInfo)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -4045,7 +4650,7 @@ func (m *RemoteProducerMetadata_TraceData) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.CollectedSpans = append(m.CollectedSpans, cockroach_util_tracing.RecordedSpan{}) + m.CollectedSpans = append(m.CollectedSpans, tracing.RecordedSpan{}) if err := m.CollectedSpans[len(m.CollectedSpans)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -4452,9 +5057,9 @@ var ( ErrIntOverflowData = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("sql/distsqlpb/data.proto", fileDescriptorData) } +func init() { proto.RegisterFile("sql/distsqlpb/data.proto", fileDescriptor_data_f33d9406823eb329) } -var fileDescriptorData = []byte{ +var fileDescriptor_data_f33d9406823eb329 = []byte{ // 1681 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x57, 0x4f, 0x6f, 0x1b, 0xb9, 0x15, 0xd7, 0xc8, 0xfa, 0xfb, 0x24, 0x2b, 0x32, 0x1b, 0xa4, 0x82, 0x9a, 0x4a, 0xde, 0x69, 0x8b, diff --git a/pkg/sql/distsqlpb/flow_diagram_test.go b/pkg/sql/distsqlpb/flow_diagram_test.go index 0787b727acff..fb443a78d08f 100644 --- a/pkg/sql/distsqlpb/flow_diagram_test.go +++ b/pkg/sql/distsqlpb/flow_diagram_test.go @@ -114,8 +114,10 @@ func TestPlanDiagramIndexJoin(t *testing.T) { }, { Input: []InputSyncSpec{{ - Type: InputSyncSpec_ORDERED, - Ordering: Ordering{Columns: []Ordering_Column{{1, Ordering_Column_ASC}}}, + Type: InputSyncSpec_ORDERED, + Ordering: Ordering{Columns: []Ordering_Column{ + {ColIdx: 1, Direction: Ordering_Column_ASC}}, + }, Streams: []StreamEndpointSpec{ {StreamID: 0}, {StreamID: 1}, diff --git a/pkg/sql/distsqlpb/processors.pb.go b/pkg/sql/distsqlpb/processors.pb.go index 379d4ebae0b8..88e2c96a208c 100644 --- a/pkg/sql/distsqlpb/processors.pb.go +++ b/pkg/sql/distsqlpb/processors.pb.go @@ -6,19 +6,17 @@ package distsqlpb import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" -import cockroach_sql_jobs_jobspb "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" -import cockroach_roachpb1 "github.com/cockroachdb/cockroach/pkg/roachpb" -import cockroach_roachpb4 "github.com/cockroachdb/cockroach/pkg/roachpb" -import cockroach_sql_sqlbase1 "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" -import cockroach_sql_sqlbase3 "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" -import cockroach_util_hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" +import jobspb "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" +import roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" +import sqlbase "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" +import hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" import time "time" import github_com_cockroachdb_cockroach_pkg_roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" import github_com_cockroachdb_cockroach_pkg_sql_sqlbase "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" -import sortkeys "github.com/gogo/protobuf/sortkeys" -import binary "encoding/binary" +import github_com_gogo_protobuf_sortkeys "github.com/gogo/protobuf/sortkeys" +import encoding_binary "encoding/binary" import io "io" @@ -28,6 +26,12 @@ var _ = fmt.Errorf var _ = math.Inf var _ = time.Kitchen +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package + // ScanVisibility controls which columns are seen by scans - just normal // columns, or normal columns and also in-progress schema change columns. type ScanVisibility int32 @@ -62,7 +66,9 @@ func (x *ScanVisibility) UnmarshalJSON(data []byte) error { *x = ScanVisibility(value) return nil } -func (ScanVisibility) EnumDescriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{0} } +func (ScanVisibility) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_processors_bf707fc3f8e6bbc6, []int{0} +} type SketchType int32 @@ -95,7 +101,9 @@ func (x *SketchType) UnmarshalJSON(data []byte) error { *x = SketchType(value) return nil } -func (SketchType) EnumDescriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{1} } +func (SketchType) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_processors_bf707fc3f8e6bbc6, []int{1} +} // These mirror the aggregate functions supported by sql/parser. See // sql/parser/aggregate_builtins.go. @@ -190,7 +198,7 @@ func (x *AggregatorSpec_Func) UnmarshalJSON(data []byte) error { return nil } func (AggregatorSpec_Func) EnumDescriptor() ([]byte, []int) { - return fileDescriptorProcessors, []int{17, 0} + return fileDescriptor_processors_bf707fc3f8e6bbc6, []int{17, 0} } type AggregatorSpec_Type int32 @@ -236,7 +244,7 @@ func (x *AggregatorSpec_Type) UnmarshalJSON(data []byte) error { return nil } func (AggregatorSpec_Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptorProcessors, []int{17, 1} + return fileDescriptor_processors_bf707fc3f8e6bbc6, []int{17, 1} } type BackfillerSpec_Type int32 @@ -275,7 +283,7 @@ func (x *BackfillerSpec_Type) UnmarshalJSON(data []byte) error { return nil } func (BackfillerSpec_Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptorProcessors, []int{18, 0} + return fileDescriptor_processors_bf707fc3f8e6bbc6, []int{18, 0} } type WindowerSpec_WindowFunc int32 @@ -339,7 +347,7 @@ func (x *WindowerSpec_WindowFunc) UnmarshalJSON(data []byte) error { return nil } func (WindowerSpec_WindowFunc) EnumDescriptor() ([]byte, []int) { - return fileDescriptorProcessors, []int{29, 0} + return fileDescriptor_processors_bf707fc3f8e6bbc6, []int{29, 0} } // Mode indicates which mode of framing is used. @@ -383,7 +391,7 @@ func (x *WindowerSpec_Frame_Mode) UnmarshalJSON(data []byte) error { return nil } func (WindowerSpec_Frame_Mode) EnumDescriptor() ([]byte, []int) { - return fileDescriptorProcessors, []int{29, 1, 0} + return fileDescriptor_processors_bf707fc3f8e6bbc6, []int{29, 1, 0} } // BoundType indicates which type of boundary is used. @@ -430,7 +438,7 @@ func (x *WindowerSpec_Frame_BoundType) UnmarshalJSON(data []byte) error { return nil } func (WindowerSpec_Frame_BoundType) EnumDescriptor() ([]byte, []int) { - return fileDescriptorProcessors, []int{29, 1, 1} + return fileDescriptor_processors_bf707fc3f8e6bbc6, []int{29, 1, 1} } // Each processor has the following components: @@ -470,13 +478,39 @@ type ProcessorSpec struct { // has no consequence on the running of flows, but is useful for plan // diagrams and linking information like tracing spans and log messages to // processors. - ProcessorID int32 `protobuf:"varint,6,opt,name=processor_id,json=processorId" json:"processor_id"` + ProcessorID int32 `protobuf:"varint,6,opt,name=processor_id,json=processorId" json:"processor_id"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ProcessorSpec) Reset() { *m = ProcessorSpec{} } +func (m *ProcessorSpec) String() string { return proto.CompactTextString(m) } +func (*ProcessorSpec) ProtoMessage() {} +func (*ProcessorSpec) Descriptor() ([]byte, []int) { + return fileDescriptor_processors_bf707fc3f8e6bbc6, []int{0} +} +func (m *ProcessorSpec) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ProcessorSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ProcessorSpec) XXX_Merge(src proto.Message) { + xxx_messageInfo_ProcessorSpec.Merge(dst, src) +} +func (m *ProcessorSpec) XXX_Size() int { + return m.Size() +} +func (m *ProcessorSpec) XXX_DiscardUnknown() { + xxx_messageInfo_ProcessorSpec.DiscardUnknown(m) } -func (m *ProcessorSpec) Reset() { *m = ProcessorSpec{} } -func (m *ProcessorSpec) String() string { return proto.CompactTextString(m) } -func (*ProcessorSpec) ProtoMessage() {} -func (*ProcessorSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{0} } +var xxx_messageInfo_ProcessorSpec proto.InternalMessageInfo // PostProcessSpec describes the processing required to obtain the output // (filtering, projection). It operates on the internal schema of the processor @@ -505,13 +539,39 @@ type PostProcessSpec struct { Offset uint64 `protobuf:"varint,5,opt,name=offset" json:"offset"` // If nonzero, the processor will stop after emitting this many rows. The rows // suppressed by , if any, do not count towards this limit. - Limit uint64 `protobuf:"varint,6,opt,name=limit" json:"limit"` + Limit uint64 `protobuf:"varint,6,opt,name=limit" json:"limit"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *PostProcessSpec) Reset() { *m = PostProcessSpec{} } -func (m *PostProcessSpec) String() string { return proto.CompactTextString(m) } -func (*PostProcessSpec) ProtoMessage() {} -func (*PostProcessSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{1} } +func (m *PostProcessSpec) Reset() { *m = PostProcessSpec{} } +func (m *PostProcessSpec) String() string { return proto.CompactTextString(m) } +func (*PostProcessSpec) ProtoMessage() {} +func (*PostProcessSpec) Descriptor() ([]byte, []int) { + return fileDescriptor_processors_bf707fc3f8e6bbc6, []int{1} +} +func (m *PostProcessSpec) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *PostProcessSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *PostProcessSpec) XXX_Merge(src proto.Message) { + xxx_messageInfo_PostProcessSpec.Merge(dst, src) +} +func (m *PostProcessSpec) XXX_Size() int { + return m.Size() +} +func (m *PostProcessSpec) XXX_DiscardUnknown() { + xxx_messageInfo_PostProcessSpec.DiscardUnknown(m) +} + +var xxx_messageInfo_PostProcessSpec proto.InternalMessageInfo type ProcessorCoreUnion struct { Noop *NoopCoreSpec `protobuf:"bytes,1,opt,name=noop" json:"noop,omitempty"` @@ -538,43 +598,145 @@ type ProcessorCoreUnion struct { LocalPlanNode *LocalPlanNodeSpec `protobuf:"bytes,24,opt,name=localPlanNode" json:"localPlanNode,omitempty"` ChangeAggregator *ChangeAggregatorSpec `protobuf:"bytes,25,opt,name=changeAggregator" json:"changeAggregator,omitempty"` ChangeFrontier *ChangeFrontierSpec `protobuf:"bytes,26,opt,name=changeFrontier" json:"changeFrontier,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ProcessorCoreUnion) Reset() { *m = ProcessorCoreUnion{} } +func (m *ProcessorCoreUnion) String() string { return proto.CompactTextString(m) } +func (*ProcessorCoreUnion) ProtoMessage() {} +func (*ProcessorCoreUnion) Descriptor() ([]byte, []int) { + return fileDescriptor_processors_bf707fc3f8e6bbc6, []int{2} +} +func (m *ProcessorCoreUnion) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ProcessorCoreUnion) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ProcessorCoreUnion) XXX_Merge(src proto.Message) { + xxx_messageInfo_ProcessorCoreUnion.Merge(dst, src) +} +func (m *ProcessorCoreUnion) XXX_Size() int { + return m.Size() +} +func (m *ProcessorCoreUnion) XXX_DiscardUnknown() { + xxx_messageInfo_ProcessorCoreUnion.DiscardUnknown(m) } -func (m *ProcessorCoreUnion) Reset() { *m = ProcessorCoreUnion{} } -func (m *ProcessorCoreUnion) String() string { return proto.CompactTextString(m) } -func (*ProcessorCoreUnion) ProtoMessage() {} -func (*ProcessorCoreUnion) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{2} } +var xxx_messageInfo_ProcessorCoreUnion proto.InternalMessageInfo // NoopCoreSpec indicates a "no-op" processor core. This is used when we just // need post-processing or when only a synchronizer is required (e.g. at the // final endpoint). type NoopCoreSpec struct { + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *NoopCoreSpec) Reset() { *m = NoopCoreSpec{} } +func (m *NoopCoreSpec) String() string { return proto.CompactTextString(m) } +func (*NoopCoreSpec) ProtoMessage() {} +func (*NoopCoreSpec) Descriptor() ([]byte, []int) { + return fileDescriptor_processors_bf707fc3f8e6bbc6, []int{3} +} +func (m *NoopCoreSpec) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *NoopCoreSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *NoopCoreSpec) XXX_Merge(src proto.Message) { + xxx_messageInfo_NoopCoreSpec.Merge(dst, src) +} +func (m *NoopCoreSpec) XXX_Size() int { + return m.Size() +} +func (m *NoopCoreSpec) XXX_DiscardUnknown() { + xxx_messageInfo_NoopCoreSpec.DiscardUnknown(m) } -func (m *NoopCoreSpec) Reset() { *m = NoopCoreSpec{} } -func (m *NoopCoreSpec) String() string { return proto.CompactTextString(m) } -func (*NoopCoreSpec) ProtoMessage() {} -func (*NoopCoreSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{3} } +var xxx_messageInfo_NoopCoreSpec proto.InternalMessageInfo type MetadataTestSenderSpec struct { - ID string `protobuf:"bytes,1,opt,name=id" json:"id"` + ID string `protobuf:"bytes,1,opt,name=id" json:"id"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *MetadataTestSenderSpec) Reset() { *m = MetadataTestSenderSpec{} } -func (m *MetadataTestSenderSpec) String() string { return proto.CompactTextString(m) } -func (*MetadataTestSenderSpec) ProtoMessage() {} -func (*MetadataTestSenderSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{4} } +func (m *MetadataTestSenderSpec) Reset() { *m = MetadataTestSenderSpec{} } +func (m *MetadataTestSenderSpec) String() string { return proto.CompactTextString(m) } +func (*MetadataTestSenderSpec) ProtoMessage() {} +func (*MetadataTestSenderSpec) Descriptor() ([]byte, []int) { + return fileDescriptor_processors_bf707fc3f8e6bbc6, []int{4} +} +func (m *MetadataTestSenderSpec) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *MetadataTestSenderSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *MetadataTestSenderSpec) XXX_Merge(src proto.Message) { + xxx_messageInfo_MetadataTestSenderSpec.Merge(dst, src) +} +func (m *MetadataTestSenderSpec) XXX_Size() int { + return m.Size() +} +func (m *MetadataTestSenderSpec) XXX_DiscardUnknown() { + xxx_messageInfo_MetadataTestSenderSpec.DiscardUnknown(m) +} + +var xxx_messageInfo_MetadataTestSenderSpec proto.InternalMessageInfo type MetadataTestReceiverSpec struct { - SenderIDs []string `protobuf:"bytes,1,rep,name=sender_ids,json=senderIds" json:"sender_ids,omitempty"` + SenderIDs []string `protobuf:"bytes,1,rep,name=sender_ids,json=senderIds" json:"sender_ids,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *MetadataTestReceiverSpec) Reset() { *m = MetadataTestReceiverSpec{} } func (m *MetadataTestReceiverSpec) String() string { return proto.CompactTextString(m) } func (*MetadataTestReceiverSpec) ProtoMessage() {} func (*MetadataTestReceiverSpec) Descriptor() ([]byte, []int) { - return fileDescriptorProcessors, []int{5} + return fileDescriptor_processors_bf707fc3f8e6bbc6, []int{5} } +func (m *MetadataTestReceiverSpec) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *MetadataTestReceiverSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *MetadataTestReceiverSpec) XXX_Merge(src proto.Message) { + xxx_messageInfo_MetadataTestReceiverSpec.Merge(dst, src) +} +func (m *MetadataTestReceiverSpec) XXX_Size() int { + return m.Size() +} +func (m *MetadataTestReceiverSpec) XXX_DiscardUnknown() { + xxx_messageInfo_MetadataTestReceiverSpec.DiscardUnknown(m) +} + +var xxx_messageInfo_MetadataTestReceiverSpec proto.InternalMessageInfo // ValuesCoreSpec is the core of a processor that has no inputs and generates // "pre-canned" rows. This is not intended to be used for very large datasets. @@ -586,25 +748,77 @@ type ValuesCoreSpec struct { NumRows uint64 `protobuf:"varint,3,opt,name=num_rows,json=numRows" json:"num_rows"` // Each raw block encodes one or more data rows; each datum is encoded // according to the corresponding DatumInfo. - RawBytes [][]byte `protobuf:"bytes,2,rep,name=raw_bytes,json=rawBytes" json:"raw_bytes,omitempty"` + RawBytes [][]byte `protobuf:"bytes,2,rep,name=raw_bytes,json=rawBytes" json:"raw_bytes,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *ValuesCoreSpec) Reset() { *m = ValuesCoreSpec{} } -func (m *ValuesCoreSpec) String() string { return proto.CompactTextString(m) } -func (*ValuesCoreSpec) ProtoMessage() {} -func (*ValuesCoreSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{6} } +func (m *ValuesCoreSpec) Reset() { *m = ValuesCoreSpec{} } +func (m *ValuesCoreSpec) String() string { return proto.CompactTextString(m) } +func (*ValuesCoreSpec) ProtoMessage() {} +func (*ValuesCoreSpec) Descriptor() ([]byte, []int) { + return fileDescriptor_processors_bf707fc3f8e6bbc6, []int{6} +} +func (m *ValuesCoreSpec) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ValuesCoreSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ValuesCoreSpec) XXX_Merge(src proto.Message) { + xxx_messageInfo_ValuesCoreSpec.Merge(dst, src) +} +func (m *ValuesCoreSpec) XXX_Size() int { + return m.Size() +} +func (m *ValuesCoreSpec) XXX_DiscardUnknown() { + xxx_messageInfo_ValuesCoreSpec.DiscardUnknown(m) +} + +var xxx_messageInfo_ValuesCoreSpec proto.InternalMessageInfo type TableReaderSpan struct { // TODO(radu): the dist_sql APIs should be agnostic to how we map tables to // KVs. The span should be described as starting and ending lists of values // for a prefix of the index columns, along with inclusive/exclusive flags. - Span cockroach_roachpb1.Span `protobuf:"bytes,1,opt,name=span" json:"span"` + Span roachpb.Span `protobuf:"bytes,1,opt,name=span" json:"span"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *TableReaderSpan) Reset() { *m = TableReaderSpan{} } -func (m *TableReaderSpan) String() string { return proto.CompactTextString(m) } -func (*TableReaderSpan) ProtoMessage() {} -func (*TableReaderSpan) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{7} } +func (m *TableReaderSpan) Reset() { *m = TableReaderSpan{} } +func (m *TableReaderSpan) String() string { return proto.CompactTextString(m) } +func (*TableReaderSpan) ProtoMessage() {} +func (*TableReaderSpan) Descriptor() ([]byte, []int) { + return fileDescriptor_processors_bf707fc3f8e6bbc6, []int{7} +} +func (m *TableReaderSpan) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *TableReaderSpan) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *TableReaderSpan) XXX_Merge(src proto.Message) { + xxx_messageInfo_TableReaderSpan.Merge(dst, src) +} +func (m *TableReaderSpan) XXX_Size() int { + return m.Size() +} +func (m *TableReaderSpan) XXX_DiscardUnknown() { + xxx_messageInfo_TableReaderSpan.DiscardUnknown(m) +} + +var xxx_messageInfo_TableReaderSpan proto.InternalMessageInfo // TableReaderSpec is the specification for a "table reader". A table reader // performs KV operations to retrieve rows for a table and outputs the desired @@ -619,7 +833,7 @@ func (*TableReaderSpan) Descriptor() ([]byte, []int) { return fileDescriptorProc // - Primary key as a string, if it was obtainable. // - JSON of all decoded column values. type TableReaderSpec struct { - Table cockroach_sql_sqlbase1.TableDescriptor `protobuf:"bytes,1,opt,name=table" json:"table"` + Table sqlbase.TableDescriptor `protobuf:"bytes,1,opt,name=table" json:"table"` // If 0, we use the primary index. If non-zero, we use the index_idx-th index, // i.e. table.indexes[index_idx-1] IndexIdx uint32 `protobuf:"varint,2,opt,name=index_idx,json=indexIdx" json:"index_idx"` @@ -642,13 +856,39 @@ type TableReaderSpec struct { Visibility ScanVisibility `protobuf:"varint,7,opt,name=visibility,enum=cockroach.sql.distsqlpb.ScanVisibility" json:"visibility"` // If non-zero, this is a guarantee for the upper bound of rows a TableReader // will read. If 0, the number of results is unbounded. - MaxResults uint64 `protobuf:"varint,8,opt,name=max_results,json=maxResults" json:"max_results"` + MaxResults uint64 `protobuf:"varint,8,opt,name=max_results,json=maxResults" json:"max_results"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *TableReaderSpec) Reset() { *m = TableReaderSpec{} } -func (m *TableReaderSpec) String() string { return proto.CompactTextString(m) } -func (*TableReaderSpec) ProtoMessage() {} -func (*TableReaderSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{8} } +func (m *TableReaderSpec) Reset() { *m = TableReaderSpec{} } +func (m *TableReaderSpec) String() string { return proto.CompactTextString(m) } +func (*TableReaderSpec) ProtoMessage() {} +func (*TableReaderSpec) Descriptor() ([]byte, []int) { + return fileDescriptor_processors_bf707fc3f8e6bbc6, []int{8} +} +func (m *TableReaderSpec) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *TableReaderSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *TableReaderSpec) XXX_Merge(src proto.Message) { + xxx_messageInfo_TableReaderSpec.Merge(dst, src) +} +func (m *TableReaderSpec) XXX_Size() int { + return m.Size() +} +func (m *TableReaderSpec) XXX_DiscardUnknown() { + xxx_messageInfo_TableReaderSpec.DiscardUnknown(m) +} + +var xxx_messageInfo_TableReaderSpec proto.InternalMessageInfo // JoinReaderSpec is the specification for a "join reader". A join reader // performs KV operations to retrieve specific rows that correspond to the @@ -670,7 +910,7 @@ func (*TableReaderSpec) Descriptor() ([]byte, []int) { return fileDescriptorProc // If performing an index join (where a = c and b = d) (lookup columns is []): // Internal columns: | c | d | e | type JoinReaderSpec struct { - Table cockroach_sql_sqlbase1.TableDescriptor `protobuf:"bytes,1,opt,name=table" json:"table"` + Table sqlbase.TableDescriptor `protobuf:"bytes,1,opt,name=table" json:"table"` // If 0, we use the primary index; each row in the input stream has a value // for each primary key. // TODO(radu): figure out the correct semantics when joining with an index. @@ -696,19 +936,45 @@ type JoinReaderSpec struct { IndexFilterExpr Expression `protobuf:"bytes,5,opt,name=index_filter_expr,json=indexFilterExpr" json:"index_filter_expr"` // For lookup joins. Only JoinType_INNER and JoinType_LEFT_OUTER are // supported. - Type cockroach_sql_sqlbase3.JoinType `protobuf:"varint,6,opt,name=type,enum=cockroach.sql.sqlbase.JoinType" json:"type"` + Type sqlbase.JoinType `protobuf:"varint,6,opt,name=type,enum=cockroach.sql.sqlbase.JoinType" json:"type"` // For index joins that are sources to mutation statements - what visibility // of columns should we return? Mutations sometimes need to see in-progress // schema change columns, in which case this field will be changed from its // default PUBLIC state. Causes the index join to return these schema change // columns. - Visibility ScanVisibility `protobuf:"varint,7,opt,name=visibility,enum=cockroach.sql.distsqlpb.ScanVisibility" json:"visibility"` + Visibility ScanVisibility `protobuf:"varint,7,opt,name=visibility,enum=cockroach.sql.distsqlpb.ScanVisibility" json:"visibility"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *JoinReaderSpec) Reset() { *m = JoinReaderSpec{} } -func (m *JoinReaderSpec) String() string { return proto.CompactTextString(m) } -func (*JoinReaderSpec) ProtoMessage() {} -func (*JoinReaderSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{9} } +func (m *JoinReaderSpec) Reset() { *m = JoinReaderSpec{} } +func (m *JoinReaderSpec) String() string { return proto.CompactTextString(m) } +func (*JoinReaderSpec) ProtoMessage() {} +func (*JoinReaderSpec) Descriptor() ([]byte, []int) { + return fileDescriptor_processors_bf707fc3f8e6bbc6, []int{9} +} +func (m *JoinReaderSpec) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *JoinReaderSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *JoinReaderSpec) XXX_Merge(src proto.Message) { + xxx_messageInfo_JoinReaderSpec.Merge(dst, src) +} +func (m *JoinReaderSpec) XXX_Size() int { + return m.Size() +} +func (m *JoinReaderSpec) XXX_DiscardUnknown() { + xxx_messageInfo_JoinReaderSpec.DiscardUnknown(m) +} + +var xxx_messageInfo_JoinReaderSpec proto.InternalMessageInfo // SorterSpec is the specification for a "sorting aggregator". A sorting // processor sorts elements in the input stream providing a certain output @@ -722,13 +988,39 @@ type SorterSpec struct { // Ordering match length, specifying that the input is already sorted by the // first 'n' output ordering columns, can be optionally specified for // possible speed-ups taking advantage of the partial orderings. - OrderingMatchLen uint32 `protobuf:"varint,2,opt,name=ordering_match_len,json=orderingMatchLen" json:"ordering_match_len"` + OrderingMatchLen uint32 `protobuf:"varint,2,opt,name=ordering_match_len,json=orderingMatchLen" json:"ordering_match_len"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *SorterSpec) Reset() { *m = SorterSpec{} } -func (m *SorterSpec) String() string { return proto.CompactTextString(m) } -func (*SorterSpec) ProtoMessage() {} -func (*SorterSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{10} } +func (m *SorterSpec) Reset() { *m = SorterSpec{} } +func (m *SorterSpec) String() string { return proto.CompactTextString(m) } +func (*SorterSpec) ProtoMessage() {} +func (*SorterSpec) Descriptor() ([]byte, []int) { + return fileDescriptor_processors_bf707fc3f8e6bbc6, []int{10} +} +func (m *SorterSpec) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SorterSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *SorterSpec) XXX_Merge(src proto.Message) { + xxx_messageInfo_SorterSpec.Merge(dst, src) +} +func (m *SorterSpec) XXX_Size() int { + return m.Size() +} +func (m *SorterSpec) XXX_DiscardUnknown() { + xxx_messageInfo_SorterSpec.DiscardUnknown(m) +} + +var xxx_messageInfo_SorterSpec proto.InternalMessageInfo type DistinctSpec struct { // The ordered columns in the input stream can be optionally specified for @@ -742,13 +1034,39 @@ type DistinctSpec struct { // 4th column D which is not included in distinct_columns, its values are not // considered, so rows A1,B1,C1,D1 and A1,B1,C1,D2 are considered equal and // only one of them (the first) is output. - DistinctColumns []uint32 `protobuf:"varint,2,rep,name=distinct_columns,json=distinctColumns" json:"distinct_columns,omitempty"` + DistinctColumns []uint32 `protobuf:"varint,2,rep,name=distinct_columns,json=distinctColumns" json:"distinct_columns,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *DistinctSpec) Reset() { *m = DistinctSpec{} } -func (m *DistinctSpec) String() string { return proto.CompactTextString(m) } -func (*DistinctSpec) ProtoMessage() {} -func (*DistinctSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{11} } +func (m *DistinctSpec) Reset() { *m = DistinctSpec{} } +func (m *DistinctSpec) String() string { return proto.CompactTextString(m) } +func (*DistinctSpec) ProtoMessage() {} +func (*DistinctSpec) Descriptor() ([]byte, []int) { + return fileDescriptor_processors_bf707fc3f8e6bbc6, []int{11} +} +func (m *DistinctSpec) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *DistinctSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *DistinctSpec) XXX_Merge(src proto.Message) { + xxx_messageInfo_DistinctSpec.Merge(dst, src) +} +func (m *DistinctSpec) XXX_Size() int { + return m.Size() +} +func (m *DistinctSpec) XXX_DiscardUnknown() { + xxx_messageInfo_DistinctSpec.DiscardUnknown(m) +} + +var xxx_messageInfo_DistinctSpec proto.InternalMessageInfo // ZigzagJoinerSpec is the specification for a zigzag join processor. The // processor's current implementation fetches the rows using internal @@ -760,7 +1078,7 @@ func (*DistinctSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcess type ZigzagJoinerSpec struct { // TODO(pbardea): Replace these with inputs that conform to a RowSource-like // interface. - Tables []cockroach_sql_sqlbase1.TableDescriptor `protobuf:"bytes,1,rep,name=tables" json:"tables"` + Tables []sqlbase.TableDescriptor `protobuf:"bytes,1,rep,name=tables" json:"tables"` // An array of arrays. The array at eq_columns[side_idx] contains the // equality columns for that side. All arrays in eq_columns should have // equal length. @@ -773,14 +1091,40 @@ type ZigzagJoinerSpec struct { // columns in the right stream. OnExpr Expression `protobuf:"bytes,4,opt,name=on_expr,json=onExpr" json:"on_expr"` // Fixed values at the start of indices. - FixedValues []*ValuesCoreSpec `protobuf:"bytes,5,rep,name=fixed_values,json=fixedValues" json:"fixed_values,omitempty"` - Type cockroach_sql_sqlbase3.JoinType `protobuf:"varint,6,opt,name=type,enum=cockroach.sql.sqlbase.JoinType" json:"type"` + FixedValues []*ValuesCoreSpec `protobuf:"bytes,5,rep,name=fixed_values,json=fixedValues" json:"fixed_values,omitempty"` + Type sqlbase.JoinType `protobuf:"varint,6,opt,name=type,enum=cockroach.sql.sqlbase.JoinType" json:"type"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *ZigzagJoinerSpec) Reset() { *m = ZigzagJoinerSpec{} } -func (m *ZigzagJoinerSpec) String() string { return proto.CompactTextString(m) } -func (*ZigzagJoinerSpec) ProtoMessage() {} -func (*ZigzagJoinerSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{12} } +func (m *ZigzagJoinerSpec) Reset() { *m = ZigzagJoinerSpec{} } +func (m *ZigzagJoinerSpec) String() string { return proto.CompactTextString(m) } +func (*ZigzagJoinerSpec) ProtoMessage() {} +func (*ZigzagJoinerSpec) Descriptor() ([]byte, []int) { + return fileDescriptor_processors_bf707fc3f8e6bbc6, []int{12} +} +func (m *ZigzagJoinerSpec) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ZigzagJoinerSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ZigzagJoinerSpec) XXX_Merge(src proto.Message) { + xxx_messageInfo_ZigzagJoinerSpec.Merge(dst, src) +} +func (m *ZigzagJoinerSpec) XXX_Size() int { + return m.Size() +} +func (m *ZigzagJoinerSpec) XXX_DiscardUnknown() { + xxx_messageInfo_ZigzagJoinerSpec.DiscardUnknown(m) +} + +var xxx_messageInfo_ZigzagJoinerSpec proto.InternalMessageInfo // LocalPlanNodeSpec is the specification for a local planNode wrapping // processor. It's created for situations where a planNode has no DistSQL @@ -790,24 +1134,76 @@ func (*ZigzagJoinerSpec) Descriptor() ([]byte, []int) { return fileDescriptorPro // planNodes. See LocalProcessors and LocalProcessorIndexes on // distsqlplan.PhysicalPlan. type LocalPlanNodeSpec struct { - RowSourceIdx *uint32 `protobuf:"varint,1,opt,name=RowSourceIdx" json:"RowSourceIdx,omitempty"` - NumInputs *uint32 `protobuf:"varint,2,opt,name=NumInputs" json:"NumInputs,omitempty"` - Name *string `protobuf:"bytes,3,opt,name=Name" json:"Name,omitempty"` + RowSourceIdx *uint32 `protobuf:"varint,1,opt,name=RowSourceIdx" json:"RowSourceIdx,omitempty"` + NumInputs *uint32 `protobuf:"varint,2,opt,name=NumInputs" json:"NumInputs,omitempty"` + Name *string `protobuf:"bytes,3,opt,name=Name" json:"Name,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *LocalPlanNodeSpec) Reset() { *m = LocalPlanNodeSpec{} } -func (m *LocalPlanNodeSpec) String() string { return proto.CompactTextString(m) } -func (*LocalPlanNodeSpec) ProtoMessage() {} -func (*LocalPlanNodeSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{13} } +func (m *LocalPlanNodeSpec) Reset() { *m = LocalPlanNodeSpec{} } +func (m *LocalPlanNodeSpec) String() string { return proto.CompactTextString(m) } +func (*LocalPlanNodeSpec) ProtoMessage() {} +func (*LocalPlanNodeSpec) Descriptor() ([]byte, []int) { + return fileDescriptor_processors_bf707fc3f8e6bbc6, []int{13} +} +func (m *LocalPlanNodeSpec) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *LocalPlanNodeSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *LocalPlanNodeSpec) XXX_Merge(src proto.Message) { + xxx_messageInfo_LocalPlanNodeSpec.Merge(dst, src) +} +func (m *LocalPlanNodeSpec) XXX_Size() int { + return m.Size() +} +func (m *LocalPlanNodeSpec) XXX_DiscardUnknown() { + xxx_messageInfo_LocalPlanNodeSpec.DiscardUnknown(m) +} + +var xxx_messageInfo_LocalPlanNodeSpec proto.InternalMessageInfo type Columns struct { - Columns []uint32 `protobuf:"varint,1,rep,packed,name=columns" json:"columns,omitempty"` + Columns []uint32 `protobuf:"varint,1,rep,packed,name=columns" json:"columns,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *Columns) Reset() { *m = Columns{} } -func (m *Columns) String() string { return proto.CompactTextString(m) } -func (*Columns) ProtoMessage() {} -func (*Columns) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{14} } +func (m *Columns) Reset() { *m = Columns{} } +func (m *Columns) String() string { return proto.CompactTextString(m) } +func (*Columns) ProtoMessage() {} +func (*Columns) Descriptor() ([]byte, []int) { + return fileDescriptor_processors_bf707fc3f8e6bbc6, []int{14} +} +func (m *Columns) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Columns) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *Columns) XXX_Merge(src proto.Message) { + xxx_messageInfo_Columns.Merge(dst, src) +} +func (m *Columns) XXX_Size() int { + return m.Size() +} +func (m *Columns) XXX_DiscardUnknown() { + xxx_messageInfo_Columns.DiscardUnknown(m) +} + +var xxx_messageInfo_Columns proto.InternalMessageInfo // MergeJoinerSpec is the specification for a merge join processor. The processor // has two inputs and one output. The inputs must have the same ordering on the @@ -840,18 +1236,44 @@ type MergeJoinerSpec struct { // stream has M columns, in this expression ordinal references @1 to @N refer // to columns of the left stream and variables @(N+1) to @(N+M) refer to // columns in the right stream. - OnExpr Expression `protobuf:"bytes,5,opt,name=on_expr,json=onExpr" json:"on_expr"` - Type cockroach_sql_sqlbase3.JoinType `protobuf:"varint,6,opt,name=type,enum=cockroach.sql.sqlbase.JoinType" json:"type"` + OnExpr Expression `protobuf:"bytes,5,opt,name=on_expr,json=onExpr" json:"on_expr"` + Type sqlbase.JoinType `protobuf:"varint,6,opt,name=type,enum=cockroach.sql.sqlbase.JoinType" json:"type"` // NullEquality indicates that NULL = NULL should be considered true. // This allows OUTER JOINs to consider NULL values meaningfully. An // example of this is during SCRUB checks on secondary indexes. - NullEquality bool `protobuf:"varint,7,opt,name=null_equality,json=nullEquality" json:"null_equality"` + NullEquality bool `protobuf:"varint,7,opt,name=null_equality,json=nullEquality" json:"null_equality"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *MergeJoinerSpec) Reset() { *m = MergeJoinerSpec{} } +func (m *MergeJoinerSpec) String() string { return proto.CompactTextString(m) } +func (*MergeJoinerSpec) ProtoMessage() {} +func (*MergeJoinerSpec) Descriptor() ([]byte, []int) { + return fileDescriptor_processors_bf707fc3f8e6bbc6, []int{15} +} +func (m *MergeJoinerSpec) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *MergeJoinerSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *MergeJoinerSpec) XXX_Merge(src proto.Message) { + xxx_messageInfo_MergeJoinerSpec.Merge(dst, src) +} +func (m *MergeJoinerSpec) XXX_Size() int { + return m.Size() +} +func (m *MergeJoinerSpec) XXX_DiscardUnknown() { + xxx_messageInfo_MergeJoinerSpec.DiscardUnknown(m) } -func (m *MergeJoinerSpec) Reset() { *m = MergeJoinerSpec{} } -func (m *MergeJoinerSpec) String() string { return proto.CompactTextString(m) } -func (*MergeJoinerSpec) ProtoMessage() {} -func (*MergeJoinerSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{15} } +var xxx_messageInfo_MergeJoinerSpec proto.InternalMessageInfo // HashJoinerSpec is the specification for a hash join processor. The processor // has two inputs and one output. @@ -889,8 +1311,8 @@ type HashJoinerSpec struct { // columns of the left stream and variables @N to @(N+M) refer to columns in // the right stream. // Having "ON" expression implies no merged columns. - OnExpr Expression `protobuf:"bytes,5,opt,name=on_expr,json=onExpr" json:"on_expr"` - Type cockroach_sql_sqlbase3.JoinType `protobuf:"varint,6,opt,name=type,enum=cockroach.sql.sqlbase.JoinType" json:"type"` + OnExpr Expression `protobuf:"bytes,5,opt,name=on_expr,json=onExpr" json:"on_expr"` + Type sqlbase.JoinType `protobuf:"varint,6,opt,name=type,enum=cockroach.sql.sqlbase.JoinType" json:"type"` // If true, it is guaranteed that the left equality columns form a key for // the left input. In other words, no two rows from the left input have the // same set of values on the left equality columns. @@ -908,13 +1330,39 @@ type HashJoinerSpec struct { // // This has been deprecated; the distsqlrun layer still supports it for // backward compatibility during upgrade. - MergedColumns bool `protobuf:"varint,7,opt,name=merged_columns,json=mergedColumns" json:"merged_columns"` + MergedColumns bool `protobuf:"varint,7,opt,name=merged_columns,json=mergedColumns" json:"merged_columns"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *HashJoinerSpec) Reset() { *m = HashJoinerSpec{} } -func (m *HashJoinerSpec) String() string { return proto.CompactTextString(m) } -func (*HashJoinerSpec) ProtoMessage() {} -func (*HashJoinerSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{16} } +func (m *HashJoinerSpec) Reset() { *m = HashJoinerSpec{} } +func (m *HashJoinerSpec) String() string { return proto.CompactTextString(m) } +func (*HashJoinerSpec) ProtoMessage() {} +func (*HashJoinerSpec) Descriptor() ([]byte, []int) { + return fileDescriptor_processors_bf707fc3f8e6bbc6, []int{16} +} +func (m *HashJoinerSpec) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *HashJoinerSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *HashJoinerSpec) XXX_Merge(src proto.Message) { + xxx_messageInfo_HashJoinerSpec.Merge(dst, src) +} +func (m *HashJoinerSpec) XXX_Size() int { + return m.Size() +} +func (m *HashJoinerSpec) XXX_DiscardUnknown() { + xxx_messageInfo_HashJoinerSpec.DiscardUnknown(m) +} + +var xxx_messageInfo_HashJoinerSpec proto.InternalMessageInfo // AggregatorSpec is the specification for an "aggregator" (processor core // type, not the logical plan computation stage). An aggregator performs @@ -930,13 +1378,39 @@ type AggregatorSpec struct { GroupCols []uint32 `protobuf:"varint,2,rep,packed,name=group_cols,json=groupCols" json:"group_cols,omitempty"` Aggregations []AggregatorSpec_Aggregation `protobuf:"bytes,3,rep,name=aggregations" json:"aggregations"` // A subset of the GROUP BY columns which are ordered in the input. - OrderedGroupCols []uint32 `protobuf:"varint,4,rep,packed,name=ordered_group_cols,json=orderedGroupCols" json:"ordered_group_cols,omitempty"` + OrderedGroupCols []uint32 `protobuf:"varint,4,rep,packed,name=ordered_group_cols,json=orderedGroupCols" json:"ordered_group_cols,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *AggregatorSpec) Reset() { *m = AggregatorSpec{} } -func (m *AggregatorSpec) String() string { return proto.CompactTextString(m) } -func (*AggregatorSpec) ProtoMessage() {} -func (*AggregatorSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{17} } +func (m *AggregatorSpec) Reset() { *m = AggregatorSpec{} } +func (m *AggregatorSpec) String() string { return proto.CompactTextString(m) } +func (*AggregatorSpec) ProtoMessage() {} +func (*AggregatorSpec) Descriptor() ([]byte, []int) { + return fileDescriptor_processors_bf707fc3f8e6bbc6, []int{17} +} +func (m *AggregatorSpec) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *AggregatorSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *AggregatorSpec) XXX_Merge(src proto.Message) { + xxx_messageInfo_AggregatorSpec.Merge(dst, src) +} +func (m *AggregatorSpec) XXX_Size() int { + return m.Size() +} +func (m *AggregatorSpec) XXX_DiscardUnknown() { + xxx_messageInfo_AggregatorSpec.DiscardUnknown(m) +} + +var xxx_messageInfo_AggregatorSpec proto.InternalMessageInfo type AggregatorSpec_Aggregation struct { Func AggregatorSpec_Func `protobuf:"varint,1,opt,name=func,enum=cockroach.sql.distsqlpb.AggregatorSpec_Func" json:"func"` @@ -957,15 +1431,39 @@ type AggregatorSpec_Aggregation struct { // SELECT SUM(x) FILTER (WHERE y > 1), SUM(x) FILTER (WHERE y < 1) FROM t FilterColIdx *uint32 `protobuf:"varint,4,opt,name=filter_col_idx,json=filterColIdx" json:"filter_col_idx,omitempty"` // Arguments are const expressions passed to aggregation functions. - Arguments []Expression `protobuf:"bytes,6,rep,name=arguments" json:"arguments"` + Arguments []Expression `protobuf:"bytes,6,rep,name=arguments" json:"arguments"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *AggregatorSpec_Aggregation) Reset() { *m = AggregatorSpec_Aggregation{} } func (m *AggregatorSpec_Aggregation) String() string { return proto.CompactTextString(m) } func (*AggregatorSpec_Aggregation) ProtoMessage() {} func (*AggregatorSpec_Aggregation) Descriptor() ([]byte, []int) { - return fileDescriptorProcessors, []int{17, 0} + return fileDescriptor_processors_bf707fc3f8e6bbc6, []int{17, 0} } +func (m *AggregatorSpec_Aggregation) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *AggregatorSpec_Aggregation) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *AggregatorSpec_Aggregation) XXX_Merge(src proto.Message) { + xxx_messageInfo_AggregatorSpec_Aggregation.Merge(dst, src) +} +func (m *AggregatorSpec_Aggregation) XXX_Size() int { + return m.Size() +} +func (m *AggregatorSpec_Aggregation) XXX_DiscardUnknown() { + xxx_messageInfo_AggregatorSpec_Aggregation.DiscardUnknown(m) +} + +var xxx_messageInfo_AggregatorSpec_Aggregation proto.InternalMessageInfo // BackfillerSpec is the specification for a "schema change backfiller". // The created backfill processor runs a backfill for the first mutations in @@ -976,8 +1474,8 @@ func (*AggregatorSpec_Aggregation) Descriptor() ([]byte, []int) { // descriptor in the database, and doesn't emit any rows nor support // any post-processing. type BackfillerSpec struct { - Type BackfillerSpec_Type `protobuf:"varint,1,opt,name=type,enum=cockroach.sql.distsqlpb.BackfillerSpec_Type" json:"type"` - Table cockroach_sql_sqlbase1.TableDescriptor `protobuf:"bytes,2,opt,name=table" json:"table"` + Type BackfillerSpec_Type `protobuf:"varint,1,opt,name=type,enum=cockroach.sql.distsqlpb.BackfillerSpec_Type" json:"type"` + Table sqlbase.TableDescriptor `protobuf:"bytes,2,opt,name=table" json:"table"` // Sections of the table to be backfilled. Spans []TableReaderSpan `protobuf:"bytes,3,rep,name=spans" json:"spans"` // Run the backfill for approximately this duration. @@ -991,29 +1489,81 @@ type BackfillerSpec struct { // Any other (leased) table descriptors necessary for the // backfiller to do its job, such as the descriptors for tables with fk // relationships to the table being modified. - OtherTables []cockroach_sql_sqlbase1.TableDescriptor `protobuf:"bytes,6,rep,name=other_tables,json=otherTables" json:"other_tables"` + OtherTables []sqlbase.TableDescriptor `protobuf:"bytes,6,rep,name=other_tables,json=otherTables" json:"other_tables"` // The timestamp to perform index backfill historical scans at. - ReadAsOf cockroach_util_hlc.Timestamp `protobuf:"bytes,7,opt,name=readAsOf" json:"readAsOf"` + ReadAsOf hlc.Timestamp `protobuf:"bytes,7,opt,name=readAsOf" json:"readAsOf"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *BackfillerSpec) Reset() { *m = BackfillerSpec{} } +func (m *BackfillerSpec) String() string { return proto.CompactTextString(m) } +func (*BackfillerSpec) ProtoMessage() {} +func (*BackfillerSpec) Descriptor() ([]byte, []int) { + return fileDescriptor_processors_bf707fc3f8e6bbc6, []int{18} +} +func (m *BackfillerSpec) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *BackfillerSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *BackfillerSpec) XXX_Merge(src proto.Message) { + xxx_messageInfo_BackfillerSpec.Merge(dst, src) +} +func (m *BackfillerSpec) XXX_Size() int { + return m.Size() +} +func (m *BackfillerSpec) XXX_DiscardUnknown() { + xxx_messageInfo_BackfillerSpec.DiscardUnknown(m) } -func (m *BackfillerSpec) Reset() { *m = BackfillerSpec{} } -func (m *BackfillerSpec) String() string { return proto.CompactTextString(m) } -func (*BackfillerSpec) ProtoMessage() {} -func (*BackfillerSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{18} } +var xxx_messageInfo_BackfillerSpec proto.InternalMessageInfo // FlowSpec describes a "flow" which is a subgraph of a distributed SQL // computation consisting of processors and streams. type FlowSpec struct { FlowID FlowID `protobuf:"bytes,1,opt,name=flow_id,json=flowId,customtype=FlowID" json:"flow_id"` // The NodeID of the gateway that planned this Flow. Used for debugging. - Gateway github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,3,opt,name=gateway,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"gateway"` - Processors []ProcessorSpec `protobuf:"bytes,2,rep,name=processors" json:"processors"` + Gateway github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,3,opt,name=gateway,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"gateway"` + Processors []ProcessorSpec `protobuf:"bytes,2,rep,name=processors" json:"processors"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *FlowSpec) Reset() { *m = FlowSpec{} } -func (m *FlowSpec) String() string { return proto.CompactTextString(m) } -func (*FlowSpec) ProtoMessage() {} -func (*FlowSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{19} } +func (m *FlowSpec) Reset() { *m = FlowSpec{} } +func (m *FlowSpec) String() string { return proto.CompactTextString(m) } +func (*FlowSpec) ProtoMessage() {} +func (*FlowSpec) Descriptor() ([]byte, []int) { + return fileDescriptor_processors_bf707fc3f8e6bbc6, []int{19} +} +func (m *FlowSpec) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *FlowSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *FlowSpec) XXX_Merge(src proto.Message) { + xxx_messageInfo_FlowSpec.Merge(dst, src) +} +func (m *FlowSpec) XXX_Size() int { + return m.Size() +} +func (m *FlowSpec) XXX_DiscardUnknown() { + xxx_messageInfo_FlowSpec.DiscardUnknown(m) +} + +var xxx_messageInfo_FlowSpec proto.InternalMessageInfo // JobProgress identifies the job to report progress on. This reporting // happens outside this package. @@ -1023,16 +1573,42 @@ type JobProgress struct { // process. Contribution float32 `protobuf:"fixed32,2,opt,name=contribution" json:"contribution"` // slot is the index into the job details for this processor's completion. - Slot int32 `protobuf:"varint,3,opt,name=slot" json:"slot"` + Slot int32 `protobuf:"varint,3,opt,name=slot" json:"slot"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *JobProgress) Reset() { *m = JobProgress{} } -func (m *JobProgress) String() string { return proto.CompactTextString(m) } -func (*JobProgress) ProtoMessage() {} -func (*JobProgress) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{20} } +func (m *JobProgress) Reset() { *m = JobProgress{} } +func (m *JobProgress) String() string { return proto.CompactTextString(m) } +func (*JobProgress) ProtoMessage() {} +func (*JobProgress) Descriptor() ([]byte, []int) { + return fileDescriptor_processors_bf707fc3f8e6bbc6, []int{20} +} +func (m *JobProgress) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *JobProgress) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *JobProgress) XXX_Merge(src proto.Message) { + xxx_messageInfo_JobProgress.Merge(dst, src) +} +func (m *JobProgress) XXX_Size() int { + return m.Size() +} +func (m *JobProgress) XXX_DiscardUnknown() { + xxx_messageInfo_JobProgress.DiscardUnknown(m) +} + +var xxx_messageInfo_JobProgress proto.InternalMessageInfo type ReadImportDataSpec struct { - Format cockroach_roachpb4.IOFileFormat `protobuf:"bytes,8,opt,name=format" json:"format"` + Format roachpb.IOFileFormat `protobuf:"bytes,8,opt,name=format" json:"format"` // sample_size is the rate at which to output rows, based on an input row's size. SampleSize int32 `protobuf:"varint,2,opt,name=sample_size,json=sampleSize" json:"sample_size"` // tables is an alternative to table_desc for input formats that can read @@ -1046,18 +1622,44 @@ type ReadImportDataSpec struct { // TableDescriptor with the corresponding descriptor ID key. If tables is // empty (and table_desc above is not specified), the processor should read // all tables in the input, determining their schemas on the fly. - Tables map[string]*cockroach_sql_sqlbase1.TableDescriptor `protobuf:"bytes,9,rep,name=tables" json:"tables,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + Tables map[string]*sqlbase.TableDescriptor `protobuf:"bytes,9,rep,name=tables" json:"tables,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` // uri is a storageccl.ExportStorage URI pointing to the CSV files to be // read. The map key must be unique across the entire IMPORT job. Uri map[int32]string `protobuf:"bytes,7,rep,name=uri" json:"uri,omitempty" protobuf_key:"varint,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` Progress JobProgress `protobuf:"bytes,6,opt,name=progress" json:"progress"` SkipMissingForeignKeys bool `protobuf:"varint,10,opt,name=skip_missing_foreign_keys,json=skipMissingForeignKeys" json:"skip_missing_foreign_keys"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ReadImportDataSpec) Reset() { *m = ReadImportDataSpec{} } +func (m *ReadImportDataSpec) String() string { return proto.CompactTextString(m) } +func (*ReadImportDataSpec) ProtoMessage() {} +func (*ReadImportDataSpec) Descriptor() ([]byte, []int) { + return fileDescriptor_processors_bf707fc3f8e6bbc6, []int{21} +} +func (m *ReadImportDataSpec) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ReadImportDataSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ReadImportDataSpec) XXX_Merge(src proto.Message) { + xxx_messageInfo_ReadImportDataSpec.Merge(dst, src) +} +func (m *ReadImportDataSpec) XXX_Size() int { + return m.Size() +} +func (m *ReadImportDataSpec) XXX_DiscardUnknown() { + xxx_messageInfo_ReadImportDataSpec.DiscardUnknown(m) } -func (m *ReadImportDataSpec) Reset() { *m = ReadImportDataSpec{} } -func (m *ReadImportDataSpec) String() string { return proto.CompactTextString(m) } -func (*ReadImportDataSpec) ProtoMessage() {} -func (*ReadImportDataSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{21} } +var xxx_messageInfo_ReadImportDataSpec proto.InternalMessageInfo // SSTWriterSpec is the specification for a processor that consumes rows, uses // tempStorage to sort them, then writes them to SST files at uri. walltime is @@ -1071,28 +1673,78 @@ type SSTWriterSpec struct { // walltimeNanos is the MVCC time at which the created KVs will be written. WalltimeNanos int64 `protobuf:"varint,3,opt,name=walltimeNanos" json:"walltimeNanos"` // spans is an array of span boundaries and corresponding filenames. - Spans []SSTWriterSpec_SpanName `protobuf:"bytes,4,rep,name=spans" json:"spans"` - Progress JobProgress `protobuf:"bytes,5,opt,name=progress" json:"progress"` + Spans []SSTWriterSpec_SpanName `protobuf:"bytes,4,rep,name=spans" json:"spans"` + Progress JobProgress `protobuf:"bytes,5,opt,name=progress" json:"progress"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *SSTWriterSpec) Reset() { *m = SSTWriterSpec{} } -func (m *SSTWriterSpec) String() string { return proto.CompactTextString(m) } -func (*SSTWriterSpec) ProtoMessage() {} -func (*SSTWriterSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{22} } +func (m *SSTWriterSpec) Reset() { *m = SSTWriterSpec{} } +func (m *SSTWriterSpec) String() string { return proto.CompactTextString(m) } +func (*SSTWriterSpec) ProtoMessage() {} +func (*SSTWriterSpec) Descriptor() ([]byte, []int) { + return fileDescriptor_processors_bf707fc3f8e6bbc6, []int{22} +} +func (m *SSTWriterSpec) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SSTWriterSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *SSTWriterSpec) XXX_Merge(src proto.Message) { + xxx_messageInfo_SSTWriterSpec.Merge(dst, src) +} +func (m *SSTWriterSpec) XXX_Size() int { + return m.Size() +} +func (m *SSTWriterSpec) XXX_DiscardUnknown() { + xxx_messageInfo_SSTWriterSpec.DiscardUnknown(m) +} + +var xxx_messageInfo_SSTWriterSpec proto.InternalMessageInfo type SSTWriterSpec_SpanName struct { // name is the file name that will be written by the export store. Name string `protobuf:"bytes,1,opt,name=name" json:"name"` // end is the end key of a span. - End []byte `protobuf:"bytes,2,opt,name=end" json:"end,omitempty"` + End []byte `protobuf:"bytes,2,opt,name=end" json:"end,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *SSTWriterSpec_SpanName) Reset() { *m = SSTWriterSpec_SpanName{} } func (m *SSTWriterSpec_SpanName) String() string { return proto.CompactTextString(m) } func (*SSTWriterSpec_SpanName) ProtoMessage() {} func (*SSTWriterSpec_SpanName) Descriptor() ([]byte, []int) { - return fileDescriptorProcessors, []int{22, 0} + return fileDescriptor_processors_bf707fc3f8e6bbc6, []int{22, 0} +} +func (m *SSTWriterSpec_SpanName) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SSTWriterSpec_SpanName) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil } +func (dst *SSTWriterSpec_SpanName) XXX_Merge(src proto.Message) { + xxx_messageInfo_SSTWriterSpec_SpanName.Merge(dst, src) +} +func (m *SSTWriterSpec_SpanName) XXX_Size() int { + return m.Size() +} +func (m *SSTWriterSpec_SpanName) XXX_DiscardUnknown() { + xxx_messageInfo_SSTWriterSpec_SpanName.DiscardUnknown(m) +} + +var xxx_messageInfo_SSTWriterSpec_SpanName proto.InternalMessageInfo // CSVWriterSpec is the specification for a processor that consumes rows and // writes them to CSV files at uri. It outputs a row per file written with @@ -1100,17 +1752,43 @@ func (*SSTWriterSpec_SpanName) Descriptor() ([]byte, []int) { type CSVWriterSpec struct { // destination as a storageccl.ExportStorage URI pointing to an export store // location (directory). - Destination string `protobuf:"bytes,1,opt,name=destination" json:"destination"` - NamePattern string `protobuf:"bytes,2,opt,name=name_pattern,json=namePattern" json:"name_pattern"` - Options cockroach_roachpb4.CSVOptions `protobuf:"bytes,3,opt,name=options" json:"options"` + Destination string `protobuf:"bytes,1,opt,name=destination" json:"destination"` + NamePattern string `protobuf:"bytes,2,opt,name=name_pattern,json=namePattern" json:"name_pattern"` + Options roachpb.CSVOptions `protobuf:"bytes,3,opt,name=options" json:"options"` // chunk_rows is num rows to write per file. 0 = no limit. - ChunkRows int64 `protobuf:"varint,4,opt,name=chunk_rows,json=chunkRows" json:"chunk_rows"` + ChunkRows int64 `protobuf:"varint,4,opt,name=chunk_rows,json=chunkRows" json:"chunk_rows"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *CSVWriterSpec) Reset() { *m = CSVWriterSpec{} } +func (m *CSVWriterSpec) String() string { return proto.CompactTextString(m) } +func (*CSVWriterSpec) ProtoMessage() {} +func (*CSVWriterSpec) Descriptor() ([]byte, []int) { + return fileDescriptor_processors_bf707fc3f8e6bbc6, []int{23} +} +func (m *CSVWriterSpec) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *CSVWriterSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *CSVWriterSpec) XXX_Merge(src proto.Message) { + xxx_messageInfo_CSVWriterSpec.Merge(dst, src) +} +func (m *CSVWriterSpec) XXX_Size() int { + return m.Size() +} +func (m *CSVWriterSpec) XXX_DiscardUnknown() { + xxx_messageInfo_CSVWriterSpec.DiscardUnknown(m) } -func (m *CSVWriterSpec) Reset() { *m = CSVWriterSpec{} } -func (m *CSVWriterSpec) String() string { return proto.CompactTextString(m) } -func (*CSVWriterSpec) ProtoMessage() {} -func (*CSVWriterSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{23} } +var xxx_messageInfo_CSVWriterSpec proto.InternalMessageInfo // SketchSpec contains the specification for a generated statistic. type SketchSpec struct { @@ -1125,13 +1803,39 @@ type SketchSpec struct { // Only used by the SampleAggregator. HistogramMaxBuckets uint32 `protobuf:"varint,4,opt,name=histogram_max_buckets,json=histogramMaxBuckets" json:"histogram_max_buckets"` // Only used by the SampleAggregator. - StatName string `protobuf:"bytes,5,opt,name=stat_name,json=statName" json:"stat_name"` + StatName string `protobuf:"bytes,5,opt,name=stat_name,json=statName" json:"stat_name"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *SketchSpec) Reset() { *m = SketchSpec{} } -func (m *SketchSpec) String() string { return proto.CompactTextString(m) } -func (*SketchSpec) ProtoMessage() {} -func (*SketchSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{24} } +func (m *SketchSpec) Reset() { *m = SketchSpec{} } +func (m *SketchSpec) String() string { return proto.CompactTextString(m) } +func (*SketchSpec) ProtoMessage() {} +func (*SketchSpec) Descriptor() ([]byte, []int) { + return fileDescriptor_processors_bf707fc3f8e6bbc6, []int{24} +} +func (m *SketchSpec) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SketchSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *SketchSpec) XXX_Merge(src proto.Message) { + xxx_messageInfo_SketchSpec.Merge(dst, src) +} +func (m *SketchSpec) XXX_Size() int { + return m.Size() +} +func (m *SketchSpec) XXX_DiscardUnknown() { + xxx_messageInfo_SketchSpec.DiscardUnknown(m) +} + +var xxx_messageInfo_SketchSpec proto.InternalMessageInfo // SamplerSpec is the specification of a "sampler" processor which // returns a sample (random subset) of the input columns and computes @@ -1165,14 +1869,40 @@ func (*SketchSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessor // Rows have NULLs on either all the sampled row columns or on all the // sketch columns. type SamplerSpec struct { - Sketches []SketchSpec `protobuf:"bytes,1,rep,name=sketches" json:"sketches"` - SampleSize uint32 `protobuf:"varint,2,opt,name=sample_size,json=sampleSize" json:"sample_size"` + Sketches []SketchSpec `protobuf:"bytes,1,rep,name=sketches" json:"sketches"` + SampleSize uint32 `protobuf:"varint,2,opt,name=sample_size,json=sampleSize" json:"sample_size"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *SamplerSpec) Reset() { *m = SamplerSpec{} } +func (m *SamplerSpec) String() string { return proto.CompactTextString(m) } +func (*SamplerSpec) ProtoMessage() {} +func (*SamplerSpec) Descriptor() ([]byte, []int) { + return fileDescriptor_processors_bf707fc3f8e6bbc6, []int{25} +} +func (m *SamplerSpec) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SamplerSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *SamplerSpec) XXX_Merge(src proto.Message) { + xxx_messageInfo_SamplerSpec.Merge(dst, src) +} +func (m *SamplerSpec) XXX_Size() int { + return m.Size() +} +func (m *SamplerSpec) XXX_DiscardUnknown() { + xxx_messageInfo_SamplerSpec.DiscardUnknown(m) } -func (m *SamplerSpec) Reset() { *m = SamplerSpec{} } -func (m *SamplerSpec) String() string { return proto.CompactTextString(m) } -func (*SamplerSpec) ProtoMessage() {} -func (*SamplerSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{25} } +var xxx_messageInfo_SamplerSpec proto.InternalMessageInfo // SampleAggregatorSpec is the specification of a processor that aggregates the // results from multiple sampler processors and writes out the statistics to @@ -1196,14 +1926,40 @@ type SampleAggregatorSpec struct { SampleSize uint32 `protobuf:"varint,2,opt,name=sample_size,json=sampleSize" json:"sample_size"` // The i-th value indicates the ColumnID of the i-th sampled row column. // These are necessary for writing out the statistic data. - SampledColumnIDs []github_com_cockroachdb_cockroach_pkg_sql_sqlbase.ColumnID `protobuf:"varint,3,rep,name=sampled_column_ids,json=sampledColumnIds,casttype=github.com/cockroachdb/cockroach/pkg/sql/sqlbase.ColumnID" json:"sampled_column_ids,omitempty"` - TableID github_com_cockroachdb_cockroach_pkg_sql_sqlbase.ID `protobuf:"varint,4,opt,name=table_id,json=tableId,casttype=github.com/cockroachdb/cockroach/pkg/sql/sqlbase.ID" json:"table_id"` + SampledColumnIDs []github_com_cockroachdb_cockroach_pkg_sql_sqlbase.ColumnID `protobuf:"varint,3,rep,name=sampled_column_ids,json=sampledColumnIds,casttype=github.com/cockroachdb/cockroach/pkg/sql/sqlbase.ColumnID" json:"sampled_column_ids,omitempty"` + TableID github_com_cockroachdb_cockroach_pkg_sql_sqlbase.ID `protobuf:"varint,4,opt,name=table_id,json=tableId,casttype=github.com/cockroachdb/cockroach/pkg/sql/sqlbase.ID" json:"table_id"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *SampleAggregatorSpec) Reset() { *m = SampleAggregatorSpec{} } +func (m *SampleAggregatorSpec) String() string { return proto.CompactTextString(m) } +func (*SampleAggregatorSpec) ProtoMessage() {} +func (*SampleAggregatorSpec) Descriptor() ([]byte, []int) { + return fileDescriptor_processors_bf707fc3f8e6bbc6, []int{26} +} +func (m *SampleAggregatorSpec) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SampleAggregatorSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *SampleAggregatorSpec) XXX_Merge(src proto.Message) { + xxx_messageInfo_SampleAggregatorSpec.Merge(dst, src) +} +func (m *SampleAggregatorSpec) XXX_Size() int { + return m.Size() +} +func (m *SampleAggregatorSpec) XXX_DiscardUnknown() { + xxx_messageInfo_SampleAggregatorSpec.DiscardUnknown(m) } -func (m *SampleAggregatorSpec) Reset() { *m = SampleAggregatorSpec{} } -func (m *SampleAggregatorSpec) String() string { return proto.CompactTextString(m) } -func (*SampleAggregatorSpec) ProtoMessage() {} -func (*SampleAggregatorSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{26} } +var xxx_messageInfo_SampleAggregatorSpec proto.InternalMessageInfo // InterleavedReaderJoinerSpec is the specification for a processor that performs // KV operations to retrieve rows from 2+ tables from an interleaved hierarchy, @@ -1235,19 +1991,43 @@ type InterleavedReaderJoinerSpec struct { // table stream has M columns, in this expression ordinal references @1 to @N // refer to columns of the left table and variables @(N+1) to @(N+M) refer to // columns in the right table. - OnExpr Expression `protobuf:"bytes,4,opt,name=on_expr,json=onExpr" json:"on_expr"` - Type cockroach_sql_sqlbase3.JoinType `protobuf:"varint,5,opt,name=type,enum=cockroach.sql.sqlbase.JoinType" json:"type"` + OnExpr Expression `protobuf:"bytes,4,opt,name=on_expr,json=onExpr" json:"on_expr"` + Type sqlbase.JoinType `protobuf:"varint,5,opt,name=type,enum=cockroach.sql.sqlbase.JoinType" json:"type"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *InterleavedReaderJoinerSpec) Reset() { *m = InterleavedReaderJoinerSpec{} } func (m *InterleavedReaderJoinerSpec) String() string { return proto.CompactTextString(m) } func (*InterleavedReaderJoinerSpec) ProtoMessage() {} func (*InterleavedReaderJoinerSpec) Descriptor() ([]byte, []int) { - return fileDescriptorProcessors, []int{27} + return fileDescriptor_processors_bf707fc3f8e6bbc6, []int{27} +} +func (m *InterleavedReaderJoinerSpec) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) } +func (m *InterleavedReaderJoinerSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *InterleavedReaderJoinerSpec) XXX_Merge(src proto.Message) { + xxx_messageInfo_InterleavedReaderJoinerSpec.Merge(dst, src) +} +func (m *InterleavedReaderJoinerSpec) XXX_Size() int { + return m.Size() +} +func (m *InterleavedReaderJoinerSpec) XXX_DiscardUnknown() { + xxx_messageInfo_InterleavedReaderJoinerSpec.DiscardUnknown(m) +} + +var xxx_messageInfo_InterleavedReaderJoinerSpec proto.InternalMessageInfo type InterleavedReaderJoinerSpec_Table struct { - Desc cockroach_sql_sqlbase1.TableDescriptor `protobuf:"bytes,1,opt,name=desc" json:"desc"` + Desc sqlbase.TableDescriptor `protobuf:"bytes,1,opt,name=desc" json:"desc"` // If 0, we use the primary index. If non-zero, we use the index_idx-th index, // i.e. desc.indexes[index_idx-1] IndexIdx uint32 `protobuf:"varint,2,opt,name=index_idx,json=indexIdx" json:"index_idx"` @@ -1270,15 +2050,39 @@ type InterleavedReaderJoinerSpec_Table struct { // all tables to do a single pass-through scan. InterleavedReaderJoiner will // then check if a given row for a table is within any of its spans. // There must exist at least one non-empty set of spans for some table. - Spans []TableReaderSpan `protobuf:"bytes,5,rep,name=spans" json:"spans"` + Spans []TableReaderSpan `protobuf:"bytes,5,rep,name=spans" json:"spans"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *InterleavedReaderJoinerSpec_Table) Reset() { *m = InterleavedReaderJoinerSpec_Table{} } func (m *InterleavedReaderJoinerSpec_Table) String() string { return proto.CompactTextString(m) } func (*InterleavedReaderJoinerSpec_Table) ProtoMessage() {} func (*InterleavedReaderJoinerSpec_Table) Descriptor() ([]byte, []int) { - return fileDescriptorProcessors, []int{27, 0} + return fileDescriptor_processors_bf707fc3f8e6bbc6, []int{27, 0} } +func (m *InterleavedReaderJoinerSpec_Table) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *InterleavedReaderJoinerSpec_Table) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *InterleavedReaderJoinerSpec_Table) XXX_Merge(src proto.Message) { + xxx_messageInfo_InterleavedReaderJoinerSpec_Table.Merge(dst, src) +} +func (m *InterleavedReaderJoinerSpec_Table) XXX_Size() int { + return m.Size() +} +func (m *InterleavedReaderJoinerSpec_Table) XXX_DiscardUnknown() { + xxx_messageInfo_InterleavedReaderJoinerSpec_Table.DiscardUnknown(m) +} + +var xxx_messageInfo_InterleavedReaderJoinerSpec_Table proto.InternalMessageInfo // ProjectSetSpec is the specification of a processor which applies a set of // expressions, which may be set-returning functions, to its input. @@ -1286,15 +2090,41 @@ type ProjectSetSpec struct { // Expressions to be applied Exprs []Expression `protobuf:"bytes,1,rep,name=exprs" json:"exprs"` // Column types for the generated values - GeneratedColumns []cockroach_sql_sqlbase1.ColumnType `protobuf:"bytes,2,rep,name=generated_columns,json=generatedColumns" json:"generated_columns"` + GeneratedColumns []sqlbase.ColumnType `protobuf:"bytes,2,rep,name=generated_columns,json=generatedColumns" json:"generated_columns"` // The number of columns each expression returns. Same length as exprs. - NumColsPerGen []uint32 `protobuf:"varint,3,rep,name=num_cols_per_gen,json=numColsPerGen" json:"num_cols_per_gen,omitempty"` + NumColsPerGen []uint32 `protobuf:"varint,3,rep,name=num_cols_per_gen,json=numColsPerGen" json:"num_cols_per_gen,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *ProjectSetSpec) Reset() { *m = ProjectSetSpec{} } -func (m *ProjectSetSpec) String() string { return proto.CompactTextString(m) } -func (*ProjectSetSpec) ProtoMessage() {} -func (*ProjectSetSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{28} } +func (m *ProjectSetSpec) Reset() { *m = ProjectSetSpec{} } +func (m *ProjectSetSpec) String() string { return proto.CompactTextString(m) } +func (*ProjectSetSpec) ProtoMessage() {} +func (*ProjectSetSpec) Descriptor() ([]byte, []int) { + return fileDescriptor_processors_bf707fc3f8e6bbc6, []int{28} +} +func (m *ProjectSetSpec) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ProjectSetSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ProjectSetSpec) XXX_Merge(src proto.Message) { + xxx_messageInfo_ProjectSetSpec.Merge(dst, src) +} +func (m *ProjectSetSpec) XXX_Size() int { + return m.Size() +} +func (m *ProjectSetSpec) XXX_DiscardUnknown() { + xxx_messageInfo_ProjectSetSpec.DiscardUnknown(m) +} + +var xxx_messageInfo_ProjectSetSpec proto.InternalMessageInfo // WindowerSpec is the specification of a processor that performs computations // of window functions that have the same PARTITION BY clause. For a particular @@ -1306,36 +2136,114 @@ type WindowerSpec struct { // PartitionBy specifies how to partition rows for all window functions. PartitionBy []uint32 `protobuf:"varint,1,rep,name=partitionBy" json:"partitionBy,omitempty"` // WindowFns is the specification of all window functions to be computed. - WindowFns []WindowerSpec_WindowFn `protobuf:"bytes,2,rep,name=windowFns" json:"windowFns"` + WindowFns []WindowerSpec_WindowFn `protobuf:"bytes,2,rep,name=windowFns" json:"windowFns"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *WindowerSpec) Reset() { *m = WindowerSpec{} } -func (m *WindowerSpec) String() string { return proto.CompactTextString(m) } -func (*WindowerSpec) ProtoMessage() {} -func (*WindowerSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{29} } +func (m *WindowerSpec) Reset() { *m = WindowerSpec{} } +func (m *WindowerSpec) String() string { return proto.CompactTextString(m) } +func (*WindowerSpec) ProtoMessage() {} +func (*WindowerSpec) Descriptor() ([]byte, []int) { + return fileDescriptor_processors_bf707fc3f8e6bbc6, []int{29} +} +func (m *WindowerSpec) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *WindowerSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *WindowerSpec) XXX_Merge(src proto.Message) { + xxx_messageInfo_WindowerSpec.Merge(dst, src) +} +func (m *WindowerSpec) XXX_Size() int { + return m.Size() +} +func (m *WindowerSpec) XXX_DiscardUnknown() { + xxx_messageInfo_WindowerSpec.DiscardUnknown(m) +} + +var xxx_messageInfo_WindowerSpec proto.InternalMessageInfo // Func specifies which function to compute. It can either be built-in // aggregate or built-in window function. type WindowerSpec_Func struct { - AggregateFunc *AggregatorSpec_Func `protobuf:"varint,1,opt,name=aggregateFunc,enum=cockroach.sql.distsqlpb.AggregatorSpec_Func" json:"aggregateFunc,omitempty"` - WindowFunc *WindowerSpec_WindowFunc `protobuf:"varint,2,opt,name=windowFunc,enum=cockroach.sql.distsqlpb.WindowerSpec_WindowFunc" json:"windowFunc,omitempty"` + AggregateFunc *AggregatorSpec_Func `protobuf:"varint,1,opt,name=aggregateFunc,enum=cockroach.sql.distsqlpb.AggregatorSpec_Func" json:"aggregateFunc,omitempty"` + WindowFunc *WindowerSpec_WindowFunc `protobuf:"varint,2,opt,name=windowFunc,enum=cockroach.sql.distsqlpb.WindowerSpec_WindowFunc" json:"windowFunc,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *WindowerSpec_Func) Reset() { *m = WindowerSpec_Func{} } -func (m *WindowerSpec_Func) String() string { return proto.CompactTextString(m) } -func (*WindowerSpec_Func) ProtoMessage() {} -func (*WindowerSpec_Func) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{29, 0} } +func (m *WindowerSpec_Func) Reset() { *m = WindowerSpec_Func{} } +func (m *WindowerSpec_Func) String() string { return proto.CompactTextString(m) } +func (*WindowerSpec_Func) ProtoMessage() {} +func (*WindowerSpec_Func) Descriptor() ([]byte, []int) { + return fileDescriptor_processors_bf707fc3f8e6bbc6, []int{29, 0} +} +func (m *WindowerSpec_Func) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *WindowerSpec_Func) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *WindowerSpec_Func) XXX_Merge(src proto.Message) { + xxx_messageInfo_WindowerSpec_Func.Merge(dst, src) +} +func (m *WindowerSpec_Func) XXX_Size() int { + return m.Size() +} +func (m *WindowerSpec_Func) XXX_DiscardUnknown() { + xxx_messageInfo_WindowerSpec_Func.DiscardUnknown(m) +} + +var xxx_messageInfo_WindowerSpec_Func proto.InternalMessageInfo // Frame is the specification of a single window frame for a window function. type WindowerSpec_Frame struct { - Mode WindowerSpec_Frame_Mode `protobuf:"varint,1,opt,name=mode,enum=cockroach.sql.distsqlpb.WindowerSpec_Frame_Mode" json:"mode"` - Bounds WindowerSpec_Frame_Bounds `protobuf:"bytes,2,opt,name=bounds" json:"bounds"` + Mode WindowerSpec_Frame_Mode `protobuf:"varint,1,opt,name=mode,enum=cockroach.sql.distsqlpb.WindowerSpec_Frame_Mode" json:"mode"` + Bounds WindowerSpec_Frame_Bounds `protobuf:"bytes,2,opt,name=bounds" json:"bounds"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *WindowerSpec_Frame) Reset() { *m = WindowerSpec_Frame{} } -func (m *WindowerSpec_Frame) String() string { return proto.CompactTextString(m) } -func (*WindowerSpec_Frame) ProtoMessage() {} -func (*WindowerSpec_Frame) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{29, 1} } +func (m *WindowerSpec_Frame) Reset() { *m = WindowerSpec_Frame{} } +func (m *WindowerSpec_Frame) String() string { return proto.CompactTextString(m) } +func (*WindowerSpec_Frame) ProtoMessage() {} +func (*WindowerSpec_Frame) Descriptor() ([]byte, []int) { + return fileDescriptor_processors_bf707fc3f8e6bbc6, []int{29, 1} +} +func (m *WindowerSpec_Frame) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *WindowerSpec_Frame) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *WindowerSpec_Frame) XXX_Merge(src proto.Message) { + xxx_messageInfo_WindowerSpec_Frame.Merge(dst, src) +} +func (m *WindowerSpec_Frame) XXX_Size() int { + return m.Size() +} +func (m *WindowerSpec_Frame) XXX_DiscardUnknown() { + xxx_messageInfo_WindowerSpec_Frame.DiscardUnknown(m) +} + +var xxx_messageInfo_WindowerSpec_Frame proto.InternalMessageInfo // Bound specifies the type of boundary and the offset (if present). type WindowerSpec_Frame_Bound struct { @@ -1343,31 +2251,79 @@ type WindowerSpec_Frame_Bound struct { // For UNBOUNDED_PRECEDING, UNBOUNDED_FOLLOWING, and CURRENT_ROW offset // is ignored. Integer offset for ROWS mode is stored in int_offset while // an encoded datum and the type information are stored for RANGE mode. - IntOffset uint32 `protobuf:"varint,2,opt,name=int_offset,json=intOffset" json:"int_offset"` - TypedOffset []byte `protobuf:"bytes,3,opt,name=typed_offset,json=typedOffset" json:"typed_offset,omitempty"` - OffsetType DatumInfo `protobuf:"bytes,4,opt,name=offset_type,json=offsetType" json:"offset_type"` + IntOffset uint32 `protobuf:"varint,2,opt,name=int_offset,json=intOffset" json:"int_offset"` + TypedOffset []byte `protobuf:"bytes,3,opt,name=typed_offset,json=typedOffset" json:"typed_offset,omitempty"` + OffsetType DatumInfo `protobuf:"bytes,4,opt,name=offset_type,json=offsetType" json:"offset_type"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *WindowerSpec_Frame_Bound) Reset() { *m = WindowerSpec_Frame_Bound{} } func (m *WindowerSpec_Frame_Bound) String() string { return proto.CompactTextString(m) } func (*WindowerSpec_Frame_Bound) ProtoMessage() {} func (*WindowerSpec_Frame_Bound) Descriptor() ([]byte, []int) { - return fileDescriptorProcessors, []int{29, 1, 0} + return fileDescriptor_processors_bf707fc3f8e6bbc6, []int{29, 1, 0} +} +func (m *WindowerSpec_Frame_Bound) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *WindowerSpec_Frame_Bound) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *WindowerSpec_Frame_Bound) XXX_Merge(src proto.Message) { + xxx_messageInfo_WindowerSpec_Frame_Bound.Merge(dst, src) +} +func (m *WindowerSpec_Frame_Bound) XXX_Size() int { + return m.Size() +} +func (m *WindowerSpec_Frame_Bound) XXX_DiscardUnknown() { + xxx_messageInfo_WindowerSpec_Frame_Bound.DiscardUnknown(m) } +var xxx_messageInfo_WindowerSpec_Frame_Bound proto.InternalMessageInfo + // Bounds specifies boundaries of the window frame. type WindowerSpec_Frame_Bounds struct { // Start bound must always be present whereas end bound might be omitted. - Start WindowerSpec_Frame_Bound `protobuf:"bytes,1,opt,name=start" json:"start"` - End *WindowerSpec_Frame_Bound `protobuf:"bytes,2,opt,name=end" json:"end,omitempty"` + Start WindowerSpec_Frame_Bound `protobuf:"bytes,1,opt,name=start" json:"start"` + End *WindowerSpec_Frame_Bound `protobuf:"bytes,2,opt,name=end" json:"end,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *WindowerSpec_Frame_Bounds) Reset() { *m = WindowerSpec_Frame_Bounds{} } func (m *WindowerSpec_Frame_Bounds) String() string { return proto.CompactTextString(m) } func (*WindowerSpec_Frame_Bounds) ProtoMessage() {} func (*WindowerSpec_Frame_Bounds) Descriptor() ([]byte, []int) { - return fileDescriptorProcessors, []int{29, 1, 1} + return fileDescriptor_processors_bf707fc3f8e6bbc6, []int{29, 1, 1} +} +func (m *WindowerSpec_Frame_Bounds) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *WindowerSpec_Frame_Bounds) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil } +func (dst *WindowerSpec_Frame_Bounds) XXX_Merge(src proto.Message) { + xxx_messageInfo_WindowerSpec_Frame_Bounds.Merge(dst, src) +} +func (m *WindowerSpec_Frame_Bounds) XXX_Size() int { + return m.Size() +} +func (m *WindowerSpec_Frame_Bounds) XXX_DiscardUnknown() { + xxx_messageInfo_WindowerSpec_Frame_Bounds.DiscardUnknown(m) +} + +var xxx_messageInfo_WindowerSpec_Frame_Bounds proto.InternalMessageInfo // WindowFn is the specification of a single window function. type WindowerSpec_WindowFn struct { @@ -1385,41 +2341,115 @@ type WindowerSpec_WindowFn struct { Frame *WindowerSpec_Frame `protobuf:"bytes,5,opt,name=frame" json:"frame,omitempty"` // Optional index of a column over which filtering of rows will be done. // Special value -1 indicates that filter is not present. - FilterColIdx int32 `protobuf:"varint,6,opt,name=filterColIdx" json:"filterColIdx"` + FilterColIdx int32 `protobuf:"varint,6,opt,name=filterColIdx" json:"filterColIdx"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *WindowerSpec_WindowFn) Reset() { *m = WindowerSpec_WindowFn{} } func (m *WindowerSpec_WindowFn) String() string { return proto.CompactTextString(m) } func (*WindowerSpec_WindowFn) ProtoMessage() {} func (*WindowerSpec_WindowFn) Descriptor() ([]byte, []int) { - return fileDescriptorProcessors, []int{29, 2} + return fileDescriptor_processors_bf707fc3f8e6bbc6, []int{29, 2} +} +func (m *WindowerSpec_WindowFn) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *WindowerSpec_WindowFn) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *WindowerSpec_WindowFn) XXX_Merge(src proto.Message) { + xxx_messageInfo_WindowerSpec_WindowFn.Merge(dst, src) +} +func (m *WindowerSpec_WindowFn) XXX_Size() int { + return m.Size() } +func (m *WindowerSpec_WindowFn) XXX_DiscardUnknown() { + xxx_messageInfo_WindowerSpec_WindowFn.DiscardUnknown(m) +} + +var xxx_messageInfo_WindowerSpec_WindowFn proto.InternalMessageInfo // ChangeAggregatorSpec is the specification for a processor that watches for // changes in a set of spans. Each span may cross multiple ranges. type ChangeAggregatorSpec struct { Watches []ChangeAggregatorSpec_Watch `protobuf:"bytes,1,rep,name=watches" json:"watches"` // Feed is the specification for this changefeed. - Feed cockroach_sql_jobs_jobspb.ChangefeedDetails `protobuf:"bytes,2,opt,name=feed" json:"feed"` + Feed jobspb.ChangefeedDetails `protobuf:"bytes,2,opt,name=feed" json:"feed"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ChangeAggregatorSpec) Reset() { *m = ChangeAggregatorSpec{} } +func (m *ChangeAggregatorSpec) String() string { return proto.CompactTextString(m) } +func (*ChangeAggregatorSpec) ProtoMessage() {} +func (*ChangeAggregatorSpec) Descriptor() ([]byte, []int) { + return fileDescriptor_processors_bf707fc3f8e6bbc6, []int{30} +} +func (m *ChangeAggregatorSpec) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ChangeAggregatorSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ChangeAggregatorSpec) XXX_Merge(src proto.Message) { + xxx_messageInfo_ChangeAggregatorSpec.Merge(dst, src) +} +func (m *ChangeAggregatorSpec) XXX_Size() int { + return m.Size() +} +func (m *ChangeAggregatorSpec) XXX_DiscardUnknown() { + xxx_messageInfo_ChangeAggregatorSpec.DiscardUnknown(m) } -func (m *ChangeAggregatorSpec) Reset() { *m = ChangeAggregatorSpec{} } -func (m *ChangeAggregatorSpec) String() string { return proto.CompactTextString(m) } -func (*ChangeAggregatorSpec) ProtoMessage() {} -func (*ChangeAggregatorSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{30} } +var xxx_messageInfo_ChangeAggregatorSpec proto.InternalMessageInfo type ChangeAggregatorSpec_Watch struct { - InitialResolved cockroach_util_hlc.Timestamp `protobuf:"bytes,1,opt,name=initial_resolved,json=initialResolved" json:"initial_resolved"` - Span cockroach_roachpb1.Span `protobuf:"bytes,2,opt,name=span" json:"span"` + InitialResolved hlc.Timestamp `protobuf:"bytes,1,opt,name=initial_resolved,json=initialResolved" json:"initial_resolved"` + Span roachpb.Span `protobuf:"bytes,2,opt,name=span" json:"span"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *ChangeAggregatorSpec_Watch) Reset() { *m = ChangeAggregatorSpec_Watch{} } func (m *ChangeAggregatorSpec_Watch) String() string { return proto.CompactTextString(m) } func (*ChangeAggregatorSpec_Watch) ProtoMessage() {} func (*ChangeAggregatorSpec_Watch) Descriptor() ([]byte, []int) { - return fileDescriptorProcessors, []int{30, 0} + return fileDescriptor_processors_bf707fc3f8e6bbc6, []int{30, 0} +} +func (m *ChangeAggregatorSpec_Watch) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ChangeAggregatorSpec_Watch) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ChangeAggregatorSpec_Watch) XXX_Merge(src proto.Message) { + xxx_messageInfo_ChangeAggregatorSpec_Watch.Merge(dst, src) +} +func (m *ChangeAggregatorSpec_Watch) XXX_Size() int { + return m.Size() +} +func (m *ChangeAggregatorSpec_Watch) XXX_DiscardUnknown() { + xxx_messageInfo_ChangeAggregatorSpec_Watch.DiscardUnknown(m) } +var xxx_messageInfo_ChangeAggregatorSpec_Watch proto.InternalMessageInfo + // ChangeFrontierSpec is the specification for a processor that receives // span-level resolved timestamps, track them, and emits the changefeed-level // resolved timestamp whenever it changes. @@ -1427,17 +2457,43 @@ type ChangeFrontierSpec struct { // TrackedSpans is the entire span set being watched. Once all these spans // have been resolved at a certain timestamp, then it's safe to resolve the // changefeed at that timestamp. - TrackedSpans []cockroach_roachpb1.Span `protobuf:"bytes,1,rep,name=tracked_spans,json=trackedSpans" json:"tracked_spans"` + TrackedSpans []roachpb.Span `protobuf:"bytes,1,rep,name=tracked_spans,json=trackedSpans" json:"tracked_spans"` // Feed is the specification for this changefeed. - Feed cockroach_sql_jobs_jobspb.ChangefeedDetails `protobuf:"bytes,2,opt,name=feed" json:"feed"` + Feed jobspb.ChangefeedDetails `protobuf:"bytes,2,opt,name=feed" json:"feed"` // JobID is the id of this changefeed in the system jobs. - JobID int64 `protobuf:"varint,3,opt,name=job_id,json=jobId" json:"job_id"` + JobID int64 `protobuf:"varint,3,opt,name=job_id,json=jobId" json:"job_id"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ChangeFrontierSpec) Reset() { *m = ChangeFrontierSpec{} } +func (m *ChangeFrontierSpec) String() string { return proto.CompactTextString(m) } +func (*ChangeFrontierSpec) ProtoMessage() {} +func (*ChangeFrontierSpec) Descriptor() ([]byte, []int) { + return fileDescriptor_processors_bf707fc3f8e6bbc6, []int{31} +} +func (m *ChangeFrontierSpec) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ChangeFrontierSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ChangeFrontierSpec) XXX_Merge(src proto.Message) { + xxx_messageInfo_ChangeFrontierSpec.Merge(dst, src) +} +func (m *ChangeFrontierSpec) XXX_Size() int { + return m.Size() +} +func (m *ChangeFrontierSpec) XXX_DiscardUnknown() { + xxx_messageInfo_ChangeFrontierSpec.DiscardUnknown(m) } -func (m *ChangeFrontierSpec) Reset() { *m = ChangeFrontierSpec{} } -func (m *ChangeFrontierSpec) String() string { return proto.CompactTextString(m) } -func (*ChangeFrontierSpec) ProtoMessage() {} -func (*ChangeFrontierSpec) Descriptor() ([]byte, []int) { return fileDescriptorProcessors, []int{31} } +var xxx_messageInfo_ChangeFrontierSpec proto.InternalMessageInfo func init() { proto.RegisterType((*ProcessorSpec)(nil), "cockroach.sql.distsqlpb.ProcessorSpec") @@ -1463,6 +2519,8 @@ func init() { proto.RegisterType((*FlowSpec)(nil), "cockroach.sql.distsqlpb.FlowSpec") proto.RegisterType((*JobProgress)(nil), "cockroach.sql.distsqlpb.JobProgress") proto.RegisterType((*ReadImportDataSpec)(nil), "cockroach.sql.distsqlpb.ReadImportDataSpec") + proto.RegisterMapType((map[string]*sqlbase.TableDescriptor)(nil), "cockroach.sql.distsqlpb.ReadImportDataSpec.TablesEntry") + proto.RegisterMapType((map[int32]string)(nil), "cockroach.sql.distsqlpb.ReadImportDataSpec.UriEntry") proto.RegisterType((*SSTWriterSpec)(nil), "cockroach.sql.distsqlpb.SSTWriterSpec") proto.RegisterType((*SSTWriterSpec_SpanName)(nil), "cockroach.sql.distsqlpb.SSTWriterSpec.SpanName") proto.RegisterType((*CSVWriterSpec)(nil), "cockroach.sql.distsqlpb.CSVWriterSpec") @@ -2777,7 +3835,7 @@ func (m *JobProgress) MarshalTo(dAtA []byte) (int, error) { i = encodeVarintProcessors(dAtA, i, uint64(m.JobID)) dAtA[i] = 0x15 i++ - binary.LittleEndian.PutUint32(dAtA[i:], uint32(math.Float32bits(float32(m.Contribution)))) + encoding_binary.LittleEndian.PutUint32(dAtA[i:], uint32(math.Float32bits(float32(m.Contribution)))) i += 4 dAtA[i] = 0x18 i++ @@ -2816,7 +3874,7 @@ func (m *ReadImportDataSpec) MarshalTo(dAtA []byte) (int, error) { for k := range m.Uri { keysForUri = append(keysForUri, int32(k)) } - sortkeys.Int32s(keysForUri) + github_com_gogo_protobuf_sortkeys.Int32s(keysForUri) for _, k := range keysForUri { dAtA[i] = 0x3a i++ @@ -2845,7 +3903,7 @@ func (m *ReadImportDataSpec) MarshalTo(dAtA []byte) (int, error) { for k := range m.Tables { keysForTables = append(keysForTables, string(k)) } - sortkeys.Strings(keysForTables) + github_com_gogo_protobuf_sortkeys.Strings(keysForTables) for _, k := range keysForTables { dAtA[i] = 0x4a i++ @@ -3615,6 +4673,9 @@ func encodeVarintProcessors(dAtA []byte, offset int, v uint64) int { return offset + 1 } func (m *ProcessorSpec) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.Input) > 0 { @@ -3639,6 +4700,9 @@ func (m *ProcessorSpec) Size() (n int) { } func (m *PostProcessSpec) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.Filter.Size() @@ -3663,6 +4727,9 @@ func (m *PostProcessSpec) Size() (n int) { } func (m *ProcessorCoreUnion) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Noop != nil { @@ -3765,12 +4832,18 @@ func (m *ProcessorCoreUnion) Size() (n int) { } func (m *NoopCoreSpec) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l return n } func (m *MetadataTestSenderSpec) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.ID) @@ -3779,6 +4852,9 @@ func (m *MetadataTestSenderSpec) Size() (n int) { } func (m *MetadataTestReceiverSpec) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.SenderIDs) > 0 { @@ -3791,6 +4867,9 @@ func (m *MetadataTestReceiverSpec) Size() (n int) { } func (m *ValuesCoreSpec) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.Columns) > 0 { @@ -3810,6 +4889,9 @@ func (m *ValuesCoreSpec) Size() (n int) { } func (m *TableReaderSpan) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.Span.Size() @@ -3818,6 +4900,9 @@ func (m *TableReaderSpan) Size() (n int) { } func (m *TableReaderSpec) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.Table.Size() @@ -3838,6 +4923,9 @@ func (m *TableReaderSpec) Size() (n int) { } func (m *JoinReaderSpec) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.Table.Size() @@ -3860,6 +4948,9 @@ func (m *JoinReaderSpec) Size() (n int) { } func (m *SorterSpec) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.OutputOrdering.Size() @@ -3869,6 +4960,9 @@ func (m *SorterSpec) Size() (n int) { } func (m *DistinctSpec) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.OrderedColumns) > 0 { @@ -3885,6 +4979,9 @@ func (m *DistinctSpec) Size() (n int) { } func (m *ZigzagJoinerSpec) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.Tables) > 0 { @@ -3919,6 +5016,9 @@ func (m *ZigzagJoinerSpec) Size() (n int) { } func (m *LocalPlanNodeSpec) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.RowSourceIdx != nil { @@ -3935,6 +5035,9 @@ func (m *LocalPlanNodeSpec) Size() (n int) { } func (m *Columns) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.Columns) > 0 { @@ -3948,6 +5051,9 @@ func (m *Columns) Size() (n int) { } func (m *MergeJoinerSpec) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.LeftOrdering.Size() @@ -3962,6 +5068,9 @@ func (m *MergeJoinerSpec) Size() (n int) { } func (m *HashJoinerSpec) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.LeftEqColumns) > 0 { @@ -3988,6 +5097,9 @@ func (m *HashJoinerSpec) Size() (n int) { } func (m *AggregatorSpec) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.GroupCols) > 0 { @@ -4015,6 +5127,9 @@ func (m *AggregatorSpec) Size() (n int) { } func (m *AggregatorSpec_Aggregation) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovProcessors(uint64(m.Func)) @@ -4037,6 +5152,9 @@ func (m *AggregatorSpec_Aggregation) Size() (n int) { } func (m *BackfillerSpec) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovProcessors(uint64(m.Type)) @@ -4062,6 +5180,9 @@ func (m *BackfillerSpec) Size() (n int) { } func (m *FlowSpec) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.FlowID.Size() @@ -4077,6 +5198,9 @@ func (m *FlowSpec) Size() (n int) { } func (m *JobProgress) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovProcessors(uint64(m.JobID)) @@ -4086,6 +5210,9 @@ func (m *JobProgress) Size() (n int) { } func (m *ReadImportDataSpec) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovProcessors(uint64(m.SampleSize)) @@ -4119,6 +5246,9 @@ func (m *ReadImportDataSpec) Size() (n int) { } func (m *SSTWriterSpec) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Destination) @@ -4136,6 +5266,9 @@ func (m *SSTWriterSpec) Size() (n int) { } func (m *SSTWriterSpec_SpanName) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Name) @@ -4148,6 +5281,9 @@ func (m *SSTWriterSpec_SpanName) Size() (n int) { } func (m *CSVWriterSpec) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Destination) @@ -4161,6 +5297,9 @@ func (m *CSVWriterSpec) Size() (n int) { } func (m *SketchSpec) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovProcessors(uint64(m.SketchType)) @@ -4177,6 +5316,9 @@ func (m *SketchSpec) Size() (n int) { } func (m *SamplerSpec) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.Sketches) > 0 { @@ -4190,6 +5332,9 @@ func (m *SamplerSpec) Size() (n int) { } func (m *SampleAggregatorSpec) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.Sketches) > 0 { @@ -4209,6 +5354,9 @@ func (m *SampleAggregatorSpec) Size() (n int) { } func (m *InterleavedReaderJoinerSpec) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.Tables) > 0 { @@ -4226,6 +5374,9 @@ func (m *InterleavedReaderJoinerSpec) Size() (n int) { } func (m *InterleavedReaderJoinerSpec_Table) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.Desc.Size() @@ -4245,6 +5396,9 @@ func (m *InterleavedReaderJoinerSpec_Table) Size() (n int) { } func (m *ProjectSetSpec) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.Exprs) > 0 { @@ -4268,6 +5422,9 @@ func (m *ProjectSetSpec) Size() (n int) { } func (m *WindowerSpec) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.PartitionBy) > 0 { @@ -4285,6 +5442,9 @@ func (m *WindowerSpec) Size() (n int) { } func (m *WindowerSpec_Func) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.AggregateFunc != nil { @@ -4297,6 +5457,9 @@ func (m *WindowerSpec_Func) Size() (n int) { } func (m *WindowerSpec_Frame) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovProcessors(uint64(m.Mode)) @@ -4306,6 +5469,9 @@ func (m *WindowerSpec_Frame) Size() (n int) { } func (m *WindowerSpec_Frame_Bound) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovProcessors(uint64(m.BoundType)) @@ -4320,6 +5486,9 @@ func (m *WindowerSpec_Frame_Bound) Size() (n int) { } func (m *WindowerSpec_Frame_Bounds) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.Start.Size() @@ -4332,6 +5501,9 @@ func (m *WindowerSpec_Frame_Bounds) Size() (n int) { } func (m *WindowerSpec_WindowFn) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.Func.Size() @@ -4349,6 +5521,9 @@ func (m *WindowerSpec_WindowFn) Size() (n int) { } func (m *ChangeAggregatorSpec) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.Watches) > 0 { @@ -4363,6 +5538,9 @@ func (m *ChangeAggregatorSpec) Size() (n int) { } func (m *ChangeAggregatorSpec_Watch) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.InitialResolved.Size() @@ -4373,6 +5551,9 @@ func (m *ChangeAggregatorSpec_Watch) Size() (n int) { } func (m *ChangeFrontierSpec) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.TrackedSpans) > 0 { @@ -4882,6 +6063,17 @@ func (m *PostProcessSpec) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + var count int + for _, integer := range dAtA { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.OutputColumns) == 0 { + m.OutputColumns = make([]uint32, 0, elementCount) + } for iNdEx < postIndex { var v uint32 for shift := uint(0); ; shift += 7 { @@ -6598,6 +7790,17 @@ func (m *JoinReaderSpec) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + var count int + for _, integer := range dAtA { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.LookupColumns) == 0 { + m.LookupColumns = make([]uint32, 0, elementCount) + } for iNdEx < postIndex { var v uint32 for shift := uint(0); ; shift += 7 { @@ -6693,7 +7896,7 @@ func (m *JoinReaderSpec) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Type |= (cockroach_sql_sqlbase3.JoinType(b) & 0x7F) << shift + m.Type |= (sqlbase.JoinType(b) & 0x7F) << shift if b < 0x80 { break } @@ -6907,6 +8110,17 @@ func (m *DistinctSpec) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + var count int + for _, integer := range dAtA { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.OrderedColumns) == 0 { + m.OrderedColumns = make([]uint32, 0, elementCount) + } for iNdEx < postIndex { var v uint32 for shift := uint(0); ; shift += 7 { @@ -6969,6 +8183,17 @@ func (m *DistinctSpec) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + var count int + for _, integer := range dAtA { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.DistinctColumns) == 0 { + m.DistinctColumns = make([]uint32, 0, elementCount) + } for iNdEx < postIndex { var v uint32 for shift := uint(0); ; shift += 7 { @@ -7066,7 +8291,7 @@ func (m *ZigzagJoinerSpec) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Tables = append(m.Tables, cockroach_sql_sqlbase1.TableDescriptor{}) + m.Tables = append(m.Tables, sqlbase.TableDescriptor{}) if err := m.Tables[len(m.Tables)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -7143,6 +8368,17 @@ func (m *ZigzagJoinerSpec) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + var count int + for _, integer := range dAtA { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.IndexIds) == 0 { + m.IndexIds = make([]uint32, 0, elementCount) + } for iNdEx < postIndex { var v uint32 for shift := uint(0); ; shift += 7 { @@ -7239,7 +8475,7 @@ func (m *ZigzagJoinerSpec) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Type |= (cockroach_sql_sqlbase3.JoinType(b) & 0x7F) << shift + m.Type |= (sqlbase.JoinType(b) & 0x7F) << shift if b < 0x80 { break } @@ -7455,6 +8691,17 @@ func (m *Columns) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + var count int + for _, integer := range dAtA { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.Columns) == 0 { + m.Columns = make([]uint32, 0, elementCount) + } for iNdEx < postIndex { var v uint32 for shift := uint(0); ; shift += 7 { @@ -7630,7 +8877,7 @@ func (m *MergeJoinerSpec) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Type |= (cockroach_sql_sqlbase3.JoinType(b) & 0x7F) << shift + m.Type |= (sqlbase.JoinType(b) & 0x7F) << shift if b < 0x80 { break } @@ -7746,6 +8993,17 @@ func (m *HashJoinerSpec) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + var count int + for _, integer := range dAtA { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.LeftEqColumns) == 0 { + m.LeftEqColumns = make([]uint32, 0, elementCount) + } for iNdEx < postIndex { var v uint32 for shift := uint(0); ; shift += 7 { @@ -7808,6 +9066,17 @@ func (m *HashJoinerSpec) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + var count int + for _, integer := range dAtA { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.RightEqColumns) == 0 { + m.RightEqColumns = make([]uint32, 0, elementCount) + } for iNdEx < postIndex { var v uint32 for shift := uint(0); ; shift += 7 { @@ -7873,7 +9142,7 @@ func (m *HashJoinerSpec) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Type |= (cockroach_sql_sqlbase3.JoinType(b) & 0x7F) << shift + m.Type |= (sqlbase.JoinType(b) & 0x7F) << shift if b < 0x80 { break } @@ -8029,6 +9298,17 @@ func (m *AggregatorSpec) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + var count int + for _, integer := range dAtA { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.GroupCols) == 0 { + m.GroupCols = make([]uint32, 0, elementCount) + } for iNdEx < postIndex { var v uint32 for shift := uint(0); ; shift += 7 { @@ -8122,6 +9402,17 @@ func (m *AggregatorSpec) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + var count int + for _, integer := range dAtA { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.OrderedGroupCols) == 0 { + m.OrderedGroupCols = make([]uint32, 0, elementCount) + } for iNdEx < postIndex { var v uint32 for shift := uint(0); ; shift += 7 { @@ -8312,6 +9603,17 @@ func (m *AggregatorSpec_Aggregation) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + var count int + for _, integer := range dAtA { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.ColIdx) == 0 { + m.ColIdx = make([]uint32, 0, elementCount) + } for iNdEx < postIndex { var v uint32 for shift := uint(0); ; shift += 7 { @@ -8558,7 +9860,7 @@ func (m *BackfillerSpec) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.OtherTables = append(m.OtherTables, cockroach_sql_sqlbase1.TableDescriptor{}) + m.OtherTables = append(m.OtherTables, sqlbase.TableDescriptor{}) if err := m.OtherTables[len(m.OtherTables)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -8800,7 +10102,7 @@ func (m *JobProgress) Unmarshal(dAtA []byte) error { if (iNdEx + 4) > l { return io.ErrUnexpectedEOF } - v = uint32(binary.LittleEndian.Uint32(dAtA[iNdEx:])) + v = uint32(encoding_binary.LittleEndian.Uint32(dAtA[iNdEx:])) iNdEx += 4 m.Contribution = float32(math.Float32frombits(v)) case 3: @@ -9085,10 +10387,10 @@ func (m *ReadImportDataSpec) Unmarshal(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Tables == nil { - m.Tables = make(map[string]*cockroach_sql_sqlbase1.TableDescriptor) + m.Tables = make(map[string]*sqlbase.TableDescriptor) } var mapkey string - var mapvalue *cockroach_sql_sqlbase1.TableDescriptor + var mapvalue *sqlbase.TableDescriptor for iNdEx < postIndex { entryPreIndex := iNdEx var wire uint64 @@ -9159,7 +10461,7 @@ func (m *ReadImportDataSpec) Unmarshal(dAtA []byte) error { if postmsgIndex > l { return io.ErrUnexpectedEOF } - mapvalue = &cockroach_sql_sqlbase1.TableDescriptor{} + mapvalue = &sqlbase.TableDescriptor{} if err := mapvalue.Unmarshal(dAtA[iNdEx:postmsgIndex]); err != nil { return err } @@ -9737,6 +11039,17 @@ func (m *SketchSpec) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + var count int + for _, integer := range dAtA { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.Columns) == 0 { + m.Columns = make([]uint32, 0, elementCount) + } for iNdEx < postIndex { var v uint32 for shift := uint(0); ; shift += 7 { @@ -10067,6 +11380,17 @@ func (m *SampleAggregatorSpec) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + var count int + for _, integer := range dAtA { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.SampledColumnIDs) == 0 { + m.SampledColumnIDs = make([]github_com_cockroachdb_cockroach_pkg_sql_sqlbase.ColumnID, 0, elementCount) + } for iNdEx < postIndex { var v github_com_cockroachdb_cockroach_pkg_sql_sqlbase.ColumnID for shift := uint(0); ; shift += 7 { @@ -10271,7 +11595,7 @@ func (m *InterleavedReaderJoinerSpec) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Type |= (cockroach_sql_sqlbase3.JoinType(b) & 0x7F) << shift + m.Type |= (sqlbase.JoinType(b) & 0x7F) << shift if b < 0x80 { break } @@ -10573,7 +11897,7 @@ func (m *ProjectSetSpec) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.GeneratedColumns = append(m.GeneratedColumns, cockroach_sql_sqlbase1.ColumnType{}) + m.GeneratedColumns = append(m.GeneratedColumns, sqlbase.ColumnType{}) if err := m.GeneratedColumns[len(m.GeneratedColumns)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -10619,6 +11943,17 @@ func (m *ProjectSetSpec) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + var count int + for _, integer := range dAtA { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.NumColsPerGen) == 0 { + m.NumColsPerGen = make([]uint32, 0, elementCount) + } for iNdEx < postIndex { var v uint32 for shift := uint(0); ; shift += 7 { @@ -10731,6 +12066,17 @@ func (m *WindowerSpec) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + var count int + for _, integer := range dAtA { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.PartitionBy) == 0 { + m.PartitionBy = make([]uint32, 0, elementCount) + } for iNdEx < postIndex { var v uint32 for shift := uint(0); ; shift += 7 { @@ -11731,7 +13077,7 @@ func (m *ChangeFrontierSpec) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.TrackedSpans = append(m.TrackedSpans, cockroach_roachpb1.Span{}) + m.TrackedSpans = append(m.TrackedSpans, roachpb.Span{}) if err := m.TrackedSpans[len(m.TrackedSpans)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -11911,9 +13257,11 @@ var ( ErrIntOverflowProcessors = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("sql/distsqlpb/processors.proto", fileDescriptorProcessors) } +func init() { + proto.RegisterFile("sql/distsqlpb/processors.proto", fileDescriptor_processors_bf707fc3f8e6bbc6) +} -var fileDescriptorProcessors = []byte{ +var fileDescriptor_processors_bf707fc3f8e6bbc6 = []byte{ // 4047 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x5a, 0x3d, 0x70, 0x1b, 0x59, 0x72, 0x16, 0x80, 0xc1, 0x5f, 0xe3, 0x87, 0xb3, 0x4f, 0xd4, 0x0a, 0xcb, 0x5d, 0x8b, 0xd2, 0xec, diff --git a/pkg/sql/distsqlrun/cluster_test.go b/pkg/sql/distsqlrun/cluster_test.go index 5b0a83c6cbe0..96c34ff65cac 100644 --- a/pkg/sql/distsqlrun/cluster_test.go +++ b/pkg/sql/distsqlrun/cluster_test.go @@ -36,7 +36,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/randutil" "github.com/cockroachdb/cockroach/pkg/util/uuid" - "github.com/opentracing/opentracing-go" + opentracing "github.com/opentracing/opentracing-go" ) func TestClusterFlow(t *testing.T) { diff --git a/pkg/sql/distsqlrun/stats.pb.go b/pkg/sql/distsqlrun/stats.pb.go index 3ad44695872e..37791ecec6ce 100644 --- a/pkg/sql/distsqlrun/stats.pb.go +++ b/pkg/sql/distsqlrun/stats.pb.go @@ -1,25 +1,6 @@ // Code generated by protoc-gen-gogo. DO NOT EDIT. // source: sql/distsqlrun/stats.proto -/* - Package distsqlrun is a generated protocol buffer package. - - It is generated from these files: - sql/distsqlrun/stats.proto - - It has these top-level messages: - InputStats - TableReaderStats - HashJoinerStats - AggregatorStats - DistinctStats - MergeJoinerStats - SorterStats - JoinReaderStats - OutboxStats - RouterOutputStats - WindowerStats -*/ package distsqlrun import proto "github.com/gogo/protobuf/proto" @@ -28,7 +9,7 @@ import math "math" import time "time" -import types "github.com/gogo/protobuf/types" +import github_com_gogo_protobuf_types "github.com/gogo/protobuf/types" import io "io" @@ -49,128 +30,414 @@ type InputStats struct { // num_rows is the number of rows received from the input. NumRows int64 `protobuf:"varint,1,opt,name=num_rows,json=numRows,proto3" json:"num_rows,omitempty"` // Duration in nanoseconds of the cumulative time spent stalled. - StallTime time.Duration `protobuf:"bytes,8,opt,name=stall_time,json=stallTime,stdduration" json:"stall_time"` + StallTime time.Duration `protobuf:"bytes,8,opt,name=stall_time,json=stallTime,proto3,stdduration" json:"stall_time"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *InputStats) Reset() { *m = InputStats{} } -func (m *InputStats) String() string { return proto.CompactTextString(m) } -func (*InputStats) ProtoMessage() {} -func (*InputStats) Descriptor() ([]byte, []int) { return fileDescriptorStats, []int{0} } +func (m *InputStats) Reset() { *m = InputStats{} } +func (m *InputStats) String() string { return proto.CompactTextString(m) } +func (*InputStats) ProtoMessage() {} +func (*InputStats) Descriptor() ([]byte, []int) { + return fileDescriptor_stats_7c106cd515ff5229, []int{0} +} +func (m *InputStats) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *InputStats) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *InputStats) XXX_Merge(src proto.Message) { + xxx_messageInfo_InputStats.Merge(dst, src) +} +func (m *InputStats) XXX_Size() int { + return m.Size() +} +func (m *InputStats) XXX_DiscardUnknown() { + xxx_messageInfo_InputStats.DiscardUnknown(m) +} + +var xxx_messageInfo_InputStats proto.InternalMessageInfo // TableReaderStats are the stats collected during a tableReader run. type TableReaderStats struct { - InputStats InputStats `protobuf:"bytes,1,opt,name=input_stats,json=inputStats" json:"input_stats"` + InputStats InputStats `protobuf:"bytes,1,opt,name=input_stats,json=inputStats,proto3" json:"input_stats"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *TableReaderStats) Reset() { *m = TableReaderStats{} } -func (m *TableReaderStats) String() string { return proto.CompactTextString(m) } -func (*TableReaderStats) ProtoMessage() {} -func (*TableReaderStats) Descriptor() ([]byte, []int) { return fileDescriptorStats, []int{1} } +func (m *TableReaderStats) Reset() { *m = TableReaderStats{} } +func (m *TableReaderStats) String() string { return proto.CompactTextString(m) } +func (*TableReaderStats) ProtoMessage() {} +func (*TableReaderStats) Descriptor() ([]byte, []int) { + return fileDescriptor_stats_7c106cd515ff5229, []int{1} +} +func (m *TableReaderStats) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *TableReaderStats) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *TableReaderStats) XXX_Merge(src proto.Message) { + xxx_messageInfo_TableReaderStats.Merge(dst, src) +} +func (m *TableReaderStats) XXX_Size() int { + return m.Size() +} +func (m *TableReaderStats) XXX_DiscardUnknown() { + xxx_messageInfo_TableReaderStats.DiscardUnknown(m) +} + +var xxx_messageInfo_TableReaderStats proto.InternalMessageInfo // HashJoinerStats are the stats collected during a hashJoiner run. type HashJoinerStats struct { - LeftInputStats InputStats `protobuf:"bytes,1,opt,name=left_input_stats,json=leftInputStats" json:"left_input_stats"` - RightInputStats InputStats `protobuf:"bytes,2,opt,name=right_input_stats,json=rightInputStats" json:"right_input_stats"` - StoredSide string `protobuf:"bytes,3,opt,name=stored_side,json=storedSide,proto3" json:"stored_side,omitempty"` - MaxAllocatedMem int64 `protobuf:"varint,4,opt,name=max_allocated_mem,json=maxAllocatedMem,proto3" json:"max_allocated_mem,omitempty"` - MaxAllocatedDisk int64 `protobuf:"varint,5,opt,name=max_allocated_disk,json=maxAllocatedDisk,proto3" json:"max_allocated_disk,omitempty"` + LeftInputStats InputStats `protobuf:"bytes,1,opt,name=left_input_stats,json=leftInputStats,proto3" json:"left_input_stats"` + RightInputStats InputStats `protobuf:"bytes,2,opt,name=right_input_stats,json=rightInputStats,proto3" json:"right_input_stats"` + StoredSide string `protobuf:"bytes,3,opt,name=stored_side,json=storedSide,proto3" json:"stored_side,omitempty"` + MaxAllocatedMem int64 `protobuf:"varint,4,opt,name=max_allocated_mem,json=maxAllocatedMem,proto3" json:"max_allocated_mem,omitempty"` + MaxAllocatedDisk int64 `protobuf:"varint,5,opt,name=max_allocated_disk,json=maxAllocatedDisk,proto3" json:"max_allocated_disk,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *HashJoinerStats) Reset() { *m = HashJoinerStats{} } -func (m *HashJoinerStats) String() string { return proto.CompactTextString(m) } -func (*HashJoinerStats) ProtoMessage() {} -func (*HashJoinerStats) Descriptor() ([]byte, []int) { return fileDescriptorStats, []int{2} } +func (m *HashJoinerStats) Reset() { *m = HashJoinerStats{} } +func (m *HashJoinerStats) String() string { return proto.CompactTextString(m) } +func (*HashJoinerStats) ProtoMessage() {} +func (*HashJoinerStats) Descriptor() ([]byte, []int) { + return fileDescriptor_stats_7c106cd515ff5229, []int{2} +} +func (m *HashJoinerStats) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *HashJoinerStats) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *HashJoinerStats) XXX_Merge(src proto.Message) { + xxx_messageInfo_HashJoinerStats.Merge(dst, src) +} +func (m *HashJoinerStats) XXX_Size() int { + return m.Size() +} +func (m *HashJoinerStats) XXX_DiscardUnknown() { + xxx_messageInfo_HashJoinerStats.DiscardUnknown(m) +} + +var xxx_messageInfo_HashJoinerStats proto.InternalMessageInfo // AggregatorStats are the stats collected during an aggregator run. type AggregatorStats struct { - InputStats InputStats `protobuf:"bytes,1,opt,name=input_stats,json=inputStats" json:"input_stats"` - MaxAllocatedMem int64 `protobuf:"varint,2,opt,name=max_allocated_mem,json=maxAllocatedMem,proto3" json:"max_allocated_mem,omitempty"` + InputStats InputStats `protobuf:"bytes,1,opt,name=input_stats,json=inputStats,proto3" json:"input_stats"` + MaxAllocatedMem int64 `protobuf:"varint,2,opt,name=max_allocated_mem,json=maxAllocatedMem,proto3" json:"max_allocated_mem,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *AggregatorStats) Reset() { *m = AggregatorStats{} } -func (m *AggregatorStats) String() string { return proto.CompactTextString(m) } -func (*AggregatorStats) ProtoMessage() {} -func (*AggregatorStats) Descriptor() ([]byte, []int) { return fileDescriptorStats, []int{3} } +func (m *AggregatorStats) Reset() { *m = AggregatorStats{} } +func (m *AggregatorStats) String() string { return proto.CompactTextString(m) } +func (*AggregatorStats) ProtoMessage() {} +func (*AggregatorStats) Descriptor() ([]byte, []int) { + return fileDescriptor_stats_7c106cd515ff5229, []int{3} +} +func (m *AggregatorStats) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *AggregatorStats) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *AggregatorStats) XXX_Merge(src proto.Message) { + xxx_messageInfo_AggregatorStats.Merge(dst, src) +} +func (m *AggregatorStats) XXX_Size() int { + return m.Size() +} +func (m *AggregatorStats) XXX_DiscardUnknown() { + xxx_messageInfo_AggregatorStats.DiscardUnknown(m) +} + +var xxx_messageInfo_AggregatorStats proto.InternalMessageInfo // DistinctStats are the stats collected during a distinct run. type DistinctStats struct { - InputStats InputStats `protobuf:"bytes,1,opt,name=input_stats,json=inputStats" json:"input_stats"` - MaxAllocatedMem int64 `protobuf:"varint,2,opt,name=max_allocated_mem,json=maxAllocatedMem,proto3" json:"max_allocated_mem,omitempty"` + InputStats InputStats `protobuf:"bytes,1,opt,name=input_stats,json=inputStats,proto3" json:"input_stats"` + MaxAllocatedMem int64 `protobuf:"varint,2,opt,name=max_allocated_mem,json=maxAllocatedMem,proto3" json:"max_allocated_mem,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *DistinctStats) Reset() { *m = DistinctStats{} } +func (m *DistinctStats) String() string { return proto.CompactTextString(m) } +func (*DistinctStats) ProtoMessage() {} +func (*DistinctStats) Descriptor() ([]byte, []int) { + return fileDescriptor_stats_7c106cd515ff5229, []int{4} +} +func (m *DistinctStats) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *DistinctStats) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *DistinctStats) XXX_Merge(src proto.Message) { + xxx_messageInfo_DistinctStats.Merge(dst, src) +} +func (m *DistinctStats) XXX_Size() int { + return m.Size() +} +func (m *DistinctStats) XXX_DiscardUnknown() { + xxx_messageInfo_DistinctStats.DiscardUnknown(m) } -func (m *DistinctStats) Reset() { *m = DistinctStats{} } -func (m *DistinctStats) String() string { return proto.CompactTextString(m) } -func (*DistinctStats) ProtoMessage() {} -func (*DistinctStats) Descriptor() ([]byte, []int) { return fileDescriptorStats, []int{4} } +var xxx_messageInfo_DistinctStats proto.InternalMessageInfo // MergeJoinerStats are the stats collected during a mergeJoiner run. type MergeJoinerStats struct { - LeftInputStats InputStats `protobuf:"bytes,1,opt,name=left_input_stats,json=leftInputStats" json:"left_input_stats"` - RightInputStats InputStats `protobuf:"bytes,2,opt,name=right_input_stats,json=rightInputStats" json:"right_input_stats"` - MaxAllocatedMem int64 `protobuf:"varint,3,opt,name=max_allocated_mem,json=maxAllocatedMem,proto3" json:"max_allocated_mem,omitempty"` + LeftInputStats InputStats `protobuf:"bytes,1,opt,name=left_input_stats,json=leftInputStats,proto3" json:"left_input_stats"` + RightInputStats InputStats `protobuf:"bytes,2,opt,name=right_input_stats,json=rightInputStats,proto3" json:"right_input_stats"` + MaxAllocatedMem int64 `protobuf:"varint,3,opt,name=max_allocated_mem,json=maxAllocatedMem,proto3" json:"max_allocated_mem,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *MergeJoinerStats) Reset() { *m = MergeJoinerStats{} } -func (m *MergeJoinerStats) String() string { return proto.CompactTextString(m) } -func (*MergeJoinerStats) ProtoMessage() {} -func (*MergeJoinerStats) Descriptor() ([]byte, []int) { return fileDescriptorStats, []int{5} } +func (m *MergeJoinerStats) Reset() { *m = MergeJoinerStats{} } +func (m *MergeJoinerStats) String() string { return proto.CompactTextString(m) } +func (*MergeJoinerStats) ProtoMessage() {} +func (*MergeJoinerStats) Descriptor() ([]byte, []int) { + return fileDescriptor_stats_7c106cd515ff5229, []int{5} +} +func (m *MergeJoinerStats) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *MergeJoinerStats) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *MergeJoinerStats) XXX_Merge(src proto.Message) { + xxx_messageInfo_MergeJoinerStats.Merge(dst, src) +} +func (m *MergeJoinerStats) XXX_Size() int { + return m.Size() +} +func (m *MergeJoinerStats) XXX_DiscardUnknown() { + xxx_messageInfo_MergeJoinerStats.DiscardUnknown(m) +} + +var xxx_messageInfo_MergeJoinerStats proto.InternalMessageInfo // SorterStats are the stats collected during a sorter run. type SorterStats struct { - InputStats InputStats `protobuf:"bytes,1,opt,name=input_stats,json=inputStats" json:"input_stats"` - MaxAllocatedMem int64 `protobuf:"varint,2,opt,name=max_allocated_mem,json=maxAllocatedMem,proto3" json:"max_allocated_mem,omitempty"` - MaxAllocatedDisk int64 `protobuf:"varint,3,opt,name=max_allocated_disk,json=maxAllocatedDisk,proto3" json:"max_allocated_disk,omitempty"` + InputStats InputStats `protobuf:"bytes,1,opt,name=input_stats,json=inputStats,proto3" json:"input_stats"` + MaxAllocatedMem int64 `protobuf:"varint,2,opt,name=max_allocated_mem,json=maxAllocatedMem,proto3" json:"max_allocated_mem,omitempty"` + MaxAllocatedDisk int64 `protobuf:"varint,3,opt,name=max_allocated_disk,json=maxAllocatedDisk,proto3" json:"max_allocated_disk,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *SorterStats) Reset() { *m = SorterStats{} } -func (m *SorterStats) String() string { return proto.CompactTextString(m) } -func (*SorterStats) ProtoMessage() {} -func (*SorterStats) Descriptor() ([]byte, []int) { return fileDescriptorStats, []int{6} } +func (m *SorterStats) Reset() { *m = SorterStats{} } +func (m *SorterStats) String() string { return proto.CompactTextString(m) } +func (*SorterStats) ProtoMessage() {} +func (*SorterStats) Descriptor() ([]byte, []int) { + return fileDescriptor_stats_7c106cd515ff5229, []int{6} +} +func (m *SorterStats) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SorterStats) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *SorterStats) XXX_Merge(src proto.Message) { + xxx_messageInfo_SorterStats.Merge(dst, src) +} +func (m *SorterStats) XXX_Size() int { + return m.Size() +} +func (m *SorterStats) XXX_DiscardUnknown() { + xxx_messageInfo_SorterStats.DiscardUnknown(m) +} + +var xxx_messageInfo_SorterStats proto.InternalMessageInfo // JoinReaderStats are the stats collected during a joinReader run. type JoinReaderStats struct { - InputStats InputStats `protobuf:"bytes,1,opt,name=input_stats,json=inputStats" json:"input_stats"` - IndexLookupStats InputStats `protobuf:"bytes,2,opt,name=index_lookup_stats,json=indexLookupStats" json:"index_lookup_stats"` - PrimaryIndexLookupStats *InputStats `protobuf:"bytes,3,opt,name=primary_index_lookup_stats,json=primaryIndexLookupStats" json:"primary_index_lookup_stats,omitempty"` + InputStats InputStats `protobuf:"bytes,1,opt,name=input_stats,json=inputStats,proto3" json:"input_stats"` + IndexLookupStats InputStats `protobuf:"bytes,2,opt,name=index_lookup_stats,json=indexLookupStats,proto3" json:"index_lookup_stats"` + PrimaryIndexLookupStats *InputStats `protobuf:"bytes,3,opt,name=primary_index_lookup_stats,json=primaryIndexLookupStats,proto3" json:"primary_index_lookup_stats,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *JoinReaderStats) Reset() { *m = JoinReaderStats{} } +func (m *JoinReaderStats) String() string { return proto.CompactTextString(m) } +func (*JoinReaderStats) ProtoMessage() {} +func (*JoinReaderStats) Descriptor() ([]byte, []int) { + return fileDescriptor_stats_7c106cd515ff5229, []int{7} +} +func (m *JoinReaderStats) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *JoinReaderStats) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *JoinReaderStats) XXX_Merge(src proto.Message) { + xxx_messageInfo_JoinReaderStats.Merge(dst, src) +} +func (m *JoinReaderStats) XXX_Size() int { + return m.Size() +} +func (m *JoinReaderStats) XXX_DiscardUnknown() { + xxx_messageInfo_JoinReaderStats.DiscardUnknown(m) } -func (m *JoinReaderStats) Reset() { *m = JoinReaderStats{} } -func (m *JoinReaderStats) String() string { return proto.CompactTextString(m) } -func (*JoinReaderStats) ProtoMessage() {} -func (*JoinReaderStats) Descriptor() ([]byte, []int) { return fileDescriptorStats, []int{7} } +var xxx_messageInfo_JoinReaderStats proto.InternalMessageInfo // OutboxStats are the stats collected by an outbox. type OutboxStats struct { - BytesSent int64 `protobuf:"varint,1,opt,name=bytes_sent,json=bytesSent,proto3" json:"bytes_sent,omitempty"` + BytesSent int64 `protobuf:"varint,1,opt,name=bytes_sent,json=bytesSent,proto3" json:"bytes_sent,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *OutboxStats) Reset() { *m = OutboxStats{} } -func (m *OutboxStats) String() string { return proto.CompactTextString(m) } -func (*OutboxStats) ProtoMessage() {} -func (*OutboxStats) Descriptor() ([]byte, []int) { return fileDescriptorStats, []int{8} } +func (m *OutboxStats) Reset() { *m = OutboxStats{} } +func (m *OutboxStats) String() string { return proto.CompactTextString(m) } +func (*OutboxStats) ProtoMessage() {} +func (*OutboxStats) Descriptor() ([]byte, []int) { + return fileDescriptor_stats_7c106cd515ff5229, []int{8} +} +func (m *OutboxStats) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *OutboxStats) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *OutboxStats) XXX_Merge(src proto.Message) { + xxx_messageInfo_OutboxStats.Merge(dst, src) +} +func (m *OutboxStats) XXX_Size() int { + return m.Size() +} +func (m *OutboxStats) XXX_DiscardUnknown() { + xxx_messageInfo_OutboxStats.DiscardUnknown(m) +} + +var xxx_messageInfo_OutboxStats proto.InternalMessageInfo // RouterOutputStats are the stats collected by a single router output stream. type RouterOutputStats struct { - NumRows int64 `protobuf:"varint,1,opt,name=num_rows,json=numRows,proto3" json:"num_rows,omitempty"` - MaxAllocatedMem int64 `protobuf:"varint,2,opt,name=max_allocated_mem,json=maxAllocatedMem,proto3" json:"max_allocated_mem,omitempty"` - MaxAllocatedDisk int64 `protobuf:"varint,3,opt,name=max_allocated_disk,json=maxAllocatedDisk,proto3" json:"max_allocated_disk,omitempty"` + NumRows int64 `protobuf:"varint,1,opt,name=num_rows,json=numRows,proto3" json:"num_rows,omitempty"` + MaxAllocatedMem int64 `protobuf:"varint,2,opt,name=max_allocated_mem,json=maxAllocatedMem,proto3" json:"max_allocated_mem,omitempty"` + MaxAllocatedDisk int64 `protobuf:"varint,3,opt,name=max_allocated_disk,json=maxAllocatedDisk,proto3" json:"max_allocated_disk,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *RouterOutputStats) Reset() { *m = RouterOutputStats{} } +func (m *RouterOutputStats) String() string { return proto.CompactTextString(m) } +func (*RouterOutputStats) ProtoMessage() {} +func (*RouterOutputStats) Descriptor() ([]byte, []int) { + return fileDescriptor_stats_7c106cd515ff5229, []int{9} +} +func (m *RouterOutputStats) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RouterOutputStats) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RouterOutputStats) XXX_Merge(src proto.Message) { + xxx_messageInfo_RouterOutputStats.Merge(dst, src) +} +func (m *RouterOutputStats) XXX_Size() int { + return m.Size() +} +func (m *RouterOutputStats) XXX_DiscardUnknown() { + xxx_messageInfo_RouterOutputStats.DiscardUnknown(m) } -func (m *RouterOutputStats) Reset() { *m = RouterOutputStats{} } -func (m *RouterOutputStats) String() string { return proto.CompactTextString(m) } -func (*RouterOutputStats) ProtoMessage() {} -func (*RouterOutputStats) Descriptor() ([]byte, []int) { return fileDescriptorStats, []int{9} } +var xxx_messageInfo_RouterOutputStats proto.InternalMessageInfo // WindowerStats are the stats collected during a windower run. type WindowerStats struct { - InputStats InputStats `protobuf:"bytes,1,opt,name=input_stats,json=inputStats" json:"input_stats"` - MaxAllocatedMem int64 `protobuf:"varint,2,opt,name=max_allocated_mem,json=maxAllocatedMem,proto3" json:"max_allocated_mem,omitempty"` + InputStats InputStats `protobuf:"bytes,1,opt,name=input_stats,json=inputStats,proto3" json:"input_stats"` + MaxAllocatedMem int64 `protobuf:"varint,2,opt,name=max_allocated_mem,json=maxAllocatedMem,proto3" json:"max_allocated_mem,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *WindowerStats) Reset() { *m = WindowerStats{} } +func (m *WindowerStats) String() string { return proto.CompactTextString(m) } +func (*WindowerStats) ProtoMessage() {} +func (*WindowerStats) Descriptor() ([]byte, []int) { + return fileDescriptor_stats_7c106cd515ff5229, []int{10} +} +func (m *WindowerStats) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *WindowerStats) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *WindowerStats) XXX_Merge(src proto.Message) { + xxx_messageInfo_WindowerStats.Merge(dst, src) +} +func (m *WindowerStats) XXX_Size() int { + return m.Size() +} +func (m *WindowerStats) XXX_DiscardUnknown() { + xxx_messageInfo_WindowerStats.DiscardUnknown(m) } -func (m *WindowerStats) Reset() { *m = WindowerStats{} } -func (m *WindowerStats) String() string { return proto.CompactTextString(m) } -func (*WindowerStats) ProtoMessage() {} -func (*WindowerStats) Descriptor() ([]byte, []int) { return fileDescriptorStats, []int{10} } +var xxx_messageInfo_WindowerStats proto.InternalMessageInfo func init() { proto.RegisterType((*InputStats)(nil), "cockroach.sql.distsqlrun.InputStats") @@ -207,8 +474,8 @@ func (m *InputStats) MarshalTo(dAtA []byte) (int, error) { } dAtA[i] = 0x42 i++ - i = encodeVarintStats(dAtA, i, uint64(types.SizeOfStdDuration(m.StallTime))) - n1, err := types.StdDurationMarshalTo(m.StallTime, dAtA[i:]) + i = encodeVarintStats(dAtA, i, uint64(github_com_gogo_protobuf_types.SizeOfStdDuration(m.StallTime))) + n1, err := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.StallTime, dAtA[i:]) if err != nil { return 0, err } @@ -570,17 +837,23 @@ func encodeVarintStats(dAtA []byte, offset int, v uint64) int { return offset + 1 } func (m *InputStats) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.NumRows != 0 { n += 1 + sovStats(uint64(m.NumRows)) } - l = types.SizeOfStdDuration(m.StallTime) + l = github_com_gogo_protobuf_types.SizeOfStdDuration(m.StallTime) n += 1 + l + sovStats(uint64(l)) return n } func (m *TableReaderStats) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.InputStats.Size() @@ -589,6 +862,9 @@ func (m *TableReaderStats) Size() (n int) { } func (m *HashJoinerStats) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.LeftInputStats.Size() @@ -609,6 +885,9 @@ func (m *HashJoinerStats) Size() (n int) { } func (m *AggregatorStats) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.InputStats.Size() @@ -620,6 +899,9 @@ func (m *AggregatorStats) Size() (n int) { } func (m *DistinctStats) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.InputStats.Size() @@ -631,6 +913,9 @@ func (m *DistinctStats) Size() (n int) { } func (m *MergeJoinerStats) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.LeftInputStats.Size() @@ -644,6 +929,9 @@ func (m *MergeJoinerStats) Size() (n int) { } func (m *SorterStats) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.InputStats.Size() @@ -658,6 +946,9 @@ func (m *SorterStats) Size() (n int) { } func (m *JoinReaderStats) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.InputStats.Size() @@ -672,6 +963,9 @@ func (m *JoinReaderStats) Size() (n int) { } func (m *OutboxStats) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.BytesSent != 0 { @@ -681,6 +975,9 @@ func (m *OutboxStats) Size() (n int) { } func (m *RouterOutputStats) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.NumRows != 0 { @@ -696,6 +993,9 @@ func (m *RouterOutputStats) Size() (n int) { } func (m *WindowerStats) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.InputStats.Size() @@ -793,7 +1093,7 @@ func (m *InputStats) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if err := types.StdDurationUnmarshal(&m.StallTime, dAtA[iNdEx:postIndex]); err != nil { + if err := github_com_gogo_protobuf_types.StdDurationUnmarshal(&m.StallTime, dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -2043,9 +2343,9 @@ var ( ErrIntOverflowStats = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("sql/distsqlrun/stats.proto", fileDescriptorStats) } +func init() { proto.RegisterFile("sql/distsqlrun/stats.proto", fileDescriptor_stats_7c106cd515ff5229) } -var fileDescriptorStats = []byte{ +var fileDescriptor_stats_7c106cd515ff5229 = []byte{ // 581 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0x95, 0xcf, 0x6e, 0xd3, 0x40, 0x10, 0xc6, 0xbb, 0x49, 0x81, 0x76, 0x2c, 0x48, 0x62, 0x21, 0xe1, 0x46, 0xc2, 0x89, 0xac, 0x1e, diff --git a/pkg/sql/pgwire/pgerror/errors.pb.go b/pkg/sql/pgwire/pgerror/errors.pb.go index ca1cdcc8773b..130f7244743f 100644 --- a/pkg/sql/pgwire/pgerror/errors.pb.go +++ b/pkg/sql/pgwire/pgerror/errors.pb.go @@ -1,15 +1,6 @@ // Code generated by protoc-gen-gogo. DO NOT EDIT. // source: sql/pgwire/pgerror/errors.proto -/* - Package pgerror is a generated protocol buffer package. - - It is generated from these files: - sql/pgwire/pgerror/errors.proto - - It has these top-level messages: - Error -*/ package pgerror import proto "github.com/gogo/protobuf/proto" @@ -38,26 +29,78 @@ type Error struct { Message string `protobuf:"bytes,2,opt,name=message,proto3" json:"message,omitempty"` Detail string `protobuf:"bytes,3,opt,name=detail,proto3" json:"detail,omitempty"` Hint string `protobuf:"bytes,4,opt,name=hint,proto3" json:"hint,omitempty"` - Source *Error_Source `protobuf:"bytes,5,opt,name=source" json:"source,omitempty"` + Source *Error_Source `protobuf:"bytes,5,opt,name=source,proto3" json:"source,omitempty"` // "Internal query: the text of a failed internally-generated command." - InternalCommand string `protobuf:"bytes,6,opt,name=internalCommand,proto3" json:"internalCommand,omitempty"` + InternalCommand string `protobuf:"bytes,6,opt,name=internalCommand,proto3" json:"internalCommand,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *Error) Reset() { *m = Error{} } -func (m *Error) String() string { return proto.CompactTextString(m) } -func (*Error) ProtoMessage() {} -func (*Error) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{0} } +func (m *Error) Reset() { *m = Error{} } +func (m *Error) String() string { return proto.CompactTextString(m) } +func (*Error) ProtoMessage() {} +func (*Error) Descriptor() ([]byte, []int) { + return fileDescriptor_errors_080b775addbd2a6b, []int{0} +} +func (m *Error) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Error) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *Error) XXX_Merge(src proto.Message) { + xxx_messageInfo_Error.Merge(dst, src) +} +func (m *Error) XXX_Size() int { + return m.Size() +} +func (m *Error) XXX_DiscardUnknown() { + xxx_messageInfo_Error.DiscardUnknown(m) +} + +var xxx_messageInfo_Error proto.InternalMessageInfo type Error_Source struct { - File string `protobuf:"bytes,1,opt,name=file,proto3" json:"file,omitempty"` - Line int32 `protobuf:"varint,2,opt,name=line,proto3" json:"line,omitempty"` - Function string `protobuf:"bytes,3,opt,name=function,proto3" json:"function,omitempty"` + File string `protobuf:"bytes,1,opt,name=file,proto3" json:"file,omitempty"` + Line int32 `protobuf:"varint,2,opt,name=line,proto3" json:"line,omitempty"` + Function string `protobuf:"bytes,3,opt,name=function,proto3" json:"function,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Error_Source) Reset() { *m = Error_Source{} } +func (m *Error_Source) String() string { return proto.CompactTextString(m) } +func (*Error_Source) ProtoMessage() {} +func (*Error_Source) Descriptor() ([]byte, []int) { + return fileDescriptor_errors_080b775addbd2a6b, []int{0, 0} +} +func (m *Error_Source) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Error_Source) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *Error_Source) XXX_Merge(src proto.Message) { + xxx_messageInfo_Error_Source.Merge(dst, src) +} +func (m *Error_Source) XXX_Size() int { + return m.Size() +} +func (m *Error_Source) XXX_DiscardUnknown() { + xxx_messageInfo_Error_Source.DiscardUnknown(m) } -func (m *Error_Source) Reset() { *m = Error_Source{} } -func (m *Error_Source) String() string { return proto.CompactTextString(m) } -func (*Error_Source) ProtoMessage() {} -func (*Error_Source) Descriptor() ([]byte, []int) { return fileDescriptorErrors, []int{0, 0} } +var xxx_messageInfo_Error_Source proto.InternalMessageInfo func init() { proto.RegisterType((*Error)(nil), "cockroach.pgerror.Error") @@ -166,6 +209,9 @@ func encodeVarintErrors(dAtA []byte, offset int, v uint64) int { return offset + 1 } func (m *Error) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Code) @@ -196,6 +242,9 @@ func (m *Error) Size() (n int) { } func (m *Error_Source) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.File) @@ -685,9 +734,11 @@ var ( ErrIntOverflowErrors = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("sql/pgwire/pgerror/errors.proto", fileDescriptorErrors) } +func init() { + proto.RegisterFile("sql/pgwire/pgerror/errors.proto", fileDescriptor_errors_080b775addbd2a6b) +} -var fileDescriptorErrors = []byte{ +var fileDescriptor_errors_080b775addbd2a6b = []byte{ // 268 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x64, 0x50, 0xb1, 0x52, 0x33, 0x21, 0x18, 0x0c, 0xf9, 0x73, 0xe4, 0x17, 0x0b, 0x47, 0x0a, 0x87, 0x49, 0x41, 0xa2, 0xd5, 0x55, 0xdc, diff --git a/pkg/sql/sqlbase/encoded_datum.pb.go b/pkg/sql/sqlbase/encoded_datum.pb.go index 1e53587ae136..72b4d22d6d83 100644 --- a/pkg/sql/sqlbase/encoded_datum.pb.go +++ b/pkg/sql/sqlbase/encoded_datum.pb.go @@ -1,30 +1,6 @@ // Code generated by protoc-gen-gogo. DO NOT EDIT. // source: sql/sqlbase/encoded_datum.proto -/* -Package sqlbase is a generated protocol buffer package. - -It is generated from these files: - sql/sqlbase/encoded_datum.proto - sql/sqlbase/join_type.proto - sql/sqlbase/privilege.proto - sql/sqlbase/structured.proto - -It has these top-level messages: - UserPrivileges - PrivilegeDescriptor - ColumnType - ForeignKeyReference - ColumnDescriptor - ColumnFamilyDescriptor - InterleaveDescriptor - PartitioningDescriptor - IndexDescriptor - DescriptorMutation - TableDescriptor - DatabaseDescriptor - Descriptor -*/ package sqlbase import proto "github.com/gogo/protobuf/proto" @@ -83,15 +59,19 @@ func (x *DatumEncoding) UnmarshalJSON(data []byte) error { *x = DatumEncoding(value) return nil } -func (DatumEncoding) EnumDescriptor() ([]byte, []int) { return fileDescriptorEncodedDatum, []int{0} } +func (DatumEncoding) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_encoded_datum_d3fede89e16b165a, []int{0} +} func init() { proto.RegisterEnum("cockroach.sql.sqlbase.DatumEncoding", DatumEncoding_name, DatumEncoding_value) } -func init() { proto.RegisterFile("sql/sqlbase/encoded_datum.proto", fileDescriptorEncodedDatum) } +func init() { + proto.RegisterFile("sql/sqlbase/encoded_datum.proto", fileDescriptor_encoded_datum_d3fede89e16b165a) +} -var fileDescriptorEncodedDatum = []byte{ +var fileDescriptor_encoded_datum_d3fede89e16b165a = []byte{ // 172 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x92, 0x2f, 0x2e, 0xcc, 0xd1, 0x2f, 0x2e, 0xcc, 0x49, 0x4a, 0x2c, 0x4e, 0xd5, 0x4f, 0xcd, 0x4b, 0xce, 0x4f, 0x49, 0x4d, 0x89, diff --git a/pkg/sql/sqlbase/join_type.pb.go b/pkg/sql/sqlbase/join_type.pb.go index b6aaf0283c3a..675a182a9367 100644 --- a/pkg/sql/sqlbase/join_type.pb.go +++ b/pkg/sql/sqlbase/join_type.pb.go @@ -12,6 +12,12 @@ var _ = proto.Marshal var _ = fmt.Errorf var _ = math.Inf +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package + // JoinType is the particular type of a join (or join-like) operation. Not all // values are used in all contexts. type JoinType int32 @@ -112,15 +118,19 @@ func (x *JoinType) UnmarshalJSON(data []byte) error { *x = JoinType(value) return nil } -func (JoinType) EnumDescriptor() ([]byte, []int) { return fileDescriptorJoinType, []int{0} } +func (JoinType) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_join_type_88efa33afe97ce73, []int{0} +} func init() { proto.RegisterEnum("cockroach.sql.sqlbase.JoinType", JoinType_name, JoinType_value) } -func init() { proto.RegisterFile("sql/sqlbase/join_type.proto", fileDescriptorJoinType) } +func init() { + proto.RegisterFile("sql/sqlbase/join_type.proto", fileDescriptor_join_type_88efa33afe97ce73) +} -var fileDescriptorJoinType = []byte{ +var fileDescriptor_join_type_88efa33afe97ce73 = []byte{ // 211 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x92, 0x2e, 0x2e, 0xcc, 0xd1, 0x2f, 0x2e, 0xcc, 0x49, 0x4a, 0x2c, 0x4e, 0xd5, 0xcf, 0xca, 0xcf, 0xcc, 0x8b, 0x2f, 0xa9, 0x2c, diff --git a/pkg/sql/sqlbase/privilege.pb.go b/pkg/sql/sqlbase/privilege.pb.go index 236908bb1d74..0a667cc34706 100644 --- a/pkg/sql/sqlbase/privilege.pb.go +++ b/pkg/sql/sqlbase/privilege.pb.go @@ -14,28 +14,86 @@ var _ = proto.Marshal var _ = fmt.Errorf var _ = math.Inf +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package + // UserPrivileges describes the list of privileges available for a given user. type UserPrivileges struct { User string `protobuf:"bytes,1,opt,name=user" json:"user"` // privileges is a bitfield of 1< 0 { @@ -423,9 +487,11 @@ var ( ErrIntOverflowPrivilege = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("sql/sqlbase/privilege.proto", fileDescriptorPrivilege) } +func init() { + proto.RegisterFile("sql/sqlbase/privilege.proto", fileDescriptor_privilege_a8444a679d17e9af) +} -var fileDescriptorPrivilege = []byte{ +var fileDescriptor_privilege_a8444a679d17e9af = []byte{ // 210 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x92, 0x2e, 0x2e, 0xcc, 0xd1, 0x2f, 0x2e, 0xcc, 0x49, 0x4a, 0x2c, 0x4e, 0xd5, 0x2f, 0x28, 0xca, 0x2c, 0xcb, 0xcc, 0x49, 0x4d, diff --git a/pkg/sql/sqlbase/structured.pb.go b/pkg/sql/sqlbase/structured.pb.go index 8b03c04f7932..4ff2e787d3aa 100644 --- a/pkg/sql/sqlbase/structured.pb.go +++ b/pkg/sql/sqlbase/structured.pb.go @@ -6,7 +6,7 @@ package sqlbase import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" -import cockroach_util_hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" +import hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" import github_com_cockroachdb_cockroach_pkg_roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -17,6 +17,12 @@ var _ = proto.Marshal var _ = fmt.Errorf var _ = math.Inf +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package + type ConstraintValidity int32 const ( @@ -49,7 +55,9 @@ func (x *ConstraintValidity) UnmarshalJSON(data []byte) error { *x = ConstraintValidity(value) return nil } -func (ConstraintValidity) EnumDescriptor() ([]byte, []int) { return fileDescriptorStructured, []int{0} } +func (ConstraintValidity) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_structured_712cb25350946e6b, []int{0} +} // These mirror the types supported by sql/coltypes. // @@ -156,7 +164,7 @@ func (x *ColumnType_SemanticType) UnmarshalJSON(data []byte) error { return nil } func (ColumnType_SemanticType) EnumDescriptor() ([]byte, []int) { - return fileDescriptorStructured, []int{0, 0} + return fileDescriptor_structured_712cb25350946e6b, []int{0, 0} } type ColumnType_VisibleType int32 @@ -216,7 +224,7 @@ func (x *ColumnType_VisibleType) UnmarshalJSON(data []byte) error { return nil } func (ColumnType_VisibleType) EnumDescriptor() ([]byte, []int) { - return fileDescriptorStructured, []int{0, 1} + return fileDescriptor_structured_712cb25350946e6b, []int{0, 1} } type ForeignKeyReference_Action int32 @@ -261,7 +269,7 @@ func (x *ForeignKeyReference_Action) UnmarshalJSON(data []byte) error { return nil } func (ForeignKeyReference_Action) EnumDescriptor() ([]byte, []int) { - return fileDescriptorStructured, []int{1, 0} + return fileDescriptor_structured_712cb25350946e6b, []int{1, 0} } // Match is the algorithm used to compare composite keys. @@ -298,7 +306,7 @@ func (x *ForeignKeyReference_Match) UnmarshalJSON(data []byte) error { return nil } func (ForeignKeyReference_Match) EnumDescriptor() ([]byte, []int) { - return fileDescriptorStructured, []int{1, 1} + return fileDescriptor_structured_712cb25350946e6b, []int{1, 1} } // The direction of a column in the index. @@ -335,7 +343,7 @@ func (x *IndexDescriptor_Direction) UnmarshalJSON(data []byte) error { return nil } func (IndexDescriptor_Direction) EnumDescriptor() ([]byte, []int) { - return fileDescriptorStructured, []int{6, 0} + return fileDescriptor_structured_712cb25350946e6b, []int{6, 0} } // The direction of a column in the index. @@ -372,7 +380,7 @@ func (x *IndexDescriptor_Type) UnmarshalJSON(data []byte) error { return nil } func (IndexDescriptor_Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptorStructured, []int{6, 1} + return fileDescriptor_structured_712cb25350946e6b, []int{6, 1} } // A descriptor within a mutation is unavailable for reads, writes @@ -437,7 +445,7 @@ func (x *DescriptorMutation_State) UnmarshalJSON(data []byte) error { return nil } func (DescriptorMutation_State) EnumDescriptor() ([]byte, []int) { - return fileDescriptorStructured, []int{7, 0} + return fileDescriptor_structured_712cb25350946e6b, []int{7, 0} } // Direction of mutation. @@ -480,7 +488,7 @@ func (x *DescriptorMutation_Direction) UnmarshalJSON(data []byte) error { return nil } func (DescriptorMutation_Direction) EnumDescriptor() ([]byte, []int) { - return fileDescriptorStructured, []int{7, 1} + return fileDescriptor_structured_712cb25350946e6b, []int{7, 1} } // State is set if this TableDescriptor is in the process of being added or deleted. @@ -527,7 +535,7 @@ func (x *TableDescriptor_State) UnmarshalJSON(data []byte) error { return nil } func (TableDescriptor_State) EnumDescriptor() ([]byte, []int) { - return fileDescriptorStructured, []int{8, 0} + return fileDescriptor_structured_712cb25350946e6b, []int{8, 0} } // AuditMode indicates which auditing actions to take when this table is used. @@ -564,7 +572,7 @@ func (x *TableDescriptor_AuditMode) UnmarshalJSON(data []byte) error { return nil } func (TableDescriptor_AuditMode) EnumDescriptor() ([]byte, []int) { - return fileDescriptorStructured, []int{8, 1} + return fileDescriptor_structured_712cb25350946e6b, []int{8, 1} } type ColumnType struct { @@ -586,14 +594,40 @@ type ColumnType struct { // Only used if the kind is ARRAY. ArrayContents *ColumnType_SemanticType `protobuf:"varint,7,opt,name=array_contents,json=arrayContents,enum=cockroach.sql.sqlbase.ColumnType_SemanticType" json:"array_contents,omitempty"` // Only used if the kind is TUPLE - TupleContents []ColumnType `protobuf:"bytes,8,rep,name=tuple_contents,json=tupleContents" json:"tuple_contents"` - TupleLabels []string `protobuf:"bytes,9,rep,name=tuple_labels,json=tupleLabels" json:"tuple_labels,omitempty"` + TupleContents []ColumnType `protobuf:"bytes,8,rep,name=tuple_contents,json=tupleContents" json:"tuple_contents"` + TupleLabels []string `protobuf:"bytes,9,rep,name=tuple_labels,json=tupleLabels" json:"tuple_labels,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *ColumnType) Reset() { *m = ColumnType{} } -func (m *ColumnType) String() string { return proto.CompactTextString(m) } -func (*ColumnType) ProtoMessage() {} -func (*ColumnType) Descriptor() ([]byte, []int) { return fileDescriptorStructured, []int{0} } +func (m *ColumnType) Reset() { *m = ColumnType{} } +func (m *ColumnType) String() string { return proto.CompactTextString(m) } +func (*ColumnType) ProtoMessage() {} +func (*ColumnType) Descriptor() ([]byte, []int) { + return fileDescriptor_structured_712cb25350946e6b, []int{0} +} +func (m *ColumnType) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ColumnType) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ColumnType) XXX_Merge(src proto.Message) { + xxx_messageInfo_ColumnType.Merge(dst, src) +} +func (m *ColumnType) XXX_Size() int { + return m.Size() +} +func (m *ColumnType) XXX_DiscardUnknown() { + xxx_messageInfo_ColumnType.DiscardUnknown(m) +} + +var xxx_messageInfo_ColumnType proto.InternalMessageInfo type ForeignKeyReference struct { Table ID `protobuf:"varint,1,opt,name=table,casttype=ID" json:"table"` @@ -607,13 +641,39 @@ type ForeignKeyReference struct { OnUpdate ForeignKeyReference_Action `protobuf:"varint,7,opt,name=on_update,json=onUpdate,enum=cockroach.sql.sqlbase.ForeignKeyReference_Action" json:"on_update"` // This is only important for composite keys. For all prior matches before // the addition of this value, MATCH SIMPLE will be used. - Match ForeignKeyReference_Match `protobuf:"varint,8,opt,name=match,enum=cockroach.sql.sqlbase.ForeignKeyReference_Match" json:"match"` + Match ForeignKeyReference_Match `protobuf:"varint,8,opt,name=match,enum=cockroach.sql.sqlbase.ForeignKeyReference_Match" json:"match"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ForeignKeyReference) Reset() { *m = ForeignKeyReference{} } +func (m *ForeignKeyReference) String() string { return proto.CompactTextString(m) } +func (*ForeignKeyReference) ProtoMessage() {} +func (*ForeignKeyReference) Descriptor() ([]byte, []int) { + return fileDescriptor_structured_712cb25350946e6b, []int{1} +} +func (m *ForeignKeyReference) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ForeignKeyReference) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ForeignKeyReference) XXX_Merge(src proto.Message) { + xxx_messageInfo_ForeignKeyReference.Merge(dst, src) +} +func (m *ForeignKeyReference) XXX_Size() int { + return m.Size() +} +func (m *ForeignKeyReference) XXX_DiscardUnknown() { + xxx_messageInfo_ForeignKeyReference.DiscardUnknown(m) } -func (m *ForeignKeyReference) Reset() { *m = ForeignKeyReference{} } -func (m *ForeignKeyReference) String() string { return proto.CompactTextString(m) } -func (*ForeignKeyReference) ProtoMessage() {} -func (*ForeignKeyReference) Descriptor() ([]byte, []int) { return fileDescriptorStructured, []int{1} } +var xxx_messageInfo_ForeignKeyReference proto.InternalMessageInfo type ColumnDescriptor struct { Name string `protobuf:"bytes,1,opt,name=name" json:"name"` @@ -628,13 +688,39 @@ type ColumnDescriptor struct { UsesSequenceIds []ID `protobuf:"varint,10,rep,name=uses_sequence_ids,json=usesSequenceIds,casttype=ID" json:"uses_sequence_ids,omitempty"` // Expression to use to compute the value of this column if this is a // computed column. - ComputeExpr *string `protobuf:"bytes,11,opt,name=compute_expr,json=computeExpr" json:"compute_expr,omitempty"` + ComputeExpr *string `protobuf:"bytes,11,opt,name=compute_expr,json=computeExpr" json:"compute_expr,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *ColumnDescriptor) Reset() { *m = ColumnDescriptor{} } -func (m *ColumnDescriptor) String() string { return proto.CompactTextString(m) } -func (*ColumnDescriptor) ProtoMessage() {} -func (*ColumnDescriptor) Descriptor() ([]byte, []int) { return fileDescriptorStructured, []int{2} } +func (m *ColumnDescriptor) Reset() { *m = ColumnDescriptor{} } +func (m *ColumnDescriptor) String() string { return proto.CompactTextString(m) } +func (*ColumnDescriptor) ProtoMessage() {} +func (*ColumnDescriptor) Descriptor() ([]byte, []int) { + return fileDescriptor_structured_712cb25350946e6b, []int{2} +} +func (m *ColumnDescriptor) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ColumnDescriptor) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ColumnDescriptor) XXX_Merge(src proto.Message) { + xxx_messageInfo_ColumnDescriptor.Merge(dst, src) +} +func (m *ColumnDescriptor) XXX_Size() int { + return m.Size() +} +func (m *ColumnDescriptor) XXX_DiscardUnknown() { + xxx_messageInfo_ColumnDescriptor.DiscardUnknown(m) +} + +var xxx_messageInfo_ColumnDescriptor proto.InternalMessageInfo // ColumnFamilyDescriptor is set of columns stored together in one kv entry. type ColumnFamilyDescriptor struct { @@ -655,13 +741,39 @@ type ColumnFamilyDescriptor struct { // column is written without the column id prefix. Because more columns could // be added, it would be ambiguous which column was stored when read back in, // so this field supplies it. - DefaultColumnID ColumnID `protobuf:"varint,5,opt,name=default_column_id,json=defaultColumnId,casttype=ColumnID" json:"default_column_id"` + DefaultColumnID ColumnID `protobuf:"varint,5,opt,name=default_column_id,json=defaultColumnId,casttype=ColumnID" json:"default_column_id"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ColumnFamilyDescriptor) Reset() { *m = ColumnFamilyDescriptor{} } +func (m *ColumnFamilyDescriptor) String() string { return proto.CompactTextString(m) } +func (*ColumnFamilyDescriptor) ProtoMessage() {} +func (*ColumnFamilyDescriptor) Descriptor() ([]byte, []int) { + return fileDescriptor_structured_712cb25350946e6b, []int{3} +} +func (m *ColumnFamilyDescriptor) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ColumnFamilyDescriptor) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ColumnFamilyDescriptor) XXX_Merge(src proto.Message) { + xxx_messageInfo_ColumnFamilyDescriptor.Merge(dst, src) +} +func (m *ColumnFamilyDescriptor) XXX_Size() int { + return m.Size() +} +func (m *ColumnFamilyDescriptor) XXX_DiscardUnknown() { + xxx_messageInfo_ColumnFamilyDescriptor.DiscardUnknown(m) } -func (m *ColumnFamilyDescriptor) Reset() { *m = ColumnFamilyDescriptor{} } -func (m *ColumnFamilyDescriptor) String() string { return proto.CompactTextString(m) } -func (*ColumnFamilyDescriptor) ProtoMessage() {} -func (*ColumnFamilyDescriptor) Descriptor() ([]byte, []int) { return fileDescriptorStructured, []int{3} } +var xxx_messageInfo_ColumnFamilyDescriptor proto.InternalMessageInfo // InterleaveDescriptor represents an index (either primary or secondary) that // is interleaved into another table's data. @@ -677,13 +789,39 @@ type InterleaveDescriptor struct { // Ancestors contains the nesting of interleaves in the order they appear in // an encoded key. This means they are always in the far-to-near ancestor // order (e.g. grand-grand-parent, grand-parent, parent). - Ancestors []InterleaveDescriptor_Ancestor `protobuf:"bytes,1,rep,name=ancestors" json:"ancestors"` + Ancestors []InterleaveDescriptor_Ancestor `protobuf:"bytes,1,rep,name=ancestors" json:"ancestors"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *InterleaveDescriptor) Reset() { *m = InterleaveDescriptor{} } +func (m *InterleaveDescriptor) String() string { return proto.CompactTextString(m) } +func (*InterleaveDescriptor) ProtoMessage() {} +func (*InterleaveDescriptor) Descriptor() ([]byte, []int) { + return fileDescriptor_structured_712cb25350946e6b, []int{4} +} +func (m *InterleaveDescriptor) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *InterleaveDescriptor) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *InterleaveDescriptor) XXX_Merge(src proto.Message) { + xxx_messageInfo_InterleaveDescriptor.Merge(dst, src) +} +func (m *InterleaveDescriptor) XXX_Size() int { + return m.Size() +} +func (m *InterleaveDescriptor) XXX_DiscardUnknown() { + xxx_messageInfo_InterleaveDescriptor.DiscardUnknown(m) } -func (m *InterleaveDescriptor) Reset() { *m = InterleaveDescriptor{} } -func (m *InterleaveDescriptor) String() string { return proto.CompactTextString(m) } -func (*InterleaveDescriptor) ProtoMessage() {} -func (*InterleaveDescriptor) Descriptor() ([]byte, []int) { return fileDescriptorStructured, []int{4} } +var xxx_messageInfo_InterleaveDescriptor proto.InternalMessageInfo type InterleaveDescriptor_Ancestor struct { // TableID the ID of the table being interleaved into. @@ -697,16 +835,40 @@ type InterleaveDescriptor_Ancestor struct { // being interleaved. // In cockroach 1.0, this value did not exist and thus a check for > 0 // must be performed prior to its use. - SharedPrefixLen uint32 `protobuf:"varint,3,opt,name=shared_prefix_len,json=sharedPrefixLen" json:"shared_prefix_len"` + SharedPrefixLen uint32 `protobuf:"varint,3,opt,name=shared_prefix_len,json=sharedPrefixLen" json:"shared_prefix_len"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *InterleaveDescriptor_Ancestor) Reset() { *m = InterleaveDescriptor_Ancestor{} } func (m *InterleaveDescriptor_Ancestor) String() string { return proto.CompactTextString(m) } func (*InterleaveDescriptor_Ancestor) ProtoMessage() {} func (*InterleaveDescriptor_Ancestor) Descriptor() ([]byte, []int) { - return fileDescriptorStructured, []int{4, 0} + return fileDescriptor_structured_712cb25350946e6b, []int{4, 0} +} +func (m *InterleaveDescriptor_Ancestor) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *InterleaveDescriptor_Ancestor) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *InterleaveDescriptor_Ancestor) XXX_Merge(src proto.Message) { + xxx_messageInfo_InterleaveDescriptor_Ancestor.Merge(dst, src) +} +func (m *InterleaveDescriptor_Ancestor) XXX_Size() int { + return m.Size() +} +func (m *InterleaveDescriptor_Ancestor) XXX_DiscardUnknown() { + xxx_messageInfo_InterleaveDescriptor_Ancestor.DiscardUnknown(m) } +var xxx_messageInfo_InterleaveDescriptor_Ancestor proto.InternalMessageInfo + // PartitioningDescriptor represents the partitioning of an index into spans // of keys addressable by a zone config. The key encoding is unchanged. Each // partition may optionally be itself divided into further partitions, called @@ -719,14 +881,40 @@ type PartitioningDescriptor struct { NumColumns uint32 `protobuf:"varint,1,opt,name=num_columns,json=numColumns" json:"num_columns"` // Exactly one of List or Range is required to be non-empty if NumColumns is // non-zero. - List []PartitioningDescriptor_List `protobuf:"bytes,2,rep,name=list" json:"list"` - Range []PartitioningDescriptor_Range `protobuf:"bytes,3,rep,name=range" json:"range"` + List []PartitioningDescriptor_List `protobuf:"bytes,2,rep,name=list" json:"list"` + Range []PartitioningDescriptor_Range `protobuf:"bytes,3,rep,name=range" json:"range"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *PartitioningDescriptor) Reset() { *m = PartitioningDescriptor{} } -func (m *PartitioningDescriptor) String() string { return proto.CompactTextString(m) } -func (*PartitioningDescriptor) ProtoMessage() {} -func (*PartitioningDescriptor) Descriptor() ([]byte, []int) { return fileDescriptorStructured, []int{5} } +func (m *PartitioningDescriptor) Reset() { *m = PartitioningDescriptor{} } +func (m *PartitioningDescriptor) String() string { return proto.CompactTextString(m) } +func (*PartitioningDescriptor) ProtoMessage() {} +func (*PartitioningDescriptor) Descriptor() ([]byte, []int) { + return fileDescriptor_structured_712cb25350946e6b, []int{5} +} +func (m *PartitioningDescriptor) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *PartitioningDescriptor) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *PartitioningDescriptor) XXX_Merge(src proto.Message) { + xxx_messageInfo_PartitioningDescriptor.Merge(dst, src) +} +func (m *PartitioningDescriptor) XXX_Size() int { + return m.Size() +} +func (m *PartitioningDescriptor) XXX_DiscardUnknown() { + xxx_messageInfo_PartitioningDescriptor.DiscardUnknown(m) +} + +var xxx_messageInfo_PartitioningDescriptor proto.InternalMessageInfo // List represents a list partitioning, which maps individual tuples to // partitions. @@ -739,16 +927,40 @@ type PartitioningDescriptor_List struct { // uvarint. Values [][]byte `protobuf:"bytes,2,rep,name=values" json:"values,omitempty"` // Subpartitioning represents a further partitioning of this list partition. - Subpartitioning PartitioningDescriptor `protobuf:"bytes,3,opt,name=subpartitioning" json:"subpartitioning"` + Subpartitioning PartitioningDescriptor `protobuf:"bytes,3,opt,name=subpartitioning" json:"subpartitioning"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *PartitioningDescriptor_List) Reset() { *m = PartitioningDescriptor_List{} } func (m *PartitioningDescriptor_List) String() string { return proto.CompactTextString(m) } func (*PartitioningDescriptor_List) ProtoMessage() {} func (*PartitioningDescriptor_List) Descriptor() ([]byte, []int) { - return fileDescriptorStructured, []int{5, 0} + return fileDescriptor_structured_712cb25350946e6b, []int{5, 0} +} +func (m *PartitioningDescriptor_List) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *PartitioningDescriptor_List) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *PartitioningDescriptor_List) XXX_Merge(src proto.Message) { + xxx_messageInfo_PartitioningDescriptor_List.Merge(dst, src) +} +func (m *PartitioningDescriptor_List) XXX_Size() int { + return m.Size() +} +func (m *PartitioningDescriptor_List) XXX_DiscardUnknown() { + xxx_messageInfo_PartitioningDescriptor_List.DiscardUnknown(m) } +var xxx_messageInfo_PartitioningDescriptor_List proto.InternalMessageInfo + // Range represents a range partitioning, which maps ranges of tuples to // partitions by specifying exclusive upper bounds. The range partitions in a // PartitioningDescriptor are required be sorted by UpperBound. @@ -762,16 +974,40 @@ type PartitioningDescriptor_Range struct { FromInclusive []byte `protobuf:"bytes,3,opt,name=from_inclusive,json=fromInclusive" json:"from_inclusive,omitempty"` // ToExclusive is the exclusive upper bound of this range partition. It is // encoded in the same way as From. - ToExclusive []byte `protobuf:"bytes,2,opt,name=to_exclusive,json=toExclusive" json:"to_exclusive,omitempty"` + ToExclusive []byte `protobuf:"bytes,2,opt,name=to_exclusive,json=toExclusive" json:"to_exclusive,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *PartitioningDescriptor_Range) Reset() { *m = PartitioningDescriptor_Range{} } func (m *PartitioningDescriptor_Range) String() string { return proto.CompactTextString(m) } func (*PartitioningDescriptor_Range) ProtoMessage() {} func (*PartitioningDescriptor_Range) Descriptor() ([]byte, []int) { - return fileDescriptorStructured, []int{5, 1} + return fileDescriptor_structured_712cb25350946e6b, []int{5, 1} +} +func (m *PartitioningDescriptor_Range) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *PartitioningDescriptor_Range) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *PartitioningDescriptor_Range) XXX_Merge(src proto.Message) { + xxx_messageInfo_PartitioningDescriptor_Range.Merge(dst, src) +} +func (m *PartitioningDescriptor_Range) XXX_Size() int { + return m.Size() +} +func (m *PartitioningDescriptor_Range) XXX_DiscardUnknown() { + xxx_messageInfo_PartitioningDescriptor_Range.DiscardUnknown(m) } +var xxx_messageInfo_PartitioningDescriptor_Range proto.InternalMessageInfo + // IndexDescriptor describes an index (primary or secondary). // // Sample field values on the following table: @@ -872,13 +1108,39 @@ type IndexDescriptor struct { // is partitioned into spans of keys each addressable by zone configs. Partitioning PartitioningDescriptor `protobuf:"bytes,15,opt,name=partitioning" json:"partitioning"` // Type is the type of index, inverted or forward. - Type IndexDescriptor_Type `protobuf:"varint,16,opt,name=type,enum=cockroach.sql.sqlbase.IndexDescriptor_Type" json:"type"` + Type IndexDescriptor_Type `protobuf:"varint,16,opt,name=type,enum=cockroach.sql.sqlbase.IndexDescriptor_Type" json:"type"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *IndexDescriptor) Reset() { *m = IndexDescriptor{} } -func (m *IndexDescriptor) String() string { return proto.CompactTextString(m) } -func (*IndexDescriptor) ProtoMessage() {} -func (*IndexDescriptor) Descriptor() ([]byte, []int) { return fileDescriptorStructured, []int{6} } +func (m *IndexDescriptor) Reset() { *m = IndexDescriptor{} } +func (m *IndexDescriptor) String() string { return proto.CompactTextString(m) } +func (*IndexDescriptor) ProtoMessage() {} +func (*IndexDescriptor) Descriptor() ([]byte, []int) { + return fileDescriptor_structured_712cb25350946e6b, []int{6} +} +func (m *IndexDescriptor) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *IndexDescriptor) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *IndexDescriptor) XXX_Merge(src proto.Message) { + xxx_messageInfo_IndexDescriptor.Merge(dst, src) +} +func (m *IndexDescriptor) XXX_Size() int { + return m.Size() +} +func (m *IndexDescriptor) XXX_DiscardUnknown() { + xxx_messageInfo_IndexDescriptor.DiscardUnknown(m) +} + +var xxx_messageInfo_IndexDescriptor proto.InternalMessageInfo // A DescriptorMutation represents a column or an index that // has either been added or dropped and hasn't yet transitioned @@ -899,13 +1161,39 @@ type DescriptorMutation struct { // unique constraint index. MutationID MutationID `protobuf:"varint,5,opt,name=mutation_id,json=mutationId,casttype=MutationID" json:"mutation_id"` // Indicates that this mutation is a rollback. - Rollback bool `protobuf:"varint,7,opt,name=rollback" json:"rollback"` + Rollback bool `protobuf:"varint,7,opt,name=rollback" json:"rollback"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *DescriptorMutation) Reset() { *m = DescriptorMutation{} } -func (m *DescriptorMutation) String() string { return proto.CompactTextString(m) } -func (*DescriptorMutation) ProtoMessage() {} -func (*DescriptorMutation) Descriptor() ([]byte, []int) { return fileDescriptorStructured, []int{7} } +func (m *DescriptorMutation) Reset() { *m = DescriptorMutation{} } +func (m *DescriptorMutation) String() string { return proto.CompactTextString(m) } +func (*DescriptorMutation) ProtoMessage() {} +func (*DescriptorMutation) Descriptor() ([]byte, []int) { + return fileDescriptor_structured_712cb25350946e6b, []int{7} +} +func (m *DescriptorMutation) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *DescriptorMutation) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *DescriptorMutation) XXX_Merge(src proto.Message) { + xxx_messageInfo_DescriptorMutation.Merge(dst, src) +} +func (m *DescriptorMutation) XXX_Size() int { + return m.Size() +} +func (m *DescriptorMutation) XXX_DiscardUnknown() { + xxx_messageInfo_DescriptorMutation.DiscardUnknown(m) +} + +var xxx_messageInfo_DescriptorMutation proto.InternalMessageInfo type isDescriptorMutation_Descriptor_ interface { isDescriptorMutation_Descriptor_() @@ -1003,12 +1291,12 @@ func _DescriptorMutation_OneofSizer(msg proto.Message) (n int) { switch x := m.Descriptor_.(type) { case *DescriptorMutation_Column: s := proto.Size(x.Column) - n += proto.SizeVarint(1<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *DescriptorMutation_Index: s := proto.Size(x.Index) - n += proto.SizeVarint(2<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case nil: @@ -1046,8 +1334,8 @@ type TableDescriptor struct { // particular version increment. Version DescriptorVersion `protobuf:"varint,5,opt,name=version,casttype=DescriptorVersion" json:"version"` // Last modification time of the table descriptor. - ModificationTime cockroach_util_hlc.Timestamp `protobuf:"bytes,7,opt,name=modification_time,json=modificationTime" json:"modification_time"` - Columns []ColumnDescriptor `protobuf:"bytes,8,rep,name=columns" json:"columns"` + ModificationTime hlc.Timestamp `protobuf:"bytes,7,opt,name=modification_time,json=modificationTime" json:"modification_time"` + Columns []ColumnDescriptor `protobuf:"bytes,8,rep,name=columns" json:"columns"` // next_column_id is used to ensure that deleted column ids are not reused. NextColumnID ColumnID `protobuf:"varint,9,opt,name=next_column_id,json=nextColumnId,casttype=ColumnID" json:"next_column_id"` Families []ColumnFamilyDescriptor `protobuf:"bytes,22,rep,name=families" json:"families"` @@ -1127,13 +1415,39 @@ type TableDescriptor struct { // // TODO(vivekmenezes): This is currently only used by the non-interleaved drop // index case. Also use for dropped interleaved indexes and columns. - GCMutations []TableDescriptor_GCDescriptorMutation `protobuf:"bytes,33,rep,name=gc_mutations,json=gcMutations" json:"gc_mutations"` + GCMutations []TableDescriptor_GCDescriptorMutation `protobuf:"bytes,33,rep,name=gc_mutations,json=gcMutations" json:"gc_mutations"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *TableDescriptor) Reset() { *m = TableDescriptor{} } +func (m *TableDescriptor) String() string { return proto.CompactTextString(m) } +func (*TableDescriptor) ProtoMessage() {} +func (*TableDescriptor) Descriptor() ([]byte, []int) { + return fileDescriptor_structured_712cb25350946e6b, []int{8} +} +func (m *TableDescriptor) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *TableDescriptor) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *TableDescriptor) XXX_Merge(src proto.Message) { + xxx_messageInfo_TableDescriptor.Merge(dst, src) +} +func (m *TableDescriptor) XXX_Size() int { + return m.Size() +} +func (m *TableDescriptor) XXX_DiscardUnknown() { + xxx_messageInfo_TableDescriptor.DiscardUnknown(m) } -func (m *TableDescriptor) Reset() { *m = TableDescriptor{} } -func (m *TableDescriptor) String() string { return proto.CompactTextString(m) } -func (*TableDescriptor) ProtoMessage() {} -func (*TableDescriptor) Descriptor() ([]byte, []int) { return fileDescriptorStructured, []int{8} } +var xxx_messageInfo_TableDescriptor proto.InternalMessageInfo func (m *TableDescriptor) GetName() string { if m != nil { @@ -1163,11 +1477,11 @@ func (m *TableDescriptor) GetVersion() DescriptorVersion { return 0 } -func (m *TableDescriptor) GetModificationTime() cockroach_util_hlc.Timestamp { +func (m *TableDescriptor) GetModificationTime() hlc.Timestamp { if m != nil { return m.ModificationTime } - return cockroach_util_hlc.Timestamp{} + return hlc.Timestamp{} } func (m *TableDescriptor) GetColumns() []ColumnDescriptor { @@ -1353,30 +1667,78 @@ func (m *TableDescriptor) GetGCMutations() []TableDescriptor_GCDescriptorMutatio type TableDescriptor_SchemaChangeLease struct { NodeID github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,1,opt,name=node_id,json=nodeId,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"node_id"` // Nanoseconds since the Unix epoch. - ExpirationTime int64 `protobuf:"varint,2,opt,name=expiration_time,json=expirationTime" json:"expiration_time"` + ExpirationTime int64 `protobuf:"varint,2,opt,name=expiration_time,json=expirationTime" json:"expiration_time"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *TableDescriptor_SchemaChangeLease) Reset() { *m = TableDescriptor_SchemaChangeLease{} } func (m *TableDescriptor_SchemaChangeLease) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_SchemaChangeLease) ProtoMessage() {} func (*TableDescriptor_SchemaChangeLease) Descriptor() ([]byte, []int) { - return fileDescriptorStructured, []int{8, 0} + return fileDescriptor_structured_712cb25350946e6b, []int{8, 0} +} +func (m *TableDescriptor_SchemaChangeLease) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *TableDescriptor_SchemaChangeLease) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *TableDescriptor_SchemaChangeLease) XXX_Merge(src proto.Message) { + xxx_messageInfo_TableDescriptor_SchemaChangeLease.Merge(dst, src) +} +func (m *TableDescriptor_SchemaChangeLease) XXX_Size() int { + return m.Size() +} +func (m *TableDescriptor_SchemaChangeLease) XXX_DiscardUnknown() { + xxx_messageInfo_TableDescriptor_SchemaChangeLease.DiscardUnknown(m) } +var xxx_messageInfo_TableDescriptor_SchemaChangeLease proto.InternalMessageInfo + type TableDescriptor_CheckConstraint struct { Expr string `protobuf:"bytes,1,opt,name=expr" json:"expr"` Name string `protobuf:"bytes,2,opt,name=name" json:"name"` Validity ConstraintValidity `protobuf:"varint,3,opt,name=validity,enum=cockroach.sql.sqlbase.ConstraintValidity" json:"validity"` // An ordered list of column IDs used by the check constraint. - ColumnIDs []ColumnID `protobuf:"varint,5,rep,name=column_ids,json=columnIds,casttype=ColumnID" json:"column_ids,omitempty"` + ColumnIDs []ColumnID `protobuf:"varint,5,rep,name=column_ids,json=columnIds,casttype=ColumnID" json:"column_ids,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *TableDescriptor_CheckConstraint) Reset() { *m = TableDescriptor_CheckConstraint{} } func (m *TableDescriptor_CheckConstraint) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_CheckConstraint) ProtoMessage() {} func (*TableDescriptor_CheckConstraint) Descriptor() ([]byte, []int) { - return fileDescriptorStructured, []int{8, 1} + return fileDescriptor_structured_712cb25350946e6b, []int{8, 1} +} +func (m *TableDescriptor_CheckConstraint) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *TableDescriptor_CheckConstraint) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *TableDescriptor_CheckConstraint) XXX_Merge(src proto.Message) { + xxx_messageInfo_TableDescriptor_CheckConstraint.Merge(dst, src) +} +func (m *TableDescriptor_CheckConstraint) XXX_Size() int { + return m.Size() } +func (m *TableDescriptor_CheckConstraint) XXX_DiscardUnknown() { + xxx_messageInfo_TableDescriptor_CheckConstraint.DiscardUnknown(m) +} + +var xxx_messageInfo_TableDescriptor_CheckConstraint proto.InternalMessageInfo // A table descriptor is named through a name map stored in the // system.namespace table: a map from {parent_id, table_name} -> id. @@ -1448,16 +1810,40 @@ func (*TableDescriptor_CheckConstraint) Descriptor() ([]byte, []int) { type TableDescriptor_NameInfo struct { // The database that the table belonged to before the rename (tables can be // renamed from one db to another). - ParentID ID `protobuf:"varint,1,opt,name=parent_id,json=parentId,casttype=ID" json:"parent_id"` - Name string `protobuf:"bytes,2,opt,name=name" json:"name"` + ParentID ID `protobuf:"varint,1,opt,name=parent_id,json=parentId,casttype=ID" json:"parent_id"` + Name string `protobuf:"bytes,2,opt,name=name" json:"name"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *TableDescriptor_NameInfo) Reset() { *m = TableDescriptor_NameInfo{} } func (m *TableDescriptor_NameInfo) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_NameInfo) ProtoMessage() {} func (*TableDescriptor_NameInfo) Descriptor() ([]byte, []int) { - return fileDescriptorStructured, []int{8, 2} + return fileDescriptor_structured_712cb25350946e6b, []int{8, 2} +} +func (m *TableDescriptor_NameInfo) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *TableDescriptor_NameInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *TableDescriptor_NameInfo) XXX_Merge(src proto.Message) { + xxx_messageInfo_TableDescriptor_NameInfo.Merge(dst, src) +} +func (m *TableDescriptor_NameInfo) XXX_Size() int { + return m.Size() } +func (m *TableDescriptor_NameInfo) XXX_DiscardUnknown() { + xxx_messageInfo_TableDescriptor_NameInfo.DiscardUnknown(m) +} + +var xxx_messageInfo_TableDescriptor_NameInfo proto.InternalMessageInfo type TableDescriptor_Reference struct { // The ID of the relation that depends on this one. @@ -1467,31 +1853,79 @@ type TableDescriptor_Reference struct { IndexID IndexID `protobuf:"varint,2,opt,name=index_id,json=indexId,casttype=IndexID" json:"index_id"` // The IDs of this table's columns that are referenced by the dependent // relation. - ColumnIDs []ColumnID `protobuf:"varint,3,rep,name=column_ids,json=columnIds,casttype=ColumnID" json:"column_ids,omitempty"` + ColumnIDs []ColumnID `protobuf:"varint,3,rep,name=column_ids,json=columnIds,casttype=ColumnID" json:"column_ids,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *TableDescriptor_Reference) Reset() { *m = TableDescriptor_Reference{} } func (m *TableDescriptor_Reference) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_Reference) ProtoMessage() {} func (*TableDescriptor_Reference) Descriptor() ([]byte, []int) { - return fileDescriptorStructured, []int{8, 3} + return fileDescriptor_structured_712cb25350946e6b, []int{8, 3} +} +func (m *TableDescriptor_Reference) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *TableDescriptor_Reference) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *TableDescriptor_Reference) XXX_Merge(src proto.Message) { + xxx_messageInfo_TableDescriptor_Reference.Merge(dst, src) } +func (m *TableDescriptor_Reference) XXX_Size() int { + return m.Size() +} +func (m *TableDescriptor_Reference) XXX_DiscardUnknown() { + xxx_messageInfo_TableDescriptor_Reference.DiscardUnknown(m) +} + +var xxx_messageInfo_TableDescriptor_Reference proto.InternalMessageInfo type TableDescriptor_MutationJob struct { // The mutation id of this mutation job. MutationID MutationID `protobuf:"varint,1,opt,name=mutation_id,json=mutationId,casttype=MutationID" json:"mutation_id"` // The job id for a mutation job is the id in the system.jobs table of the // schema change job executing the mutation referenced by mutation_id. - JobID int64 `protobuf:"varint,2,opt,name=job_id,json=jobId" json:"job_id"` + JobID int64 `protobuf:"varint,2,opt,name=job_id,json=jobId" json:"job_id"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *TableDescriptor_MutationJob) Reset() { *m = TableDescriptor_MutationJob{} } func (m *TableDescriptor_MutationJob) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_MutationJob) ProtoMessage() {} func (*TableDescriptor_MutationJob) Descriptor() ([]byte, []int) { - return fileDescriptorStructured, []int{8, 4} + return fileDescriptor_structured_712cb25350946e6b, []int{8, 4} +} +func (m *TableDescriptor_MutationJob) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *TableDescriptor_MutationJob) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *TableDescriptor_MutationJob) XXX_Merge(src proto.Message) { + xxx_messageInfo_TableDescriptor_MutationJob.Merge(dst, src) +} +func (m *TableDescriptor_MutationJob) XXX_Size() int { + return m.Size() +} +func (m *TableDescriptor_MutationJob) XXX_DiscardUnknown() { + xxx_messageInfo_TableDescriptor_MutationJob.DiscardUnknown(m) } +var xxx_messageInfo_TableDescriptor_MutationJob proto.InternalMessageInfo + type TableDescriptor_SequenceOpts struct { // How much to increment the sequence by when nextval() is called. Increment int64 `protobuf:"varint,1,opt,name=increment" json:"increment"` @@ -1502,57 +1936,155 @@ type TableDescriptor_SequenceOpts struct { // Start value of the sequence. Start int64 `protobuf:"varint,4,opt,name=start" json:"start"` // Whether the sequence is virtual. - Virtual bool `protobuf:"varint,5,opt,name=virtual" json:"virtual"` + Virtual bool `protobuf:"varint,5,opt,name=virtual" json:"virtual"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *TableDescriptor_SequenceOpts) Reset() { *m = TableDescriptor_SequenceOpts{} } func (m *TableDescriptor_SequenceOpts) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_SequenceOpts) ProtoMessage() {} func (*TableDescriptor_SequenceOpts) Descriptor() ([]byte, []int) { - return fileDescriptorStructured, []int{8, 5} + return fileDescriptor_structured_712cb25350946e6b, []int{8, 5} +} +func (m *TableDescriptor_SequenceOpts) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *TableDescriptor_SequenceOpts) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *TableDescriptor_SequenceOpts) XXX_Merge(src proto.Message) { + xxx_messageInfo_TableDescriptor_SequenceOpts.Merge(dst, src) +} +func (m *TableDescriptor_SequenceOpts) XXX_Size() int { + return m.Size() +} +func (m *TableDescriptor_SequenceOpts) XXX_DiscardUnknown() { + xxx_messageInfo_TableDescriptor_SequenceOpts.DiscardUnknown(m) } +var xxx_messageInfo_TableDescriptor_SequenceOpts proto.InternalMessageInfo + type TableDescriptor_Replacement struct { - ID ID `protobuf:"varint,1,opt,name=id,casttype=ID" json:"id"` - Time cockroach_util_hlc.Timestamp `protobuf:"bytes,2,opt,name=time" json:"time"` + ID ID `protobuf:"varint,1,opt,name=id,casttype=ID" json:"id"` + Time hlc.Timestamp `protobuf:"bytes,2,opt,name=time" json:"time"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *TableDescriptor_Replacement) Reset() { *m = TableDescriptor_Replacement{} } func (m *TableDescriptor_Replacement) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_Replacement) ProtoMessage() {} func (*TableDescriptor_Replacement) Descriptor() ([]byte, []int) { - return fileDescriptorStructured, []int{8, 6} + return fileDescriptor_structured_712cb25350946e6b, []int{8, 6} +} +func (m *TableDescriptor_Replacement) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *TableDescriptor_Replacement) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil } +func (dst *TableDescriptor_Replacement) XXX_Merge(src proto.Message) { + xxx_messageInfo_TableDescriptor_Replacement.Merge(dst, src) +} +func (m *TableDescriptor_Replacement) XXX_Size() int { + return m.Size() +} +func (m *TableDescriptor_Replacement) XXX_DiscardUnknown() { + xxx_messageInfo_TableDescriptor_Replacement.DiscardUnknown(m) +} + +var xxx_messageInfo_TableDescriptor_Replacement proto.InternalMessageInfo type TableDescriptor_GCDescriptorMutation struct { IndexID IndexID `protobuf:"varint,1,opt,name=index_id,json=indexId,casttype=IndexID" json:"index_id"` DropTime int64 `protobuf:"varint,2,opt,name=drop_time,json=dropTime" json:"drop_time"` // The job id for a mutation job is the id in the system.jobs table of the // schema change job executing the mutation referenced by mutation_id. - JobID int64 `protobuf:"varint,3,opt,name=job_id,json=jobId" json:"job_id"` + JobID int64 `protobuf:"varint,3,opt,name=job_id,json=jobId" json:"job_id"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *TableDescriptor_GCDescriptorMutation) Reset() { *m = TableDescriptor_GCDescriptorMutation{} } func (m *TableDescriptor_GCDescriptorMutation) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_GCDescriptorMutation) ProtoMessage() {} func (*TableDescriptor_GCDescriptorMutation) Descriptor() ([]byte, []int) { - return fileDescriptorStructured, []int{8, 7} + return fileDescriptor_structured_712cb25350946e6b, []int{8, 7} +} +func (m *TableDescriptor_GCDescriptorMutation) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *TableDescriptor_GCDescriptorMutation) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *TableDescriptor_GCDescriptorMutation) XXX_Merge(src proto.Message) { + xxx_messageInfo_TableDescriptor_GCDescriptorMutation.Merge(dst, src) +} +func (m *TableDescriptor_GCDescriptorMutation) XXX_Size() int { + return m.Size() } +func (m *TableDescriptor_GCDescriptorMutation) XXX_DiscardUnknown() { + xxx_messageInfo_TableDescriptor_GCDescriptorMutation.DiscardUnknown(m) +} + +var xxx_messageInfo_TableDescriptor_GCDescriptorMutation proto.InternalMessageInfo // DatabaseDescriptor represents a namespace (aka database) and is stored // in a structured metadata key. The DatabaseDescriptor has a globally-unique // ID shared with the TableDescriptor ID. // Permissions are applied to all tables in the namespace. type DatabaseDescriptor struct { - Name string `protobuf:"bytes,1,opt,name=name" json:"name"` - ID ID `protobuf:"varint,2,opt,name=id,casttype=ID" json:"id"` - Privileges *PrivilegeDescriptor `protobuf:"bytes,3,opt,name=privileges" json:"privileges,omitempty"` + Name string `protobuf:"bytes,1,opt,name=name" json:"name"` + ID ID `protobuf:"varint,2,opt,name=id,casttype=ID" json:"id"` + Privileges *PrivilegeDescriptor `protobuf:"bytes,3,opt,name=privileges" json:"privileges,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *DatabaseDescriptor) Reset() { *m = DatabaseDescriptor{} } -func (m *DatabaseDescriptor) String() string { return proto.CompactTextString(m) } -func (*DatabaseDescriptor) ProtoMessage() {} -func (*DatabaseDescriptor) Descriptor() ([]byte, []int) { return fileDescriptorStructured, []int{9} } +func (m *DatabaseDescriptor) Reset() { *m = DatabaseDescriptor{} } +func (m *DatabaseDescriptor) String() string { return proto.CompactTextString(m) } +func (*DatabaseDescriptor) ProtoMessage() {} +func (*DatabaseDescriptor) Descriptor() ([]byte, []int) { + return fileDescriptor_structured_712cb25350946e6b, []int{9} +} +func (m *DatabaseDescriptor) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *DatabaseDescriptor) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *DatabaseDescriptor) XXX_Merge(src proto.Message) { + xxx_messageInfo_DatabaseDescriptor.Merge(dst, src) +} +func (m *DatabaseDescriptor) XXX_Size() int { + return m.Size() +} +func (m *DatabaseDescriptor) XXX_DiscardUnknown() { + xxx_messageInfo_DatabaseDescriptor.DiscardUnknown(m) +} + +var xxx_messageInfo_DatabaseDescriptor proto.InternalMessageInfo func (m *DatabaseDescriptor) GetName() string { if m != nil { @@ -1580,13 +2112,39 @@ type Descriptor struct { // Types that are valid to be assigned to Union: // *Descriptor_Table // *Descriptor_Database - Union isDescriptor_Union `protobuf_oneof:"union"` + Union isDescriptor_Union `protobuf_oneof:"union"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *Descriptor) Reset() { *m = Descriptor{} } -func (m *Descriptor) String() string { return proto.CompactTextString(m) } -func (*Descriptor) ProtoMessage() {} -func (*Descriptor) Descriptor() ([]byte, []int) { return fileDescriptorStructured, []int{10} } +func (m *Descriptor) Reset() { *m = Descriptor{} } +func (m *Descriptor) String() string { return proto.CompactTextString(m) } +func (*Descriptor) ProtoMessage() {} +func (*Descriptor) Descriptor() ([]byte, []int) { + return fileDescriptor_structured_712cb25350946e6b, []int{10} +} +func (m *Descriptor) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Descriptor) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *Descriptor) XXX_Merge(src proto.Message) { + xxx_messageInfo_Descriptor.Merge(dst, src) +} +func (m *Descriptor) XXX_Size() int { + return m.Size() +} +func (m *Descriptor) XXX_DiscardUnknown() { + xxx_messageInfo_Descriptor.DiscardUnknown(m) +} + +var xxx_messageInfo_Descriptor proto.InternalMessageInfo type isDescriptor_Union interface { isDescriptor_Union() @@ -1684,12 +2242,12 @@ func _Descriptor_OneofSizer(msg proto.Message) (n int) { switch x := m.Union.(type) { case *Descriptor_Table: s := proto.Size(x.Table) - n += proto.SizeVarint(1<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case *Descriptor_Database: s := proto.Size(x.Database) - n += proto.SizeVarint(2<<3 | proto.WireBytes) + n += 1 // tag and wire n += proto.SizeVarint(uint64(s)) n += s case nil: @@ -3037,6 +3595,9 @@ func encodeVarintStructured(dAtA []byte, offset int, v uint64) int { return offset + 1 } func (m *ColumnType) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovStructured(uint64(m.SemanticType)) @@ -3071,6 +3632,9 @@ func (m *ColumnType) Size() (n int) { } func (m *ForeignKeyReference) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovStructured(uint64(m.Table)) @@ -3086,6 +3650,9 @@ func (m *ForeignKeyReference) Size() (n int) { } func (m *ColumnDescriptor) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Name) @@ -3112,6 +3679,9 @@ func (m *ColumnDescriptor) Size() (n int) { } func (m *ColumnFamilyDescriptor) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Name) @@ -3133,6 +3703,9 @@ func (m *ColumnFamilyDescriptor) Size() (n int) { } func (m *InterleaveDescriptor) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.Ancestors) > 0 { @@ -3145,6 +3718,9 @@ func (m *InterleaveDescriptor) Size() (n int) { } func (m *InterleaveDescriptor_Ancestor) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovStructured(uint64(m.TableID)) @@ -3154,6 +3730,9 @@ func (m *InterleaveDescriptor_Ancestor) Size() (n int) { } func (m *PartitioningDescriptor) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovStructured(uint64(m.NumColumns)) @@ -3173,6 +3752,9 @@ func (m *PartitioningDescriptor) Size() (n int) { } func (m *PartitioningDescriptor_List) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Name) @@ -3189,6 +3771,9 @@ func (m *PartitioningDescriptor_List) Size() (n int) { } func (m *PartitioningDescriptor_Range) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Name) @@ -3205,6 +3790,9 @@ func (m *PartitioningDescriptor_Range) Size() (n int) { } func (m *IndexDescriptor) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Name) @@ -3271,6 +3859,9 @@ func (m *IndexDescriptor) Size() (n int) { } func (m *DescriptorMutation) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Descriptor_ != nil { @@ -3284,6 +3875,9 @@ func (m *DescriptorMutation) Size() (n int) { } func (m *DescriptorMutation_Column) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Column != nil { @@ -3293,6 +3887,9 @@ func (m *DescriptorMutation_Column) Size() (n int) { return n } func (m *DescriptorMutation_Index) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Index != nil { @@ -3302,6 +3899,9 @@ func (m *DescriptorMutation_Index) Size() (n int) { return n } func (m *TableDescriptor) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Name) @@ -3401,6 +4001,9 @@ func (m *TableDescriptor) Size() (n int) { } func (m *TableDescriptor_SchemaChangeLease) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovStructured(uint64(m.NodeID)) @@ -3409,6 +4012,9 @@ func (m *TableDescriptor_SchemaChangeLease) Size() (n int) { } func (m *TableDescriptor_CheckConstraint) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Expr) @@ -3425,6 +4031,9 @@ func (m *TableDescriptor_CheckConstraint) Size() (n int) { } func (m *TableDescriptor_NameInfo) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovStructured(uint64(m.ParentID)) @@ -3434,6 +4043,9 @@ func (m *TableDescriptor_NameInfo) Size() (n int) { } func (m *TableDescriptor_Reference) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovStructured(uint64(m.ID)) @@ -3447,6 +4059,9 @@ func (m *TableDescriptor_Reference) Size() (n int) { } func (m *TableDescriptor_MutationJob) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovStructured(uint64(m.MutationID)) @@ -3455,6 +4070,9 @@ func (m *TableDescriptor_MutationJob) Size() (n int) { } func (m *TableDescriptor_SequenceOpts) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovStructured(uint64(m.Increment)) @@ -3466,6 +4084,9 @@ func (m *TableDescriptor_SequenceOpts) Size() (n int) { } func (m *TableDescriptor_Replacement) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovStructured(uint64(m.ID)) @@ -3475,6 +4096,9 @@ func (m *TableDescriptor_Replacement) Size() (n int) { } func (m *TableDescriptor_GCDescriptorMutation) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovStructured(uint64(m.IndexID)) @@ -3484,6 +4108,9 @@ func (m *TableDescriptor_GCDescriptorMutation) Size() (n int) { } func (m *DatabaseDescriptor) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Name) @@ -3497,6 +4124,9 @@ func (m *DatabaseDescriptor) Size() (n int) { } func (m *Descriptor) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Union != nil { @@ -3506,6 +4136,9 @@ func (m *Descriptor) Size() (n int) { } func (m *Descriptor_Table) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Table != nil { @@ -3515,6 +4148,9 @@ func (m *Descriptor_Table) Size() (n int) { return n } func (m *Descriptor_Database) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Database != nil { @@ -3664,6 +4300,17 @@ func (m *ColumnType) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + var count int + for _, integer := range dAtA { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.ArrayDimensions) == 0 { + m.ArrayDimensions = make([]int32, 0, elementCount) + } for iNdEx < postIndex { var v int32 for shift := uint(0); ; shift += 7 { @@ -4265,6 +4912,17 @@ func (m *ColumnDescriptor) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + var count int + for _, integer := range dAtA { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.UsesSequenceIds) == 0 { + m.UsesSequenceIds = make([]ID, 0, elementCount) + } for iNdEx < postIndex { var v ID for shift := uint(0); ; shift += 7 { @@ -4484,6 +5142,17 @@ func (m *ColumnFamilyDescriptor) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + var count int + for _, integer := range dAtA { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.ColumnIDs) == 0 { + m.ColumnIDs = make([]ColumnID, 0, elementCount) + } for iNdEx < postIndex { var v ColumnID for shift := uint(0); ; shift += 7 { @@ -5339,6 +6008,17 @@ func (m *IndexDescriptor) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + var count int + for _, integer := range dAtA { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.ColumnIDs) == 0 { + m.ColumnIDs = make([]ColumnID, 0, elementCount) + } for iNdEx < postIndex { var v ColumnID for shift := uint(0); ; shift += 7 { @@ -5401,6 +6081,17 @@ func (m *IndexDescriptor) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + var count int + for _, integer := range dAtA { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.ExtraColumnIDs) == 0 { + m.ExtraColumnIDs = make([]ColumnID, 0, elementCount) + } for iNdEx < postIndex { var v ColumnID for shift := uint(0); ; shift += 7 { @@ -5463,6 +6154,10 @@ func (m *IndexDescriptor) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + if elementCount != 0 && len(m.ColumnDirections) == 0 { + m.ColumnDirections = make([]IndexDescriptor_Direction, 0, elementCount) + } for iNdEx < postIndex { var v IndexDescriptor_Direction for shift := uint(0); ; shift += 7 { @@ -5647,6 +6342,17 @@ func (m *IndexDescriptor) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + var count int + for _, integer := range dAtA { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.CompositeColumnIDs) == 0 { + m.CompositeColumnIDs = make([]ColumnID, 0, elementCount) + } for iNdEx < postIndex { var v ColumnID for shift := uint(0); ; shift += 7 { @@ -5709,6 +6415,17 @@ func (m *IndexDescriptor) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + var count int + for _, integer := range dAtA { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.StoreColumnIDs) == 0 { + m.StoreColumnIDs = make([]ColumnID, 0, elementCount) + } for iNdEx < postIndex { var v ColumnID for shift := uint(0); ; shift += 7 { @@ -6602,6 +7319,17 @@ func (m *TableDescriptor) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + var count int + for _, integer := range dAtA { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.DependsOn) == 0 { + m.DependsOn = make([]ID, 0, elementCount) + } for iNdEx < postIndex { var v ID for shift := uint(0); ; shift += 7 { @@ -7092,6 +7820,17 @@ func (m *TableDescriptor_CheckConstraint) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + var count int + for _, integer := range dAtA { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.ColumnIDs) == 0 { + m.ColumnIDs = make([]ColumnID, 0, elementCount) + } for iNdEx < postIndex { var v ColumnID for shift := uint(0); ; shift += 7 { @@ -7340,6 +8079,17 @@ func (m *TableDescriptor_Reference) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + var count int + for _, integer := range dAtA { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.ColumnIDs) == 0 { + m.ColumnIDs = make([]ColumnID, 0, elementCount) + } for iNdEx < postIndex { var v ColumnID for shift := uint(0); ; shift += 7 { @@ -8172,9 +8922,11 @@ var ( ErrIntOverflowStructured = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("sql/sqlbase/structured.proto", fileDescriptorStructured) } +func init() { + proto.RegisterFile("sql/sqlbase/structured.proto", fileDescriptor_structured_712cb25350946e6b) +} -var fileDescriptorStructured = []byte{ +var fileDescriptor_structured_712cb25350946e6b = []byte{ // 3170 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x59, 0xcd, 0x6f, 0x1b, 0x47, 0x96, 0x57, 0x93, 0x4d, 0xb2, 0xf9, 0xf8, 0xd5, 0x2a, 0xcb, 0x0e, 0xcd, 0x38, 0x92, 0xcc, 0xc4, diff --git a/pkg/sql/sqlbase/table_test.go b/pkg/sql/sqlbase/table_test.go index 20f3684a80ec..304c204517bc 100644 --- a/pkg/sql/sqlbase/table_test.go +++ b/pkg/sql/sqlbase/table_test.go @@ -511,7 +511,7 @@ func TestMarshalColumnValue(t *testing.T) { if actual, err := MarshalColumnValue(col, testCase.datum); err != nil { t.Errorf("%d: unexpected error with column type %v: %v", i, typ, err) } else if !reflect.DeepEqual(actual, testCase.exp) { - t.Errorf("%d: MarshalColumnValue() got %s, expected %v", i, actual, testCase.exp) + t.Errorf("%d: MarshalColumnValue() got %v, expected %v", i, actual, testCase.exp) } } } diff --git a/pkg/sql/stats/histogram.pb.go b/pkg/sql/stats/histogram.pb.go index ec03620bbfa1..afbfee1274f7 100644 --- a/pkg/sql/stats/histogram.pb.go +++ b/pkg/sql/stats/histogram.pb.go @@ -1,21 +1,12 @@ // Code generated by protoc-gen-gogo. DO NOT EDIT. // source: sql/stats/histogram.proto -/* - Package stats is a generated protocol buffer package. - - It is generated from these files: - sql/stats/histogram.proto - - It has these top-level messages: - HistogramData -*/ package stats import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" -import cockroach_sql_sqlbase1 "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" +import sqlbase "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" import io "io" @@ -34,16 +25,42 @@ const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package // distribution of values on a specific column. type HistogramData struct { // Value type for the column. - ColumnType cockroach_sql_sqlbase1.ColumnType `protobuf:"bytes,2,opt,name=column_type,json=columnType" json:"column_type"` + ColumnType sqlbase.ColumnType `protobuf:"bytes,2,opt,name=column_type,json=columnType,proto3" json:"column_type"` // Histogram buckets. Note that NULL values are excluded from the // histogram. - Buckets []HistogramData_Bucket `protobuf:"bytes,1,rep,name=buckets" json:"buckets"` + Buckets []HistogramData_Bucket `protobuf:"bytes,1,rep,name=buckets,proto3" json:"buckets"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *HistogramData) Reset() { *m = HistogramData{} } -func (m *HistogramData) String() string { return proto.CompactTextString(m) } -func (*HistogramData) ProtoMessage() {} -func (*HistogramData) Descriptor() ([]byte, []int) { return fileDescriptorHistogram, []int{0} } +func (m *HistogramData) Reset() { *m = HistogramData{} } +func (m *HistogramData) String() string { return proto.CompactTextString(m) } +func (*HistogramData) ProtoMessage() {} +func (*HistogramData) Descriptor() ([]byte, []int) { + return fileDescriptor_histogram_96c931f453c9d8b8, []int{0} +} +func (m *HistogramData) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *HistogramData) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *HistogramData) XXX_Merge(src proto.Message) { + xxx_messageInfo_HistogramData.Merge(dst, src) +} +func (m *HistogramData) XXX_Size() int { + return m.Size() +} +func (m *HistogramData) XXX_DiscardUnknown() { + xxx_messageInfo_HistogramData.DiscardUnknown(m) +} + +var xxx_messageInfo_HistogramData proto.InternalMessageInfo type HistogramData_Bucket struct { // The estimated number of values that are equal to upper_bound. @@ -59,13 +76,39 @@ type HistogramData_Bucket struct { NumRange int64 `protobuf:"varint,2,opt,name=num_range,json=numRange,proto3" json:"num_range,omitempty"` // The upper boundary of the bucket. The column values for the upper bound // are encoded using the ascending key encoding of the column type. - UpperBound []byte `protobuf:"bytes,3,opt,name=upper_bound,json=upperBound,proto3" json:"upper_bound,omitempty"` + UpperBound []byte `protobuf:"bytes,3,opt,name=upper_bound,json=upperBound,proto3" json:"upper_bound,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *HistogramData_Bucket) Reset() { *m = HistogramData_Bucket{} } +func (m *HistogramData_Bucket) String() string { return proto.CompactTextString(m) } +func (*HistogramData_Bucket) ProtoMessage() {} +func (*HistogramData_Bucket) Descriptor() ([]byte, []int) { + return fileDescriptor_histogram_96c931f453c9d8b8, []int{0, 0} +} +func (m *HistogramData_Bucket) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *HistogramData_Bucket) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *HistogramData_Bucket) XXX_Merge(src proto.Message) { + xxx_messageInfo_HistogramData_Bucket.Merge(dst, src) +} +func (m *HistogramData_Bucket) XXX_Size() int { + return m.Size() +} +func (m *HistogramData_Bucket) XXX_DiscardUnknown() { + xxx_messageInfo_HistogramData_Bucket.DiscardUnknown(m) } -func (m *HistogramData_Bucket) Reset() { *m = HistogramData_Bucket{} } -func (m *HistogramData_Bucket) String() string { return proto.CompactTextString(m) } -func (*HistogramData_Bucket) ProtoMessage() {} -func (*HistogramData_Bucket) Descriptor() ([]byte, []int) { return fileDescriptorHistogram, []int{0, 0} } +var xxx_messageInfo_HistogramData_Bucket proto.InternalMessageInfo func init() { proto.RegisterType((*HistogramData)(nil), "cockroach.sql.stats.HistogramData") @@ -153,6 +196,9 @@ func encodeVarintHistogram(dAtA []byte, offset int, v uint64) int { return offset + 1 } func (m *HistogramData) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.Buckets) > 0 { @@ -167,6 +213,9 @@ func (m *HistogramData) Size() (n int) { } func (m *HistogramData_Bucket) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.NumEq != 0 { @@ -530,9 +579,11 @@ var ( ErrIntOverflowHistogram = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("sql/stats/histogram.proto", fileDescriptorHistogram) } +func init() { + proto.RegisterFile("sql/stats/histogram.proto", fileDescriptor_histogram_96c931f453c9d8b8) +} -var fileDescriptorHistogram = []byte{ +var fileDescriptor_histogram_96c931f453c9d8b8 = []byte{ // 304 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x6c, 0x90, 0x3f, 0x4e, 0xc3, 0x30, 0x18, 0xc5, 0xeb, 0x86, 0x16, 0x70, 0x60, 0x31, 0x20, 0x85, 0x82, 0xdc, 0xc0, 0x14, 0x16, 0x47, diff --git a/pkg/storage/allocator.go b/pkg/storage/allocator.go index 8b86f39ab6f9..cc3f976d4ced 100644 --- a/pkg/storage/allocator.go +++ b/pkg/storage/allocator.go @@ -20,6 +20,7 @@ import ( "fmt" "math" "math/rand" + "strings" "time" "github.com/cockroachdb/cockroach/pkg/config" @@ -162,7 +163,19 @@ func (ae *allocatorError) Error() string { } return baseMsg + "; likely not enough nodes in cluster" } - return fmt.Sprintf("%s; must match constraints %v", baseMsg, ae.constraints) + var b strings.Builder + b.WriteString(baseMsg) + b.WriteString("; must match constraints [") + for i := range ae.constraints { + if i > 0 { + b.WriteByte(' ') + } + b.WriteByte('{') + b.WriteString(ae.constraints[i].String()) + b.WriteByte('}') + } + b.WriteString("]") + return b.String() } func (*allocatorError) purgatoryErrorMarker() {} diff --git a/pkg/storage/allocator_test.go b/pkg/storage/allocator_test.go index 0253adddbde8..248719c6eb9e 100644 --- a/pkg/storage/allocator_test.go +++ b/pkg/storage/allocator_test.go @@ -5073,15 +5073,20 @@ func TestAllocatorError(t *testing.T) { {allocatorError{constraints: nil, existingReplicas: 1, aliveStores: 2, throttledStores: 1}, "0 of 2 live stores are able to take a new replica for the range (1 throttled, 1 already has a replica)"}, {allocatorError{constraints: constraint, existingReplicas: 1, aliveStores: 1}, - "0 of 1 live stores are able to take a new replica for the range (1 already has a replica); must match constraints [{0 [+one]}]"}, + `0 of 1 live stores are able to take a new replica for the range (1 already has a replica); ` + + `must match constraints [{num_replicas:0 constraints: }]`}, {allocatorError{constraints: constraint, existingReplicas: 1, aliveStores: 2}, - "0 of 2 live stores are able to take a new replica for the range (1 already has a replica); must match constraints [{0 [+one]}]"}, + `0 of 2 live stores are able to take a new replica for the range (1 already has a replica); ` + + `must match constraints [{num_replicas:0 constraints: }]`}, {allocatorError{constraints: constraints, existingReplicas: 1, aliveStores: 1}, - "0 of 1 live stores are able to take a new replica for the range (1 already has a replica); must match constraints [{0 [+one +two]}]"}, + `0 of 1 live stores are able to take a new replica for the range (1 already has a replica); ` + + `must match constraints [{num_replicas:0 constraints: constraints: }]`}, {allocatorError{constraints: constraints, existingReplicas: 1, aliveStores: 2}, - "0 of 2 live stores are able to take a new replica for the range (1 already has a replica); must match constraints [{0 [+one +two]}]"}, + `0 of 2 live stores are able to take a new replica for the range (1 already has a replica); ` + + `must match constraints [{num_replicas:0 constraints: constraints: }]`}, {allocatorError{constraints: constraint, existingReplicas: 1, aliveStores: 2, throttledStores: 1}, - "0 of 2 live stores are able to take a new replica for the range (1 throttled, 1 already has a replica); must match constraints [{0 [+one]}]"}, + `0 of 2 live stores are able to take a new replica for the range (1 throttled, 1 already has a replica); ` + + `must match constraints [{num_replicas:0 constraints: }]`}, } for i, testCase := range testCases { diff --git a/pkg/storage/api.pb.go b/pkg/storage/api.pb.go index b1f7193194e5..7ea8b81c7540 100644 --- a/pkg/storage/api.pb.go +++ b/pkg/storage/api.pb.go @@ -1,43 +1,21 @@ // Code generated by protoc-gen-gogo. DO NOT EDIT. // source: storage/api.proto -/* - Package storage is a generated protocol buffer package. - - It is generated from these files: - storage/api.proto - storage/raft.proto - - It has these top-level messages: - StoreRequestHeader - CollectChecksumRequest - CollectChecksumResponse - WaitForApplicationRequest - WaitForApplicationResponse - WaitForReplicaInitRequest - WaitForReplicaInitResponse - RaftHeartbeat - RaftMessageRequest - RaftMessageRequestBatch - RaftMessageResponseUnion - RaftMessageResponse - SnapshotRequest - SnapshotResponse - ConfChangeContext -*/ package storage import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" -import cockroach_roachpb1 "github.com/cockroachdb/cockroach/pkg/roachpb" -import cockroach_storage_engine_enginepb "github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb" +import roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" +import enginepb "github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb" import github_com_cockroachdb_cockroach_pkg_roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" import github_com_cockroachdb_cockroach_pkg_util_uuid "github.com/cockroachdb/cockroach/pkg/util/uuid" -import context "context" -import grpc "google.golang.org/grpc" +import ( + context "context" + grpc "google.golang.org/grpc" +) import io "io" @@ -54,29 +32,81 @@ const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package // StoreRequestHeader locates a Store on a Node. type StoreRequestHeader struct { - NodeID github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,1,opt,name=node_id,json=nodeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"node_id,omitempty"` - StoreID github_com_cockroachdb_cockroach_pkg_roachpb.StoreID `protobuf:"varint,2,opt,name=store_id,json=storeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.StoreID" json:"store_id,omitempty"` + NodeID github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,1,opt,name=node_id,json=nodeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"node_id,omitempty"` + StoreID github_com_cockroachdb_cockroach_pkg_roachpb.StoreID `protobuf:"varint,2,opt,name=store_id,json=storeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.StoreID" json:"store_id,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *StoreRequestHeader) Reset() { *m = StoreRequestHeader{} } +func (m *StoreRequestHeader) String() string { return proto.CompactTextString(m) } +func (*StoreRequestHeader) ProtoMessage() {} +func (*StoreRequestHeader) Descriptor() ([]byte, []int) { + return fileDescriptor_api_07e0411937110fca, []int{0} +} +func (m *StoreRequestHeader) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *StoreRequestHeader) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *StoreRequestHeader) XXX_Merge(src proto.Message) { + xxx_messageInfo_StoreRequestHeader.Merge(dst, src) +} +func (m *StoreRequestHeader) XXX_Size() int { + return m.Size() +} +func (m *StoreRequestHeader) XXX_DiscardUnknown() { + xxx_messageInfo_StoreRequestHeader.DiscardUnknown(m) } -func (m *StoreRequestHeader) Reset() { *m = StoreRequestHeader{} } -func (m *StoreRequestHeader) String() string { return proto.CompactTextString(m) } -func (*StoreRequestHeader) ProtoMessage() {} -func (*StoreRequestHeader) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{0} } +var xxx_messageInfo_StoreRequestHeader proto.InternalMessageInfo // A CollectChecksumRequest asks the addressed replica for the result of a // roachpb.ComputeChecksumRequest. type CollectChecksumRequest struct { - StoreRequestHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` + StoreRequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` RangeID github_com_cockroachdb_cockroach_pkg_roachpb.RangeID `protobuf:"varint,2,opt,name=range_id,json=rangeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" json:"range_id,omitempty"` // checksum_id identifies the corresponding roachpb.ComputeChecksumRequest. - ChecksumID github_com_cockroachdb_cockroach_pkg_util_uuid.UUID `protobuf:"bytes,3,opt,name=checksum_id,json=checksumId,proto3,customtype=github.com/cockroachdb/cockroach/pkg/util/uuid.UUID" json:"checksum_id"` - Checksum []byte `protobuf:"bytes,4,opt,name=checksum,proto3" json:"checksum,omitempty"` + ChecksumID github_com_cockroachdb_cockroach_pkg_util_uuid.UUID `protobuf:"bytes,3,opt,name=checksum_id,json=checksumId,proto3,customtype=github.com/cockroachdb/cockroach/pkg/util/uuid.UUID" json:"checksum_id"` + Checksum []byte `protobuf:"bytes,4,opt,name=checksum,proto3" json:"checksum,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *CollectChecksumRequest) Reset() { *m = CollectChecksumRequest{} } +func (m *CollectChecksumRequest) String() string { return proto.CompactTextString(m) } +func (*CollectChecksumRequest) ProtoMessage() {} +func (*CollectChecksumRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_api_07e0411937110fca, []int{1} +} +func (m *CollectChecksumRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *CollectChecksumRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *CollectChecksumRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_CollectChecksumRequest.Merge(dst, src) +} +func (m *CollectChecksumRequest) XXX_Size() int { + return m.Size() +} +func (m *CollectChecksumRequest) XXX_DiscardUnknown() { + xxx_messageInfo_CollectChecksumRequest.DiscardUnknown(m) } -func (m *CollectChecksumRequest) Reset() { *m = CollectChecksumRequest{} } -func (m *CollectChecksumRequest) String() string { return proto.CompactTextString(m) } -func (*CollectChecksumRequest) ProtoMessage() {} -func (*CollectChecksumRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{1} } +var xxx_messageInfo_CollectChecksumRequest proto.InternalMessageInfo type CollectChecksumResponse struct { Checksum []byte `protobuf:"bytes,1,opt,name=checksum,proto3" json:"checksum,omitempty"` @@ -85,54 +115,184 @@ type CollectChecksumResponse struct { // // TODO(tschottdorf): with larger ranges, this is no longer tenable. // See https://github.com/cockroachdb/cockroach/issues/21128. - Snapshot *cockroach_roachpb1.RaftSnapshotData `protobuf:"bytes,2,opt,name=snapshot" json:"snapshot,omitempty"` + Snapshot *roachpb.RaftSnapshotData `protobuf:"bytes,2,opt,name=snapshot,proto3" json:"snapshot,omitempty"` // delta carries the stats of the range minus the recomputed stats. - Delta cockroach_storage_engine_enginepb.MVCCStatsDelta `protobuf:"bytes,3,opt,name=delta" json:"delta"` + Delta enginepb.MVCCStatsDelta `protobuf:"bytes,3,opt,name=delta,proto3" json:"delta"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *CollectChecksumResponse) Reset() { *m = CollectChecksumResponse{} } -func (m *CollectChecksumResponse) String() string { return proto.CompactTextString(m) } -func (*CollectChecksumResponse) ProtoMessage() {} -func (*CollectChecksumResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{2} } +func (m *CollectChecksumResponse) Reset() { *m = CollectChecksumResponse{} } +func (m *CollectChecksumResponse) String() string { return proto.CompactTextString(m) } +func (*CollectChecksumResponse) ProtoMessage() {} +func (*CollectChecksumResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_api_07e0411937110fca, []int{2} +} +func (m *CollectChecksumResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *CollectChecksumResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *CollectChecksumResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_CollectChecksumResponse.Merge(dst, src) +} +func (m *CollectChecksumResponse) XXX_Size() int { + return m.Size() +} +func (m *CollectChecksumResponse) XXX_DiscardUnknown() { + xxx_messageInfo_CollectChecksumResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_CollectChecksumResponse proto.InternalMessageInfo // WaitForApplicationRequest blocks until the addressed replica has applied the // command with the specified lease index. type WaitForApplicationRequest struct { - StoreRequestHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` - RangeID github_com_cockroachdb_cockroach_pkg_roachpb.RangeID `protobuf:"varint,2,opt,name=range_id,json=rangeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" json:"range_id,omitempty"` - LeaseIndex uint64 `protobuf:"varint,3,opt,name=lease_index,json=leaseIndex,proto3" json:"lease_index,omitempty"` + StoreRequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + RangeID github_com_cockroachdb_cockroach_pkg_roachpb.RangeID `protobuf:"varint,2,opt,name=range_id,json=rangeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" json:"range_id,omitempty"` + LeaseIndex uint64 `protobuf:"varint,3,opt,name=lease_index,json=leaseIndex,proto3" json:"lease_index,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *WaitForApplicationRequest) Reset() { *m = WaitForApplicationRequest{} } +func (m *WaitForApplicationRequest) String() string { return proto.CompactTextString(m) } +func (*WaitForApplicationRequest) ProtoMessage() {} +func (*WaitForApplicationRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_api_07e0411937110fca, []int{3} +} +func (m *WaitForApplicationRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *WaitForApplicationRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *WaitForApplicationRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_WaitForApplicationRequest.Merge(dst, src) +} +func (m *WaitForApplicationRequest) XXX_Size() int { + return m.Size() +} +func (m *WaitForApplicationRequest) XXX_DiscardUnknown() { + xxx_messageInfo_WaitForApplicationRequest.DiscardUnknown(m) } -func (m *WaitForApplicationRequest) Reset() { *m = WaitForApplicationRequest{} } -func (m *WaitForApplicationRequest) String() string { return proto.CompactTextString(m) } -func (*WaitForApplicationRequest) ProtoMessage() {} -func (*WaitForApplicationRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{3} } +var xxx_messageInfo_WaitForApplicationRequest proto.InternalMessageInfo type WaitForApplicationResponse struct { + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *WaitForApplicationResponse) Reset() { *m = WaitForApplicationResponse{} } +func (m *WaitForApplicationResponse) String() string { return proto.CompactTextString(m) } +func (*WaitForApplicationResponse) ProtoMessage() {} +func (*WaitForApplicationResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_api_07e0411937110fca, []int{4} +} +func (m *WaitForApplicationResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *WaitForApplicationResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *WaitForApplicationResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_WaitForApplicationResponse.Merge(dst, src) +} +func (m *WaitForApplicationResponse) XXX_Size() int { + return m.Size() +} +func (m *WaitForApplicationResponse) XXX_DiscardUnknown() { + xxx_messageInfo_WaitForApplicationResponse.DiscardUnknown(m) } -func (m *WaitForApplicationResponse) Reset() { *m = WaitForApplicationResponse{} } -func (m *WaitForApplicationResponse) String() string { return proto.CompactTextString(m) } -func (*WaitForApplicationResponse) ProtoMessage() {} -func (*WaitForApplicationResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{4} } +var xxx_messageInfo_WaitForApplicationResponse proto.InternalMessageInfo type WaitForReplicaInitRequest struct { - StoreRequestHeader `protobuf:"bytes,1,opt,name=header,embedded=header" json:"header"` - RangeID github_com_cockroachdb_cockroach_pkg_roachpb.RangeID `protobuf:"varint,2,opt,name=range_id,json=rangeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" json:"range_id,omitempty"` + StoreRequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` + RangeID github_com_cockroachdb_cockroach_pkg_roachpb.RangeID `protobuf:"varint,2,opt,name=range_id,json=rangeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" json:"range_id,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *WaitForReplicaInitRequest) Reset() { *m = WaitForReplicaInitRequest{} } -func (m *WaitForReplicaInitRequest) String() string { return proto.CompactTextString(m) } -func (*WaitForReplicaInitRequest) ProtoMessage() {} -func (*WaitForReplicaInitRequest) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{5} } +func (m *WaitForReplicaInitRequest) Reset() { *m = WaitForReplicaInitRequest{} } +func (m *WaitForReplicaInitRequest) String() string { return proto.CompactTextString(m) } +func (*WaitForReplicaInitRequest) ProtoMessage() {} +func (*WaitForReplicaInitRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_api_07e0411937110fca, []int{5} +} +func (m *WaitForReplicaInitRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *WaitForReplicaInitRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *WaitForReplicaInitRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_WaitForReplicaInitRequest.Merge(dst, src) +} +func (m *WaitForReplicaInitRequest) XXX_Size() int { + return m.Size() +} +func (m *WaitForReplicaInitRequest) XXX_DiscardUnknown() { + xxx_messageInfo_WaitForReplicaInitRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_WaitForReplicaInitRequest proto.InternalMessageInfo type WaitForReplicaInitResponse struct { + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *WaitForReplicaInitResponse) Reset() { *m = WaitForReplicaInitResponse{} } +func (m *WaitForReplicaInitResponse) String() string { return proto.CompactTextString(m) } +func (*WaitForReplicaInitResponse) ProtoMessage() {} +func (*WaitForReplicaInitResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_api_07e0411937110fca, []int{6} +} +func (m *WaitForReplicaInitResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *WaitForReplicaInitResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *WaitForReplicaInitResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_WaitForReplicaInitResponse.Merge(dst, src) +} +func (m *WaitForReplicaInitResponse) XXX_Size() int { + return m.Size() +} +func (m *WaitForReplicaInitResponse) XXX_DiscardUnknown() { + xxx_messageInfo_WaitForReplicaInitResponse.DiscardUnknown(m) } -func (m *WaitForReplicaInitResponse) Reset() { *m = WaitForReplicaInitResponse{} } -func (m *WaitForReplicaInitResponse) String() string { return proto.CompactTextString(m) } -func (*WaitForReplicaInitResponse) ProtoMessage() {} -func (*WaitForReplicaInitResponse) Descriptor() ([]byte, []int) { return fileDescriptorApi, []int{6} } +var xxx_messageInfo_WaitForReplicaInitResponse proto.InternalMessageInfo func init() { proto.RegisterType((*StoreRequestHeader)(nil), "cockroach.storage.StoreRequestHeader") @@ -152,8 +312,9 @@ var _ grpc.ClientConn // is compatible with the grpc package it is being compiled against. const _ = grpc.SupportPackageIsVersion4 -// Client API for PerReplica service - +// PerReplicaClient is the client API for PerReplica service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream. type PerReplicaClient interface { CollectChecksum(ctx context.Context, in *CollectChecksumRequest, opts ...grpc.CallOption) (*CollectChecksumResponse, error) WaitForApplication(ctx context.Context, in *WaitForApplicationRequest, opts ...grpc.CallOption) (*WaitForApplicationResponse, error) @@ -170,7 +331,7 @@ func NewPerReplicaClient(cc *grpc.ClientConn) PerReplicaClient { func (c *perReplicaClient) CollectChecksum(ctx context.Context, in *CollectChecksumRequest, opts ...grpc.CallOption) (*CollectChecksumResponse, error) { out := new(CollectChecksumResponse) - err := grpc.Invoke(ctx, "/cockroach.storage.PerReplica/CollectChecksum", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/cockroach.storage.PerReplica/CollectChecksum", in, out, opts...) if err != nil { return nil, err } @@ -179,7 +340,7 @@ func (c *perReplicaClient) CollectChecksum(ctx context.Context, in *CollectCheck func (c *perReplicaClient) WaitForApplication(ctx context.Context, in *WaitForApplicationRequest, opts ...grpc.CallOption) (*WaitForApplicationResponse, error) { out := new(WaitForApplicationResponse) - err := grpc.Invoke(ctx, "/cockroach.storage.PerReplica/WaitForApplication", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/cockroach.storage.PerReplica/WaitForApplication", in, out, opts...) if err != nil { return nil, err } @@ -188,15 +349,14 @@ func (c *perReplicaClient) WaitForApplication(ctx context.Context, in *WaitForAp func (c *perReplicaClient) WaitForReplicaInit(ctx context.Context, in *WaitForReplicaInitRequest, opts ...grpc.CallOption) (*WaitForReplicaInitResponse, error) { out := new(WaitForReplicaInitResponse) - err := grpc.Invoke(ctx, "/cockroach.storage.PerReplica/WaitForReplicaInit", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/cockroach.storage.PerReplica/WaitForReplicaInit", in, out, opts...) if err != nil { return nil, err } return out, nil } -// Server API for PerReplica service - +// PerReplicaServer is the server API for PerReplica service. type PerReplicaServer interface { CollectChecksum(context.Context, *CollectChecksumRequest) (*CollectChecksumResponse, error) WaitForApplication(context.Context, *WaitForApplicationRequest) (*WaitForApplicationResponse, error) @@ -510,6 +670,9 @@ func encodeVarintApi(dAtA []byte, offset int, v uint64) int { return offset + 1 } func (m *StoreRequestHeader) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.NodeID != 0 { @@ -522,6 +685,9 @@ func (m *StoreRequestHeader) Size() (n int) { } func (m *CollectChecksumRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.StoreRequestHeader.Size() @@ -539,6 +705,9 @@ func (m *CollectChecksumRequest) Size() (n int) { } func (m *CollectChecksumResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Checksum) @@ -555,6 +724,9 @@ func (m *CollectChecksumResponse) Size() (n int) { } func (m *WaitForApplicationRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.StoreRequestHeader.Size() @@ -569,12 +741,18 @@ func (m *WaitForApplicationRequest) Size() (n int) { } func (m *WaitForApplicationResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l return n } func (m *WaitForReplicaInitRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.StoreRequestHeader.Size() @@ -586,6 +764,9 @@ func (m *WaitForReplicaInitRequest) Size() (n int) { } func (m *WaitForReplicaInitResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l return n @@ -939,7 +1120,7 @@ func (m *CollectChecksumResponse) Unmarshal(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Snapshot == nil { - m.Snapshot = &cockroach_roachpb1.RaftSnapshotData{} + m.Snapshot = &roachpb.RaftSnapshotData{} } if err := m.Snapshot.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err @@ -1418,9 +1599,9 @@ var ( ErrIntOverflowApi = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("storage/api.proto", fileDescriptorApi) } +func init() { proto.RegisterFile("storage/api.proto", fileDescriptor_api_07e0411937110fca) } -var fileDescriptorApi = []byte{ +var fileDescriptor_api_07e0411937110fca = []byte{ // 628 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0x55, 0x5f, 0x6f, 0xd3, 0x3e, 0x14, 0xad, 0xf7, 0xa7, 0xad, 0xdc, 0x9f, 0xf4, 0xd3, 0x2c, 0x34, 0x46, 0x41, 0xcd, 0xc8, 0x84, diff --git a/pkg/storage/batcheval/cmd_refresh_range_test.go b/pkg/storage/batcheval/cmd_refresh_range_test.go index 43fe8af2441e..ad0d1d753656 100644 --- a/pkg/storage/batcheval/cmd_refresh_range_test.go +++ b/pkg/storage/batcheval/cmd_refresh_range_test.go @@ -123,7 +123,7 @@ func TestRefreshRangeTimeBoundIterator(t *testing.T) { } else if intent != nil { t.Fatalf("got unexpected intent: %v", intent) } else if !val.EqualData(v) { - t.Fatalf("expected %s, got %s", v, val) + t.Fatalf("expected %v, got %v", v, val) } // Now the real test: a transaction at ts2 has been pushed to ts3 diff --git a/pkg/storage/closedts/ctpb/entry.pb.go b/pkg/storage/closedts/ctpb/entry.pb.go index 7bbec1fe5156..cb30dc02ad11 100644 --- a/pkg/storage/closedts/ctpb/entry.pb.go +++ b/pkg/storage/closedts/ctpb/entry.pb.go @@ -1,29 +1,21 @@ // Code generated by protoc-gen-gogo. DO NOT EDIT. // source: storage/closedts/ctpb/entry.proto -/* - Package ctpb is a generated protocol buffer package. - - It is generated from these files: - storage/closedts/ctpb/entry.proto - - It has these top-level messages: - Entry - Reaction -*/ package ctpb import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" -import cockroach_util_hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" +import hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" import github_com_cockroachdb_cockroach_pkg_roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" -import context "context" -import grpc "google.golang.org/grpc" +import ( + context "context" + grpc "google.golang.org/grpc" +) -import sortkeys "github.com/gogo/protobuf/sortkeys" +import github_com_gogo_protobuf_sortkeys "github.com/gogo/protobuf/sortkeys" import io "io" @@ -50,8 +42,8 @@ const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package // the state accumulated since the last full Entry is the true full state. type Entry struct { Epoch Epoch `protobuf:"varint,1,opt,name=epoch,proto3,casttype=Epoch" json:"epoch,omitempty"` - ClosedTimestamp cockroach_util_hlc.Timestamp `protobuf:"bytes,2,opt,name=closed_timestamp,json=closedTimestamp" json:"closed_timestamp"` - MLAI map[github_com_cockroachdb_cockroach_pkg_roachpb.RangeID]LAI `protobuf:"bytes,3,rep,name=mlai,castkey=github.com/cockroachdb/cockroach/pkg/roachpb.RangeID,castvalue=LAI" json:"mlai,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` + ClosedTimestamp hlc.Timestamp `protobuf:"bytes,2,opt,name=closed_timestamp,json=closedTimestamp,proto3" json:"closed_timestamp"` + MLAI map[github_com_cockroachdb_cockroach_pkg_roachpb.RangeID]LAI `protobuf:"bytes,3,rep,name=mlai,proto3,castkey=github.com/cockroachdb/cockroach/pkg/roachpb.RangeID,castvalue=LAI" json:"mlai,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` // Full is true if the emitter promises that any future write to any range // mentioned in this Entry will be reflected in a subsequent Entry before any // stale follower reads are possible. For example, if range 1 is assigned an @@ -62,12 +54,38 @@ type Entry struct { // In practice, a Full message is received when a stream of Entries is first // established (or the Epoch changes), and all other updates are incremental // (i.e. not Full). - Full bool `protobuf:"varint,4,opt,name=full,proto3" json:"full,omitempty"` + Full bool `protobuf:"varint,4,opt,name=full,proto3" json:"full,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Entry) Reset() { *m = Entry{} } +func (*Entry) ProtoMessage() {} +func (*Entry) Descriptor() ([]byte, []int) { + return fileDescriptor_entry_e42ec3f943fd7ecf, []int{0} +} +func (m *Entry) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Entry) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *Entry) XXX_Merge(src proto.Message) { + xxx_messageInfo_Entry.Merge(dst, src) +} +func (m *Entry) XXX_Size() int { + return m.Size() +} +func (m *Entry) XXX_DiscardUnknown() { + xxx_messageInfo_Entry.DiscardUnknown(m) } -func (m *Entry) Reset() { *m = Entry{} } -func (*Entry) ProtoMessage() {} -func (*Entry) Descriptor() ([]byte, []int) { return fileDescriptorEntry, []int{0} } +var xxx_messageInfo_Entry proto.InternalMessageInfo // Reactions flow in the direction opposite to Entries and request for ranges to // be included in the next Entry. Under rare circumstances, ranges may be omitted @@ -75,15 +93,42 @@ func (*Entry) Descriptor() ([]byte, []int) { return fileDescriptorEntry, []int{0 // fail. The Reaction mechanism serves to explicitly request the missing information // when that happens. type Reaction struct { - Requested []github_com_cockroachdb_cockroach_pkg_roachpb.RangeID `protobuf:"varint,1,rep,packed,name=Requested,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" json:"Requested,omitempty"` + Requested []github_com_cockroachdb_cockroach_pkg_roachpb.RangeID `protobuf:"varint,1,rep,packed,name=Requested,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" json:"Requested,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Reaction) Reset() { *m = Reaction{} } +func (*Reaction) ProtoMessage() {} +func (*Reaction) Descriptor() ([]byte, []int) { + return fileDescriptor_entry_e42ec3f943fd7ecf, []int{1} +} +func (m *Reaction) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Reaction) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *Reaction) XXX_Merge(src proto.Message) { + xxx_messageInfo_Reaction.Merge(dst, src) +} +func (m *Reaction) XXX_Size() int { + return m.Size() +} +func (m *Reaction) XXX_DiscardUnknown() { + xxx_messageInfo_Reaction.DiscardUnknown(m) } -func (m *Reaction) Reset() { *m = Reaction{} } -func (*Reaction) ProtoMessage() {} -func (*Reaction) Descriptor() ([]byte, []int) { return fileDescriptorEntry, []int{1} } +var xxx_messageInfo_Reaction proto.InternalMessageInfo func init() { proto.RegisterType((*Entry)(nil), "cockroach.storage.ctupdate.Entry") + proto.RegisterMapType((map[github_com_cockroachdb_cockroach_pkg_roachpb.RangeID]LAI)(nil), "cockroach.storage.ctupdate.Entry.MlaiEntry") proto.RegisterType((*Reaction)(nil), "cockroach.storage.ctupdate.Reaction") } @@ -95,8 +140,9 @@ var _ grpc.ClientConn // is compatible with the grpc package it is being compiled against. const _ = grpc.SupportPackageIsVersion4 -// Client API for ClosedTimestamp service - +// ClosedTimestampClient is the client API for ClosedTimestamp service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream. type ClosedTimestampClient interface { Get(ctx context.Context, opts ...grpc.CallOption) (ClosedTimestamp_GetClient, error) } @@ -110,7 +156,7 @@ func NewClosedTimestampClient(cc *grpc.ClientConn) ClosedTimestampClient { } func (c *closedTimestampClient) Get(ctx context.Context, opts ...grpc.CallOption) (ClosedTimestamp_GetClient, error) { - stream, err := grpc.NewClientStream(ctx, &_ClosedTimestamp_serviceDesc.Streams[0], c.cc, "/cockroach.storage.ctupdate.ClosedTimestamp/Get", opts...) + stream, err := c.cc.NewStream(ctx, &_ClosedTimestamp_serviceDesc.Streams[0], "/cockroach.storage.ctupdate.ClosedTimestamp/Get", opts...) if err != nil { return nil, err } @@ -140,8 +186,7 @@ func (x *closedTimestampGetClient) Recv() (*Entry, error) { return m, nil } -// Server API for ClosedTimestamp service - +// ClosedTimestampServer is the server API for ClosedTimestamp service. type ClosedTimestampServer interface { Get(ClosedTimestamp_GetServer) error } @@ -224,7 +269,7 @@ func (m *Entry) MarshalTo(dAtA []byte) (int, error) { for k := range m.MLAI { keysForMLAI = append(keysForMLAI, int32(k)) } - sortkeys.Int32s(keysForMLAI) + github_com_gogo_protobuf_sortkeys.Int32s(keysForMLAI) for _, k := range keysForMLAI { dAtA[i] = 0x1a i++ @@ -298,6 +343,9 @@ func encodeVarintEntry(dAtA []byte, offset int, v uint64) int { return offset + 1 } func (m *Entry) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Epoch != 0 { @@ -320,6 +368,9 @@ func (m *Entry) Size() (n int) { } func (m *Reaction) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.Requested) > 0 { @@ -630,6 +681,17 @@ func (m *Reaction) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } + var elementCount int + var count int + for _, integer := range dAtA { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.Requested) == 0 { + m.Requested = make([]github_com_cockroachdb_cockroach_pkg_roachpb.RangeID, 0, elementCount) + } for iNdEx < postIndex { var v github_com_cockroachdb_cockroach_pkg_roachpb.RangeID for shift := uint(0); ; shift += 7 { @@ -777,9 +839,11 @@ var ( ErrIntOverflowEntry = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("storage/closedts/ctpb/entry.proto", fileDescriptorEntry) } +func init() { + proto.RegisterFile("storage/closedts/ctpb/entry.proto", fileDescriptor_entry_e42ec3f943fd7ecf) +} -var fileDescriptorEntry = []byte{ +var fileDescriptor_entry_e42ec3f943fd7ecf = []byte{ // 459 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x52, 0x4f, 0x6b, 0xd4, 0x40, 0x1c, 0xcd, 0x6c, 0x12, 0xe9, 0x4e, 0x0f, 0x2d, 0x43, 0x0f, 0x21, 0x68, 0x92, 0x2e, 0x1e, 0x02, diff --git a/pkg/storage/consistency_queue_test.go b/pkg/storage/consistency_queue_test.go index d59ae7bea7b3..bf2f926ff27e 100644 --- a/pkg/storage/consistency_queue_test.go +++ b/pkg/storage/consistency_queue_test.go @@ -186,7 +186,7 @@ func TestCheckConsistencyInconsistent(t *testing.T) { sc.ConsistencyTestingKnobs.BadChecksumReportDiff = func(s roachpb.StoreIdent, diff storage.ReplicaSnapshotDiffSlice) { if s != *mtc.Store(0).Ident { - t.Errorf("BadChecksumReportDiff called from follower (StoreIdent = %s)", s) + t.Errorf("BadChecksumReportDiff called from follower (StoreIdent = %v)", s) return } if len(diff) != 1 { @@ -219,7 +219,7 @@ func TestCheckConsistencyInconsistent(t *testing.T) { notifyPanic := make(chan struct{}, 1) sc.ConsistencyTestingKnobs.BadChecksumPanic = func(s roachpb.StoreIdent) { if s != *mtc.Store(0).Ident { - t.Errorf("BadChecksumPanic called from follower (StoreIdent = %s)", s) + t.Errorf("BadChecksumPanic called from follower (StoreIdent = %v)", s) return } notifyPanic <- struct{}{} diff --git a/pkg/storage/engine/enginepb/file_registry.pb.go b/pkg/storage/engine/enginepb/file_registry.pb.go index 3f7d4996c55d..172222619a4c 100644 --- a/pkg/storage/engine/enginepb/file_registry.pb.go +++ b/pkg/storage/engine/enginepb/file_registry.pb.go @@ -1,40 +1,13 @@ // Code generated by protoc-gen-gogo. DO NOT EDIT. // source: storage/engine/enginepb/file_registry.proto -/* - Package enginepb is a generated protocol buffer package. - - It is generated from these files: - storage/engine/enginepb/file_registry.proto - storage/engine/enginepb/mvcc.proto - storage/engine/enginepb/mvcc3.proto - storage/engine/enginepb/rocksdb.proto - - It has these top-level messages: - FileRegistry - FileEntry - MVCCMetadata - MVCCStats - TxnMeta - MVCCStatsDelta - MVCCPersistentStats - RangeAppliedState - MVCCWriteValueOp - MVCCWriteIntentOp - MVCCUpdateIntentOp - MVCCCommitIntentOp - MVCCAbortIntentOp - MVCCLogicalOp - SSTUserProperties - SSTUserPropertiesCollection -*/ package enginepb import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" -import sortkeys "github.com/gogo/protobuf/sortkeys" +import github_com_gogo_protobuf_sortkeys "github.com/gogo/protobuf/sortkeys" import io "io" @@ -66,7 +39,9 @@ var RegistryVersion_value = map[string]int32{ func (x RegistryVersion) String() string { return proto.EnumName(RegistryVersion_name, int32(x)) } -func (RegistryVersion) EnumDescriptor() ([]byte, []int) { return fileDescriptorFileRegistry, []int{0} } +func (RegistryVersion) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_file_registry_4c7143274eb677fa, []int{0} +} // EnvType determines which rocksdb::Env is used and for what purpose. type EnvType int32 @@ -97,7 +72,9 @@ var EnvType_value = map[string]int32{ func (x EnvType) String() string { return proto.EnumName(EnvType_name, int32(x)) } -func (EnvType) EnumDescriptor() ([]byte, []int) { return fileDescriptorFileRegistry, []int{1} } +func (EnvType) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_file_registry_4c7143274eb677fa, []int{1} +} // Registry describes how a files are handled. This includes the // rockdb::Env responsible for each file as well as opaque env details. @@ -108,13 +85,39 @@ type FileRegistry struct { // Filename is relative to the rocksdb dir if the file is inside it. // Otherwise it is an absolute path. // TODO(mberhault): figure out if we need anything special for Windows. - Files map[string]*FileEntry `protobuf:"bytes,2,rep,name=files" json:"files,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value"` + Files map[string]*FileEntry `protobuf:"bytes,2,rep,name=files,proto3" json:"files,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *FileRegistry) Reset() { *m = FileRegistry{} } +func (m *FileRegistry) String() string { return proto.CompactTextString(m) } +func (*FileRegistry) ProtoMessage() {} +func (*FileRegistry) Descriptor() ([]byte, []int) { + return fileDescriptor_file_registry_4c7143274eb677fa, []int{0} +} +func (m *FileRegistry) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *FileRegistry) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *FileRegistry) XXX_Merge(src proto.Message) { + xxx_messageInfo_FileRegistry.Merge(dst, src) +} +func (m *FileRegistry) XXX_Size() int { + return m.Size() +} +func (m *FileRegistry) XXX_DiscardUnknown() { + xxx_messageInfo_FileRegistry.DiscardUnknown(m) } -func (m *FileRegistry) Reset() { *m = FileRegistry{} } -func (m *FileRegistry) String() string { return proto.CompactTextString(m) } -func (*FileRegistry) ProtoMessage() {} -func (*FileRegistry) Descriptor() ([]byte, []int) { return fileDescriptorFileRegistry, []int{0} } +var xxx_messageInfo_FileRegistry proto.InternalMessageInfo type FileEntry struct { // Env type identifies which rocksdb::Env is responsible for this file. @@ -122,16 +125,43 @@ type FileEntry struct { // Env-specific fields for non-0 env. These are known by CCL code only. // This is a serialized protobuf. We cannot use protobuf.Any since we use // MessageLite in C++. - EncryptionSettings []byte `protobuf:"bytes,2,opt,name=encryption_settings,json=encryptionSettings,proto3" json:"encryption_settings,omitempty"` + EncryptionSettings []byte `protobuf:"bytes,2,opt,name=encryption_settings,json=encryptionSettings,proto3" json:"encryption_settings,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *FileEntry) Reset() { *m = FileEntry{} } -func (m *FileEntry) String() string { return proto.CompactTextString(m) } -func (*FileEntry) ProtoMessage() {} -func (*FileEntry) Descriptor() ([]byte, []int) { return fileDescriptorFileRegistry, []int{1} } +func (m *FileEntry) Reset() { *m = FileEntry{} } +func (m *FileEntry) String() string { return proto.CompactTextString(m) } +func (*FileEntry) ProtoMessage() {} +func (*FileEntry) Descriptor() ([]byte, []int) { + return fileDescriptor_file_registry_4c7143274eb677fa, []int{1} +} +func (m *FileEntry) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *FileEntry) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *FileEntry) XXX_Merge(src proto.Message) { + xxx_messageInfo_FileEntry.Merge(dst, src) +} +func (m *FileEntry) XXX_Size() int { + return m.Size() +} +func (m *FileEntry) XXX_DiscardUnknown() { + xxx_messageInfo_FileEntry.DiscardUnknown(m) +} + +var xxx_messageInfo_FileEntry proto.InternalMessageInfo func init() { proto.RegisterType((*FileRegistry)(nil), "cockroach.storage.engine.enginepb.FileRegistry") + proto.RegisterMapType((map[string]*FileEntry)(nil), "cockroach.storage.engine.enginepb.FileRegistry.FilesEntry") proto.RegisterType((*FileEntry)(nil), "cockroach.storage.engine.enginepb.FileEntry") proto.RegisterEnum("cockroach.storage.engine.enginepb.RegistryVersion", RegistryVersion_name, RegistryVersion_value) proto.RegisterEnum("cockroach.storage.engine.enginepb.EnvType", EnvType_name, EnvType_value) @@ -161,7 +191,7 @@ func (m *FileRegistry) MarshalTo(dAtA []byte) (int, error) { for k := range m.Files { keysForFiles = append(keysForFiles, string(k)) } - sortkeys.Strings(keysForFiles) + github_com_gogo_protobuf_sortkeys.Strings(keysForFiles) for _, k := range keysForFiles { dAtA[i] = 0x12 i++ @@ -231,6 +261,9 @@ func encodeVarintFileRegistry(dAtA []byte, offset int, v uint64) int { return offset + 1 } func (m *FileRegistry) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Version != 0 { @@ -253,6 +286,9 @@ func (m *FileRegistry) Size() (n int) { } func (m *FileEntry) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.EnvType != 0 { @@ -676,10 +712,10 @@ var ( ) func init() { - proto.RegisterFile("storage/engine/enginepb/file_registry.proto", fileDescriptorFileRegistry) + proto.RegisterFile("storage/engine/enginepb/file_registry.proto", fileDescriptor_file_registry_4c7143274eb677fa) } -var fileDescriptorFileRegistry = []byte{ +var fileDescriptor_file_registry_4c7143274eb677fa = []byte{ // 371 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x92, 0xbf, 0x8e, 0xd3, 0x40, 0x10, 0xc6, 0xbd, 0x0e, 0x21, 0xf1, 0x24, 0x80, 0xb5, 0x50, 0x44, 0x41, 0xb2, 0x42, 0xaa, 0x28, diff --git a/pkg/storage/engine/enginepb/mvcc.pb.go b/pkg/storage/engine/enginepb/mvcc.pb.go index 0b62942f8665..a613bccd067c 100644 --- a/pkg/storage/engine/enginepb/mvcc.pb.go +++ b/pkg/storage/engine/enginepb/mvcc.pb.go @@ -6,11 +6,11 @@ package enginepb import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" -import cockroach_util_hlc1 "github.com/cockroachdb/cockroach/pkg/util/hlc" +import hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" import bytes "bytes" -import binary "encoding/binary" +import encoding_binary "encoding/binary" import io "io" @@ -19,6 +19,12 @@ var _ = proto.Marshal var _ = fmt.Errorf var _ = math.Inf +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package + // MVCCMetadata holds MVCC metadata for a key. Used by storage/engine/mvcc.go. // An MVCCMetadata is stored for a versioned key while there is an intent on // that key. @@ -28,7 +34,7 @@ type MVCCMetadata struct { // value that may have multiple versions. For values which may have // only one version, the data is stored inline (via raw_bytes), and // timestamp is set to zero. - Timestamp cockroach_util_hlc1.LegacyTimestamp `protobuf:"bytes,2,opt,name=timestamp" json:"timestamp"` + Timestamp hlc.LegacyTimestamp `protobuf:"bytes,2,opt,name=timestamp" json:"timestamp"` // Is the most recent value a deletion tombstone? Deleted bool `protobuf:"varint,3,opt,name=deleted" json:"deleted"` // The size in bytes of the most recent encoded key. @@ -50,13 +56,39 @@ type MVCCMetadata struct { IntentHistory []MVCCMetadata_SequencedIntent `protobuf:"bytes,8,rep,name=intent_history,json=intentHistory" json:"intent_history"` // This provides a measure of protection against replays caused by // Raft duplicating merge commands. - MergeTimestamp *cockroach_util_hlc1.LegacyTimestamp `protobuf:"bytes,7,opt,name=merge_timestamp,json=mergeTimestamp" json:"merge_timestamp,omitempty"` + MergeTimestamp *hlc.LegacyTimestamp `protobuf:"bytes,7,opt,name=merge_timestamp,json=mergeTimestamp" json:"merge_timestamp,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *MVCCMetadata) Reset() { *m = MVCCMetadata{} } +func (m *MVCCMetadata) String() string { return proto.CompactTextString(m) } +func (*MVCCMetadata) ProtoMessage() {} +func (*MVCCMetadata) Descriptor() ([]byte, []int) { + return fileDescriptor_mvcc_09e838457798996e, []int{0} +} +func (m *MVCCMetadata) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *MVCCMetadata) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *MVCCMetadata) XXX_Merge(src proto.Message) { + xxx_messageInfo_MVCCMetadata.Merge(dst, src) +} +func (m *MVCCMetadata) XXX_Size() int { + return m.Size() +} +func (m *MVCCMetadata) XXX_DiscardUnknown() { + xxx_messageInfo_MVCCMetadata.DiscardUnknown(m) } -func (m *MVCCMetadata) Reset() { *m = MVCCMetadata{} } -func (m *MVCCMetadata) String() string { return proto.CompactTextString(m) } -func (*MVCCMetadata) ProtoMessage() {} -func (*MVCCMetadata) Descriptor() ([]byte, []int) { return fileDescriptorMvcc, []int{0} } +var xxx_messageInfo_MVCCMetadata proto.InternalMessageInfo // SequencedIntent stores a value at a given key and the sequence number it was // written at - to be stored in an IntentHistory of a key during a transaction. @@ -67,15 +99,39 @@ type MVCCMetadata_SequencedIntent struct { Sequence int32 `protobuf:"varint,1,opt,name=sequence" json:"sequence"` // Value is the value written to the key as part of the transaction at // the above Sequence. - Value []byte `protobuf:"bytes,2,opt,name=value" json:"value,omitempty"` + Value []byte `protobuf:"bytes,2,opt,name=value" json:"value,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *MVCCMetadata_SequencedIntent) Reset() { *m = MVCCMetadata_SequencedIntent{} } func (m *MVCCMetadata_SequencedIntent) String() string { return proto.CompactTextString(m) } func (*MVCCMetadata_SequencedIntent) ProtoMessage() {} func (*MVCCMetadata_SequencedIntent) Descriptor() ([]byte, []int) { - return fileDescriptorMvcc, []int{0, 0} + return fileDescriptor_mvcc_09e838457798996e, []int{0, 0} +} +func (m *MVCCMetadata_SequencedIntent) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *MVCCMetadata_SequencedIntent) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *MVCCMetadata_SequencedIntent) XXX_Merge(src proto.Message) { + xxx_messageInfo_MVCCMetadata_SequencedIntent.Merge(dst, src) +} +func (m *MVCCMetadata_SequencedIntent) XXX_Size() int { + return m.Size() } +func (m *MVCCMetadata_SequencedIntent) XXX_DiscardUnknown() { + xxx_messageInfo_MVCCMetadata_SequencedIntent.DiscardUnknown(m) +} + +var xxx_messageInfo_MVCCMetadata_SequencedIntent proto.InternalMessageInfo // MVCCStats tracks byte and instance counts for various groups of keys, // values, or key-value pairs; see the field comments for details. @@ -159,13 +215,39 @@ type MVCCStats struct { // and is prefixed by either LocalRangeIDPrefix or LocalRangePrefix. SysBytes int64 `protobuf:"fixed64,12,opt,name=sys_bytes,json=sysBytes" json:"sys_bytes"` // sys_count is the number of meta keys tracked under sys_bytes. - SysCount int64 `protobuf:"fixed64,13,opt,name=sys_count,json=sysCount" json:"sys_count"` + SysCount int64 `protobuf:"fixed64,13,opt,name=sys_count,json=sysCount" json:"sys_count"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *MVCCStats) Reset() { *m = MVCCStats{} } -func (m *MVCCStats) String() string { return proto.CompactTextString(m) } -func (*MVCCStats) ProtoMessage() {} -func (*MVCCStats) Descriptor() ([]byte, []int) { return fileDescriptorMvcc, []int{1} } +func (m *MVCCStats) Reset() { *m = MVCCStats{} } +func (m *MVCCStats) String() string { return proto.CompactTextString(m) } +func (*MVCCStats) ProtoMessage() {} +func (*MVCCStats) Descriptor() ([]byte, []int) { + return fileDescriptor_mvcc_09e838457798996e, []int{1} +} +func (m *MVCCStats) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *MVCCStats) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *MVCCStats) XXX_Merge(src proto.Message) { + xxx_messageInfo_MVCCStats.Merge(dst, src) +} +func (m *MVCCStats) XXX_Size() int { + return m.Size() +} +func (m *MVCCStats) XXX_DiscardUnknown() { + xxx_messageInfo_MVCCStats.DiscardUnknown(m) +} + +var xxx_messageInfo_MVCCStats proto.InternalMessageInfo func init() { proto.RegisterType((*MVCCMetadata)(nil), "cockroach.storage.engine.enginepb.MVCCMetadata") @@ -434,55 +516,55 @@ func (m *MVCCStats) MarshalTo(dAtA []byte) (int, error) { _ = l dAtA[i] = 0x9 i++ - binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.LastUpdateNanos)) + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.LastUpdateNanos)) i += 8 dAtA[i] = 0x11 i++ - binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.IntentAge)) + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.IntentAge)) i += 8 dAtA[i] = 0x19 i++ - binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.GCBytesAge)) + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.GCBytesAge)) i += 8 dAtA[i] = 0x21 i++ - binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.LiveBytes)) + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.LiveBytes)) i += 8 dAtA[i] = 0x29 i++ - binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.LiveCount)) + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.LiveCount)) i += 8 dAtA[i] = 0x31 i++ - binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.KeyBytes)) + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.KeyBytes)) i += 8 dAtA[i] = 0x39 i++ - binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.KeyCount)) + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.KeyCount)) i += 8 dAtA[i] = 0x41 i++ - binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.ValBytes)) + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.ValBytes)) i += 8 dAtA[i] = 0x49 i++ - binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.ValCount)) + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.ValCount)) i += 8 dAtA[i] = 0x51 i++ - binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.IntentBytes)) + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.IntentBytes)) i += 8 dAtA[i] = 0x59 i++ - binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.IntentCount)) + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.IntentCount)) i += 8 dAtA[i] = 0x61 i++ - binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.SysBytes)) + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.SysBytes)) i += 8 dAtA[i] = 0x69 i++ - binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.SysCount)) + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.SysCount)) i += 8 dAtA[i] = 0x70 i++ @@ -509,7 +591,7 @@ func NewPopulatedMVCCMetadata(r randyMvcc, easy bool) *MVCCMetadata { if r.Intn(10) != 0 { this.Txn = NewPopulatedTxnMeta(r, easy) } - v1 := cockroach_util_hlc1.NewPopulatedLegacyTimestamp(r, easy) + v1 := hlc.NewPopulatedLegacyTimestamp(r, easy) this.Timestamp = *v1 this.Deleted = bool(bool(r.Intn(2) == 0)) this.KeyBytes = int64(r.Int63()) @@ -528,7 +610,7 @@ func NewPopulatedMVCCMetadata(r randyMvcc, easy bool) *MVCCMetadata { } } if r.Intn(10) != 0 { - this.MergeTimestamp = cockroach_util_hlc1.NewPopulatedLegacyTimestamp(r, easy) + this.MergeTimestamp = hlc.NewPopulatedLegacyTimestamp(r, easy) } if r.Intn(10) != 0 { v3 := r.Intn(5) @@ -694,6 +776,9 @@ func encodeVarintPopulateMvcc(dAtA []byte, v uint64) []byte { return dAtA } func (m *MVCCMetadata) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Txn != nil { @@ -723,6 +808,9 @@ func (m *MVCCMetadata) Size() (n int) { } func (m *MVCCMetadata_SequencedIntent) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovMvcc(uint64(m.Sequence)) @@ -734,6 +822,9 @@ func (m *MVCCMetadata_SequencedIntent) Size() (n int) { } func (m *MVCCStats) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 9 @@ -974,7 +1065,7 @@ func (m *MVCCMetadata) Unmarshal(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.MergeTimestamp == nil { - m.MergeTimestamp = &cockroach_util_hlc1.LegacyTimestamp{} + m.MergeTimestamp = &hlc.LegacyTimestamp{} } if err := m.MergeTimestamp.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err @@ -1169,7 +1260,7 @@ func (m *MVCCStats) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - m.LastUpdateNanos = int64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + m.LastUpdateNanos = int64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 case 2: if wireType != 1 { @@ -1179,7 +1270,7 @@ func (m *MVCCStats) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - m.IntentAge = int64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + m.IntentAge = int64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 case 3: if wireType != 1 { @@ -1189,7 +1280,7 @@ func (m *MVCCStats) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - m.GCBytesAge = int64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + m.GCBytesAge = int64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 case 4: if wireType != 1 { @@ -1199,7 +1290,7 @@ func (m *MVCCStats) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - m.LiveBytes = int64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + m.LiveBytes = int64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 case 5: if wireType != 1 { @@ -1209,7 +1300,7 @@ func (m *MVCCStats) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - m.LiveCount = int64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + m.LiveCount = int64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 case 6: if wireType != 1 { @@ -1219,7 +1310,7 @@ func (m *MVCCStats) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - m.KeyBytes = int64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + m.KeyBytes = int64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 case 7: if wireType != 1 { @@ -1229,7 +1320,7 @@ func (m *MVCCStats) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - m.KeyCount = int64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + m.KeyCount = int64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 case 8: if wireType != 1 { @@ -1239,7 +1330,7 @@ func (m *MVCCStats) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - m.ValBytes = int64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + m.ValBytes = int64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 case 9: if wireType != 1 { @@ -1249,7 +1340,7 @@ func (m *MVCCStats) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - m.ValCount = int64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + m.ValCount = int64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 case 10: if wireType != 1 { @@ -1259,7 +1350,7 @@ func (m *MVCCStats) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - m.IntentBytes = int64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + m.IntentBytes = int64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 case 11: if wireType != 1 { @@ -1269,7 +1360,7 @@ func (m *MVCCStats) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - m.IntentCount = int64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + m.IntentCount = int64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 case 12: if wireType != 1 { @@ -1279,7 +1370,7 @@ func (m *MVCCStats) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - m.SysBytes = int64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + m.SysBytes = int64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 case 13: if wireType != 1 { @@ -1289,7 +1380,7 @@ func (m *MVCCStats) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - m.SysCount = int64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + m.SysCount = int64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 case 14: if wireType != 0 { @@ -1437,9 +1528,11 @@ var ( ErrIntOverflowMvcc = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("storage/engine/enginepb/mvcc.proto", fileDescriptorMvcc) } +func init() { + proto.RegisterFile("storage/engine/enginepb/mvcc.proto", fileDescriptor_mvcc_09e838457798996e) +} -var fileDescriptorMvcc = []byte{ +var fileDescriptor_mvcc_09e838457798996e = []byte{ // 632 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x93, 0x4f, 0x4f, 0xd4, 0x40, 0x14, 0xc0, 0xa9, 0xbb, 0x40, 0x77, 0xb6, 0x80, 0x4c, 0x38, 0x34, 0x98, 0x74, 0xcb, 0x72, 0x70, diff --git a/pkg/storage/engine/enginepb/mvcc3.pb.go b/pkg/storage/engine/enginepb/mvcc3.pb.go index fb5921e68a46..dc7f89938cbe 100644 --- a/pkg/storage/engine/enginepb/mvcc3.pb.go +++ b/pkg/storage/engine/enginepb/mvcc3.pb.go @@ -6,13 +6,13 @@ package enginepb import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" -import cockroach_util_hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" +import hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" import github_com_cockroachdb_cockroach_pkg_util_uuid "github.com/cockroachdb/cockroach/pkg/util/uuid" import bytes "bytes" -import binary "encoding/binary" +import encoding_binary "encoding/binary" import io "io" @@ -21,6 +21,12 @@ var _ = proto.Marshal var _ = fmt.Errorf var _ = math.Inf +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package + // TxnMeta is the metadata of a Transaction record. type TxnMeta struct { // id is a unique UUID value which identifies the transaction. @@ -88,20 +94,46 @@ type TxnMeta struct { // value will need to be rewritten at the forwarded timestamp if the // transaction commits. // - Timestamp cockroach_util_hlc.Timestamp `protobuf:"bytes,5,opt,name=timestamp" json:"timestamp"` - Priority int32 `protobuf:"varint,6,opt,name=priority,proto3" json:"priority,omitempty"` + Timestamp hlc.Timestamp `protobuf:"bytes,5,opt,name=timestamp,proto3" json:"timestamp"` + Priority int32 `protobuf:"varint,6,opt,name=priority,proto3" json:"priority,omitempty"` // A one-indexed sequence number which is increased on each request // sent as part of the transaction. When set in the header of a batch // of requests, the value will correspond to the sequence number of the // last request. Used to prevent replay and out-of-order application // protection (by means of a transaction retry). - Sequence int32 `protobuf:"varint,7,opt,name=sequence,proto3" json:"sequence,omitempty"` + Sequence int32 `protobuf:"varint,7,opt,name=sequence,proto3" json:"sequence,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *TxnMeta) Reset() { *m = TxnMeta{} } -func (m *TxnMeta) String() string { return proto.CompactTextString(m) } -func (*TxnMeta) ProtoMessage() {} -func (*TxnMeta) Descriptor() ([]byte, []int) { return fileDescriptorMvcc3, []int{0} } +func (m *TxnMeta) Reset() { *m = TxnMeta{} } +func (m *TxnMeta) String() string { return proto.CompactTextString(m) } +func (*TxnMeta) ProtoMessage() {} +func (*TxnMeta) Descriptor() ([]byte, []int) { + return fileDescriptor_mvcc3_e549160c12464c1c, []int{0} +} +func (m *TxnMeta) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *TxnMeta) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *TxnMeta) XXX_Merge(src proto.Message) { + xxx_messageInfo_TxnMeta.Merge(dst, src) +} +func (m *TxnMeta) XXX_Size() int { + return m.Size() +} +func (m *TxnMeta) XXX_DiscardUnknown() { + xxx_messageInfo_TxnMeta.DiscardUnknown(m) +} + +var xxx_messageInfo_TxnMeta proto.InternalMessageInfo // MVCCStatsDelta is convertible to MVCCStats, but uses signed variable width // encodings for most fields that make it more efficient to store negative @@ -110,52 +142,104 @@ type MVCCStatsDelta struct { // TODO(nvanbenschoten): now that we've split MVCCPersistentStats // from this MVCCStatsDelta type, we can turn contains_estimates // into a three-valued type ('UNCHANGED', 'NO', and 'YES'). - ContainsEstimates bool `protobuf:"varint,14,opt,name=contains_estimates,json=containsEstimates,proto3" json:"contains_estimates,omitempty"` - LastUpdateNanos int64 `protobuf:"fixed64,1,opt,name=last_update_nanos,json=lastUpdateNanos,proto3" json:"last_update_nanos,omitempty"` - IntentAge int64 `protobuf:"fixed64,2,opt,name=intent_age,json=intentAge,proto3" json:"intent_age,omitempty"` - GCBytesAge int64 `protobuf:"fixed64,3,opt,name=gc_bytes_age,json=gcBytesAge,proto3" json:"gc_bytes_age,omitempty"` - LiveBytes int64 `protobuf:"zigzag64,4,opt,name=live_bytes,json=liveBytes,proto3" json:"live_bytes,omitempty"` - LiveCount int64 `protobuf:"zigzag64,5,opt,name=live_count,json=liveCount,proto3" json:"live_count,omitempty"` - KeyBytes int64 `protobuf:"zigzag64,6,opt,name=key_bytes,json=keyBytes,proto3" json:"key_bytes,omitempty"` - KeyCount int64 `protobuf:"zigzag64,7,opt,name=key_count,json=keyCount,proto3" json:"key_count,omitempty"` - ValBytes int64 `protobuf:"zigzag64,8,opt,name=val_bytes,json=valBytes,proto3" json:"val_bytes,omitempty"` - ValCount int64 `protobuf:"zigzag64,9,opt,name=val_count,json=valCount,proto3" json:"val_count,omitempty"` - IntentBytes int64 `protobuf:"zigzag64,10,opt,name=intent_bytes,json=intentBytes,proto3" json:"intent_bytes,omitempty"` - IntentCount int64 `protobuf:"zigzag64,11,opt,name=intent_count,json=intentCount,proto3" json:"intent_count,omitempty"` - SysBytes int64 `protobuf:"zigzag64,12,opt,name=sys_bytes,json=sysBytes,proto3" json:"sys_bytes,omitempty"` - SysCount int64 `protobuf:"zigzag64,13,opt,name=sys_count,json=sysCount,proto3" json:"sys_count,omitempty"` -} - -func (m *MVCCStatsDelta) Reset() { *m = MVCCStatsDelta{} } -func (m *MVCCStatsDelta) String() string { return proto.CompactTextString(m) } -func (*MVCCStatsDelta) ProtoMessage() {} -func (*MVCCStatsDelta) Descriptor() ([]byte, []int) { return fileDescriptorMvcc3, []int{1} } + ContainsEstimates bool `protobuf:"varint,14,opt,name=contains_estimates,json=containsEstimates,proto3" json:"contains_estimates,omitempty"` + LastUpdateNanos int64 `protobuf:"fixed64,1,opt,name=last_update_nanos,json=lastUpdateNanos,proto3" json:"last_update_nanos,omitempty"` + IntentAge int64 `protobuf:"fixed64,2,opt,name=intent_age,json=intentAge,proto3" json:"intent_age,omitempty"` + GCBytesAge int64 `protobuf:"fixed64,3,opt,name=gc_bytes_age,json=gcBytesAge,proto3" json:"gc_bytes_age,omitempty"` + LiveBytes int64 `protobuf:"zigzag64,4,opt,name=live_bytes,json=liveBytes,proto3" json:"live_bytes,omitempty"` + LiveCount int64 `protobuf:"zigzag64,5,opt,name=live_count,json=liveCount,proto3" json:"live_count,omitempty"` + KeyBytes int64 `protobuf:"zigzag64,6,opt,name=key_bytes,json=keyBytes,proto3" json:"key_bytes,omitempty"` + KeyCount int64 `protobuf:"zigzag64,7,opt,name=key_count,json=keyCount,proto3" json:"key_count,omitempty"` + ValBytes int64 `protobuf:"zigzag64,8,opt,name=val_bytes,json=valBytes,proto3" json:"val_bytes,omitempty"` + ValCount int64 `protobuf:"zigzag64,9,opt,name=val_count,json=valCount,proto3" json:"val_count,omitempty"` + IntentBytes int64 `protobuf:"zigzag64,10,opt,name=intent_bytes,json=intentBytes,proto3" json:"intent_bytes,omitempty"` + IntentCount int64 `protobuf:"zigzag64,11,opt,name=intent_count,json=intentCount,proto3" json:"intent_count,omitempty"` + SysBytes int64 `protobuf:"zigzag64,12,opt,name=sys_bytes,json=sysBytes,proto3" json:"sys_bytes,omitempty"` + SysCount int64 `protobuf:"zigzag64,13,opt,name=sys_count,json=sysCount,proto3" json:"sys_count,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *MVCCStatsDelta) Reset() { *m = MVCCStatsDelta{} } +func (m *MVCCStatsDelta) String() string { return proto.CompactTextString(m) } +func (*MVCCStatsDelta) ProtoMessage() {} +func (*MVCCStatsDelta) Descriptor() ([]byte, []int) { + return fileDescriptor_mvcc3_e549160c12464c1c, []int{1} +} +func (m *MVCCStatsDelta) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *MVCCStatsDelta) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *MVCCStatsDelta) XXX_Merge(src proto.Message) { + xxx_messageInfo_MVCCStatsDelta.Merge(dst, src) +} +func (m *MVCCStatsDelta) XXX_Size() int { + return m.Size() +} +func (m *MVCCStatsDelta) XXX_DiscardUnknown() { + xxx_messageInfo_MVCCStatsDelta.DiscardUnknown(m) +} + +var xxx_messageInfo_MVCCStatsDelta proto.InternalMessageInfo // MVCCPersistentStats is convertible to MVCCStats, but uses signed variable // width encodings for most fields that make it efficient to store positive // values but inefficient to store negative values. This makes the encodings // incompatible. type MVCCPersistentStats struct { - ContainsEstimates bool `protobuf:"varint,14,opt,name=contains_estimates,json=containsEstimates,proto3" json:"contains_estimates,omitempty"` - LastUpdateNanos int64 `protobuf:"fixed64,1,opt,name=last_update_nanos,json=lastUpdateNanos,proto3" json:"last_update_nanos,omitempty"` - IntentAge int64 `protobuf:"fixed64,2,opt,name=intent_age,json=intentAge,proto3" json:"intent_age,omitempty"` - GCBytesAge int64 `protobuf:"fixed64,3,opt,name=gc_bytes_age,json=gcBytesAge,proto3" json:"gc_bytes_age,omitempty"` - LiveBytes int64 `protobuf:"varint,4,opt,name=live_bytes,json=liveBytes,proto3" json:"live_bytes,omitempty"` - LiveCount int64 `protobuf:"varint,5,opt,name=live_count,json=liveCount,proto3" json:"live_count,omitempty"` - KeyBytes int64 `protobuf:"varint,6,opt,name=key_bytes,json=keyBytes,proto3" json:"key_bytes,omitempty"` - KeyCount int64 `protobuf:"varint,7,opt,name=key_count,json=keyCount,proto3" json:"key_count,omitempty"` - ValBytes int64 `protobuf:"varint,8,opt,name=val_bytes,json=valBytes,proto3" json:"val_bytes,omitempty"` - ValCount int64 `protobuf:"varint,9,opt,name=val_count,json=valCount,proto3" json:"val_count,omitempty"` - IntentBytes int64 `protobuf:"varint,10,opt,name=intent_bytes,json=intentBytes,proto3" json:"intent_bytes,omitempty"` - IntentCount int64 `protobuf:"varint,11,opt,name=intent_count,json=intentCount,proto3" json:"intent_count,omitempty"` - SysBytes int64 `protobuf:"varint,12,opt,name=sys_bytes,json=sysBytes,proto3" json:"sys_bytes,omitempty"` - SysCount int64 `protobuf:"varint,13,opt,name=sys_count,json=sysCount,proto3" json:"sys_count,omitempty"` -} - -func (m *MVCCPersistentStats) Reset() { *m = MVCCPersistentStats{} } -func (m *MVCCPersistentStats) String() string { return proto.CompactTextString(m) } -func (*MVCCPersistentStats) ProtoMessage() {} -func (*MVCCPersistentStats) Descriptor() ([]byte, []int) { return fileDescriptorMvcc3, []int{2} } + ContainsEstimates bool `protobuf:"varint,14,opt,name=contains_estimates,json=containsEstimates,proto3" json:"contains_estimates,omitempty"` + LastUpdateNanos int64 `protobuf:"fixed64,1,opt,name=last_update_nanos,json=lastUpdateNanos,proto3" json:"last_update_nanos,omitempty"` + IntentAge int64 `protobuf:"fixed64,2,opt,name=intent_age,json=intentAge,proto3" json:"intent_age,omitempty"` + GCBytesAge int64 `protobuf:"fixed64,3,opt,name=gc_bytes_age,json=gcBytesAge,proto3" json:"gc_bytes_age,omitempty"` + LiveBytes int64 `protobuf:"varint,4,opt,name=live_bytes,json=liveBytes,proto3" json:"live_bytes,omitempty"` + LiveCount int64 `protobuf:"varint,5,opt,name=live_count,json=liveCount,proto3" json:"live_count,omitempty"` + KeyBytes int64 `protobuf:"varint,6,opt,name=key_bytes,json=keyBytes,proto3" json:"key_bytes,omitempty"` + KeyCount int64 `protobuf:"varint,7,opt,name=key_count,json=keyCount,proto3" json:"key_count,omitempty"` + ValBytes int64 `protobuf:"varint,8,opt,name=val_bytes,json=valBytes,proto3" json:"val_bytes,omitempty"` + ValCount int64 `protobuf:"varint,9,opt,name=val_count,json=valCount,proto3" json:"val_count,omitempty"` + IntentBytes int64 `protobuf:"varint,10,opt,name=intent_bytes,json=intentBytes,proto3" json:"intent_bytes,omitempty"` + IntentCount int64 `protobuf:"varint,11,opt,name=intent_count,json=intentCount,proto3" json:"intent_count,omitempty"` + SysBytes int64 `protobuf:"varint,12,opt,name=sys_bytes,json=sysBytes,proto3" json:"sys_bytes,omitempty"` + SysCount int64 `protobuf:"varint,13,opt,name=sys_count,json=sysCount,proto3" json:"sys_count,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *MVCCPersistentStats) Reset() { *m = MVCCPersistentStats{} } +func (m *MVCCPersistentStats) String() string { return proto.CompactTextString(m) } +func (*MVCCPersistentStats) ProtoMessage() {} +func (*MVCCPersistentStats) Descriptor() ([]byte, []int) { + return fileDescriptor_mvcc3_e549160c12464c1c, []int{2} +} +func (m *MVCCPersistentStats) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *MVCCPersistentStats) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *MVCCPersistentStats) XXX_Merge(src proto.Message) { + xxx_messageInfo_MVCCPersistentStats.Merge(dst, src) +} +func (m *MVCCPersistentStats) XXX_Size() int { + return m.Size() +} +func (m *MVCCPersistentStats) XXX_DiscardUnknown() { + xxx_messageInfo_MVCCPersistentStats.DiscardUnknown(m) +} + +var xxx_messageInfo_MVCCPersistentStats proto.InternalMessageInfo // RangeAppliedState combines the raft and lease applied indices with // mvcc stats. These are all persisted on each transition of the Raft @@ -170,90 +254,272 @@ type RangeAppliedState struct { LeaseAppliedIndex uint64 `protobuf:"varint,2,opt,name=lease_applied_index,json=leaseAppliedIndex,proto3" json:"lease_applied_index,omitempty"` // range_stats is the set of mvcc stats that accounts for the current value // of the Raft state machine. - RangeStats MVCCPersistentStats `protobuf:"bytes,3,opt,name=range_stats,json=rangeStats" json:"range_stats"` + RangeStats MVCCPersistentStats `protobuf:"bytes,3,opt,name=range_stats,json=rangeStats,proto3" json:"range_stats"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *RangeAppliedState) Reset() { *m = RangeAppliedState{} } -func (m *RangeAppliedState) String() string { return proto.CompactTextString(m) } -func (*RangeAppliedState) ProtoMessage() {} -func (*RangeAppliedState) Descriptor() ([]byte, []int) { return fileDescriptorMvcc3, []int{3} } +func (m *RangeAppliedState) Reset() { *m = RangeAppliedState{} } +func (m *RangeAppliedState) String() string { return proto.CompactTextString(m) } +func (*RangeAppliedState) ProtoMessage() {} +func (*RangeAppliedState) Descriptor() ([]byte, []int) { + return fileDescriptor_mvcc3_e549160c12464c1c, []int{3} +} +func (m *RangeAppliedState) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RangeAppliedState) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RangeAppliedState) XXX_Merge(src proto.Message) { + xxx_messageInfo_RangeAppliedState.Merge(dst, src) +} +func (m *RangeAppliedState) XXX_Size() int { + return m.Size() +} +func (m *RangeAppliedState) XXX_DiscardUnknown() { + xxx_messageInfo_RangeAppliedState.DiscardUnknown(m) +} + +var xxx_messageInfo_RangeAppliedState proto.InternalMessageInfo // MVCCWriteValueOp corresponds to a value being written outside of a // transaction. type MVCCWriteValueOp struct { - Key []byte `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"` - Timestamp cockroach_util_hlc.Timestamp `protobuf:"bytes,2,opt,name=timestamp" json:"timestamp"` - Value []byte `protobuf:"bytes,3,opt,name=value,proto3" json:"value,omitempty"` + Key []byte `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"` + Timestamp hlc.Timestamp `protobuf:"bytes,2,opt,name=timestamp,proto3" json:"timestamp"` + Value []byte `protobuf:"bytes,3,opt,name=value,proto3" json:"value,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *MVCCWriteValueOp) Reset() { *m = MVCCWriteValueOp{} } -func (m *MVCCWriteValueOp) String() string { return proto.CompactTextString(m) } -func (*MVCCWriteValueOp) ProtoMessage() {} -func (*MVCCWriteValueOp) Descriptor() ([]byte, []int) { return fileDescriptorMvcc3, []int{4} } +func (m *MVCCWriteValueOp) Reset() { *m = MVCCWriteValueOp{} } +func (m *MVCCWriteValueOp) String() string { return proto.CompactTextString(m) } +func (*MVCCWriteValueOp) ProtoMessage() {} +func (*MVCCWriteValueOp) Descriptor() ([]byte, []int) { + return fileDescriptor_mvcc3_e549160c12464c1c, []int{4} +} +func (m *MVCCWriteValueOp) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *MVCCWriteValueOp) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *MVCCWriteValueOp) XXX_Merge(src proto.Message) { + xxx_messageInfo_MVCCWriteValueOp.Merge(dst, src) +} +func (m *MVCCWriteValueOp) XXX_Size() int { + return m.Size() +} +func (m *MVCCWriteValueOp) XXX_DiscardUnknown() { + xxx_messageInfo_MVCCWriteValueOp.DiscardUnknown(m) +} + +var xxx_messageInfo_MVCCWriteValueOp proto.InternalMessageInfo // MVCCUpdateIntentOp corresponds to an intent being written for a given // transaction. type MVCCWriteIntentOp struct { - TxnID github_com_cockroachdb_cockroach_pkg_util_uuid.UUID `protobuf:"bytes,1,opt,name=txn_id,json=txnId,proto3,customtype=github.com/cockroachdb/cockroach/pkg/util/uuid.UUID" json:"txn_id"` - TxnKey []byte `protobuf:"bytes,2,opt,name=txn_key,json=txnKey,proto3" json:"txn_key,omitempty"` - Timestamp cockroach_util_hlc.Timestamp `protobuf:"bytes,3,opt,name=timestamp" json:"timestamp"` + TxnID github_com_cockroachdb_cockroach_pkg_util_uuid.UUID `protobuf:"bytes,1,opt,name=txn_id,json=txnId,proto3,customtype=github.com/cockroachdb/cockroach/pkg/util/uuid.UUID" json:"txn_id"` + TxnKey []byte `protobuf:"bytes,2,opt,name=txn_key,json=txnKey,proto3" json:"txn_key,omitempty"` + Timestamp hlc.Timestamp `protobuf:"bytes,3,opt,name=timestamp,proto3" json:"timestamp"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *MVCCWriteIntentOp) Reset() { *m = MVCCWriteIntentOp{} } -func (m *MVCCWriteIntentOp) String() string { return proto.CompactTextString(m) } -func (*MVCCWriteIntentOp) ProtoMessage() {} -func (*MVCCWriteIntentOp) Descriptor() ([]byte, []int) { return fileDescriptorMvcc3, []int{5} } +func (m *MVCCWriteIntentOp) Reset() { *m = MVCCWriteIntentOp{} } +func (m *MVCCWriteIntentOp) String() string { return proto.CompactTextString(m) } +func (*MVCCWriteIntentOp) ProtoMessage() {} +func (*MVCCWriteIntentOp) Descriptor() ([]byte, []int) { + return fileDescriptor_mvcc3_e549160c12464c1c, []int{5} +} +func (m *MVCCWriteIntentOp) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *MVCCWriteIntentOp) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *MVCCWriteIntentOp) XXX_Merge(src proto.Message) { + xxx_messageInfo_MVCCWriteIntentOp.Merge(dst, src) +} +func (m *MVCCWriteIntentOp) XXX_Size() int { + return m.Size() +} +func (m *MVCCWriteIntentOp) XXX_DiscardUnknown() { + xxx_messageInfo_MVCCWriteIntentOp.DiscardUnknown(m) +} + +var xxx_messageInfo_MVCCWriteIntentOp proto.InternalMessageInfo // MVCCUpdateIntentOp corresponds to an intent being updates at a larger // timestamp for a given transaction. type MVCCUpdateIntentOp struct { - TxnID github_com_cockroachdb_cockroach_pkg_util_uuid.UUID `protobuf:"bytes,1,opt,name=txn_id,json=txnId,proto3,customtype=github.com/cockroachdb/cockroach/pkg/util/uuid.UUID" json:"txn_id"` - Timestamp cockroach_util_hlc.Timestamp `protobuf:"bytes,2,opt,name=timestamp" json:"timestamp"` + TxnID github_com_cockroachdb_cockroach_pkg_util_uuid.UUID `protobuf:"bytes,1,opt,name=txn_id,json=txnId,proto3,customtype=github.com/cockroachdb/cockroach/pkg/util/uuid.UUID" json:"txn_id"` + Timestamp hlc.Timestamp `protobuf:"bytes,2,opt,name=timestamp,proto3" json:"timestamp"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *MVCCUpdateIntentOp) Reset() { *m = MVCCUpdateIntentOp{} } -func (m *MVCCUpdateIntentOp) String() string { return proto.CompactTextString(m) } -func (*MVCCUpdateIntentOp) ProtoMessage() {} -func (*MVCCUpdateIntentOp) Descriptor() ([]byte, []int) { return fileDescriptorMvcc3, []int{6} } +func (m *MVCCUpdateIntentOp) Reset() { *m = MVCCUpdateIntentOp{} } +func (m *MVCCUpdateIntentOp) String() string { return proto.CompactTextString(m) } +func (*MVCCUpdateIntentOp) ProtoMessage() {} +func (*MVCCUpdateIntentOp) Descriptor() ([]byte, []int) { + return fileDescriptor_mvcc3_e549160c12464c1c, []int{6} +} +func (m *MVCCUpdateIntentOp) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *MVCCUpdateIntentOp) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *MVCCUpdateIntentOp) XXX_Merge(src proto.Message) { + xxx_messageInfo_MVCCUpdateIntentOp.Merge(dst, src) +} +func (m *MVCCUpdateIntentOp) XXX_Size() int { + return m.Size() +} +func (m *MVCCUpdateIntentOp) XXX_DiscardUnknown() { + xxx_messageInfo_MVCCUpdateIntentOp.DiscardUnknown(m) +} + +var xxx_messageInfo_MVCCUpdateIntentOp proto.InternalMessageInfo // MVCCCommitIntentOp corresponds to an intent being committed for a given // transaction. type MVCCCommitIntentOp struct { - TxnID github_com_cockroachdb_cockroach_pkg_util_uuid.UUID `protobuf:"bytes,1,opt,name=txn_id,json=txnId,proto3,customtype=github.com/cockroachdb/cockroach/pkg/util/uuid.UUID" json:"txn_id"` - Key []byte `protobuf:"bytes,2,opt,name=key,proto3" json:"key,omitempty"` - Timestamp cockroach_util_hlc.Timestamp `protobuf:"bytes,3,opt,name=timestamp" json:"timestamp"` - Value []byte `protobuf:"bytes,4,opt,name=value,proto3" json:"value,omitempty"` + TxnID github_com_cockroachdb_cockroach_pkg_util_uuid.UUID `protobuf:"bytes,1,opt,name=txn_id,json=txnId,proto3,customtype=github.com/cockroachdb/cockroach/pkg/util/uuid.UUID" json:"txn_id"` + Key []byte `protobuf:"bytes,2,opt,name=key,proto3" json:"key,omitempty"` + Timestamp hlc.Timestamp `protobuf:"bytes,3,opt,name=timestamp,proto3" json:"timestamp"` + Value []byte `protobuf:"bytes,4,opt,name=value,proto3" json:"value,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *MVCCCommitIntentOp) Reset() { *m = MVCCCommitIntentOp{} } +func (m *MVCCCommitIntentOp) String() string { return proto.CompactTextString(m) } +func (*MVCCCommitIntentOp) ProtoMessage() {} +func (*MVCCCommitIntentOp) Descriptor() ([]byte, []int) { + return fileDescriptor_mvcc3_e549160c12464c1c, []int{7} +} +func (m *MVCCCommitIntentOp) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *MVCCCommitIntentOp) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *MVCCCommitIntentOp) XXX_Merge(src proto.Message) { + xxx_messageInfo_MVCCCommitIntentOp.Merge(dst, src) +} +func (m *MVCCCommitIntentOp) XXX_Size() int { + return m.Size() +} +func (m *MVCCCommitIntentOp) XXX_DiscardUnknown() { + xxx_messageInfo_MVCCCommitIntentOp.DiscardUnknown(m) } -func (m *MVCCCommitIntentOp) Reset() { *m = MVCCCommitIntentOp{} } -func (m *MVCCCommitIntentOp) String() string { return proto.CompactTextString(m) } -func (*MVCCCommitIntentOp) ProtoMessage() {} -func (*MVCCCommitIntentOp) Descriptor() ([]byte, []int) { return fileDescriptorMvcc3, []int{7} } +var xxx_messageInfo_MVCCCommitIntentOp proto.InternalMessageInfo // MVCCAbortIntentOp corresponds to an intent being aborted for a given // transaction. type MVCCAbortIntentOp struct { - TxnID github_com_cockroachdb_cockroach_pkg_util_uuid.UUID `protobuf:"bytes,1,opt,name=txn_id,json=txnId,proto3,customtype=github.com/cockroachdb/cockroach/pkg/util/uuid.UUID" json:"txn_id"` + TxnID github_com_cockroachdb_cockroach_pkg_util_uuid.UUID `protobuf:"bytes,1,opt,name=txn_id,json=txnId,proto3,customtype=github.com/cockroachdb/cockroach/pkg/util/uuid.UUID" json:"txn_id"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *MVCCAbortIntentOp) Reset() { *m = MVCCAbortIntentOp{} } +func (m *MVCCAbortIntentOp) String() string { return proto.CompactTextString(m) } +func (*MVCCAbortIntentOp) ProtoMessage() {} +func (*MVCCAbortIntentOp) Descriptor() ([]byte, []int) { + return fileDescriptor_mvcc3_e549160c12464c1c, []int{8} +} +func (m *MVCCAbortIntentOp) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *MVCCAbortIntentOp) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *MVCCAbortIntentOp) XXX_Merge(src proto.Message) { + xxx_messageInfo_MVCCAbortIntentOp.Merge(dst, src) +} +func (m *MVCCAbortIntentOp) XXX_Size() int { + return m.Size() +} +func (m *MVCCAbortIntentOp) XXX_DiscardUnknown() { + xxx_messageInfo_MVCCAbortIntentOp.DiscardUnknown(m) } -func (m *MVCCAbortIntentOp) Reset() { *m = MVCCAbortIntentOp{} } -func (m *MVCCAbortIntentOp) String() string { return proto.CompactTextString(m) } -func (*MVCCAbortIntentOp) ProtoMessage() {} -func (*MVCCAbortIntentOp) Descriptor() ([]byte, []int) { return fileDescriptorMvcc3, []int{8} } +var xxx_messageInfo_MVCCAbortIntentOp proto.InternalMessageInfo // MVCCLogicalOp is a union of all logical MVCC operation types. type MVCCLogicalOp struct { - WriteValue *MVCCWriteValueOp `protobuf:"bytes,1,opt,name=write_value,json=writeValue" json:"write_value,omitempty"` - WriteIntent *MVCCWriteIntentOp `protobuf:"bytes,2,opt,name=write_intent,json=writeIntent" json:"write_intent,omitempty"` - UpdateIntent *MVCCUpdateIntentOp `protobuf:"bytes,3,opt,name=update_intent,json=updateIntent" json:"update_intent,omitempty"` - CommitIntent *MVCCCommitIntentOp `protobuf:"bytes,4,opt,name=commit_intent,json=commitIntent" json:"commit_intent,omitempty"` - AbortIntent *MVCCAbortIntentOp `protobuf:"bytes,5,opt,name=abort_intent,json=abortIntent" json:"abort_intent,omitempty"` + WriteValue *MVCCWriteValueOp `protobuf:"bytes,1,opt,name=write_value,json=writeValue,proto3" json:"write_value,omitempty"` + WriteIntent *MVCCWriteIntentOp `protobuf:"bytes,2,opt,name=write_intent,json=writeIntent,proto3" json:"write_intent,omitempty"` + UpdateIntent *MVCCUpdateIntentOp `protobuf:"bytes,3,opt,name=update_intent,json=updateIntent,proto3" json:"update_intent,omitempty"` + CommitIntent *MVCCCommitIntentOp `protobuf:"bytes,4,opt,name=commit_intent,json=commitIntent,proto3" json:"commit_intent,omitempty"` + AbortIntent *MVCCAbortIntentOp `protobuf:"bytes,5,opt,name=abort_intent,json=abortIntent,proto3" json:"abort_intent,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *MVCCLogicalOp) Reset() { *m = MVCCLogicalOp{} } +func (m *MVCCLogicalOp) String() string { return proto.CompactTextString(m) } +func (*MVCCLogicalOp) ProtoMessage() {} +func (*MVCCLogicalOp) Descriptor() ([]byte, []int) { + return fileDescriptor_mvcc3_e549160c12464c1c, []int{9} +} +func (m *MVCCLogicalOp) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *MVCCLogicalOp) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *MVCCLogicalOp) XXX_Merge(src proto.Message) { + xxx_messageInfo_MVCCLogicalOp.Merge(dst, src) +} +func (m *MVCCLogicalOp) XXX_Size() int { + return m.Size() +} +func (m *MVCCLogicalOp) XXX_DiscardUnknown() { + xxx_messageInfo_MVCCLogicalOp.DiscardUnknown(m) } -func (m *MVCCLogicalOp) Reset() { *m = MVCCLogicalOp{} } -func (m *MVCCLogicalOp) String() string { return proto.CompactTextString(m) } -func (*MVCCLogicalOp) ProtoMessage() {} -func (*MVCCLogicalOp) Descriptor() ([]byte, []int) { return fileDescriptorMvcc3, []int{9} } +var xxx_messageInfo_MVCCLogicalOp proto.InternalMessageInfo func init() { proto.RegisterType((*TxnMeta)(nil), "cockroach.storage.engine.enginepb.TxnMeta") @@ -535,19 +801,19 @@ func (m *MVCCStatsDelta) MarshalTo(dAtA []byte) (int, error) { if m.LastUpdateNanos != 0 { dAtA[i] = 0x9 i++ - binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.LastUpdateNanos)) + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.LastUpdateNanos)) i += 8 } if m.IntentAge != 0 { dAtA[i] = 0x11 i++ - binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.IntentAge)) + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.IntentAge)) i += 8 } if m.GCBytesAge != 0 { dAtA[i] = 0x19 i++ - binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.GCBytesAge)) + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.GCBytesAge)) i += 8 } if m.LiveBytes != 0 { @@ -631,19 +897,19 @@ func (m *MVCCPersistentStats) MarshalTo(dAtA []byte) (int, error) { if m.LastUpdateNanos != 0 { dAtA[i] = 0x9 i++ - binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.LastUpdateNanos)) + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.LastUpdateNanos)) i += 8 } if m.IntentAge != 0 { dAtA[i] = 0x11 i++ - binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.IntentAge)) + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.IntentAge)) i += 8 } if m.GCBytesAge != 0 { dAtA[i] = 0x19 i++ - binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.GCBytesAge)) + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(m.GCBytesAge)) i += 8 } if m.LiveBytes != 0 { @@ -1016,7 +1282,7 @@ func NewPopulatedTxnMeta(r randyMvcc3, easy bool) *TxnMeta { this.Key[i] = byte(r.Intn(256)) } this.Epoch = uint32(r.Uint32()) - v3 := cockroach_util_hlc.NewPopulatedTimestamp(r, easy) + v3 := hlc.NewPopulatedTimestamp(r, easy) this.Timestamp = *v3 this.Priority = int32(r.Int31()) if r.Intn(2) == 0 { @@ -1175,6 +1441,9 @@ func encodeVarintPopulateMvcc3(dAtA []byte, v uint64) []byte { return dAtA } func (m *TxnMeta) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.ID.Size() @@ -1198,6 +1467,9 @@ func (m *TxnMeta) Size() (n int) { } func (m *MVCCStatsDelta) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.LastUpdateNanos != 0 { @@ -1246,6 +1518,9 @@ func (m *MVCCStatsDelta) Size() (n int) { } func (m *MVCCPersistentStats) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.LastUpdateNanos != 0 { @@ -1294,6 +1569,9 @@ func (m *MVCCPersistentStats) Size() (n int) { } func (m *RangeAppliedState) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.RaftAppliedIndex != 0 { @@ -1308,6 +1586,9 @@ func (m *RangeAppliedState) Size() (n int) { } func (m *MVCCWriteValueOp) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Key) @@ -1324,6 +1605,9 @@ func (m *MVCCWriteValueOp) Size() (n int) { } func (m *MVCCWriteIntentOp) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.TxnID.Size() @@ -1338,6 +1622,9 @@ func (m *MVCCWriteIntentOp) Size() (n int) { } func (m *MVCCUpdateIntentOp) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.TxnID.Size() @@ -1348,6 +1635,9 @@ func (m *MVCCUpdateIntentOp) Size() (n int) { } func (m *MVCCCommitIntentOp) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.TxnID.Size() @@ -1366,6 +1656,9 @@ func (m *MVCCCommitIntentOp) Size() (n int) { } func (m *MVCCAbortIntentOp) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.TxnID.Size() @@ -1374,6 +1667,9 @@ func (m *MVCCAbortIntentOp) Size() (n int) { } func (m *MVCCLogicalOp) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.WriteValue != nil { @@ -1683,7 +1979,7 @@ func (m *MVCCStatsDelta) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - m.LastUpdateNanos = int64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + m.LastUpdateNanos = int64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 case 2: if wireType != 1 { @@ -1693,7 +1989,7 @@ func (m *MVCCStatsDelta) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - m.IntentAge = int64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + m.IntentAge = int64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 case 3: if wireType != 1 { @@ -1703,7 +1999,7 @@ func (m *MVCCStatsDelta) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - m.GCBytesAge = int64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + m.GCBytesAge = int64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 case 4: if wireType != 0 { @@ -1993,7 +2289,7 @@ func (m *MVCCPersistentStats) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - m.LastUpdateNanos = int64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + m.LastUpdateNanos = int64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 case 2: if wireType != 1 { @@ -2003,7 +2299,7 @@ func (m *MVCCPersistentStats) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - m.IntentAge = int64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + m.IntentAge = int64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 case 3: if wireType != 1 { @@ -2013,7 +2309,7 @@ func (m *MVCCPersistentStats) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - m.GCBytesAge = int64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + m.GCBytesAge = int64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 case 4: if wireType != 0 { @@ -3329,9 +3625,11 @@ var ( ErrIntOverflowMvcc3 = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("storage/engine/enginepb/mvcc3.proto", fileDescriptorMvcc3) } +func init() { + proto.RegisterFile("storage/engine/enginepb/mvcc3.proto", fileDescriptor_mvcc3_e549160c12464c1c) +} -var fileDescriptorMvcc3 = []byte{ +var fileDescriptor_mvcc3_e549160c12464c1c = []byte{ // 990 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xcc, 0x56, 0x4f, 0x6f, 0xe3, 0x44, 0x14, 0xef, 0x78, 0x92, 0x36, 0x99, 0xa4, 0x25, 0x9d, 0xad, 0x84, 0x55, 0xb4, 0x49, 0x36, 0x5c, diff --git a/pkg/storage/engine/enginepb/rocksdb.pb.go b/pkg/storage/engine/enginepb/rocksdb.pb.go index 0974486572f3..5bf6c3f6a447 100644 --- a/pkg/storage/engine/enginepb/rocksdb.pb.go +++ b/pkg/storage/engine/enginepb/rocksdb.pb.go @@ -6,7 +6,7 @@ package enginepb import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" -import cockroach_util_hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" +import hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" import io "io" @@ -15,33 +15,89 @@ var _ = proto.Marshal var _ = fmt.Errorf var _ = math.Inf +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package + // SSTUserProperties contains the user-added properties of a single sstable. type SSTUserProperties struct { Path string `protobuf:"bytes,1,opt,name=path,proto3" json:"path,omitempty"` // ts_min is the minimum mvcc timestamp present in this sstable. - TsMin *cockroach_util_hlc.Timestamp `protobuf:"bytes,2,opt,name=ts_min,json=tsMin" json:"ts_min,omitempty"` + TsMin *hlc.Timestamp `protobuf:"bytes,2,opt,name=ts_min,json=tsMin,proto3" json:"ts_min,omitempty"` // ts_max is the maximum mvcc timestamp present in this sstable. - TsMax *cockroach_util_hlc.Timestamp `protobuf:"bytes,3,opt,name=ts_max,json=tsMax" json:"ts_max,omitempty"` + TsMax *hlc.Timestamp `protobuf:"bytes,3,opt,name=ts_max,json=tsMax,proto3" json:"ts_max,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *SSTUserProperties) Reset() { *m = SSTUserProperties{} } +func (m *SSTUserProperties) String() string { return proto.CompactTextString(m) } +func (*SSTUserProperties) ProtoMessage() {} +func (*SSTUserProperties) Descriptor() ([]byte, []int) { + return fileDescriptor_rocksdb_8b195a1167228ff1, []int{0} +} +func (m *SSTUserProperties) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SSTUserProperties) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *SSTUserProperties) XXX_Merge(src proto.Message) { + xxx_messageInfo_SSTUserProperties.Merge(dst, src) +} +func (m *SSTUserProperties) XXX_Size() int { + return m.Size() +} +func (m *SSTUserProperties) XXX_DiscardUnknown() { + xxx_messageInfo_SSTUserProperties.DiscardUnknown(m) } -func (m *SSTUserProperties) Reset() { *m = SSTUserProperties{} } -func (m *SSTUserProperties) String() string { return proto.CompactTextString(m) } -func (*SSTUserProperties) ProtoMessage() {} -func (*SSTUserProperties) Descriptor() ([]byte, []int) { return fileDescriptorRocksdb, []int{0} } +var xxx_messageInfo_SSTUserProperties proto.InternalMessageInfo // SSTUserPropertiesCollection contains the user-added properties of every // sstable in a RocksDB instance. type SSTUserPropertiesCollection struct { - Sst []SSTUserProperties `protobuf:"bytes,1,rep,name=sst" json:"sst"` - Error string `protobuf:"bytes,2,opt,name=error,proto3" json:"error,omitempty"` + Sst []SSTUserProperties `protobuf:"bytes,1,rep,name=sst,proto3" json:"sst"` + Error string `protobuf:"bytes,2,opt,name=error,proto3" json:"error,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *SSTUserPropertiesCollection) Reset() { *m = SSTUserPropertiesCollection{} } func (m *SSTUserPropertiesCollection) String() string { return proto.CompactTextString(m) } func (*SSTUserPropertiesCollection) ProtoMessage() {} func (*SSTUserPropertiesCollection) Descriptor() ([]byte, []int) { - return fileDescriptorRocksdb, []int{1} + return fileDescriptor_rocksdb_8b195a1167228ff1, []int{1} +} +func (m *SSTUserPropertiesCollection) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) } +func (m *SSTUserPropertiesCollection) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *SSTUserPropertiesCollection) XXX_Merge(src proto.Message) { + xxx_messageInfo_SSTUserPropertiesCollection.Merge(dst, src) +} +func (m *SSTUserPropertiesCollection) XXX_Size() int { + return m.Size() +} +func (m *SSTUserPropertiesCollection) XXX_DiscardUnknown() { + xxx_messageInfo_SSTUserPropertiesCollection.DiscardUnknown(m) +} + +var xxx_messageInfo_SSTUserPropertiesCollection proto.InternalMessageInfo func init() { proto.RegisterType((*SSTUserProperties)(nil), "cockroach.storage.engine.enginepb.SSTUserProperties") @@ -137,6 +193,9 @@ func encodeVarintRocksdb(dAtA []byte, offset int, v uint64) int { return offset + 1 } func (m *SSTUserProperties) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Path) @@ -155,6 +214,9 @@ func (m *SSTUserProperties) Size() (n int) { } func (m *SSTUserPropertiesCollection) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.Sst) > 0 { @@ -268,7 +330,7 @@ func (m *SSTUserProperties) Unmarshal(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.TsMin == nil { - m.TsMin = &cockroach_util_hlc.Timestamp{} + m.TsMin = &hlc.Timestamp{} } if err := m.TsMin.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err @@ -301,7 +363,7 @@ func (m *SSTUserProperties) Unmarshal(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.TsMax == nil { - m.TsMax = &cockroach_util_hlc.Timestamp{} + m.TsMax = &hlc.Timestamp{} } if err := m.TsMax.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err @@ -543,9 +605,11 @@ var ( ErrIntOverflowRocksdb = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("storage/engine/enginepb/rocksdb.proto", fileDescriptorRocksdb) } +func init() { + proto.RegisterFile("storage/engine/enginepb/rocksdb.proto", fileDescriptor_rocksdb_8b195a1167228ff1) +} -var fileDescriptorRocksdb = []byte{ +var fileDescriptor_rocksdb_8b195a1167228ff1 = []byte{ // 295 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x90, 0xc1, 0x4a, 0xc3, 0x30, 0x1c, 0xc6, 0x17, 0xbb, 0x0d, 0x97, 0x9d, 0x0c, 0x3b, 0x94, 0x89, 0xb1, 0x16, 0x84, 0x9e, 0x52, diff --git a/pkg/storage/engine/mvcc_test.go b/pkg/storage/engine/mvcc_test.go index 94eb627ad975..c4ccb9e5053b 100644 --- a/pkg/storage/engine/mvcc_test.go +++ b/pkg/storage/engine/mvcc_test.go @@ -1000,7 +1000,7 @@ func TestMVCCInlineWithTxn(t *testing.T) { t.Fatal(err) } if !reflect.DeepEqual(value1, *value) { - t.Errorf("the inline value should be %s; got %s", value1, *value) + t.Errorf("the inline value should be %v; got %v", value1, *value) } // Verify inline get with txn does still work (this will happen on a diff --git a/pkg/storage/raft.pb.go b/pkg/storage/raft.pb.go index dc2433e98bcf..d231257348f1 100644 --- a/pkg/storage/raft.pb.go +++ b/pkg/storage/raft.pb.go @@ -6,15 +6,16 @@ package storage import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" -import cockroach_roachpb3 "github.com/cockroachdb/cockroach/pkg/roachpb" -import cockroach_roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" -import cockroach_storage_storagepb "github.com/cockroachdb/cockroach/pkg/storage/storagepb" import raftpb "go.etcd.io/etcd/raft/raftpb" +import roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" +import storagepb "github.com/cockroachdb/cockroach/pkg/storage/storagepb" import github_com_cockroachdb_cockroach_pkg_roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" -import context "context" -import grpc "google.golang.org/grpc" +import ( + context "context" + grpc "google.golang.org/grpc" +) import io "io" @@ -23,6 +24,12 @@ var _ = proto.Marshal var _ = fmt.Errorf var _ = math.Inf +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package + type SnapshotRequest_Priority int32 const ( @@ -63,7 +70,7 @@ func (x *SnapshotRequest_Priority) UnmarshalJSON(data []byte) error { return nil } func (SnapshotRequest_Priority) EnumDescriptor() ([]byte, []int) { - return fileDescriptorRaft, []int{5, 0} + return fileDescriptor_raft_460a63b017d715a3, []int{5, 0} } type SnapshotRequest_Strategy int32 @@ -100,7 +107,7 @@ func (x *SnapshotRequest_Strategy) UnmarshalJSON(data []byte) error { return nil } func (SnapshotRequest_Strategy) EnumDescriptor() ([]byte, []int) { - return fileDescriptorRaft, []int{5, 1} + return fileDescriptor_raft_460a63b017d715a3, []int{5, 1} } type SnapshotResponse_Status int32 @@ -145,7 +152,7 @@ func (x *SnapshotResponse_Status) UnmarshalJSON(data []byte) error { return nil } func (SnapshotResponse_Status) EnumDescriptor() ([]byte, []int) { - return fileDescriptorRaft, []int{6, 0} + return fileDescriptor_raft_460a63b017d715a3, []int{6, 0} } // RaftHeartbeat is a request that contains the barebones information for a @@ -153,18 +160,44 @@ func (SnapshotResponse_Status) EnumDescriptor() ([]byte, []int) { // in a RaftMessageRequest, and reconstructed by the receiver into individual // raftpb.Message protos. type RaftHeartbeat struct { - RangeID github_com_cockroachdb_cockroach_pkg_roachpb.RangeID `protobuf:"varint,1,opt,name=range_id,json=rangeId,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" json:"range_id"` - FromReplicaID github_com_cockroachdb_cockroach_pkg_roachpb.ReplicaID `protobuf:"varint,2,opt,name=from_replica_id,json=fromReplicaId,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.ReplicaID" json:"from_replica_id"` - ToReplicaID github_com_cockroachdb_cockroach_pkg_roachpb.ReplicaID `protobuf:"varint,3,opt,name=to_replica_id,json=toReplicaId,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.ReplicaID" json:"to_replica_id"` - Term uint64 `protobuf:"varint,4,opt,name=term" json:"term"` - Commit uint64 `protobuf:"varint,5,opt,name=commit" json:"commit"` - Quiesce bool `protobuf:"varint,6,opt,name=quiesce" json:"quiesce"` + RangeID github_com_cockroachdb_cockroach_pkg_roachpb.RangeID `protobuf:"varint,1,opt,name=range_id,json=rangeId,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" json:"range_id"` + FromReplicaID github_com_cockroachdb_cockroach_pkg_roachpb.ReplicaID `protobuf:"varint,2,opt,name=from_replica_id,json=fromReplicaId,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.ReplicaID" json:"from_replica_id"` + ToReplicaID github_com_cockroachdb_cockroach_pkg_roachpb.ReplicaID `protobuf:"varint,3,opt,name=to_replica_id,json=toReplicaId,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.ReplicaID" json:"to_replica_id"` + Term uint64 `protobuf:"varint,4,opt,name=term" json:"term"` + Commit uint64 `protobuf:"varint,5,opt,name=commit" json:"commit"` + Quiesce bool `protobuf:"varint,6,opt,name=quiesce" json:"quiesce"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *RaftHeartbeat) Reset() { *m = RaftHeartbeat{} } +func (m *RaftHeartbeat) String() string { return proto.CompactTextString(m) } +func (*RaftHeartbeat) ProtoMessage() {} +func (*RaftHeartbeat) Descriptor() ([]byte, []int) { + return fileDescriptor_raft_460a63b017d715a3, []int{0} +} +func (m *RaftHeartbeat) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RaftHeartbeat) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RaftHeartbeat) XXX_Merge(src proto.Message) { + xxx_messageInfo_RaftHeartbeat.Merge(dst, src) +} +func (m *RaftHeartbeat) XXX_Size() int { + return m.Size() +} +func (m *RaftHeartbeat) XXX_DiscardUnknown() { + xxx_messageInfo_RaftHeartbeat.DiscardUnknown(m) } -func (m *RaftHeartbeat) Reset() { *m = RaftHeartbeat{} } -func (m *RaftHeartbeat) String() string { return proto.CompactTextString(m) } -func (*RaftHeartbeat) ProtoMessage() {} -func (*RaftHeartbeat) Descriptor() ([]byte, []int) { return fileDescriptorRaft, []int{0} } +var xxx_messageInfo_RaftHeartbeat proto.InternalMessageInfo // RaftMessageRequest is the request used to send raft messages using our // protobuf-based RPC codec. If a RaftMessageRequest has a non-empty number of @@ -176,8 +209,8 @@ type RaftMessageRequest struct { // Optionally, the start key of the sending replica. This is only populated // as a "hint" under certain conditions. RangeStartKey github_com_cockroachdb_cockroach_pkg_roachpb.RKey `protobuf:"bytes,8,opt,name=range_start_key,json=rangeStartKey,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.RKey" json:"range_start_key,omitempty"` - FromReplica cockroach_roachpb.ReplicaDescriptor `protobuf:"bytes,2,opt,name=from_replica,json=fromReplica" json:"from_replica"` - ToReplica cockroach_roachpb.ReplicaDescriptor `protobuf:"bytes,3,opt,name=to_replica,json=toReplica" json:"to_replica"` + FromReplica roachpb.ReplicaDescriptor `protobuf:"bytes,2,opt,name=from_replica,json=fromReplica" json:"from_replica"` + ToReplica roachpb.ReplicaDescriptor `protobuf:"bytes,3,opt,name=to_replica,json=toReplica" json:"to_replica"` Message raftpb.Message `protobuf:"bytes,4,opt,name=message" json:"message"` // Is this a quiesce request? A quiesce request is a MsgHeartbeat // which is requesting the recipient to stop ticking its local @@ -189,32 +222,110 @@ type RaftMessageRequest struct { Quiesce bool `protobuf:"varint,5,opt,name=quiesce" json:"quiesce"` // A coalesced heartbeat request is any RaftMessageRequest with a nonzero number of // heartbeats or heartbeat_resps. - Heartbeats []RaftHeartbeat `protobuf:"bytes,6,rep,name=heartbeats" json:"heartbeats"` - HeartbeatResps []RaftHeartbeat `protobuf:"bytes,7,rep,name=heartbeat_resps,json=heartbeatResps" json:"heartbeat_resps"` + Heartbeats []RaftHeartbeat `protobuf:"bytes,6,rep,name=heartbeats" json:"heartbeats"` + HeartbeatResps []RaftHeartbeat `protobuf:"bytes,7,rep,name=heartbeat_resps,json=heartbeatResps" json:"heartbeat_resps"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *RaftMessageRequest) Reset() { *m = RaftMessageRequest{} } +func (m *RaftMessageRequest) String() string { return proto.CompactTextString(m) } +func (*RaftMessageRequest) ProtoMessage() {} +func (*RaftMessageRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_raft_460a63b017d715a3, []int{1} +} +func (m *RaftMessageRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RaftMessageRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RaftMessageRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_RaftMessageRequest.Merge(dst, src) +} +func (m *RaftMessageRequest) XXX_Size() int { + return m.Size() +} +func (m *RaftMessageRequest) XXX_DiscardUnknown() { + xxx_messageInfo_RaftMessageRequest.DiscardUnknown(m) } -func (m *RaftMessageRequest) Reset() { *m = RaftMessageRequest{} } -func (m *RaftMessageRequest) String() string { return proto.CompactTextString(m) } -func (*RaftMessageRequest) ProtoMessage() {} -func (*RaftMessageRequest) Descriptor() ([]byte, []int) { return fileDescriptorRaft, []int{1} } +var xxx_messageInfo_RaftMessageRequest proto.InternalMessageInfo type RaftMessageRequestBatch struct { - Requests []RaftMessageRequest `protobuf:"bytes,1,rep,name=requests" json:"requests"` + Requests []RaftMessageRequest `protobuf:"bytes,1,rep,name=requests" json:"requests"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *RaftMessageRequestBatch) Reset() { *m = RaftMessageRequestBatch{} } +func (m *RaftMessageRequestBatch) String() string { return proto.CompactTextString(m) } +func (*RaftMessageRequestBatch) ProtoMessage() {} +func (*RaftMessageRequestBatch) Descriptor() ([]byte, []int) { + return fileDescriptor_raft_460a63b017d715a3, []int{2} +} +func (m *RaftMessageRequestBatch) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RaftMessageRequestBatch) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RaftMessageRequestBatch) XXX_Merge(src proto.Message) { + xxx_messageInfo_RaftMessageRequestBatch.Merge(dst, src) +} +func (m *RaftMessageRequestBatch) XXX_Size() int { + return m.Size() +} +func (m *RaftMessageRequestBatch) XXX_DiscardUnknown() { + xxx_messageInfo_RaftMessageRequestBatch.DiscardUnknown(m) } -func (m *RaftMessageRequestBatch) Reset() { *m = RaftMessageRequestBatch{} } -func (m *RaftMessageRequestBatch) String() string { return proto.CompactTextString(m) } -func (*RaftMessageRequestBatch) ProtoMessage() {} -func (*RaftMessageRequestBatch) Descriptor() ([]byte, []int) { return fileDescriptorRaft, []int{2} } +var xxx_messageInfo_RaftMessageRequestBatch proto.InternalMessageInfo type RaftMessageResponseUnion struct { - Error *cockroach_roachpb3.Error `protobuf:"bytes,1,opt,name=error" json:"error,omitempty"` + Error *roachpb.Error `protobuf:"bytes,1,opt,name=error" json:"error,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *RaftMessageResponseUnion) Reset() { *m = RaftMessageResponseUnion{} } +func (m *RaftMessageResponseUnion) String() string { return proto.CompactTextString(m) } +func (*RaftMessageResponseUnion) ProtoMessage() {} +func (*RaftMessageResponseUnion) Descriptor() ([]byte, []int) { + return fileDescriptor_raft_460a63b017d715a3, []int{3} +} +func (m *RaftMessageResponseUnion) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RaftMessageResponseUnion) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RaftMessageResponseUnion) XXX_Merge(src proto.Message) { + xxx_messageInfo_RaftMessageResponseUnion.Merge(dst, src) +} +func (m *RaftMessageResponseUnion) XXX_Size() int { + return m.Size() +} +func (m *RaftMessageResponseUnion) XXX_DiscardUnknown() { + xxx_messageInfo_RaftMessageResponseUnion.DiscardUnknown(m) } -func (m *RaftMessageResponseUnion) Reset() { *m = RaftMessageResponseUnion{} } -func (m *RaftMessageResponseUnion) String() string { return proto.CompactTextString(m) } -func (*RaftMessageResponseUnion) ProtoMessage() {} -func (*RaftMessageResponseUnion) Descriptor() ([]byte, []int) { return fileDescriptorRaft, []int{3} } +var xxx_messageInfo_RaftMessageResponseUnion proto.InternalMessageInfo // RaftMessageResponse may be sent to the sender of a // RaftMessageRequest. RaftMessage does not use the usual @@ -224,16 +335,42 @@ func (*RaftMessageResponseUnion) Descriptor() ([]byte, []int) { return fileDescr // RaftMessageResponse is not sent for every RaftMessageRequest, but // may be used for certain error conditions. type RaftMessageResponse struct { - RangeID github_com_cockroachdb_cockroach_pkg_roachpb.RangeID `protobuf:"varint,1,opt,name=range_id,json=rangeId,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" json:"range_id"` - FromReplica cockroach_roachpb.ReplicaDescriptor `protobuf:"bytes,2,opt,name=from_replica,json=fromReplica" json:"from_replica"` - ToReplica cockroach_roachpb.ReplicaDescriptor `protobuf:"bytes,3,opt,name=to_replica,json=toReplica" json:"to_replica"` - Union RaftMessageResponseUnion `protobuf:"bytes,4,opt,name=union" json:"union"` + RangeID github_com_cockroachdb_cockroach_pkg_roachpb.RangeID `protobuf:"varint,1,opt,name=range_id,json=rangeId,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" json:"range_id"` + FromReplica roachpb.ReplicaDescriptor `protobuf:"bytes,2,opt,name=from_replica,json=fromReplica" json:"from_replica"` + ToReplica roachpb.ReplicaDescriptor `protobuf:"bytes,3,opt,name=to_replica,json=toReplica" json:"to_replica"` + Union RaftMessageResponseUnion `protobuf:"bytes,4,opt,name=union" json:"union"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *RaftMessageResponse) Reset() { *m = RaftMessageResponse{} } +func (m *RaftMessageResponse) String() string { return proto.CompactTextString(m) } +func (*RaftMessageResponse) ProtoMessage() {} +func (*RaftMessageResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_raft_460a63b017d715a3, []int{4} +} +func (m *RaftMessageResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RaftMessageResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RaftMessageResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_RaftMessageResponse.Merge(dst, src) +} +func (m *RaftMessageResponse) XXX_Size() int { + return m.Size() +} +func (m *RaftMessageResponse) XXX_DiscardUnknown() { + xxx_messageInfo_RaftMessageResponse.DiscardUnknown(m) } -func (m *RaftMessageResponse) Reset() { *m = RaftMessageResponse{} } -func (m *RaftMessageResponse) String() string { return proto.CompactTextString(m) } -func (*RaftMessageResponse) ProtoMessage() {} -func (*RaftMessageResponse) Descriptor() ([]byte, []int) { return fileDescriptorRaft, []int{4} } +var xxx_messageInfo_RaftMessageResponse proto.InternalMessageInfo // SnapshotRequest is the request used to send streaming snapshot requests. type SnapshotRequest struct { @@ -243,14 +380,40 @@ type SnapshotRequest struct { // These are really raftpb.Entry, but we model them as raw bytes to avoid // roundtripping through memory. They are separate from the kv_batch to // allow flexibility in log implementations. - LogEntries [][]byte `protobuf:"bytes,3,rep,name=log_entries,json=logEntries" json:"log_entries,omitempty"` - Final bool `protobuf:"varint,4,opt,name=final" json:"final"` + LogEntries [][]byte `protobuf:"bytes,3,rep,name=log_entries,json=logEntries" json:"log_entries,omitempty"` + Final bool `protobuf:"varint,4,opt,name=final" json:"final"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *SnapshotRequest) Reset() { *m = SnapshotRequest{} } +func (m *SnapshotRequest) String() string { return proto.CompactTextString(m) } +func (*SnapshotRequest) ProtoMessage() {} +func (*SnapshotRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_raft_460a63b017d715a3, []int{5} +} +func (m *SnapshotRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SnapshotRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *SnapshotRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_SnapshotRequest.Merge(dst, src) +} +func (m *SnapshotRequest) XXX_Size() int { + return m.Size() +} +func (m *SnapshotRequest) XXX_DiscardUnknown() { + xxx_messageInfo_SnapshotRequest.DiscardUnknown(m) } -func (m *SnapshotRequest) Reset() { *m = SnapshotRequest{} } -func (m *SnapshotRequest) String() string { return proto.CompactTextString(m) } -func (*SnapshotRequest) ProtoMessage() {} -func (*SnapshotRequest) Descriptor() ([]byte, []int) { return fileDescriptorRaft, []int{5} } +var xxx_messageInfo_SnapshotRequest proto.InternalMessageInfo type SnapshotRequest_Header struct { // The replica state at the time the snapshot was generated. Note @@ -258,7 +421,7 @@ type SnapshotRequest_Header struct { // field which holds the updated descriptor after the new replica // has been added while ReplicaState.Desc holds the descriptor // before the new replica has been added. - State cockroach_storage_storagepb.ReplicaState `protobuf:"bytes,5,opt,name=state" json:"state"` + State storagepb.ReplicaState `protobuf:"bytes,5,opt,name=state" json:"state"` // The inner raft message is of type MsgSnap, and its snapshot data contains a UUID. RaftMessageRequest RaftMessageRequest `protobuf:"bytes,2,opt,name=raft_message_request,json=raftMessageRequest" json:"raft_message_request"` // The estimated size of the range, to be used in reservation decisions. @@ -270,23 +433,75 @@ type SnapshotRequest_Header struct { // The priority of the snapshot. Priority SnapshotRequest_Priority `protobuf:"varint,6,opt,name=priority,enum=cockroach.storage.SnapshotRequest_Priority" json:"priority"` // The strategy of the snapshot. - Strategy SnapshotRequest_Strategy `protobuf:"varint,7,opt,name=strategy,enum=cockroach.storage.SnapshotRequest_Strategy" json:"strategy"` + Strategy SnapshotRequest_Strategy `protobuf:"varint,7,opt,name=strategy,enum=cockroach.storage.SnapshotRequest_Strategy" json:"strategy"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *SnapshotRequest_Header) Reset() { *m = SnapshotRequest_Header{} } +func (m *SnapshotRequest_Header) String() string { return proto.CompactTextString(m) } +func (*SnapshotRequest_Header) ProtoMessage() {} +func (*SnapshotRequest_Header) Descriptor() ([]byte, []int) { + return fileDescriptor_raft_460a63b017d715a3, []int{5, 0} +} +func (m *SnapshotRequest_Header) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SnapshotRequest_Header) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *SnapshotRequest_Header) XXX_Merge(src proto.Message) { + xxx_messageInfo_SnapshotRequest_Header.Merge(dst, src) +} +func (m *SnapshotRequest_Header) XXX_Size() int { + return m.Size() +} +func (m *SnapshotRequest_Header) XXX_DiscardUnknown() { + xxx_messageInfo_SnapshotRequest_Header.DiscardUnknown(m) } -func (m *SnapshotRequest_Header) Reset() { *m = SnapshotRequest_Header{} } -func (m *SnapshotRequest_Header) String() string { return proto.CompactTextString(m) } -func (*SnapshotRequest_Header) ProtoMessage() {} -func (*SnapshotRequest_Header) Descriptor() ([]byte, []int) { return fileDescriptorRaft, []int{5, 0} } +var xxx_messageInfo_SnapshotRequest_Header proto.InternalMessageInfo type SnapshotResponse struct { - Status SnapshotResponse_Status `protobuf:"varint,1,opt,name=status,enum=cockroach.storage.SnapshotResponse_Status" json:"status"` - Message string `protobuf:"bytes,2,opt,name=message" json:"message"` + Status SnapshotResponse_Status `protobuf:"varint,1,opt,name=status,enum=cockroach.storage.SnapshotResponse_Status" json:"status"` + Message string `protobuf:"bytes,2,opt,name=message" json:"message"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *SnapshotResponse) Reset() { *m = SnapshotResponse{} } +func (m *SnapshotResponse) String() string { return proto.CompactTextString(m) } +func (*SnapshotResponse) ProtoMessage() {} +func (*SnapshotResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_raft_460a63b017d715a3, []int{6} +} +func (m *SnapshotResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SnapshotResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *SnapshotResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_SnapshotResponse.Merge(dst, src) +} +func (m *SnapshotResponse) XXX_Size() int { + return m.Size() +} +func (m *SnapshotResponse) XXX_DiscardUnknown() { + xxx_messageInfo_SnapshotResponse.DiscardUnknown(m) } -func (m *SnapshotResponse) Reset() { *m = SnapshotResponse{} } -func (m *SnapshotResponse) String() string { return proto.CompactTextString(m) } -func (*SnapshotResponse) ProtoMessage() {} -func (*SnapshotResponse) Descriptor() ([]byte, []int) { return fileDescriptorRaft, []int{6} } +var xxx_messageInfo_SnapshotResponse proto.InternalMessageInfo // ConfChangeContext is encoded in the raftpb.ConfChange.Context field. type ConfChangeContext struct { @@ -295,13 +510,39 @@ type ConfChangeContext struct { // storagepb.RaftCommand). Payload []byte `protobuf:"bytes,2,opt,name=payload" json:"payload,omitempty"` // Replica contains full details about the replica being added or removed. - Replica cockroach_roachpb.ReplicaDescriptor `protobuf:"bytes,3,opt,name=replica" json:"replica"` + Replica roachpb.ReplicaDescriptor `protobuf:"bytes,3,opt,name=replica" json:"replica"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ConfChangeContext) Reset() { *m = ConfChangeContext{} } +func (m *ConfChangeContext) String() string { return proto.CompactTextString(m) } +func (*ConfChangeContext) ProtoMessage() {} +func (*ConfChangeContext) Descriptor() ([]byte, []int) { + return fileDescriptor_raft_460a63b017d715a3, []int{7} +} +func (m *ConfChangeContext) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ConfChangeContext) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ConfChangeContext) XXX_Merge(src proto.Message) { + xxx_messageInfo_ConfChangeContext.Merge(dst, src) +} +func (m *ConfChangeContext) XXX_Size() int { + return m.Size() +} +func (m *ConfChangeContext) XXX_DiscardUnknown() { + xxx_messageInfo_ConfChangeContext.DiscardUnknown(m) } -func (m *ConfChangeContext) Reset() { *m = ConfChangeContext{} } -func (m *ConfChangeContext) String() string { return proto.CompactTextString(m) } -func (*ConfChangeContext) ProtoMessage() {} -func (*ConfChangeContext) Descriptor() ([]byte, []int) { return fileDescriptorRaft, []int{7} } +var xxx_messageInfo_ConfChangeContext proto.InternalMessageInfo func init() { proto.RegisterType((*RaftHeartbeat)(nil), "cockroach.storage.RaftHeartbeat") @@ -326,8 +567,9 @@ var _ grpc.ClientConn // is compatible with the grpc package it is being compiled against. const _ = grpc.SupportPackageIsVersion4 -// Client API for MultiRaft service - +// MultiRaftClient is the client API for MultiRaft service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream. type MultiRaftClient interface { RaftMessageBatch(ctx context.Context, opts ...grpc.CallOption) (MultiRaft_RaftMessageBatchClient, error) RaftSnapshot(ctx context.Context, opts ...grpc.CallOption) (MultiRaft_RaftSnapshotClient, error) @@ -342,7 +584,7 @@ func NewMultiRaftClient(cc *grpc.ClientConn) MultiRaftClient { } func (c *multiRaftClient) RaftMessageBatch(ctx context.Context, opts ...grpc.CallOption) (MultiRaft_RaftMessageBatchClient, error) { - stream, err := grpc.NewClientStream(ctx, &_MultiRaft_serviceDesc.Streams[0], c.cc, "/cockroach.storage.MultiRaft/RaftMessageBatch", opts...) + stream, err := c.cc.NewStream(ctx, &_MultiRaft_serviceDesc.Streams[0], "/cockroach.storage.MultiRaft/RaftMessageBatch", opts...) if err != nil { return nil, err } @@ -373,7 +615,7 @@ func (x *multiRaftRaftMessageBatchClient) Recv() (*RaftMessageResponse, error) { } func (c *multiRaftClient) RaftSnapshot(ctx context.Context, opts ...grpc.CallOption) (MultiRaft_RaftSnapshotClient, error) { - stream, err := grpc.NewClientStream(ctx, &_MultiRaft_serviceDesc.Streams[1], c.cc, "/cockroach.storage.MultiRaft/RaftSnapshot", opts...) + stream, err := c.cc.NewStream(ctx, &_MultiRaft_serviceDesc.Streams[1], "/cockroach.storage.MultiRaft/RaftSnapshot", opts...) if err != nil { return nil, err } @@ -403,8 +645,7 @@ func (x *multiRaftRaftSnapshotClient) Recv() (*SnapshotResponse, error) { return m, nil } -// Server API for MultiRaft service - +// MultiRaftServer is the server API for MultiRaft service. type MultiRaftServer interface { RaftMessageBatch(MultiRaft_RaftMessageBatchServer) error RaftSnapshot(MultiRaft_RaftSnapshotServer) error @@ -886,6 +1127,9 @@ func encodeVarintRaft(dAtA []byte, offset int, v uint64) int { return offset + 1 } func (m *RaftHeartbeat) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovRaft(uint64(m.RangeID)) @@ -898,6 +1142,9 @@ func (m *RaftHeartbeat) Size() (n int) { } func (m *RaftMessageRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovRaft(uint64(m.RangeID)) @@ -928,6 +1175,9 @@ func (m *RaftMessageRequest) Size() (n int) { } func (m *RaftMessageRequestBatch) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.Requests) > 0 { @@ -940,6 +1190,9 @@ func (m *RaftMessageRequestBatch) Size() (n int) { } func (m *RaftMessageResponseUnion) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Error != nil { @@ -950,6 +1203,9 @@ func (m *RaftMessageResponseUnion) Size() (n int) { } func (m *RaftMessageResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovRaft(uint64(m.RangeID)) @@ -963,6 +1219,9 @@ func (m *RaftMessageResponse) Size() (n int) { } func (m *SnapshotRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Header != nil { @@ -984,6 +1243,9 @@ func (m *SnapshotRequest) Size() (n int) { } func (m *SnapshotRequest_Header) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.RaftMessageRequest.Size() @@ -998,6 +1260,9 @@ func (m *SnapshotRequest_Header) Size() (n int) { } func (m *SnapshotResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovRaft(uint64(m.Status)) @@ -1007,6 +1272,9 @@ func (m *SnapshotResponse) Size() (n int) { } func (m *ConfChangeContext) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.CommandID) @@ -1042,7 +1310,7 @@ func (this *RaftMessageResponseUnion) GetValue() interface{} { func (this *RaftMessageResponseUnion) SetValue(value interface{}) bool { switch vt := value.(type) { - case *cockroach_roachpb3.Error: + case *roachpb.Error: this.Error = vt default: return false @@ -1623,7 +1891,7 @@ func (m *RaftMessageResponseUnion) Unmarshal(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Error == nil { - m.Error = &cockroach_roachpb3.Error{} + m.Error = &roachpb.Error{} } if err := m.Error.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err @@ -2502,9 +2770,9 @@ var ( ErrIntOverflowRaft = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("storage/raft.proto", fileDescriptorRaft) } +func init() { proto.RegisterFile("storage/raft.proto", fileDescriptor_raft_460a63b017d715a3) } -var fileDescriptorRaft = []byte{ +var fileDescriptor_raft_460a63b017d715a3 = []byte{ // 1147 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x56, 0xdd, 0x6e, 0x1b, 0xc5, 0x17, 0xf7, 0xc6, 0xdf, 0xc7, 0x76, 0xb3, 0x9d, 0x7f, 0xf5, 0x67, 0x65, 0xc0, 0x36, 0x5b, 0x5a, diff --git a/pkg/storage/rangefeed/resolved_timestamp.go b/pkg/storage/rangefeed/resolved_timestamp.go index 1d91c1d47a96..b1d4a7e233fc 100644 --- a/pkg/storage/rangefeed/resolved_timestamp.go +++ b/pkg/storage/rangefeed/resolved_timestamp.go @@ -209,7 +209,7 @@ func (rts *resolvedTimestamp) assertNoChange() { // indicate a failure of the closed timestamp mechanism. func (rts *resolvedTimestamp) assertOpAboveRTS(op enginepb.MVCCLogicalOp, opTS hlc.Timestamp) { if !rts.resolvedTS.Less(opTS) { - panic(fmt.Sprintf("resolved timestamp %s equal to or above timestamp of operation %s", + panic(fmt.Sprintf("resolved timestamp %s equal to or above timestamp of operation %v", rts.resolvedTS, op)) } } diff --git a/pkg/storage/replica_consistency.go b/pkg/storage/replica_consistency.go index 48a213af234d..d61765544848 100644 --- a/pkg/storage/replica_consistency.go +++ b/pkg/storage/replica_consistency.go @@ -182,10 +182,10 @@ func (r *Replica) collectChecksumFromReplica( } client := NewPerReplicaClient(conn) req := &CollectChecksumRequest{ - StoreRequestHeader{NodeID: replica.NodeID, StoreID: replica.StoreID}, - r.RangeID, - id, - checksum, + StoreRequestHeader: StoreRequestHeader{NodeID: replica.NodeID, StoreID: replica.StoreID}, + RangeID: r.RangeID, + ChecksumID: id, + Checksum: checksum, } resp, err := client.CollectChecksum(ctx, req) if err != nil { diff --git a/pkg/storage/storagepb/lease_status.pb.go b/pkg/storage/storagepb/lease_status.pb.go index 92e4185d7876..b7bb29982d7d 100644 --- a/pkg/storage/storagepb/lease_status.pb.go +++ b/pkg/storage/storagepb/lease_status.pb.go @@ -1,41 +1,13 @@ // Code generated by protoc-gen-gogo. DO NOT EDIT. // source: storage/storagepb/lease_status.proto -/* - Package storagepb is a generated protocol buffer package. - - It is generated from these files: - storage/storagepb/lease_status.proto - storage/storagepb/liveness.proto - storage/storagepb/log.proto - storage/storagepb/proposer_kv.proto - storage/storagepb/state.proto - - It has these top-level messages: - LeaseStatus - Liveness - RangeLogEvent - Split - Merge - ChangeReplicas - ComputeChecksum - Compaction - SuggestedCompaction - ReplicatedEvalResult - WriteBatch - LogicalOpLog - RaftCommand - ReplicaState - RangeInfo - LatchManagerInfo -*/ package storagepb import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" -import cockroach_roachpb1 "github.com/cockroachdb/cockroach/pkg/roachpb" -import cockroach_util_hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" +import roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" +import hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" import io "io" @@ -84,26 +56,54 @@ var LeaseState_value = map[string]int32{ func (x LeaseState) String() string { return proto.EnumName(LeaseState_name, int32(x)) } -func (LeaseState) EnumDescriptor() ([]byte, []int) { return fileDescriptorLeaseStatus, []int{0} } +func (LeaseState) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_lease_status_95911d6a50f866a6, []int{0} +} // LeaseStatus holds the lease state, the timestamp at which the state // is accurate, the lease and optionally the liveness if the lease is // epoch-based. type LeaseStatus struct { // Lease which this status describes. - Lease cockroach_roachpb1.Lease `protobuf:"bytes,1,opt,name=lease" json:"lease"` + Lease roachpb.Lease `protobuf:"bytes,1,opt,name=lease,proto3" json:"lease"` // Timestamp that the lease was evaluated at. - Timestamp cockroach_util_hlc.Timestamp `protobuf:"bytes,2,opt,name=timestamp" json:"timestamp"` + Timestamp hlc.Timestamp `protobuf:"bytes,2,opt,name=timestamp,proto3" json:"timestamp"` // State of the lease at timestamp. State LeaseState `protobuf:"varint,3,opt,name=state,proto3,enum=cockroach.storage.LeaseState" json:"state,omitempty"` // Liveness if this is an epoch-based lease. - Liveness *Liveness `protobuf:"bytes,4,opt,name=liveness" json:"liveness,omitempty"` + Liveness *Liveness `protobuf:"bytes,4,opt,name=liveness,proto3" json:"liveness,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *LeaseStatus) Reset() { *m = LeaseStatus{} } +func (m *LeaseStatus) String() string { return proto.CompactTextString(m) } +func (*LeaseStatus) ProtoMessage() {} +func (*LeaseStatus) Descriptor() ([]byte, []int) { + return fileDescriptor_lease_status_95911d6a50f866a6, []int{0} +} +func (m *LeaseStatus) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *LeaseStatus) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *LeaseStatus) XXX_Merge(src proto.Message) { + xxx_messageInfo_LeaseStatus.Merge(dst, src) +} +func (m *LeaseStatus) XXX_Size() int { + return m.Size() +} +func (m *LeaseStatus) XXX_DiscardUnknown() { + xxx_messageInfo_LeaseStatus.DiscardUnknown(m) } -func (m *LeaseStatus) Reset() { *m = LeaseStatus{} } -func (m *LeaseStatus) String() string { return proto.CompactTextString(m) } -func (*LeaseStatus) ProtoMessage() {} -func (*LeaseStatus) Descriptor() ([]byte, []int) { return fileDescriptorLeaseStatus, []int{0} } +var xxx_messageInfo_LeaseStatus proto.InternalMessageInfo func init() { proto.RegisterType((*LeaseStatus)(nil), "cockroach.storage.LeaseStatus") @@ -168,6 +168,9 @@ func encodeVarintLeaseStatus(dAtA []byte, offset int, v uint64) int { return offset + 1 } func (m *LeaseStatus) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.Lease.Size() @@ -464,9 +467,11 @@ var ( ErrIntOverflowLeaseStatus = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("storage/storagepb/lease_status.proto", fileDescriptorLeaseStatus) } +func init() { + proto.RegisterFile("storage/storagepb/lease_status.proto", fileDescriptor_lease_status_95911d6a50f866a6) +} -var fileDescriptorLeaseStatus = []byte{ +var fileDescriptor_lease_status_95911d6a50f866a6 = []byte{ // 346 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x6c, 0x91, 0xcf, 0x4a, 0xeb, 0x40, 0x14, 0x87, 0x33, 0xfd, 0x77, 0x6f, 0x4f, 0xa1, 0xc4, 0xc1, 0x45, 0xa8, 0x18, 0x8b, 0xba, 0x28, diff --git a/pkg/storage/storagepb/liveness.pb.go b/pkg/storage/storagepb/liveness.pb.go index dcad2bbfd300..d08757daee66 100644 --- a/pkg/storage/storagepb/liveness.pb.go +++ b/pkg/storage/storagepb/liveness.pb.go @@ -6,7 +6,7 @@ package storagepb import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" -import cockroach_util_hlc1 "github.com/cockroachdb/cockroach/pkg/util/hlc" +import hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" import github_com_cockroachdb_cockroach_pkg_roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -17,6 +17,12 @@ var _ = proto.Marshal var _ = fmt.Errorf var _ = math.Inf +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package + // NodeLivenessStatus describes the status of a node from the perspective of the // liveness system. type NodeLivenessStatus int32 @@ -58,7 +64,9 @@ var NodeLivenessStatus_value = map[string]int32{ func (x NodeLivenessStatus) String() string { return proto.EnumName(NodeLivenessStatus_name, int32(x)) } -func (NodeLivenessStatus) EnumDescriptor() ([]byte, []int) { return fileDescriptorLiveness, []int{0} } +func (NodeLivenessStatus) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_liveness_6f963fa8db838a98, []int{0} +} // Liveness holds information about a node's latest heartbeat and epoch. // @@ -71,15 +79,41 @@ type Liveness struct { // is later than the expiration timestamp). Epoch int64 `protobuf:"varint,2,opt,name=epoch,proto3" json:"epoch,omitempty"` // The timestamp at which this liveness record expires. - Expiration cockroach_util_hlc1.LegacyTimestamp `protobuf:"bytes,3,opt,name=expiration" json:"expiration"` - Draining bool `protobuf:"varint,4,opt,name=draining,proto3" json:"draining,omitempty"` - Decommissioning bool `protobuf:"varint,5,opt,name=decommissioning,proto3" json:"decommissioning,omitempty"` + Expiration hlc.LegacyTimestamp `protobuf:"bytes,3,opt,name=expiration,proto3" json:"expiration"` + Draining bool `protobuf:"varint,4,opt,name=draining,proto3" json:"draining,omitempty"` + Decommissioning bool `protobuf:"varint,5,opt,name=decommissioning,proto3" json:"decommissioning,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *Liveness) Reset() { *m = Liveness{} } -func (m *Liveness) String() string { return proto.CompactTextString(m) } -func (*Liveness) ProtoMessage() {} -func (*Liveness) Descriptor() ([]byte, []int) { return fileDescriptorLiveness, []int{0} } +func (m *Liveness) Reset() { *m = Liveness{} } +func (m *Liveness) String() string { return proto.CompactTextString(m) } +func (*Liveness) ProtoMessage() {} +func (*Liveness) Descriptor() ([]byte, []int) { + return fileDescriptor_liveness_6f963fa8db838a98, []int{0} +} +func (m *Liveness) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Liveness) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *Liveness) XXX_Merge(src proto.Message) { + xxx_messageInfo_Liveness.Merge(dst, src) +} +func (m *Liveness) XXX_Size() int { + return m.Size() +} +func (m *Liveness) XXX_DiscardUnknown() { + xxx_messageInfo_Liveness.DiscardUnknown(m) +} + +var xxx_messageInfo_Liveness proto.InternalMessageInfo func init() { proto.RegisterType((*Liveness)(nil), "cockroach.storage.Liveness") @@ -160,7 +194,7 @@ func NewPopulatedLiveness(r randyLiveness, easy bool) *Liveness { if r.Intn(2) == 0 { this.Epoch *= -1 } - v1 := cockroach_util_hlc1.NewPopulatedLegacyTimestamp(r, easy) + v1 := hlc.NewPopulatedLegacyTimestamp(r, easy) this.Expiration = *v1 this.Draining = bool(bool(r.Intn(2) == 0)) this.Decommissioning = bool(bool(r.Intn(2) == 0)) @@ -242,6 +276,9 @@ func encodeVarintPopulateLiveness(dAtA []byte, v uint64) []byte { return dAtA } func (m *Liveness) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.NodeID != 0 { @@ -537,9 +574,11 @@ var ( ErrIntOverflowLiveness = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("storage/storagepb/liveness.proto", fileDescriptorLiveness) } +func init() { + proto.RegisterFile("storage/storagepb/liveness.proto", fileDescriptor_liveness_6f963fa8db838a98) +} -var fileDescriptorLiveness = []byte{ +var fileDescriptor_liveness_6f963fa8db838a98 = []byte{ // 418 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x5c, 0x52, 0xc1, 0x8e, 0x93, 0x40, 0x18, 0xee, 0xb4, 0xb4, 0x5b, 0xa7, 0x89, 0xc5, 0x71, 0x0f, 0x4d, 0x0f, 0x40, 0xdc, 0x0b, 0xf1, diff --git a/pkg/storage/storagepb/log.pb.go b/pkg/storage/storagepb/log.pb.go index 13ee4797a7fb..b060a28d9a5d 100644 --- a/pkg/storage/storagepb/log.pb.go +++ b/pkg/storage/storagepb/log.pb.go @@ -6,12 +6,12 @@ package storagepb import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" -import cockroach_roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" +import roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" import time "time" import github_com_cockroachdb_cockroach_pkg_roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" -import types "github.com/gogo/protobuf/types" +import github_com_gogo_protobuf_types "github.com/gogo/protobuf/types" import io "io" @@ -21,6 +21,12 @@ var _ = fmt.Errorf var _ = math.Inf var _ = time.Kitchen +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package + type RangeLogEventType int32 const ( @@ -52,36 +58,90 @@ var RangeLogEventType_value = map[string]int32{ func (x RangeLogEventType) String() string { return proto.EnumName(RangeLogEventType_name, int32(x)) } -func (RangeLogEventType) EnumDescriptor() ([]byte, []int) { return fileDescriptorLog, []int{0} } +func (RangeLogEventType) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_log_ec9647c2a5c91039, []int{0} +} type RangeLogEvent struct { - Timestamp time.Time `protobuf:"bytes,1,opt,name=timestamp,stdtime" json:"timestamp"` - RangeID github_com_cockroachdb_cockroach_pkg_roachpb.RangeID `protobuf:"varint,2,opt,name=range_id,json=rangeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" json:"range_id,omitempty"` - StoreID github_com_cockroachdb_cockroach_pkg_roachpb.StoreID `protobuf:"varint,3,opt,name=store_id,json=storeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.StoreID" json:"store_id,omitempty"` - EventType RangeLogEventType `protobuf:"varint,4,opt,name=event_type,json=eventType,proto3,enum=cockroach.storage.RangeLogEventType" json:"event_type,omitempty"` - OtherRangeID github_com_cockroachdb_cockroach_pkg_roachpb.RangeID `protobuf:"varint,5,opt,name=other_range_id,json=otherRangeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" json:"other_range_id,omitempty"` - Info *RangeLogEvent_Info `protobuf:"bytes,6,opt,name=info" json:"info,omitempty"` + Timestamp time.Time `protobuf:"bytes,1,opt,name=timestamp,proto3,stdtime" json:"timestamp"` + RangeID github_com_cockroachdb_cockroach_pkg_roachpb.RangeID `protobuf:"varint,2,opt,name=range_id,json=rangeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" json:"range_id,omitempty"` + StoreID github_com_cockroachdb_cockroach_pkg_roachpb.StoreID `protobuf:"varint,3,opt,name=store_id,json=storeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.StoreID" json:"store_id,omitempty"` + EventType RangeLogEventType `protobuf:"varint,4,opt,name=event_type,json=eventType,proto3,enum=cockroach.storage.RangeLogEventType" json:"event_type,omitempty"` + OtherRangeID github_com_cockroachdb_cockroach_pkg_roachpb.RangeID `protobuf:"varint,5,opt,name=other_range_id,json=otherRangeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" json:"other_range_id,omitempty"` + Info *RangeLogEvent_Info `protobuf:"bytes,6,opt,name=info,proto3" json:"info,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *RangeLogEvent) Reset() { *m = RangeLogEvent{} } +func (m *RangeLogEvent) String() string { return proto.CompactTextString(m) } +func (*RangeLogEvent) ProtoMessage() {} +func (*RangeLogEvent) Descriptor() ([]byte, []int) { + return fileDescriptor_log_ec9647c2a5c91039, []int{0} +} +func (m *RangeLogEvent) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RangeLogEvent) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RangeLogEvent) XXX_Merge(src proto.Message) { + xxx_messageInfo_RangeLogEvent.Merge(dst, src) +} +func (m *RangeLogEvent) XXX_Size() int { + return m.Size() +} +func (m *RangeLogEvent) XXX_DiscardUnknown() { + xxx_messageInfo_RangeLogEvent.DiscardUnknown(m) } -func (m *RangeLogEvent) Reset() { *m = RangeLogEvent{} } -func (m *RangeLogEvent) String() string { return proto.CompactTextString(m) } -func (*RangeLogEvent) ProtoMessage() {} -func (*RangeLogEvent) Descriptor() ([]byte, []int) { return fileDescriptorLog, []int{0} } +var xxx_messageInfo_RangeLogEvent proto.InternalMessageInfo type RangeLogEvent_Info struct { - UpdatedDesc *cockroach_roachpb.RangeDescriptor `protobuf:"bytes,1,opt,name=updated_desc,json=updatedDesc" json:"UpdatedDesc,omitempty"` - NewDesc *cockroach_roachpb.RangeDescriptor `protobuf:"bytes,2,opt,name=new_desc,json=newDesc" json:"NewDesc,omitempty"` - RemovedDesc *cockroach_roachpb.RangeDescriptor `protobuf:"bytes,7,opt,name=removed_desc,json=removedDesc" json:"RemovedDesc,omitempty"` - AddedReplica *cockroach_roachpb.ReplicaDescriptor `protobuf:"bytes,3,opt,name=added_replica,json=addedReplica" json:"AddReplica,omitempty"` - RemovedReplica *cockroach_roachpb.ReplicaDescriptor `protobuf:"bytes,4,opt,name=removed_replica,json=removedReplica" json:"RemovedReplica,omitempty"` - Reason RangeLogEventReason `protobuf:"bytes,5,opt,name=reason,proto3,casttype=RangeLogEventReason" json:"Reason,omitempty"` - Details string `protobuf:"bytes,6,opt,name=details,proto3" json:"Details,omitempty"` + UpdatedDesc *roachpb.RangeDescriptor `protobuf:"bytes,1,opt,name=updated_desc,json=updatedDesc,proto3" json:"UpdatedDesc,omitempty"` + NewDesc *roachpb.RangeDescriptor `protobuf:"bytes,2,opt,name=new_desc,json=newDesc,proto3" json:"NewDesc,omitempty"` + RemovedDesc *roachpb.RangeDescriptor `protobuf:"bytes,7,opt,name=removed_desc,json=removedDesc,proto3" json:"RemovedDesc,omitempty"` + AddedReplica *roachpb.ReplicaDescriptor `protobuf:"bytes,3,opt,name=added_replica,json=addedReplica,proto3" json:"AddReplica,omitempty"` + RemovedReplica *roachpb.ReplicaDescriptor `protobuf:"bytes,4,opt,name=removed_replica,json=removedReplica,proto3" json:"RemovedReplica,omitempty"` + Reason RangeLogEventReason `protobuf:"bytes,5,opt,name=reason,proto3,casttype=RangeLogEventReason" json:"Reason,omitempty"` + Details string `protobuf:"bytes,6,opt,name=details,proto3" json:"Details,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *RangeLogEvent_Info) Reset() { *m = RangeLogEvent_Info{} } -func (m *RangeLogEvent_Info) String() string { return proto.CompactTextString(m) } -func (*RangeLogEvent_Info) ProtoMessage() {} -func (*RangeLogEvent_Info) Descriptor() ([]byte, []int) { return fileDescriptorLog, []int{0, 0} } +func (m *RangeLogEvent_Info) Reset() { *m = RangeLogEvent_Info{} } +func (m *RangeLogEvent_Info) String() string { return proto.CompactTextString(m) } +func (*RangeLogEvent_Info) ProtoMessage() {} +func (*RangeLogEvent_Info) Descriptor() ([]byte, []int) { + return fileDescriptor_log_ec9647c2a5c91039, []int{0, 0} +} +func (m *RangeLogEvent_Info) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RangeLogEvent_Info) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RangeLogEvent_Info) XXX_Merge(src proto.Message) { + xxx_messageInfo_RangeLogEvent_Info.Merge(dst, src) +} +func (m *RangeLogEvent_Info) XXX_Size() int { + return m.Size() +} +func (m *RangeLogEvent_Info) XXX_DiscardUnknown() { + xxx_messageInfo_RangeLogEvent_Info.DiscardUnknown(m) +} + +var xxx_messageInfo_RangeLogEvent_Info proto.InternalMessageInfo func init() { proto.RegisterType((*RangeLogEvent)(nil), "cockroach.storage.RangeLogEvent") @@ -105,8 +165,8 @@ func (m *RangeLogEvent) MarshalTo(dAtA []byte) (int, error) { _ = l dAtA[i] = 0xa i++ - i = encodeVarintLog(dAtA, i, uint64(types.SizeOfStdTime(m.Timestamp))) - n1, err := types.StdTimeMarshalTo(m.Timestamp, dAtA[i:]) + i = encodeVarintLog(dAtA, i, uint64(github_com_gogo_protobuf_types.SizeOfStdTime(m.Timestamp))) + n1, err := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Timestamp, dAtA[i:]) if err != nil { return 0, err } @@ -234,9 +294,12 @@ func encodeVarintLog(dAtA []byte, offset int, v uint64) int { return offset + 1 } func (m *RangeLogEvent) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l - l = types.SizeOfStdTime(m.Timestamp) + l = github_com_gogo_protobuf_types.SizeOfStdTime(m.Timestamp) n += 1 + l + sovLog(uint64(l)) if m.RangeID != 0 { n += 1 + sovLog(uint64(m.RangeID)) @@ -258,6 +321,9 @@ func (m *RangeLogEvent) Size() (n int) { } func (m *RangeLogEvent_Info) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.UpdatedDesc != nil { @@ -359,7 +425,7 @@ func (m *RangeLogEvent) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if err := types.StdTimeUnmarshal(&m.Timestamp, dAtA[iNdEx:postIndex]); err != nil { + if err := github_com_gogo_protobuf_types.StdTimeUnmarshal(&m.Timestamp, dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -549,7 +615,7 @@ func (m *RangeLogEvent_Info) Unmarshal(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.UpdatedDesc == nil { - m.UpdatedDesc = &cockroach_roachpb.RangeDescriptor{} + m.UpdatedDesc = &roachpb.RangeDescriptor{} } if err := m.UpdatedDesc.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err @@ -582,7 +648,7 @@ func (m *RangeLogEvent_Info) Unmarshal(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.NewDesc == nil { - m.NewDesc = &cockroach_roachpb.RangeDescriptor{} + m.NewDesc = &roachpb.RangeDescriptor{} } if err := m.NewDesc.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err @@ -615,7 +681,7 @@ func (m *RangeLogEvent_Info) Unmarshal(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.AddedReplica == nil { - m.AddedReplica = &cockroach_roachpb.ReplicaDescriptor{} + m.AddedReplica = &roachpb.ReplicaDescriptor{} } if err := m.AddedReplica.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err @@ -648,7 +714,7 @@ func (m *RangeLogEvent_Info) Unmarshal(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.RemovedReplica == nil { - m.RemovedReplica = &cockroach_roachpb.ReplicaDescriptor{} + m.RemovedReplica = &roachpb.ReplicaDescriptor{} } if err := m.RemovedReplica.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err @@ -739,7 +805,7 @@ func (m *RangeLogEvent_Info) Unmarshal(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.RemovedDesc == nil { - m.RemovedDesc = &cockroach_roachpb.RangeDescriptor{} + m.RemovedDesc = &roachpb.RangeDescriptor{} } if err := m.RemovedDesc.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err @@ -871,9 +937,9 @@ var ( ErrIntOverflowLog = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("storage/storagepb/log.proto", fileDescriptorLog) } +func init() { proto.RegisterFile("storage/storagepb/log.proto", fileDescriptor_log_ec9647c2a5c91039) } -var fileDescriptorLog = []byte{ +var fileDescriptor_log_ec9647c2a5c91039 = []byte{ // 638 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x94, 0xcd, 0x6e, 0xd3, 0x40, 0x14, 0x85, 0xe3, 0x26, 0xad, 0x93, 0x69, 0x5a, 0xd2, 0xa1, 0x45, 0x21, 0xa0, 0x38, 0x2a, 0x20, diff --git a/pkg/storage/storagepb/proposer_kv.pb.go b/pkg/storage/storagepb/proposer_kv.pb.go index c156b330ab54..d2bf4e5026b7 100644 --- a/pkg/storage/storagepb/proposer_kv.pb.go +++ b/pkg/storage/storagepb/proposer_kv.pb.go @@ -6,11 +6,9 @@ package storagepb import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" -import cockroach_roachpb1 "github.com/cockroachdb/cockroach/pkg/roachpb" -import cockroach_roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" -import cockroach_storage_engine_enginepb1 "github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb" -import cockroach_storage_engine_enginepb "github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb" -import cockroach_util_hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" +import roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" +import enginepb "github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb" +import hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" import github_com_cockroachdb_cockroach_pkg_util_uuid "github.com/cockroachdb/cockroach/pkg/util/uuid" import github_com_cockroachdb_cockroach_pkg_roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -24,44 +22,128 @@ var _ = proto.Marshal var _ = fmt.Errorf var _ = math.Inf +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package + // Split is emitted when a Replica commits a split trigger. It signals that the // Replica has prepared the on-disk state for both the left and right hand // sides of the split, and that the left hand side Replica should be updated as // well as the right hand side created. type Split struct { - cockroach_roachpb1.SplitTrigger `protobuf:"bytes,1,opt,name=trigger,embedded=trigger" json:"trigger"` + roachpb.SplitTrigger `protobuf:"bytes,1,opt,name=trigger,proto3,embedded=trigger" json:"trigger"` // RHSDelta holds the statistics for what was written to what is now the // right-hand side of the split during the batch which executed it. // The on-disk state of the right-hand side is already correct, but the // Store must learn about this delta to update its counters appropriately. - RHSDelta cockroach_storage_engine_enginepb1.MVCCStats `protobuf:"bytes,2,opt,name=rhs_delta,json=rhsDelta" json:"rhs_delta"` + RHSDelta enginepb.MVCCStats `protobuf:"bytes,2,opt,name=rhs_delta,json=rhsDelta,proto3" json:"rhs_delta"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Split) Reset() { *m = Split{} } +func (m *Split) String() string { return proto.CompactTextString(m) } +func (*Split) ProtoMessage() {} +func (*Split) Descriptor() ([]byte, []int) { + return fileDescriptor_proposer_kv_e79e54048b29cf41, []int{0} +} +func (m *Split) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Split) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *Split) XXX_Merge(src proto.Message) { + xxx_messageInfo_Split.Merge(dst, src) +} +func (m *Split) XXX_Size() int { + return m.Size() +} +func (m *Split) XXX_DiscardUnknown() { + xxx_messageInfo_Split.DiscardUnknown(m) } -func (m *Split) Reset() { *m = Split{} } -func (m *Split) String() string { return proto.CompactTextString(m) } -func (*Split) ProtoMessage() {} -func (*Split) Descriptor() ([]byte, []int) { return fileDescriptorProposerKv, []int{0} } +var xxx_messageInfo_Split proto.InternalMessageInfo // Merge is emitted by a Replica which commits a transaction with // a MergeTrigger (i.e. absorbs its right neighbor). type Merge struct { - cockroach_roachpb1.MergeTrigger `protobuf:"bytes,1,opt,name=trigger,embedded=trigger" json:"trigger"` + roachpb.MergeTrigger `protobuf:"bytes,1,opt,name=trigger,proto3,embedded=trigger" json:"trigger"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *Merge) Reset() { *m = Merge{} } -func (m *Merge) String() string { return proto.CompactTextString(m) } -func (*Merge) ProtoMessage() {} -func (*Merge) Descriptor() ([]byte, []int) { return fileDescriptorProposerKv, []int{1} } +func (m *Merge) Reset() { *m = Merge{} } +func (m *Merge) String() string { return proto.CompactTextString(m) } +func (*Merge) ProtoMessage() {} +func (*Merge) Descriptor() ([]byte, []int) { + return fileDescriptor_proposer_kv_e79e54048b29cf41, []int{1} +} +func (m *Merge) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Merge) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *Merge) XXX_Merge(src proto.Message) { + xxx_messageInfo_Merge.Merge(dst, src) +} +func (m *Merge) XXX_Size() int { + return m.Size() +} +func (m *Merge) XXX_DiscardUnknown() { + xxx_messageInfo_Merge.DiscardUnknown(m) +} + +var xxx_messageInfo_Merge proto.InternalMessageInfo // ChangeReplicas is emitted by a Replica which commits a transaction with // a ChangeReplicasTrigger. type ChangeReplicas struct { - cockroach_roachpb1.ChangeReplicasTrigger `protobuf:"bytes,1,opt,name=trigger,embedded=trigger" json:"trigger"` + roachpb.ChangeReplicasTrigger `protobuf:"bytes,1,opt,name=trigger,proto3,embedded=trigger" json:"trigger"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ChangeReplicas) Reset() { *m = ChangeReplicas{} } +func (*ChangeReplicas) ProtoMessage() {} +func (*ChangeReplicas) Descriptor() ([]byte, []int) { + return fileDescriptor_proposer_kv_e79e54048b29cf41, []int{2} +} +func (m *ChangeReplicas) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ChangeReplicas) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ChangeReplicas) XXX_Merge(src proto.Message) { + xxx_messageInfo_ChangeReplicas.Merge(dst, src) +} +func (m *ChangeReplicas) XXX_Size() int { + return m.Size() +} +func (m *ChangeReplicas) XXX_DiscardUnknown() { + xxx_messageInfo_ChangeReplicas.DiscardUnknown(m) } -func (m *ChangeReplicas) Reset() { *m = ChangeReplicas{} } -func (*ChangeReplicas) ProtoMessage() {} -func (*ChangeReplicas) Descriptor() ([]byte, []int) { return fileDescriptorProposerKv, []int{2} } +var xxx_messageInfo_ChangeReplicas proto.InternalMessageInfo // ComputeChecksum is emitted when a ComputeChecksum request is evaluated. It // instructs the replica to compute a checksum at the time the command is @@ -72,39 +154,117 @@ type ComputeChecksum struct { ChecksumID github_com_cockroachdb_cockroach_pkg_util_uuid.UUID `protobuf:"bytes,1,opt,name=checksum_id,json=checksumId,proto3,customtype=github.com/cockroachdb/cockroach/pkg/util/uuid.UUID" json:"checksum_id"` // SaveSnapshot indicates that the snapshot used to compute the checksum // should be saved so that a diff of divergent replicas can later be computed. - SaveSnapshot bool `protobuf:"varint,2,opt,name=save_snapshot,json=saveSnapshot,proto3" json:"save_snapshot,omitempty"` + SaveSnapshot bool `protobuf:"varint,2,opt,name=save_snapshot,json=saveSnapshot,proto3" json:"save_snapshot,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *ComputeChecksum) Reset() { *m = ComputeChecksum{} } -func (m *ComputeChecksum) String() string { return proto.CompactTextString(m) } -func (*ComputeChecksum) ProtoMessage() {} -func (*ComputeChecksum) Descriptor() ([]byte, []int) { return fileDescriptorProposerKv, []int{3} } +func (m *ComputeChecksum) Reset() { *m = ComputeChecksum{} } +func (m *ComputeChecksum) String() string { return proto.CompactTextString(m) } +func (*ComputeChecksum) ProtoMessage() {} +func (*ComputeChecksum) Descriptor() ([]byte, []int) { + return fileDescriptor_proposer_kv_e79e54048b29cf41, []int{3} +} +func (m *ComputeChecksum) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ComputeChecksum) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ComputeChecksum) XXX_Merge(src proto.Message) { + xxx_messageInfo_ComputeChecksum.Merge(dst, src) +} +func (m *ComputeChecksum) XXX_Size() int { + return m.Size() +} +func (m *ComputeChecksum) XXX_DiscardUnknown() { + xxx_messageInfo_ComputeChecksum.DiscardUnknown(m) +} + +var xxx_messageInfo_ComputeChecksum proto.InternalMessageInfo // Compaction holds core details about a suggested compaction. type Compaction struct { // bytes indicates the expected space reclamation from compaction. Bytes int64 `protobuf:"varint,1,opt,name=bytes,proto3" json:"bytes,omitempty"` // suggested_at is nanoseconds since the epoch. - SuggestedAtNanos int64 `protobuf:"varint,2,opt,name=suggested_at_nanos,json=suggestedAtNanos,proto3" json:"suggested_at_nanos,omitempty"` + SuggestedAtNanos int64 `protobuf:"varint,2,opt,name=suggested_at_nanos,json=suggestedAtNanos,proto3" json:"suggested_at_nanos,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *Compaction) Reset() { *m = Compaction{} } -func (m *Compaction) String() string { return proto.CompactTextString(m) } -func (*Compaction) ProtoMessage() {} -func (*Compaction) Descriptor() ([]byte, []int) { return fileDescriptorProposerKv, []int{4} } +func (m *Compaction) Reset() { *m = Compaction{} } +func (m *Compaction) String() string { return proto.CompactTextString(m) } +func (*Compaction) ProtoMessage() {} +func (*Compaction) Descriptor() ([]byte, []int) { + return fileDescriptor_proposer_kv_e79e54048b29cf41, []int{4} +} +func (m *Compaction) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Compaction) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *Compaction) XXX_Merge(src proto.Message) { + xxx_messageInfo_Compaction.Merge(dst, src) +} +func (m *Compaction) XXX_Size() int { + return m.Size() +} +func (m *Compaction) XXX_DiscardUnknown() { + xxx_messageInfo_Compaction.DiscardUnknown(m) +} + +var xxx_messageInfo_Compaction proto.InternalMessageInfo // SuggestedCompaction holds start and end keys in conjunction with // the compaction details. type SuggestedCompaction struct { - StartKey github_com_cockroachdb_cockroach_pkg_roachpb.Key `protobuf:"bytes,1,opt,name=start_key,json=startKey,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.Key" json:"start_key,omitempty"` - EndKey github_com_cockroachdb_cockroach_pkg_roachpb.Key `protobuf:"bytes,2,opt,name=end_key,json=endKey,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.Key" json:"end_key,omitempty"` - Compaction `protobuf:"bytes,3,opt,name=compaction,embedded=compaction" json:"compaction"` + StartKey github_com_cockroachdb_cockroach_pkg_roachpb.Key `protobuf:"bytes,1,opt,name=start_key,json=startKey,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.Key" json:"start_key,omitempty"` + EndKey github_com_cockroachdb_cockroach_pkg_roachpb.Key `protobuf:"bytes,2,opt,name=end_key,json=endKey,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.Key" json:"end_key,omitempty"` + Compaction `protobuf:"bytes,3,opt,name=compaction,proto3,embedded=compaction" json:"compaction"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *SuggestedCompaction) Reset() { *m = SuggestedCompaction{} } -func (m *SuggestedCompaction) String() string { return proto.CompactTextString(m) } -func (*SuggestedCompaction) ProtoMessage() {} -func (*SuggestedCompaction) Descriptor() ([]byte, []int) { return fileDescriptorProposerKv, []int{5} } +func (m *SuggestedCompaction) Reset() { *m = SuggestedCompaction{} } +func (m *SuggestedCompaction) String() string { return proto.CompactTextString(m) } +func (*SuggestedCompaction) ProtoMessage() {} +func (*SuggestedCompaction) Descriptor() ([]byte, []int) { + return fileDescriptor_proposer_kv_e79e54048b29cf41, []int{5} +} +func (m *SuggestedCompaction) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *SuggestedCompaction) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *SuggestedCompaction) XXX_Merge(src proto.Message) { + xxx_messageInfo_SuggestedCompaction.Merge(dst, src) +} +func (m *SuggestedCompaction) XXX_Size() int { + return m.Size() +} +func (m *SuggestedCompaction) XXX_DiscardUnknown() { + xxx_messageInfo_SuggestedCompaction.DiscardUnknown(m) +} + +var xxx_messageInfo_SuggestedCompaction proto.InternalMessageInfo // ReplicatedEvalResult is the structured information which together with // a RocksDB WriteBatch constitutes the proposal payload in proposer-evaluated @@ -119,37 +279,63 @@ type ReplicatedEvalResult struct { // the comment on the ReplicaState message, this field is sparsely populated // and any field set overwrites the corresponding field in the state, perhaps // with additional side effects (for instance on a descriptor update). - State *ReplicaState `protobuf:"bytes,2,opt,name=state" json:"state,omitempty"` - Split *Split `protobuf:"bytes,3,opt,name=split" json:"split,omitempty"` - Merge *Merge `protobuf:"bytes,4,opt,name=merge" json:"merge,omitempty"` - ComputeChecksum *ComputeChecksum `protobuf:"bytes,21,opt,name=compute_checksum,json=computeChecksum" json:"compute_checksum,omitempty"` + State *ReplicaState `protobuf:"bytes,2,opt,name=state,proto3" json:"state,omitempty"` + Split *Split `protobuf:"bytes,3,opt,name=split,proto3" json:"split,omitempty"` + Merge *Merge `protobuf:"bytes,4,opt,name=merge,proto3" json:"merge,omitempty"` + ComputeChecksum *ComputeChecksum `protobuf:"bytes,21,opt,name=compute_checksum,json=computeChecksum,proto3" json:"compute_checksum,omitempty"` IsLeaseRequest bool `protobuf:"varint,6,opt,name=is_lease_request,json=isLeaseRequest,proto3" json:"is_lease_request,omitempty"` // Duplicates BatchRequest.Timestamp for proposer-evaluated KV. Used // to verify the validity of the command (for lease coverage and GC // threshold). - Timestamp cockroach_util_hlc.Timestamp `protobuf:"bytes,8,opt,name=timestamp" json:"timestamp"` - IsConsistencyRelated bool `protobuf:"varint,9,opt,name=is_consistency_related,json=isConsistencyRelated,proto3" json:"is_consistency_related,omitempty"` + Timestamp hlc.Timestamp `protobuf:"bytes,8,opt,name=timestamp,proto3" json:"timestamp"` + IsConsistencyRelated bool `protobuf:"varint,9,opt,name=is_consistency_related,json=isConsistencyRelated,proto3" json:"is_consistency_related,omitempty"` // The stats delta corresponding to the data in this WriteBatch. On // a split, contains only the contributions to the left-hand side. - DeprecatedDelta *cockroach_storage_engine_enginepb1.MVCCStats `protobuf:"bytes,10,opt,name=deprecated_delta,json=deprecatedDelta" json:"deprecated_delta,omitempty"` - Delta cockroach_storage_engine_enginepb.MVCCStatsDelta `protobuf:"bytes,18,opt,name=delta" json:"delta"` - ChangeReplicas *ChangeReplicas `protobuf:"bytes,12,opt,name=change_replicas,json=changeReplicas" json:"change_replicas,omitempty"` - RaftLogDelta int64 `protobuf:"varint,13,opt,name=raft_log_delta,json=raftLogDelta,proto3" json:"raft_log_delta,omitempty"` - AddSSTable *ReplicatedEvalResult_AddSSTable `protobuf:"bytes,17,opt,name=add_sstable,json=addSstable" json:"add_sstable,omitempty"` + DeprecatedDelta *enginepb.MVCCStats `protobuf:"bytes,10,opt,name=deprecated_delta,json=deprecatedDelta,proto3" json:"deprecated_delta,omitempty"` + Delta enginepb.MVCCStatsDelta `protobuf:"bytes,18,opt,name=delta,proto3" json:"delta"` + ChangeReplicas *ChangeReplicas `protobuf:"bytes,12,opt,name=change_replicas,json=changeReplicas,proto3" json:"change_replicas,omitempty"` + RaftLogDelta int64 `protobuf:"varint,13,opt,name=raft_log_delta,json=raftLogDelta,proto3" json:"raft_log_delta,omitempty"` + AddSSTable *ReplicatedEvalResult_AddSSTable `protobuf:"bytes,17,opt,name=add_sstable,json=addSstable,proto3" json:"add_sstable,omitempty"` // suggested_compactions are sent to the engine's compactor to // reclaim storage space after garbage collection or cleared / // rebalanced ranges. - SuggestedCompactions []SuggestedCompaction `protobuf:"bytes,19,rep,name=suggested_compactions,json=suggestedCompactions" json:"suggested_compactions"` + SuggestedCompactions []SuggestedCompaction `protobuf:"bytes,19,rep,name=suggested_compactions,json=suggestedCompactions,proto3" json:"suggested_compactions"` // This is the proposal timestamp for the active lease while evaluating a lease request. // It will be used to make sure we know if a lease was extended after we sent out the request // but before we tried to apply it. - PrevLeaseProposal *cockroach_util_hlc.Timestamp `protobuf:"bytes,20,opt,name=prev_lease_proposal,json=prevLeaseProposal" json:"prev_lease_proposal,omitempty"` + PrevLeaseProposal *hlc.Timestamp `protobuf:"bytes,20,opt,name=prev_lease_proposal,json=prevLeaseProposal,proto3" json:"prev_lease_proposal,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *ReplicatedEvalResult) Reset() { *m = ReplicatedEvalResult{} } -func (m *ReplicatedEvalResult) String() string { return proto.CompactTextString(m) } -func (*ReplicatedEvalResult) ProtoMessage() {} -func (*ReplicatedEvalResult) Descriptor() ([]byte, []int) { return fileDescriptorProposerKv, []int{6} } +func (m *ReplicatedEvalResult) Reset() { *m = ReplicatedEvalResult{} } +func (m *ReplicatedEvalResult) String() string { return proto.CompactTextString(m) } +func (*ReplicatedEvalResult) ProtoMessage() {} +func (*ReplicatedEvalResult) Descriptor() ([]byte, []int) { + return fileDescriptor_proposer_kv_e79e54048b29cf41, []int{6} +} +func (m *ReplicatedEvalResult) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ReplicatedEvalResult) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ReplicatedEvalResult) XXX_Merge(src proto.Message) { + xxx_messageInfo_ReplicatedEvalResult.Merge(dst, src) +} +func (m *ReplicatedEvalResult) XXX_Size() int { + return m.Size() +} +func (m *ReplicatedEvalResult) XXX_DiscardUnknown() { + xxx_messageInfo_ReplicatedEvalResult.DiscardUnknown(m) +} + +var xxx_messageInfo_ReplicatedEvalResult proto.InternalMessageInfo // AddSSTable is a side effect that must execute before the Raft application // is committed. It must be idempotent to account for an ill-timed crash after @@ -162,42 +348,118 @@ func (*ReplicatedEvalResult) Descriptor() ([]byte, []int) { return fileDescripto // since these Ranges are not user-visible, but it is a general concern assuming // other such side effects are added. type ReplicatedEvalResult_AddSSTable struct { - Data []byte `protobuf:"bytes,1,opt,name=data,proto3" json:"data,omitempty"` - CRC32 uint32 `protobuf:"varint,2,opt,name=crc32,proto3" json:"crc32,omitempty"` + Data []byte `protobuf:"bytes,1,opt,name=data,proto3" json:"data,omitempty"` + CRC32 uint32 `protobuf:"varint,2,opt,name=crc32,proto3" json:"crc32,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *ReplicatedEvalResult_AddSSTable) Reset() { *m = ReplicatedEvalResult_AddSSTable{} } func (m *ReplicatedEvalResult_AddSSTable) String() string { return proto.CompactTextString(m) } func (*ReplicatedEvalResult_AddSSTable) ProtoMessage() {} func (*ReplicatedEvalResult_AddSSTable) Descriptor() ([]byte, []int) { - return fileDescriptorProposerKv, []int{6, 0} + return fileDescriptor_proposer_kv_e79e54048b29cf41, []int{6, 0} +} +func (m *ReplicatedEvalResult_AddSSTable) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ReplicatedEvalResult_AddSSTable) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ReplicatedEvalResult_AddSSTable) XXX_Merge(src proto.Message) { + xxx_messageInfo_ReplicatedEvalResult_AddSSTable.Merge(dst, src) +} +func (m *ReplicatedEvalResult_AddSSTable) XXX_Size() int { + return m.Size() } +func (m *ReplicatedEvalResult_AddSSTable) XXX_DiscardUnknown() { + xxx_messageInfo_ReplicatedEvalResult_AddSSTable.DiscardUnknown(m) +} + +var xxx_messageInfo_ReplicatedEvalResult_AddSSTable proto.InternalMessageInfo // WriteBatch is the serialized representation of a RocksDB write // batch. A wrapper message is used so that the absence of the field // can be distinguished from a zero-length batch, and so structs // containing pointers to it can be compared with the == operator. type WriteBatch struct { - Data []byte `protobuf:"bytes,1,opt,name=data,proto3" json:"data,omitempty"` + Data []byte `protobuf:"bytes,1,opt,name=data,proto3" json:"data,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *WriteBatch) Reset() { *m = WriteBatch{} } -func (m *WriteBatch) String() string { return proto.CompactTextString(m) } -func (*WriteBatch) ProtoMessage() {} -func (*WriteBatch) Descriptor() ([]byte, []int) { return fileDescriptorProposerKv, []int{7} } +func (m *WriteBatch) Reset() { *m = WriteBatch{} } +func (m *WriteBatch) String() string { return proto.CompactTextString(m) } +func (*WriteBatch) ProtoMessage() {} +func (*WriteBatch) Descriptor() ([]byte, []int) { + return fileDescriptor_proposer_kv_e79e54048b29cf41, []int{7} +} +func (m *WriteBatch) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *WriteBatch) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *WriteBatch) XXX_Merge(src proto.Message) { + xxx_messageInfo_WriteBatch.Merge(dst, src) +} +func (m *WriteBatch) XXX_Size() int { + return m.Size() +} +func (m *WriteBatch) XXX_DiscardUnknown() { + xxx_messageInfo_WriteBatch.DiscardUnknown(m) +} + +var xxx_messageInfo_WriteBatch proto.InternalMessageInfo // LogicalOpLog is a log of logical MVCC operations. A wrapper message // is used so that the absence of the field can be distinguished from a // zero-length batch, and so structs containing pointers to it can be // compared with the == operator. type LogicalOpLog struct { - Ops []cockroach_storage_engine_enginepb.MVCCLogicalOp `protobuf:"bytes,1,rep,name=ops" json:"ops"` + Ops []enginepb.MVCCLogicalOp `protobuf:"bytes,1,rep,name=ops,proto3" json:"ops"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *LogicalOpLog) Reset() { *m = LogicalOpLog{} } -func (m *LogicalOpLog) String() string { return proto.CompactTextString(m) } -func (*LogicalOpLog) ProtoMessage() {} -func (*LogicalOpLog) Descriptor() ([]byte, []int) { return fileDescriptorProposerKv, []int{8} } +func (m *LogicalOpLog) Reset() { *m = LogicalOpLog{} } +func (m *LogicalOpLog) String() string { return proto.CompactTextString(m) } +func (*LogicalOpLog) ProtoMessage() {} +func (*LogicalOpLog) Descriptor() ([]byte, []int) { + return fileDescriptor_proposer_kv_e79e54048b29cf41, []int{8} +} +func (m *LogicalOpLog) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *LogicalOpLog) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *LogicalOpLog) XXX_Merge(src proto.Message) { + xxx_messageInfo_LogicalOpLog.Merge(dst, src) +} +func (m *LogicalOpLog) XXX_Size() int { + return m.Size() +} +func (m *LogicalOpLog) XXX_DiscardUnknown() { + xxx_messageInfo_LogicalOpLog.DiscardUnknown(m) +} + +var xxx_messageInfo_LogicalOpLog proto.InternalMessageInfo // RaftCommand is the message written to the raft log. It contains // some metadata about the proposal itself, then either a BatchRequest @@ -206,7 +468,7 @@ func (*LogicalOpLog) Descriptor() ([]byte, []int) { return fileDescriptorPropose type RaftCommand struct { // proposer_replica is the replica which proposed this command, to be // used for lease validation. - ProposerReplica cockroach_roachpb.ReplicaDescriptor `protobuf:"bytes,2,opt,name=proposer_replica,json=proposerReplica" json:"proposer_replica"` + ProposerReplica roachpb.ReplicaDescriptor `protobuf:"bytes,2,opt,name=proposer_replica,json=proposerReplica,proto3" json:"proposer_replica"` // proposer_lease_seq is provided to verify at raft command apply-time // that the lease under which the command was proposed remains in effect. // @@ -226,7 +488,7 @@ type RaftCommand struct { // However, unless we add a check that all existing Raft logs on all nodes // in the cluster contain only "new" leases, we won't be able to remove the // legacy code. - DeprecatedProposerLease *cockroach_roachpb1.Lease `protobuf:"bytes,5,opt,name=deprecated_proposer_lease,json=deprecatedProposerLease" json:"deprecated_proposer_lease,omitempty"` + DeprecatedProposerLease *roachpb.Lease `protobuf:"bytes,5,opt,name=deprecated_proposer_lease,json=deprecatedProposerLease,proto3" json:"deprecated_proposer_lease,omitempty"` // When the command is applied, its result is an error if the lease log // counter has already reached (or exceeded) max_lease_index. // @@ -263,21 +525,47 @@ type RaftCommand struct { // replicated_eval_result is a set of structured information that instructs // replicated state changes to the part of a Range's replicated state machine // that exists outside of RocksDB. - ReplicatedEvalResult ReplicatedEvalResult `protobuf:"bytes,13,opt,name=replicated_eval_result,json=replicatedEvalResult" json:"replicated_eval_result"` + ReplicatedEvalResult ReplicatedEvalResult `protobuf:"bytes,13,opt,name=replicated_eval_result,json=replicatedEvalResult,proto3" json:"replicated_eval_result"` // write_batch is a RocksDB WriteBatch that will be applied to RockDB during // the application of the Raft command. The batch can be thought of as a // series of replicated instructions that inform a RocksDB engine on how to // change. - WriteBatch *WriteBatch `protobuf:"bytes,14,opt,name=write_batch,json=writeBatch" json:"write_batch,omitempty"` + WriteBatch *WriteBatch `protobuf:"bytes,14,opt,name=write_batch,json=writeBatch,proto3" json:"write_batch,omitempty"` // logical_op_log contains a series of logical MVCC operations that correspond // to the physical operations being made in the write_batch. - LogicalOpLog *LogicalOpLog `protobuf:"bytes,15,opt,name=logical_op_log,json=logicalOpLog" json:"logical_op_log,omitempty"` + LogicalOpLog *LogicalOpLog `protobuf:"bytes,15,opt,name=logical_op_log,json=logicalOpLog,proto3" json:"logical_op_log,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *RaftCommand) Reset() { *m = RaftCommand{} } +func (m *RaftCommand) String() string { return proto.CompactTextString(m) } +func (*RaftCommand) ProtoMessage() {} +func (*RaftCommand) Descriptor() ([]byte, []int) { + return fileDescriptor_proposer_kv_e79e54048b29cf41, []int{9} +} +func (m *RaftCommand) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RaftCommand) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RaftCommand) XXX_Merge(src proto.Message) { + xxx_messageInfo_RaftCommand.Merge(dst, src) +} +func (m *RaftCommand) XXX_Size() int { + return m.Size() +} +func (m *RaftCommand) XXX_DiscardUnknown() { + xxx_messageInfo_RaftCommand.DiscardUnknown(m) } -func (m *RaftCommand) Reset() { *m = RaftCommand{} } -func (m *RaftCommand) String() string { return proto.CompactTextString(m) } -func (*RaftCommand) ProtoMessage() {} -func (*RaftCommand) Descriptor() ([]byte, []int) { return fileDescriptorProposerKv, []int{9} } +var xxx_messageInfo_RaftCommand proto.InternalMessageInfo func init() { proto.RegisterType((*Split)(nil), "cockroach.storage.storagepb.Split") @@ -1075,6 +1363,9 @@ func encodeVarintProposerKv(dAtA []byte, offset int, v uint64) int { return offset + 1 } func (m *Split) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.SplitTrigger.Size() @@ -1085,6 +1376,9 @@ func (m *Split) Size() (n int) { } func (m *Merge) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.MergeTrigger.Size() @@ -1093,6 +1387,9 @@ func (m *Merge) Size() (n int) { } func (m *ChangeReplicas) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.ChangeReplicasTrigger.Size() @@ -1101,6 +1398,9 @@ func (m *ChangeReplicas) Size() (n int) { } func (m *ComputeChecksum) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.ChecksumID.Size() @@ -1112,6 +1412,9 @@ func (m *ComputeChecksum) Size() (n int) { } func (m *Compaction) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Bytes != 0 { @@ -1124,6 +1427,9 @@ func (m *Compaction) Size() (n int) { } func (m *SuggestedCompaction) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.StartKey) @@ -1140,6 +1446,9 @@ func (m *SuggestedCompaction) Size() (n int) { } func (m *ReplicatedEvalResult) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.BlockReads { @@ -1200,6 +1509,9 @@ func (m *ReplicatedEvalResult) Size() (n int) { } func (m *ReplicatedEvalResult_AddSSTable) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Data) @@ -1213,6 +1525,9 @@ func (m *ReplicatedEvalResult_AddSSTable) Size() (n int) { } func (m *WriteBatch) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Data) @@ -1223,6 +1538,9 @@ func (m *WriteBatch) Size() (n int) { } func (m *LogicalOpLog) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.Ops) > 0 { @@ -1235,6 +1553,9 @@ func (m *LogicalOpLog) Size() (n int) { } func (m *RaftCommand) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.ProposerReplica.Size() @@ -2120,7 +2441,7 @@ func (m *ReplicatedEvalResult) Unmarshal(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.DeprecatedDelta == nil { - m.DeprecatedDelta = &cockroach_storage_engine_enginepb1.MVCCStats{} + m.DeprecatedDelta = &enginepb.MVCCStats{} } if err := m.DeprecatedDelta.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err @@ -2299,7 +2620,7 @@ func (m *ReplicatedEvalResult) Unmarshal(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.PrevLeaseProposal == nil { - m.PrevLeaseProposal = &cockroach_util_hlc.Timestamp{} + m.PrevLeaseProposal = &hlc.Timestamp{} } if err := m.PrevLeaseProposal.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err @@ -2595,7 +2916,7 @@ func (m *LogicalOpLog) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Ops = append(m.Ops, cockroach_storage_engine_enginepb.MVCCLogicalOp{}) + m.Ops = append(m.Ops, enginepb.MVCCLogicalOp{}) if err := m.Ops[len(m.Ops)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } @@ -2726,7 +3047,7 @@ func (m *RaftCommand) Unmarshal(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.DeprecatedProposerLease == nil { - m.DeprecatedProposerLease = &cockroach_roachpb1.Lease{} + m.DeprecatedProposerLease = &roachpb.Lease{} } if err := m.DeprecatedProposerLease.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err @@ -2973,9 +3294,11 @@ var ( ErrIntOverflowProposerKv = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("storage/storagepb/proposer_kv.proto", fileDescriptorProposerKv) } +func init() { + proto.RegisterFile("storage/storagepb/proposer_kv.proto", fileDescriptor_proposer_kv_e79e54048b29cf41) +} -var fileDescriptorProposerKv = []byte{ +var fileDescriptor_proposer_kv_e79e54048b29cf41 = []byte{ // 1312 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x56, 0xcf, 0x6e, 0xdb, 0xc6, 0x13, 0xb6, 0x6c, 0xca, 0xa1, 0x47, 0xb6, 0xc4, 0x6c, 0x9c, 0x84, 0xbf, 0xfc, 0x10, 0x2b, 0x90, diff --git a/pkg/storage/storagepb/state.pb.go b/pkg/storage/storagepb/state.pb.go index 8c56a488009f..9ea05c1a1e2d 100644 --- a/pkg/storage/storagepb/state.pb.go +++ b/pkg/storage/storagepb/state.pb.go @@ -6,11 +6,9 @@ package storagepb import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" -import cockroach_storage_engine_enginepb1 "github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb" -import cockroach_roachpb2 "github.com/cockroachdb/cockroach/pkg/roachpb" -import cockroach_roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" -import cockroach_roachpb1 "github.com/cockroachdb/cockroach/pkg/roachpb" -import cockroach_util_hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" +import roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" +import enginepb "github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb" +import hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" import io "io" @@ -19,6 +17,12 @@ var _ = proto.Marshal var _ = fmt.Errorf var _ = math.Inf +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package + // ReplicaState is the part of the Range Raft state machine which is cached in // memory and which is manipulated exclusively through consensus. // @@ -41,26 +45,26 @@ type ReplicaState struct { // // Changes of the descriptor should always go through one of the // (*Replica).setDesc* methods. - Desc *cockroach_roachpb.RangeDescriptor `protobuf:"bytes,3,opt,name=desc" json:"desc,omitempty"` + Desc *roachpb.RangeDescriptor `protobuf:"bytes,3,opt,name=desc,proto3" json:"desc,omitempty"` // The latest range lease. // // Note that this message is both sent over the network and used to model // replica state in memory. In memory (storage.Replica.mu.state), the lease // is never nil (and never zero-valued), but it may be nil when sent over // the network as part of ReplicatedEvalResult. - Lease *cockroach_roachpb1.Lease `protobuf:"bytes,4,opt,name=lease" json:"lease,omitempty"` + Lease *roachpb.Lease `protobuf:"bytes,4,opt,name=lease,proto3" json:"lease,omitempty"` // The truncation state of the Raft log. - TruncatedState *cockroach_roachpb2.RaftTruncatedState `protobuf:"bytes,5,opt,name=truncated_state,json=truncatedState" json:"truncated_state,omitempty"` + TruncatedState *roachpb.RaftTruncatedState `protobuf:"bytes,5,opt,name=truncated_state,json=truncatedState,proto3" json:"truncated_state,omitempty"` // gcThreshold is the GC threshold of the Range, typically updated when keys // are garbage collected. Reads and writes at timestamps <= this time will // not be served. - GCThreshold *cockroach_util_hlc.Timestamp `protobuf:"bytes,6,opt,name=gc_threshold,json=gcThreshold" json:"gc_threshold,omitempty" cockroachdb:"randnullable"` - Stats *cockroach_storage_engine_enginepb1.MVCCStats `protobuf:"bytes,7,opt,name=stats" json:"stats,omitempty"` + GCThreshold *hlc.Timestamp `protobuf:"bytes,6,opt,name=gc_threshold,json=gcThreshold,proto3" json:"gc_threshold,omitempty" cockroachdb:"randnullable"` + Stats *enginepb.MVCCStats `protobuf:"bytes,7,opt,name=stats,proto3" json:"stats,omitempty"` // txn_span_gc_threshold is the (maximum) timestamp below which transaction // records may have been garbage collected (as measured by txn.LastActive()). // Transaction at lower timestamps must not be allowed to write their initial // transaction entry. - TxnSpanGCThreshold *cockroach_util_hlc.Timestamp `protobuf:"bytes,9,opt,name=txn_span_gc_threshold,json=txnSpanGcThreshold" json:"txn_span_gc_threshold,omitempty" cockroachdb:"randnullable"` + TxnSpanGCThreshold *hlc.Timestamp `protobuf:"bytes,9,opt,name=txn_span_gc_threshold,json=txnSpanGcThreshold,proto3" json:"txn_span_gc_threshold,omitempty" cockroachdb:"randnullable"` // using_applied_state_key specifies whether the Range has been upgraded // to begin using the RangeAppliedState key. This key holds a combination // of the Raft applied index, the lease applied index, and the MVCC stats. @@ -69,18 +73,44 @@ type ReplicaState struct { // range should begin using the RangeAppliedState key. Handling of this flag // is idempotent by Replica state machines, meaning that it is ok for multiple // Raft commands to set it to true. - UsingAppliedStateKey bool `protobuf:"varint,11,opt,name=using_applied_state_key,json=usingAppliedStateKey,proto3" json:"using_applied_state_key,omitempty"` + UsingAppliedStateKey bool `protobuf:"varint,11,opt,name=using_applied_state_key,json=usingAppliedStateKey,proto3" json:"using_applied_state_key,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ReplicaState) Reset() { *m = ReplicaState{} } +func (m *ReplicaState) String() string { return proto.CompactTextString(m) } +func (*ReplicaState) ProtoMessage() {} +func (*ReplicaState) Descriptor() ([]byte, []int) { + return fileDescriptor_state_ac40414024a1cc3f, []int{0} +} +func (m *ReplicaState) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ReplicaState) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ReplicaState) XXX_Merge(src proto.Message) { + xxx_messageInfo_ReplicaState.Merge(dst, src) +} +func (m *ReplicaState) XXX_Size() int { + return m.Size() +} +func (m *ReplicaState) XXX_DiscardUnknown() { + xxx_messageInfo_ReplicaState.DiscardUnknown(m) } -func (m *ReplicaState) Reset() { *m = ReplicaState{} } -func (m *ReplicaState) String() string { return proto.CompactTextString(m) } -func (*ReplicaState) ProtoMessage() {} -func (*ReplicaState) Descriptor() ([]byte, []int) { return fileDescriptorState, []int{0} } +var xxx_messageInfo_ReplicaState proto.InternalMessageInfo // RangeInfo is used for reporting status information about a range out through // the status server. type RangeInfo struct { - ReplicaState `protobuf:"bytes,1,opt,name=state,embedded=state" json:"state"` + ReplicaState `protobuf:"bytes,1,opt,name=state,proto3,embedded=state" json:"state"` // The highest (and last) index in the Raft log. LastIndex uint64 `protobuf:"varint,2,opt,name=last_index,json=lastIndex,proto3" json:"last_index,omitempty"` NumPending uint64 `protobuf:"varint,3,opt,name=num_pending,json=numPending,proto3" json:"num_pending,omitempty"` @@ -91,25 +121,77 @@ type RangeInfo struct { // Approximately the amount of quota available. ApproximateProposalQuota int64 `protobuf:"varint,7,opt,name=approximate_proposal_quota,json=approximateProposalQuota,proto3" json:"approximate_proposal_quota,omitempty"` // The max size the range can grow to before it will be split. - RangeMaxBytes int64 `protobuf:"varint,8,opt,name=range_max_bytes,json=rangeMaxBytes,proto3" json:"range_max_bytes,omitempty"` + RangeMaxBytes int64 `protobuf:"varint,8,opt,name=range_max_bytes,json=rangeMaxBytes,proto3" json:"range_max_bytes,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +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_state_ac40414024a1cc3f, []int{1} +} +func (m *RangeInfo) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RangeInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RangeInfo) XXX_Merge(src proto.Message) { + xxx_messageInfo_RangeInfo.Merge(dst, src) +} +func (m *RangeInfo) XXX_Size() int { + return m.Size() +} +func (m *RangeInfo) XXX_DiscardUnknown() { + xxx_messageInfo_RangeInfo.DiscardUnknown(m) } -func (m *RangeInfo) Reset() { *m = RangeInfo{} } -func (m *RangeInfo) String() string { return proto.CompactTextString(m) } -func (*RangeInfo) ProtoMessage() {} -func (*RangeInfo) Descriptor() ([]byte, []int) { return fileDescriptorState, []int{1} } +var xxx_messageInfo_RangeInfo proto.InternalMessageInfo // LatchManagerInfo is used for reporting status information about a spanlatch // manager out through the status server. type LatchManagerInfo struct { - ReadCount int64 `protobuf:"varint,1,opt,name=read_count,json=readCount,proto3" json:"read_count,omitempty"` - WriteCount int64 `protobuf:"varint,2,opt,name=write_count,json=writeCount,proto3" json:"write_count,omitempty"` + ReadCount int64 `protobuf:"varint,1,opt,name=read_count,json=readCount,proto3" json:"read_count,omitempty"` + WriteCount int64 `protobuf:"varint,2,opt,name=write_count,json=writeCount,proto3" json:"write_count,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *LatchManagerInfo) Reset() { *m = LatchManagerInfo{} } +func (m *LatchManagerInfo) String() string { return proto.CompactTextString(m) } +func (*LatchManagerInfo) ProtoMessage() {} +func (*LatchManagerInfo) Descriptor() ([]byte, []int) { + return fileDescriptor_state_ac40414024a1cc3f, []int{2} +} +func (m *LatchManagerInfo) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *LatchManagerInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *LatchManagerInfo) XXX_Merge(src proto.Message) { + xxx_messageInfo_LatchManagerInfo.Merge(dst, src) +} +func (m *LatchManagerInfo) XXX_Size() int { + return m.Size() +} +func (m *LatchManagerInfo) XXX_DiscardUnknown() { + xxx_messageInfo_LatchManagerInfo.DiscardUnknown(m) } -func (m *LatchManagerInfo) Reset() { *m = LatchManagerInfo{} } -func (m *LatchManagerInfo) String() string { return proto.CompactTextString(m) } -func (*LatchManagerInfo) ProtoMessage() {} -func (*LatchManagerInfo) Descriptor() ([]byte, []int) { return fileDescriptorState, []int{2} } +var xxx_messageInfo_LatchManagerInfo proto.InternalMessageInfo func init() { proto.RegisterType((*ReplicaState)(nil), "cockroach.storage.storagepb.ReplicaState") @@ -398,6 +480,9 @@ func encodeVarintState(dAtA []byte, offset int, v uint64) int { return offset + 1 } func (m *ReplicaState) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.RaftAppliedIndex != 0 { @@ -437,6 +522,9 @@ func (m *ReplicaState) Size() (n int) { } func (m *RangeInfo) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.ReplicaState.Size() @@ -463,6 +551,9 @@ func (m *RangeInfo) Size() (n int) { } func (m *LatchManagerInfo) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.ReadCount != 0 { @@ -581,7 +672,7 @@ func (m *ReplicaState) Unmarshal(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Desc == nil { - m.Desc = &cockroach_roachpb.RangeDescriptor{} + m.Desc = &roachpb.RangeDescriptor{} } if err := m.Desc.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err @@ -614,7 +705,7 @@ func (m *ReplicaState) Unmarshal(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Lease == nil { - m.Lease = &cockroach_roachpb1.Lease{} + m.Lease = &roachpb.Lease{} } if err := m.Lease.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err @@ -647,7 +738,7 @@ func (m *ReplicaState) Unmarshal(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.TruncatedState == nil { - m.TruncatedState = &cockroach_roachpb2.RaftTruncatedState{} + m.TruncatedState = &roachpb.RaftTruncatedState{} } if err := m.TruncatedState.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err @@ -680,7 +771,7 @@ func (m *ReplicaState) Unmarshal(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.GCThreshold == nil { - m.GCThreshold = &cockroach_util_hlc.Timestamp{} + m.GCThreshold = &hlc.Timestamp{} } if err := m.GCThreshold.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err @@ -713,7 +804,7 @@ func (m *ReplicaState) Unmarshal(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Stats == nil { - m.Stats = &cockroach_storage_engine_enginepb1.MVCCStats{} + m.Stats = &enginepb.MVCCStats{} } if err := m.Stats.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err @@ -746,7 +837,7 @@ func (m *ReplicaState) Unmarshal(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.TxnSpanGCThreshold == nil { - m.TxnSpanGCThreshold = &cockroach_util_hlc.Timestamp{} + m.TxnSpanGCThreshold = &hlc.Timestamp{} } if err := m.TxnSpanGCThreshold.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err @@ -1180,9 +1271,11 @@ var ( ErrIntOverflowState = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("storage/storagepb/state.proto", fileDescriptorState) } +func init() { + proto.RegisterFile("storage/storagepb/state.proto", fileDescriptor_state_ac40414024a1cc3f) +} -var fileDescriptorState = []byte{ +var fileDescriptor_state_ac40414024a1cc3f = []byte{ // 756 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x54, 0xcf, 0x6e, 0xdb, 0x36, 0x1c, 0x8e, 0x62, 0x39, 0xb3, 0xa9, 0x64, 0xf1, 0xb8, 0x6c, 0x13, 0x1c, 0xd8, 0x0e, 0x34, 0x6c, diff --git a/pkg/storage/store.go b/pkg/storage/store.go index 9c0ba11c2741..9cebe1d3a27b 100644 --- a/pkg/storage/store.go +++ b/pkg/storage/store.go @@ -1781,7 +1781,7 @@ func Bootstrap( ) error { exIdent, err := ReadStoreIdent(ctx, eng) if err == nil { - return errors.Errorf("engine %s is already bootstrapped with ident %s", eng, exIdent) + return errors.Errorf("engine %s is already bootstrapped with ident %s", eng, exIdent.String()) } if _, ok := err.(*NotBootstrappedError); !ok { return err @@ -1927,7 +1927,7 @@ func checkEngineEmpty(ctx context.Context, eng engine.Engine) error { for i, kv := range kvs { keyVals[i] = fmt.Sprintf("%s: %q", kv.Key, kv.Value) } - return errors.Errorf("engine belongs to store %s, contains %s", ident, keyVals) + return errors.Errorf("engine belongs to store %s, contains %s", ident.String(), keyVals) } return nil } diff --git a/pkg/testutils/lint/lint_test.go b/pkg/testutils/lint/lint_test.go index 71d8fe079298..155c69eee647 100644 --- a/pkg/testutils/lint/lint_test.go +++ b/pkg/testutils/lint/lint_test.go @@ -1173,6 +1173,7 @@ func TestLint(t *testing.T) { // _fsm.go files are allowed to dot-import the util/fsm package. stream.GrepNot("_fsm.go.*should not use dot imports"), stream.GrepNot("sql/.*exported func .* returns unexported type sql.planNode"), + stream.GrepNot("struct field (XXX_NoUnkeyedLiteral|XXX_sizecache) should be"), ), func(s string) { t.Errorf("\n%s", s) }); err != nil { diff --git a/pkg/testutils/zerofields/no_zero_field.go b/pkg/testutils/zerofields/no_zero_field.go index 2b87e03a40c9..4d77304db855 100644 --- a/pkg/testutils/zerofields/no_zero_field.go +++ b/pkg/testutils/zerofields/no_zero_field.go @@ -29,7 +29,12 @@ func NoZeroField(v interface{}) error { f := ele.Field(i) zero := reflect.Zero(f.Type()) if reflect.DeepEqual(f.Interface(), zero.Interface()) { - return fmt.Errorf("expected %s field to be non-zero", eleT.Field(i).Name) + switch field := eleT.Field(i).Name; field { + case "XXX_NoUnkeyedLiteral", "XXX_DiscardUnknown", "XXX_sizecache": + // Ignore these special protobuf fields. + default: + return fmt.Errorf("expected %s field to be non-zero", eleT.Field(i).Name) + } } } return nil diff --git a/pkg/ts/tspb/timeseries.pb.go b/pkg/ts/tspb/timeseries.pb.go index b4533b7518a5..43d5b33aff86 100644 --- a/pkg/ts/tspb/timeseries.pb.go +++ b/pkg/ts/tspb/timeseries.pb.go @@ -1,30 +1,18 @@ // Code generated by protoc-gen-gogo. DO NOT EDIT. // source: ts/tspb/timeseries.proto -/* - Package tspb is a generated protocol buffer package. - - It is generated from these files: - ts/tspb/timeseries.proto - - It has these top-level messages: - TimeSeriesDatapoint - TimeSeriesData - Query - TimeSeriesQueryRequest - TimeSeriesQueryResponse - DumpRequest -*/ package tspb import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" -import context "context" -import grpc "google.golang.org/grpc" +import ( + context "context" + grpc "google.golang.org/grpc" +) -import binary "encoding/binary" +import encoding_binary "encoding/binary" import io "io" @@ -105,7 +93,7 @@ func (x *TimeSeriesQueryAggregator) UnmarshalJSON(data []byte) error { return nil } func (TimeSeriesQueryAggregator) EnumDescriptor() ([]byte, []int) { - return fileDescriptorTimeseries, []int{0} + return fileDescriptor_timeseries_1f7ae1dea9c20e22, []int{0} } // TimeSeriesQueryDerivative describes a derivative function used to convert @@ -151,7 +139,7 @@ func (x *TimeSeriesQueryDerivative) UnmarshalJSON(data []byte) error { return nil } func (TimeSeriesQueryDerivative) EnumDescriptor() ([]byte, []int) { - return fileDescriptorTimeseries, []int{1} + return fileDescriptor_timeseries_1f7ae1dea9c20e22, []int{1} } // TimeSeriesDatapoint is a single point of time series data; a value associated @@ -161,13 +149,39 @@ type TimeSeriesDatapoint struct { // since the unix epoch. TimestampNanos int64 `protobuf:"varint,1,opt,name=timestamp_nanos,json=timestampNanos" json:"timestamp_nanos"` // A floating point representation of the value of this datapoint. - Value float64 `protobuf:"fixed64,2,opt,name=value" json:"value"` + Value float64 `protobuf:"fixed64,2,opt,name=value" json:"value"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *TimeSeriesDatapoint) Reset() { *m = TimeSeriesDatapoint{} } +func (m *TimeSeriesDatapoint) String() string { return proto.CompactTextString(m) } +func (*TimeSeriesDatapoint) ProtoMessage() {} +func (*TimeSeriesDatapoint) Descriptor() ([]byte, []int) { + return fileDescriptor_timeseries_1f7ae1dea9c20e22, []int{0} +} +func (m *TimeSeriesDatapoint) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *TimeSeriesDatapoint) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *TimeSeriesDatapoint) XXX_Merge(src proto.Message) { + xxx_messageInfo_TimeSeriesDatapoint.Merge(dst, src) +} +func (m *TimeSeriesDatapoint) XXX_Size() int { + return m.Size() +} +func (m *TimeSeriesDatapoint) XXX_DiscardUnknown() { + xxx_messageInfo_TimeSeriesDatapoint.DiscardUnknown(m) } -func (m *TimeSeriesDatapoint) Reset() { *m = TimeSeriesDatapoint{} } -func (m *TimeSeriesDatapoint) String() string { return proto.CompactTextString(m) } -func (*TimeSeriesDatapoint) ProtoMessage() {} -func (*TimeSeriesDatapoint) Descriptor() ([]byte, []int) { return fileDescriptorTimeseries, []int{0} } +var xxx_messageInfo_TimeSeriesDatapoint proto.InternalMessageInfo // TimeSeriesData is a set of measurements of a single named variable at // multiple points in time. This message contains a name and a source which, in @@ -180,13 +194,39 @@ type TimeSeriesData struct { // A string which identifies the unique source from which the variable was measured. Source string `protobuf:"bytes,2,opt,name=source" json:"source"` // Datapoints representing one or more measurements taken from the variable. - Datapoints []TimeSeriesDatapoint `protobuf:"bytes,3,rep,name=datapoints" json:"datapoints"` + Datapoints []TimeSeriesDatapoint `protobuf:"bytes,3,rep,name=datapoints" json:"datapoints"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *TimeSeriesData) Reset() { *m = TimeSeriesData{} } +func (m *TimeSeriesData) String() string { return proto.CompactTextString(m) } +func (*TimeSeriesData) ProtoMessage() {} +func (*TimeSeriesData) Descriptor() ([]byte, []int) { + return fileDescriptor_timeseries_1f7ae1dea9c20e22, []int{1} +} +func (m *TimeSeriesData) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *TimeSeriesData) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *TimeSeriesData) XXX_Merge(src proto.Message) { + xxx_messageInfo_TimeSeriesData.Merge(dst, src) +} +func (m *TimeSeriesData) XXX_Size() int { + return m.Size() +} +func (m *TimeSeriesData) XXX_DiscardUnknown() { + xxx_messageInfo_TimeSeriesData.DiscardUnknown(m) } -func (m *TimeSeriesData) Reset() { *m = TimeSeriesData{} } -func (m *TimeSeriesData) String() string { return proto.CompactTextString(m) } -func (*TimeSeriesData) ProtoMessage() {} -func (*TimeSeriesData) Descriptor() ([]byte, []int) { return fileDescriptorTimeseries, []int{1} } +var xxx_messageInfo_TimeSeriesData proto.InternalMessageInfo // Each Query defines a specific metric to query over the time span of // this request. @@ -204,13 +244,39 @@ type Query struct { Derivative *TimeSeriesQueryDerivative `protobuf:"varint,4,opt,name=derivative,enum=cockroach.ts.tspb.TimeSeriesQueryDerivative,def=0" json:"derivative,omitempty"` // An optional list of sources to restrict the time series query. If no // sources are provided, all available sources will be queried. - Sources []string `protobuf:"bytes,5,rep,name=sources" json:"sources,omitempty"` + Sources []string `protobuf:"bytes,5,rep,name=sources" json:"sources,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *Query) Reset() { *m = Query{} } -func (m *Query) String() string { return proto.CompactTextString(m) } -func (*Query) ProtoMessage() {} -func (*Query) Descriptor() ([]byte, []int) { return fileDescriptorTimeseries, []int{2} } +func (m *Query) Reset() { *m = Query{} } +func (m *Query) String() string { return proto.CompactTextString(m) } +func (*Query) ProtoMessage() {} +func (*Query) Descriptor() ([]byte, []int) { + return fileDescriptor_timeseries_1f7ae1dea9c20e22, []int{2} +} +func (m *Query) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Query) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *Query) XXX_Merge(src proto.Message) { + xxx_messageInfo_Query.Merge(dst, src) +} +func (m *Query) XXX_Size() int { + return m.Size() +} +func (m *Query) XXX_DiscardUnknown() { + xxx_messageInfo_Query.DiscardUnknown(m) +} + +var xxx_messageInfo_Query proto.InternalMessageInfo const Default_Query_Downsampler TimeSeriesQueryAggregator = TimeSeriesQueryAggregator_AVG const Default_Query_SourceAggregator TimeSeriesQueryAggregator = TimeSeriesQueryAggregator_SUM @@ -266,13 +332,39 @@ type TimeSeriesQueryRequest struct { // Duration of requested sample period in nanoseconds. Returned data for each // query will be downsampled into periods of the supplied length. The // supplied duration must be a multiple of ten seconds. - SampleNanos int64 `protobuf:"varint,4,opt,name=sample_nanos,json=sampleNanos" json:"sample_nanos"` + SampleNanos int64 `protobuf:"varint,4,opt,name=sample_nanos,json=sampleNanos" json:"sample_nanos"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *TimeSeriesQueryRequest) Reset() { *m = TimeSeriesQueryRequest{} } +func (m *TimeSeriesQueryRequest) String() string { return proto.CompactTextString(m) } +func (*TimeSeriesQueryRequest) ProtoMessage() {} +func (*TimeSeriesQueryRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_timeseries_1f7ae1dea9c20e22, []int{3} +} +func (m *TimeSeriesQueryRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *TimeSeriesQueryRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *TimeSeriesQueryRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_TimeSeriesQueryRequest.Merge(dst, src) +} +func (m *TimeSeriesQueryRequest) XXX_Size() int { + return m.Size() +} +func (m *TimeSeriesQueryRequest) XXX_DiscardUnknown() { + xxx_messageInfo_TimeSeriesQueryRequest.DiscardUnknown(m) } -func (m *TimeSeriesQueryRequest) Reset() { *m = TimeSeriesQueryRequest{} } -func (m *TimeSeriesQueryRequest) String() string { return proto.CompactTextString(m) } -func (*TimeSeriesQueryRequest) ProtoMessage() {} -func (*TimeSeriesQueryRequest) Descriptor() ([]byte, []int) { return fileDescriptorTimeseries, []int{3} } +var xxx_messageInfo_TimeSeriesQueryRequest proto.InternalMessageInfo // TimeSeriesQueryResponse is the standard response for time series queries // returned to cockroach clients. @@ -280,36 +372,110 @@ type TimeSeriesQueryResponse struct { // A set of Results; there will be one result for each Query in the matching // TimeSeriesQueryRequest, in the same order. A Result will be present for // each Query even if there are zero datapoints to return. - Results []TimeSeriesQueryResponse_Result `protobuf:"bytes,1,rep,name=results" json:"results"` + Results []TimeSeriesQueryResponse_Result `protobuf:"bytes,1,rep,name=results" json:"results"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *TimeSeriesQueryResponse) Reset() { *m = TimeSeriesQueryResponse{} } func (m *TimeSeriesQueryResponse) String() string { return proto.CompactTextString(m) } func (*TimeSeriesQueryResponse) ProtoMessage() {} func (*TimeSeriesQueryResponse) Descriptor() ([]byte, []int) { - return fileDescriptorTimeseries, []int{4} + return fileDescriptor_timeseries_1f7ae1dea9c20e22, []int{4} +} +func (m *TimeSeriesQueryResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *TimeSeriesQueryResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *TimeSeriesQueryResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_TimeSeriesQueryResponse.Merge(dst, src) +} +func (m *TimeSeriesQueryResponse) XXX_Size() int { + return m.Size() +} +func (m *TimeSeriesQueryResponse) XXX_DiscardUnknown() { + xxx_messageInfo_TimeSeriesQueryResponse.DiscardUnknown(m) } +var xxx_messageInfo_TimeSeriesQueryResponse proto.InternalMessageInfo + // Result is the data returned from a single metric query over a time span. type TimeSeriesQueryResponse_Result struct { - Query `protobuf:"bytes,1,opt,name=query,embedded=query" json:"query"` - Datapoints []TimeSeriesDatapoint `protobuf:"bytes,2,rep,name=datapoints" json:"datapoints"` + Query `protobuf:"bytes,1,opt,name=query,embedded=query" json:"query"` + Datapoints []TimeSeriesDatapoint `protobuf:"bytes,2,rep,name=datapoints" json:"datapoints"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *TimeSeriesQueryResponse_Result) Reset() { *m = TimeSeriesQueryResponse_Result{} } func (m *TimeSeriesQueryResponse_Result) String() string { return proto.CompactTextString(m) } func (*TimeSeriesQueryResponse_Result) ProtoMessage() {} func (*TimeSeriesQueryResponse_Result) Descriptor() ([]byte, []int) { - return fileDescriptorTimeseries, []int{4, 0} + return fileDescriptor_timeseries_1f7ae1dea9c20e22, []int{4, 0} +} +func (m *TimeSeriesQueryResponse_Result) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) } +func (m *TimeSeriesQueryResponse_Result) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *TimeSeriesQueryResponse_Result) XXX_Merge(src proto.Message) { + xxx_messageInfo_TimeSeriesQueryResponse_Result.Merge(dst, src) +} +func (m *TimeSeriesQueryResponse_Result) XXX_Size() int { + return m.Size() +} +func (m *TimeSeriesQueryResponse_Result) XXX_DiscardUnknown() { + xxx_messageInfo_TimeSeriesQueryResponse_Result.DiscardUnknown(m) +} + +var xxx_messageInfo_TimeSeriesQueryResponse_Result proto.InternalMessageInfo type DumpRequest struct { + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *DumpRequest) Reset() { *m = DumpRequest{} } +func (m *DumpRequest) String() string { return proto.CompactTextString(m) } +func (*DumpRequest) ProtoMessage() {} +func (*DumpRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_timeseries_1f7ae1dea9c20e22, []int{5} +} +func (m *DumpRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *DumpRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *DumpRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_DumpRequest.Merge(dst, src) +} +func (m *DumpRequest) XXX_Size() int { + return m.Size() +} +func (m *DumpRequest) XXX_DiscardUnknown() { + xxx_messageInfo_DumpRequest.DiscardUnknown(m) } -func (m *DumpRequest) Reset() { *m = DumpRequest{} } -func (m *DumpRequest) String() string { return proto.CompactTextString(m) } -func (*DumpRequest) ProtoMessage() {} -func (*DumpRequest) Descriptor() ([]byte, []int) { return fileDescriptorTimeseries, []int{5} } +var xxx_messageInfo_DumpRequest proto.InternalMessageInfo func init() { proto.RegisterType((*TimeSeriesDatapoint)(nil), "cockroach.ts.tspb.TimeSeriesDatapoint") @@ -331,8 +497,9 @@ var _ grpc.ClientConn // is compatible with the grpc package it is being compiled against. const _ = grpc.SupportPackageIsVersion4 -// Client API for TimeSeries service - +// TimeSeriesClient is the client API for TimeSeries service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream. type TimeSeriesClient interface { // URL: /ts/query Query(ctx context.Context, in *TimeSeriesQueryRequest, opts ...grpc.CallOption) (*TimeSeriesQueryResponse, error) @@ -353,7 +520,7 @@ func NewTimeSeriesClient(cc *grpc.ClientConn) TimeSeriesClient { func (c *timeSeriesClient) Query(ctx context.Context, in *TimeSeriesQueryRequest, opts ...grpc.CallOption) (*TimeSeriesQueryResponse, error) { out := new(TimeSeriesQueryResponse) - err := grpc.Invoke(ctx, "/cockroach.ts.tspb.TimeSeries/Query", in, out, c.cc, opts...) + err := c.cc.Invoke(ctx, "/cockroach.ts.tspb.TimeSeries/Query", in, out, opts...) if err != nil { return nil, err } @@ -361,7 +528,7 @@ func (c *timeSeriesClient) Query(ctx context.Context, in *TimeSeriesQueryRequest } func (c *timeSeriesClient) Dump(ctx context.Context, in *DumpRequest, opts ...grpc.CallOption) (TimeSeries_DumpClient, error) { - stream, err := grpc.NewClientStream(ctx, &_TimeSeries_serviceDesc.Streams[0], c.cc, "/cockroach.ts.tspb.TimeSeries/Dump", opts...) + stream, err := c.cc.NewStream(ctx, &_TimeSeries_serviceDesc.Streams[0], "/cockroach.ts.tspb.TimeSeries/Dump", opts...) if err != nil { return nil, err } @@ -392,8 +559,7 @@ func (x *timeSeriesDumpClient) Recv() (*TimeSeriesData, error) { return m, nil } -// Server API for TimeSeries service - +// TimeSeriesServer is the server API for TimeSeries service. type TimeSeriesServer interface { // URL: /ts/query Query(context.Context, *TimeSeriesQueryRequest) (*TimeSeriesQueryResponse, error) @@ -486,7 +652,7 @@ func (m *TimeSeriesDatapoint) MarshalTo(dAtA []byte) (int, error) { i = encodeVarintTimeseries(dAtA, i, uint64(m.TimestampNanos)) dAtA[i] = 0x11 i++ - binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.Value)))) + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.Value)))) i += 8 return i, nil } @@ -716,6 +882,9 @@ func encodeVarintTimeseries(dAtA []byte, offset int, v uint64) int { return offset + 1 } func (m *TimeSeriesDatapoint) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovTimeseries(uint64(m.TimestampNanos)) @@ -724,6 +893,9 @@ func (m *TimeSeriesDatapoint) Size() (n int) { } func (m *TimeSeriesData) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Name) @@ -740,6 +912,9 @@ func (m *TimeSeriesData) Size() (n int) { } func (m *Query) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Name) @@ -763,6 +938,9 @@ func (m *Query) Size() (n int) { } func (m *TimeSeriesQueryRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovTimeseries(uint64(m.StartNanos)) @@ -778,6 +956,9 @@ func (m *TimeSeriesQueryRequest) Size() (n int) { } func (m *TimeSeriesQueryResponse) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if len(m.Results) > 0 { @@ -790,6 +971,9 @@ func (m *TimeSeriesQueryResponse) Size() (n int) { } func (m *TimeSeriesQueryResponse_Result) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = m.Query.Size() @@ -804,6 +988,9 @@ func (m *TimeSeriesQueryResponse_Result) Size() (n int) { } func (m *DumpRequest) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l return n @@ -878,7 +1065,7 @@ func (m *TimeSeriesDatapoint) Unmarshal(dAtA []byte) error { if (iNdEx + 8) > l { return io.ErrUnexpectedEOF } - v = uint64(binary.LittleEndian.Uint64(dAtA[iNdEx:])) + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) iNdEx += 8 m.Value = float64(math.Float64frombits(v)) default: @@ -1694,9 +1881,11 @@ var ( ErrIntOverflowTimeseries = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("ts/tspb/timeseries.proto", fileDescriptorTimeseries) } +func init() { + proto.RegisterFile("ts/tspb/timeseries.proto", fileDescriptor_timeseries_1f7ae1dea9c20e22) +} -var fileDescriptorTimeseries = []byte{ +var fileDescriptor_timeseries_1f7ae1dea9c20e22 = []byte{ // 711 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x93, 0x4d, 0x6f, 0xda, 0x48, 0x18, 0xc7, 0x19, 0x30, 0x01, 0x1e, 0xb2, 0xac, 0x33, 0x1b, 0x6d, 0xbc, 0x6c, 0xe4, 0x10, 0x4b, diff --git a/pkg/util/hlc/legacy_timestamp.pb.go b/pkg/util/hlc/legacy_timestamp.pb.go index 4184da56c6b5..d56fcb3f96f2 100644 --- a/pkg/util/hlc/legacy_timestamp.pb.go +++ b/pkg/util/hlc/legacy_timestamp.pb.go @@ -1,17 +1,6 @@ // Code generated by protoc-gen-gogo. DO NOT EDIT. // source: util/hlc/legacy_timestamp.proto -/* - Package hlc is a generated protocol buffer package. - - It is generated from these files: - util/hlc/legacy_timestamp.proto - util/hlc/timestamp.proto - - It has these top-level messages: - LegacyTimestamp - Timestamp -*/ package hlc import proto "github.com/gogo/protobuf/proto" @@ -41,12 +30,38 @@ type LegacyTimestamp struct { // times are equal. It is effectively bounded by (maximum clock // skew)/(minimal ns between events) and nearly impossible to // overflow. - Logical int32 `protobuf:"varint,2,opt,name=logical" json:"logical"` + Logical int32 `protobuf:"varint,2,opt,name=logical" json:"logical"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *LegacyTimestamp) Reset() { *m = LegacyTimestamp{} } -func (*LegacyTimestamp) ProtoMessage() {} -func (*LegacyTimestamp) Descriptor() ([]byte, []int) { return fileDescriptorLegacyTimestamp, []int{0} } +func (m *LegacyTimestamp) Reset() { *m = LegacyTimestamp{} } +func (*LegacyTimestamp) ProtoMessage() {} +func (*LegacyTimestamp) Descriptor() ([]byte, []int) { + return fileDescriptor_legacy_timestamp_1f7cbb1099511bab, []int{0} +} +func (m *LegacyTimestamp) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *LegacyTimestamp) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *LegacyTimestamp) XXX_Merge(src proto.Message) { + xxx_messageInfo_LegacyTimestamp.Merge(dst, src) +} +func (m *LegacyTimestamp) XXX_Size() int { + return m.Size() +} +func (m *LegacyTimestamp) XXX_DiscardUnknown() { + xxx_messageInfo_LegacyTimestamp.DiscardUnknown(m) +} + +var xxx_messageInfo_LegacyTimestamp proto.InternalMessageInfo func init() { proto.RegisterType((*LegacyTimestamp)(nil), "cockroach.util.hlc.LegacyTimestamp") @@ -199,6 +214,9 @@ func encodeVarintPopulateLegacyTimestamp(dAtA []byte, v uint64) []byte { return dAtA } func (m *LegacyTimestamp) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l n += 1 + sovLegacyTimestamp(uint64(m.WallTime)) @@ -412,9 +430,11 @@ var ( ErrIntOverflowLegacyTimestamp = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("util/hlc/legacy_timestamp.proto", fileDescriptorLegacyTimestamp) } +func init() { + proto.RegisterFile("util/hlc/legacy_timestamp.proto", fileDescriptor_legacy_timestamp_1f7cbb1099511bab) +} -var fileDescriptorLegacyTimestamp = []byte{ +var fileDescriptor_legacy_timestamp_1f7cbb1099511bab = []byte{ // 194 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x92, 0x2f, 0x2d, 0xc9, 0xcc, 0xd1, 0xcf, 0xc8, 0x49, 0xd6, 0xcf, 0x49, 0x4d, 0x4f, 0x4c, 0xae, 0x8c, 0x2f, 0xc9, 0xcc, 0x4d, diff --git a/pkg/util/hlc/timestamp.pb.go b/pkg/util/hlc/timestamp.pb.go index 0a53893700d4..a7fbba53ff00 100644 --- a/pkg/util/hlc/timestamp.pb.go +++ b/pkg/util/hlc/timestamp.pb.go @@ -14,6 +14,12 @@ var _ = proto.Marshal var _ = fmt.Errorf var _ = math.Inf +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package + // Timestamp represents a state of the hybrid logical clock. type Timestamp struct { // Holds a wall time, typically a unix epoch time expressed in @@ -23,12 +29,38 @@ type Timestamp struct { // times are equal. It is effectively bounded by (maximum clock // skew)/(minimal ns between events) and nearly impossible to // overflow. - Logical int32 `protobuf:"varint,2,opt,name=logical,proto3" json:"logical,omitempty"` + Logical int32 `protobuf:"varint,2,opt,name=logical,proto3" json:"logical,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Timestamp) Reset() { *m = Timestamp{} } +func (*Timestamp) ProtoMessage() {} +func (*Timestamp) Descriptor() ([]byte, []int) { + return fileDescriptor_timestamp_c5e946b21c98d098, []int{0} +} +func (m *Timestamp) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Timestamp) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *Timestamp) XXX_Merge(src proto.Message) { + xxx_messageInfo_Timestamp.Merge(dst, src) +} +func (m *Timestamp) XXX_Size() int { + return m.Size() +} +func (m *Timestamp) XXX_DiscardUnknown() { + xxx_messageInfo_Timestamp.DiscardUnknown(m) } -func (m *Timestamp) Reset() { *m = Timestamp{} } -func (*Timestamp) ProtoMessage() {} -func (*Timestamp) Descriptor() ([]byte, []int) { return fileDescriptorTimestamp, []int{0} } +var xxx_messageInfo_Timestamp proto.InternalMessageInfo func init() { proto.RegisterType((*Timestamp)(nil), "cockroach.util.hlc.Timestamp") @@ -185,6 +217,9 @@ func encodeVarintPopulateTimestamp(dAtA []byte, v uint64) []byte { return dAtA } func (m *Timestamp) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.WallTime != 0 { @@ -402,9 +437,9 @@ var ( ErrIntOverflowTimestamp = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("util/hlc/timestamp.proto", fileDescriptorTimestamp) } +func init() { proto.RegisterFile("util/hlc/timestamp.proto", fileDescriptor_timestamp_c5e946b21c98d098) } -var fileDescriptorTimestamp = []byte{ +var fileDescriptor_timestamp_c5e946b21c98d098 = []byte{ // 183 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x92, 0x28, 0x2d, 0xc9, 0xcc, 0xd1, 0xcf, 0xc8, 0x49, 0xd6, 0x2f, 0xc9, 0xcc, 0x4d, 0x2d, 0x2e, 0x49, 0xcc, 0x2d, 0xd0, 0x2b, diff --git a/pkg/util/log/log.pb.go b/pkg/util/log/log.pb.go index c9f4ef671a99..b58a7f641f4d 100644 --- a/pkg/util/log/log.pb.go +++ b/pkg/util/log/log.pb.go @@ -1,17 +1,6 @@ // Code generated by protoc-gen-gogo. DO NOT EDIT. // source: util/log/log.proto -/* - Package log is a generated protocol buffer package. - - It is generated from these files: - util/log/log.proto - - It has these top-level messages: - Entry - FileDetails - FileInfo -*/ package log import proto "github.com/gogo/protobuf/proto" @@ -71,50 +60,130 @@ var Severity_value = map[string]int32{ func (x Severity) String() string { return proto.EnumName(Severity_name, int32(x)) } -func (Severity) EnumDescriptor() ([]byte, []int) { return fileDescriptorLog, []int{0} } +func (Severity) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_log_3a319d334ebe63bc, []int{0} +} // Entry represents a cockroach structured log entry. type Entry struct { Severity Severity `protobuf:"varint,1,opt,name=severity,proto3,enum=cockroach.util.log.Severity" json:"severity,omitempty"` // Nanoseconds since the epoch. - Time int64 `protobuf:"varint,2,opt,name=time,proto3" json:"time,omitempty"` - Goroutine int64 `protobuf:"varint,6,opt,name=goroutine,proto3" json:"goroutine,omitempty"` - File string `protobuf:"bytes,3,opt,name=file,proto3" json:"file,omitempty"` - Line int64 `protobuf:"varint,4,opt,name=line,proto3" json:"line,omitempty"` - Message string `protobuf:"bytes,5,opt,name=message,proto3" json:"message,omitempty"` + Time int64 `protobuf:"varint,2,opt,name=time,proto3" json:"time,omitempty"` + Goroutine int64 `protobuf:"varint,6,opt,name=goroutine,proto3" json:"goroutine,omitempty"` + File string `protobuf:"bytes,3,opt,name=file,proto3" json:"file,omitempty"` + Line int64 `protobuf:"varint,4,opt,name=line,proto3" json:"line,omitempty"` + Message string `protobuf:"bytes,5,opt,name=message,proto3" json:"message,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Entry) Reset() { *m = Entry{} } +func (m *Entry) String() string { return proto.CompactTextString(m) } +func (*Entry) ProtoMessage() {} +func (*Entry) Descriptor() ([]byte, []int) { + return fileDescriptor_log_3a319d334ebe63bc, []int{0} +} +func (m *Entry) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Entry) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *Entry) XXX_Merge(src proto.Message) { + xxx_messageInfo_Entry.Merge(dst, src) +} +func (m *Entry) XXX_Size() int { + return m.Size() +} +func (m *Entry) XXX_DiscardUnknown() { + xxx_messageInfo_Entry.DiscardUnknown(m) } -func (m *Entry) Reset() { *m = Entry{} } -func (m *Entry) String() string { return proto.CompactTextString(m) } -func (*Entry) ProtoMessage() {} -func (*Entry) Descriptor() ([]byte, []int) { return fileDescriptorLog, []int{0} } +var xxx_messageInfo_Entry proto.InternalMessageInfo // A FileDetails holds all of the particulars that can be parsed by the name of // a log file. type FileDetails struct { - Program string `protobuf:"bytes,1,opt,name=program,proto3" json:"program,omitempty"` - Host string `protobuf:"bytes,2,opt,name=host,proto3" json:"host,omitempty"` - UserName string `protobuf:"bytes,3,opt,name=user_name,json=userName,proto3" json:"user_name,omitempty"` - Time int64 `protobuf:"varint,5,opt,name=time,proto3" json:"time,omitempty"` - PID int64 `protobuf:"varint,6,opt,name=pid,proto3" json:"pid,omitempty"` + Program string `protobuf:"bytes,1,opt,name=program,proto3" json:"program,omitempty"` + Host string `protobuf:"bytes,2,opt,name=host,proto3" json:"host,omitempty"` + UserName string `protobuf:"bytes,3,opt,name=user_name,json=userName,proto3" json:"user_name,omitempty"` + Time int64 `protobuf:"varint,5,opt,name=time,proto3" json:"time,omitempty"` + PID int64 `protobuf:"varint,6,opt,name=pid,proto3" json:"pid,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *FileDetails) Reset() { *m = FileDetails{} } +func (m *FileDetails) String() string { return proto.CompactTextString(m) } +func (*FileDetails) ProtoMessage() {} +func (*FileDetails) Descriptor() ([]byte, []int) { + return fileDescriptor_log_3a319d334ebe63bc, []int{1} +} +func (m *FileDetails) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *FileDetails) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *FileDetails) XXX_Merge(src proto.Message) { + xxx_messageInfo_FileDetails.Merge(dst, src) +} +func (m *FileDetails) XXX_Size() int { + return m.Size() +} +func (m *FileDetails) XXX_DiscardUnknown() { + xxx_messageInfo_FileDetails.DiscardUnknown(m) } -func (m *FileDetails) Reset() { *m = FileDetails{} } -func (m *FileDetails) String() string { return proto.CompactTextString(m) } -func (*FileDetails) ProtoMessage() {} -func (*FileDetails) Descriptor() ([]byte, []int) { return fileDescriptorLog, []int{1} } +var xxx_messageInfo_FileDetails proto.InternalMessageInfo type FileInfo struct { - Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` - SizeBytes int64 `protobuf:"varint,2,opt,name=size_bytes,json=sizeBytes,proto3" json:"size_bytes,omitempty"` - ModTimeNanos int64 `protobuf:"varint,3,opt,name=mod_time_nanos,json=modTimeNanos,proto3" json:"mod_time_nanos,omitempty"` - Details FileDetails `protobuf:"bytes,4,opt,name=details" json:"details"` + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + SizeBytes int64 `protobuf:"varint,2,opt,name=size_bytes,json=sizeBytes,proto3" json:"size_bytes,omitempty"` + ModTimeNanos int64 `protobuf:"varint,3,opt,name=mod_time_nanos,json=modTimeNanos,proto3" json:"mod_time_nanos,omitempty"` + Details FileDetails `protobuf:"bytes,4,opt,name=details,proto3" json:"details"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *FileInfo) Reset() { *m = FileInfo{} } +func (m *FileInfo) String() string { return proto.CompactTextString(m) } +func (*FileInfo) ProtoMessage() {} +func (*FileInfo) Descriptor() ([]byte, []int) { + return fileDescriptor_log_3a319d334ebe63bc, []int{2} +} +func (m *FileInfo) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *FileInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *FileInfo) XXX_Merge(src proto.Message) { + xxx_messageInfo_FileInfo.Merge(dst, src) +} +func (m *FileInfo) XXX_Size() int { + return m.Size() +} +func (m *FileInfo) XXX_DiscardUnknown() { + xxx_messageInfo_FileInfo.DiscardUnknown(m) } -func (m *FileInfo) Reset() { *m = FileInfo{} } -func (m *FileInfo) String() string { return proto.CompactTextString(m) } -func (*FileInfo) ProtoMessage() {} -func (*FileInfo) Descriptor() ([]byte, []int) { return fileDescriptorLog, []int{2} } +var xxx_messageInfo_FileInfo proto.InternalMessageInfo func init() { proto.RegisterType((*Entry)(nil), "cockroach.util.log.Entry") @@ -270,6 +339,9 @@ func encodeVarintLog(dAtA []byte, offset int, v uint64) int { return offset + 1 } func (m *Entry) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Severity != 0 { @@ -296,6 +368,9 @@ func (m *Entry) Size() (n int) { } func (m *FileDetails) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Program) @@ -320,6 +395,9 @@ func (m *FileDetails) Size() (n int) { } func (m *FileInfo) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Name) @@ -961,9 +1039,9 @@ var ( ErrIntOverflowLog = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("util/log/log.proto", fileDescriptorLog) } +func init() { proto.RegisterFile("util/log/log.proto", fileDescriptor_log_3a319d334ebe63bc) } -var fileDescriptorLog = []byte{ +var fileDescriptor_log_3a319d334ebe63bc = []byte{ // 477 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x6c, 0x92, 0xc1, 0x8a, 0xd3, 0x40, 0x18, 0xc7, 0x3b, 0x9b, 0xa4, 0x4d, 0xbe, 0xca, 0x12, 0x06, 0x0f, 0x51, 0x77, 0xd3, 0x52, 0x3c, diff --git a/pkg/util/metric/metric.pb.go b/pkg/util/metric/metric.pb.go index 89e0e7a42743..271945aa0eba 100644 --- a/pkg/util/metric/metric.pb.go +++ b/pkg/util/metric/metric.pb.go @@ -1,25 +1,17 @@ // Code generated by protoc-gen-gogo. DO NOT EDIT. // source: util/metric/metric.proto -/* - Package metric is a generated protocol buffer package. - - It is generated from these files: - util/metric/metric.proto - - It has these top-level messages: - LabelPair - Metadata -*/ package metric import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" -import io_prometheus_client "github.com/prometheus/client_model/go" +import _go "github.com/prometheus/client_model/go" import io "io" +import github_com_gogo_protobuf_proto "github.com/gogo/protobuf/proto" + // Reference imports to suppress errors if they are not otherwise used. var _ = proto.Marshal var _ = fmt.Errorf @@ -96,7 +88,9 @@ func (x *Unit) UnmarshalJSON(data []byte) error { *x = Unit(value) return nil } -func (Unit) EnumDescriptor() ([]byte, []int) { return fileDescriptorMetric, []int{0} } +func (Unit) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_metric_31105fc50986e5b9, []int{0} +} // metric.LabelPair is a proxy for io.prometheus.client.LabelPair. // io.prometheus.client.LabelPair doesn't support gogoproto.marshaler @@ -105,31 +99,83 @@ func (Unit) EnumDescriptor() ([]byte, []int) { return fileDescriptorMetric, []in // gogoproto options, and is convertible to io.prometheus.client.LabelPair // to satisfy PrometheusExportable's GetLabels method. type LabelPair struct { - Name *string `protobuf:"bytes,1,opt,name=name" json:"name,omitempty"` - Value *string `protobuf:"bytes,2,opt,name=value" json:"value,omitempty"` + Name *string `protobuf:"bytes,1,opt,name=name" json:"name,omitempty"` + Value *string `protobuf:"bytes,2,opt,name=value" json:"value,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *LabelPair) Reset() { *m = LabelPair{} } +func (m *LabelPair) String() string { return proto.CompactTextString(m) } +func (*LabelPair) ProtoMessage() {} +func (*LabelPair) Descriptor() ([]byte, []int) { + return fileDescriptor_metric_31105fc50986e5b9, []int{0} +} +func (m *LabelPair) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *LabelPair) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *LabelPair) XXX_Merge(src proto.Message) { + xxx_messageInfo_LabelPair.Merge(dst, src) +} +func (m *LabelPair) XXX_Size() int { + return m.Size() +} +func (m *LabelPair) XXX_DiscardUnknown() { + xxx_messageInfo_LabelPair.DiscardUnknown(m) } -func (m *LabelPair) Reset() { *m = LabelPair{} } -func (m *LabelPair) String() string { return proto.CompactTextString(m) } -func (*LabelPair) ProtoMessage() {} -func (*LabelPair) Descriptor() ([]byte, []int) { return fileDescriptorMetric, []int{0} } +var xxx_messageInfo_LabelPair proto.InternalMessageInfo // Metadata holds metadata about a metric. It must be embedded in // each metric object. It's used to export information about the // metric to Prometheus and for Admin UI charts. type Metadata struct { - Name string `protobuf:"bytes,1,req,name=name" json:"name"` - Help string `protobuf:"bytes,2,req,name=help" json:"help"` - Measurement string `protobuf:"bytes,3,req,name=measurement" json:"measurement"` - Unit Unit `protobuf:"varint,4,req,name=unit,enum=cockroach.util.metric.Unit" json:"unit"` - MetricType io_prometheus_client.MetricType `protobuf:"varint,5,opt,name=metricType,enum=io.prometheus.client.MetricType" json:"metricType"` - Labels []*LabelPair `protobuf:"bytes,6,rep,name=labels" json:"labels,omitempty"` + Name string `protobuf:"bytes,1,req,name=name" json:"name"` + Help string `protobuf:"bytes,2,req,name=help" json:"help"` + Measurement string `protobuf:"bytes,3,req,name=measurement" json:"measurement"` + Unit Unit `protobuf:"varint,4,req,name=unit,enum=cockroach.util.metric.Unit" json:"unit"` + MetricType _go.MetricType `protobuf:"varint,5,opt,name=metricType,enum=io.prometheus.client.MetricType" json:"metricType"` + Labels []*LabelPair `protobuf:"bytes,6,rep,name=labels" json:"labels,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *Metadata) Reset() { *m = Metadata{} } -func (m *Metadata) String() string { return proto.CompactTextString(m) } -func (*Metadata) ProtoMessage() {} -func (*Metadata) Descriptor() ([]byte, []int) { return fileDescriptorMetric, []int{1} } +func (m *Metadata) Reset() { *m = Metadata{} } +func (m *Metadata) String() string { return proto.CompactTextString(m) } +func (*Metadata) ProtoMessage() {} +func (*Metadata) Descriptor() ([]byte, []int) { + return fileDescriptor_metric_31105fc50986e5b9, []int{1} +} +func (m *Metadata) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Metadata) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *Metadata) XXX_Merge(src proto.Message) { + xxx_messageInfo_Metadata.Merge(dst, src) +} +func (m *Metadata) XXX_Size() int { + return m.Size() +} +func (m *Metadata) XXX_DiscardUnknown() { + xxx_messageInfo_Metadata.DiscardUnknown(m) +} + +var xxx_messageInfo_Metadata proto.InternalMessageInfo func init() { proto.RegisterType((*LabelPair)(nil), "cockroach.util.metric.LabelPair") @@ -224,6 +270,9 @@ func encodeVarintMetric(dAtA []byte, offset int, v uint64) int { return offset + 1 } func (m *LabelPair) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.Name != nil { @@ -238,6 +287,9 @@ func (m *LabelPair) Size() (n int) { } func (m *Metadata) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Name) @@ -534,7 +586,7 @@ func (m *Metadata) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.MetricType |= (io_prometheus_client.MetricType(b) & 0x7F) << shift + m.MetricType |= (_go.MetricType(b) & 0x7F) << shift if b < 0x80 { break } @@ -586,16 +638,16 @@ func (m *Metadata) Unmarshal(dAtA []byte) error { } } if hasFields[0]&uint64(0x00000001) == 0 { - return proto.NewRequiredNotSetError("name") + return github_com_gogo_protobuf_proto.NewRequiredNotSetError("name") } if hasFields[0]&uint64(0x00000002) == 0 { - return proto.NewRequiredNotSetError("help") + return github_com_gogo_protobuf_proto.NewRequiredNotSetError("help") } if hasFields[0]&uint64(0x00000004) == 0 { - return proto.NewRequiredNotSetError("measurement") + return github_com_gogo_protobuf_proto.NewRequiredNotSetError("measurement") } if hasFields[0]&uint64(0x00000008) == 0 { - return proto.NewRequiredNotSetError("unit") + return github_com_gogo_protobuf_proto.NewRequiredNotSetError("unit") } if iNdEx > l { @@ -708,9 +760,9 @@ var ( ErrIntOverflowMetric = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("util/metric/metric.proto", fileDescriptorMetric) } +func init() { proto.RegisterFile("util/metric/metric.proto", fileDescriptor_metric_31105fc50986e5b9) } -var fileDescriptorMetric = []byte{ +var fileDescriptor_metric_31105fc50986e5b9 = []byte{ // 421 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x74, 0x90, 0x41, 0x8b, 0xd3, 0x40, 0x18, 0x86, 0x3b, 0x69, 0xda, 0xdd, 0x7e, 0xd5, 0x75, 0x1c, 0x56, 0x18, 0x56, 0x88, 0x61, 0x41, diff --git a/pkg/util/protoutil/clone.pb.go b/pkg/util/protoutil/clone.pb.go index 2162e46a0c37..c3996bcdf6f2 100644 --- a/pkg/util/protoutil/clone.pb.go +++ b/pkg/util/protoutil/clone.pb.go @@ -1,15 +1,6 @@ // Code generated by protoc-gen-gogo. DO NOT EDIT. // source: util/protoutil/clone.proto -/* - Package protoutil is a generated protocol buffer package. - - It is generated from these files: - util/protoutil/clone.proto - - It has these top-level messages: - RecursiveAndUncloneable -*/ package protoutil import proto "github.com/gogo/protobuf/proto" @@ -32,14 +23,40 @@ var _ = math.Inf const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package type RecursiveAndUncloneable struct { - R *RecursiveAndUncloneable `protobuf:"bytes,1,opt,name=r" json:"r,omitempty"` - Uuid github_com_cockroachdb_cockroach_pkg_util_uuid.UUID `protobuf:"bytes,2,opt,name=uuid,proto3,customtype=github.com/cockroachdb/cockroach/pkg/util/uuid.UUID" json:"uuid"` + R *RecursiveAndUncloneable `protobuf:"bytes,1,opt,name=r,proto3" json:"r,omitempty"` + Uuid github_com_cockroachdb_cockroach_pkg_util_uuid.UUID `protobuf:"bytes,2,opt,name=uuid,proto3,customtype=github.com/cockroachdb/cockroach/pkg/util/uuid.UUID" json:"uuid"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *RecursiveAndUncloneable) Reset() { *m = RecursiveAndUncloneable{} } -func (m *RecursiveAndUncloneable) String() string { return proto.CompactTextString(m) } -func (*RecursiveAndUncloneable) ProtoMessage() {} -func (*RecursiveAndUncloneable) Descriptor() ([]byte, []int) { return fileDescriptorClone, []int{0} } +func (m *RecursiveAndUncloneable) Reset() { *m = RecursiveAndUncloneable{} } +func (m *RecursiveAndUncloneable) String() string { return proto.CompactTextString(m) } +func (*RecursiveAndUncloneable) ProtoMessage() {} +func (*RecursiveAndUncloneable) Descriptor() ([]byte, []int) { + return fileDescriptor_clone_83f9ffca05b6b114, []int{0} +} +func (m *RecursiveAndUncloneable) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RecursiveAndUncloneable) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RecursiveAndUncloneable) XXX_Merge(src proto.Message) { + xxx_messageInfo_RecursiveAndUncloneable.Merge(dst, src) +} +func (m *RecursiveAndUncloneable) XXX_Size() int { + return m.Size() +} +func (m *RecursiveAndUncloneable) XXX_DiscardUnknown() { + xxx_messageInfo_RecursiveAndUncloneable.DiscardUnknown(m) +} + +var xxx_messageInfo_RecursiveAndUncloneable proto.InternalMessageInfo func init() { proto.RegisterType((*RecursiveAndUncloneable)(nil), "cockroach.util.protoutil.RecursiveAndUncloneable") @@ -90,6 +107,9 @@ func encodeVarintClone(dAtA []byte, offset int, v uint64) int { return offset + 1 } func (m *RecursiveAndUncloneable) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.R != nil { @@ -332,9 +352,9 @@ var ( ErrIntOverflowClone = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("util/protoutil/clone.proto", fileDescriptorClone) } +func init() { proto.RegisterFile("util/protoutil/clone.proto", fileDescriptor_clone_83f9ffca05b6b114) } -var fileDescriptorClone = []byte{ +var fileDescriptor_clone_83f9ffca05b6b114 = []byte{ // 218 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x92, 0x2a, 0x2d, 0xc9, 0xcc, 0xd1, 0x2f, 0x28, 0xca, 0x2f, 0xc9, 0x07, 0xb3, 0x92, 0x73, 0xf2, 0xf3, 0x52, 0xf5, 0xc0, 0x7c, diff --git a/pkg/util/tracing/recorded_span.pb.go b/pkg/util/tracing/recorded_span.pb.go index f783a768db8d..bf3cefe0a393 100644 --- a/pkg/util/tracing/recorded_span.pb.go +++ b/pkg/util/tracing/recorded_span.pb.go @@ -1,26 +1,17 @@ // Code generated by protoc-gen-gogo. DO NOT EDIT. // source: util/tracing/recorded_span.proto -/* - Package tracing is a generated protocol buffer package. - - It is generated from these files: - util/tracing/recorded_span.proto - - It has these top-level messages: - RecordedSpan -*/ package tracing import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" -import google_protobuf1 "github.com/gogo/protobuf/types" +import types "github.com/gogo/protobuf/types" import time "time" -import sortkeys "github.com/gogo/protobuf/sortkeys" -import types "github.com/gogo/protobuf/types" +import github_com_gogo_protobuf_sortkeys "github.com/gogo/protobuf/sortkeys" +import github_com_gogo_protobuf_types "github.com/gogo/protobuf/types" import io "io" @@ -50,52 +41,128 @@ type RecordedSpan struct { Operation string `protobuf:"bytes,4,opt,name=operation,proto3" json:"operation,omitempty"` // Baggage items get passed from parent to child spans (even through gRPC). // Notably, snowball tracing uses a special `sb` baggage item. - Baggage map[string]string `protobuf:"bytes,5,rep,name=baggage" json:"baggage,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Baggage map[string]string `protobuf:"bytes,5,rep,name=baggage,proto3" json:"baggage,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` // Tags associated with the span. - Tags map[string]string `protobuf:"bytes,6,rep,name=tags" json:"tags,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + Tags map[string]string `protobuf:"bytes,6,rep,name=tags,proto3" json:"tags,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` // Time when the span was started. - StartTime time.Time `protobuf:"bytes,7,opt,name=start_time,json=startTime,stdtime" json:"start_time"` + StartTime time.Time `protobuf:"bytes,7,opt,name=start_time,json=startTime,proto3,stdtime" json:"start_time"` // Duration in nanoseconds; 0 if the span is not finished. - Duration time.Duration `protobuf:"bytes,8,opt,name=duration,stdduration" json:"duration"` + Duration time.Duration `protobuf:"bytes,8,opt,name=duration,proto3,stdduration" json:"duration"` // Events logged in the span. - Logs []RecordedSpan_LogRecord `protobuf:"bytes,9,rep,name=logs" json:"logs"` + Logs []RecordedSpan_LogRecord `protobuf:"bytes,9,rep,name=logs,proto3" json:"logs"` // Stats collected in this span. - Stats *google_protobuf1.Any `protobuf:"bytes,10,opt,name=stats" json:"stats,omitempty"` + Stats *types.Any `protobuf:"bytes,10,opt,name=stats,proto3" json:"stats,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *RecordedSpan) Reset() { *m = RecordedSpan{} } -func (m *RecordedSpan) String() string { return proto.CompactTextString(m) } -func (*RecordedSpan) ProtoMessage() {} -func (*RecordedSpan) Descriptor() ([]byte, []int) { return fileDescriptorRecordedSpan, []int{0} } +func (m *RecordedSpan) Reset() { *m = RecordedSpan{} } +func (m *RecordedSpan) String() string { return proto.CompactTextString(m) } +func (*RecordedSpan) ProtoMessage() {} +func (*RecordedSpan) Descriptor() ([]byte, []int) { + return fileDescriptor_recorded_span_36cb8126afc70b66, []int{0} +} +func (m *RecordedSpan) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RecordedSpan) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RecordedSpan) XXX_Merge(src proto.Message) { + xxx_messageInfo_RecordedSpan.Merge(dst, src) +} +func (m *RecordedSpan) XXX_Size() int { + return m.Size() +} +func (m *RecordedSpan) XXX_DiscardUnknown() { + xxx_messageInfo_RecordedSpan.DiscardUnknown(m) +} + +var xxx_messageInfo_RecordedSpan proto.InternalMessageInfo type RecordedSpan_LogRecord struct { // Time of the log record. - Time time.Time `protobuf:"bytes,1,opt,name=time,stdtime" json:"time"` + Time time.Time `protobuf:"bytes,1,opt,name=time,proto3,stdtime" json:"time"` // Fields with values converted to strings. - Fields []RecordedSpan_LogRecord_Field `protobuf:"bytes,2,rep,name=fields" json:"fields"` + Fields []RecordedSpan_LogRecord_Field `protobuf:"bytes,2,rep,name=fields,proto3" json:"fields"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *RecordedSpan_LogRecord) Reset() { *m = RecordedSpan_LogRecord{} } func (m *RecordedSpan_LogRecord) String() string { return proto.CompactTextString(m) } func (*RecordedSpan_LogRecord) ProtoMessage() {} func (*RecordedSpan_LogRecord) Descriptor() ([]byte, []int) { - return fileDescriptorRecordedSpan, []int{0, 2} + return fileDescriptor_recorded_span_36cb8126afc70b66, []int{0, 2} +} +func (m *RecordedSpan_LogRecord) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RecordedSpan_LogRecord) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RecordedSpan_LogRecord) XXX_Merge(src proto.Message) { + xxx_messageInfo_RecordedSpan_LogRecord.Merge(dst, src) +} +func (m *RecordedSpan_LogRecord) XXX_Size() int { + return m.Size() +} +func (m *RecordedSpan_LogRecord) XXX_DiscardUnknown() { + xxx_messageInfo_RecordedSpan_LogRecord.DiscardUnknown(m) } +var xxx_messageInfo_RecordedSpan_LogRecord proto.InternalMessageInfo + type RecordedSpan_LogRecord_Field struct { - Key string `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"` - Value string `protobuf:"bytes,2,opt,name=value,proto3" json:"value,omitempty"` + Key string `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"` + Value string `protobuf:"bytes,2,opt,name=value,proto3" json:"value,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *RecordedSpan_LogRecord_Field) Reset() { *m = RecordedSpan_LogRecord_Field{} } func (m *RecordedSpan_LogRecord_Field) String() string { return proto.CompactTextString(m) } func (*RecordedSpan_LogRecord_Field) ProtoMessage() {} func (*RecordedSpan_LogRecord_Field) Descriptor() ([]byte, []int) { - return fileDescriptorRecordedSpan, []int{0, 2, 0} + return fileDescriptor_recorded_span_36cb8126afc70b66, []int{0, 2, 0} +} +func (m *RecordedSpan_LogRecord_Field) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RecordedSpan_LogRecord_Field) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RecordedSpan_LogRecord_Field) XXX_Merge(src proto.Message) { + xxx_messageInfo_RecordedSpan_LogRecord_Field.Merge(dst, src) +} +func (m *RecordedSpan_LogRecord_Field) XXX_Size() int { + return m.Size() +} +func (m *RecordedSpan_LogRecord_Field) XXX_DiscardUnknown() { + xxx_messageInfo_RecordedSpan_LogRecord_Field.DiscardUnknown(m) } +var xxx_messageInfo_RecordedSpan_LogRecord_Field proto.InternalMessageInfo + func init() { proto.RegisterType((*RecordedSpan)(nil), "cockroach.util.tracing.RecordedSpan") + proto.RegisterMapType((map[string]string)(nil), "cockroach.util.tracing.RecordedSpan.BaggageEntry") + proto.RegisterMapType((map[string]string)(nil), "cockroach.util.tracing.RecordedSpan.TagsEntry") proto.RegisterType((*RecordedSpan_LogRecord)(nil), "cockroach.util.tracing.RecordedSpan.LogRecord") proto.RegisterType((*RecordedSpan_LogRecord_Field)(nil), "cockroach.util.tracing.RecordedSpan.LogRecord.Field") } @@ -140,7 +207,7 @@ func (m *RecordedSpan) MarshalTo(dAtA []byte) (int, error) { for k := range m.Baggage { keysForBaggage = append(keysForBaggage, string(k)) } - sortkeys.Strings(keysForBaggage) + github_com_gogo_protobuf_sortkeys.Strings(keysForBaggage) for _, k := range keysForBaggage { dAtA[i] = 0x2a i++ @@ -162,7 +229,7 @@ func (m *RecordedSpan) MarshalTo(dAtA []byte) (int, error) { for k := range m.Tags { keysForTags = append(keysForTags, string(k)) } - sortkeys.Strings(keysForTags) + github_com_gogo_protobuf_sortkeys.Strings(keysForTags) for _, k := range keysForTags { dAtA[i] = 0x32 i++ @@ -181,16 +248,16 @@ func (m *RecordedSpan) MarshalTo(dAtA []byte) (int, error) { } dAtA[i] = 0x3a i++ - i = encodeVarintRecordedSpan(dAtA, i, uint64(types.SizeOfStdTime(m.StartTime))) - n1, err := types.StdTimeMarshalTo(m.StartTime, dAtA[i:]) + i = encodeVarintRecordedSpan(dAtA, i, uint64(github_com_gogo_protobuf_types.SizeOfStdTime(m.StartTime))) + n1, err := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.StartTime, dAtA[i:]) if err != nil { return 0, err } i += n1 dAtA[i] = 0x42 i++ - i = encodeVarintRecordedSpan(dAtA, i, uint64(types.SizeOfStdDuration(m.Duration))) - n2, err := types.StdDurationMarshalTo(m.Duration, dAtA[i:]) + i = encodeVarintRecordedSpan(dAtA, i, uint64(github_com_gogo_protobuf_types.SizeOfStdDuration(m.Duration))) + n2, err := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.Duration, dAtA[i:]) if err != nil { return 0, err } @@ -237,8 +304,8 @@ func (m *RecordedSpan_LogRecord) MarshalTo(dAtA []byte) (int, error) { _ = l dAtA[i] = 0xa i++ - i = encodeVarintRecordedSpan(dAtA, i, uint64(types.SizeOfStdTime(m.Time))) - n4, err := types.StdTimeMarshalTo(m.Time, dAtA[i:]) + i = encodeVarintRecordedSpan(dAtA, i, uint64(github_com_gogo_protobuf_types.SizeOfStdTime(m.Time))) + n4, err := github_com_gogo_protobuf_types.StdTimeMarshalTo(m.Time, dAtA[i:]) if err != nil { return 0, err } @@ -298,6 +365,9 @@ func encodeVarintRecordedSpan(dAtA []byte, offset int, v uint64) int { return offset + 1 } func (m *RecordedSpan) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l if m.TraceID != 0 { @@ -329,9 +399,9 @@ func (m *RecordedSpan) Size() (n int) { n += mapEntrySize + 1 + sovRecordedSpan(uint64(mapEntrySize)) } } - l = types.SizeOfStdTime(m.StartTime) + l = github_com_gogo_protobuf_types.SizeOfStdTime(m.StartTime) n += 1 + l + sovRecordedSpan(uint64(l)) - l = types.SizeOfStdDuration(m.Duration) + l = github_com_gogo_protobuf_types.SizeOfStdDuration(m.Duration) n += 1 + l + sovRecordedSpan(uint64(l)) if len(m.Logs) > 0 { for _, e := range m.Logs { @@ -347,9 +417,12 @@ func (m *RecordedSpan) Size() (n int) { } func (m *RecordedSpan_LogRecord) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l - l = types.SizeOfStdTime(m.Time) + l = github_com_gogo_protobuf_types.SizeOfStdTime(m.Time) n += 1 + l + sovRecordedSpan(uint64(l)) if len(m.Fields) > 0 { for _, e := range m.Fields { @@ -361,6 +434,9 @@ func (m *RecordedSpan_LogRecord) Size() (n int) { } func (m *RecordedSpan_LogRecord_Field) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.Key) @@ -764,7 +840,7 @@ func (m *RecordedSpan) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if err := types.StdTimeUnmarshal(&m.StartTime, dAtA[iNdEx:postIndex]); err != nil { + if err := github_com_gogo_protobuf_types.StdTimeUnmarshal(&m.StartTime, dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -794,7 +870,7 @@ func (m *RecordedSpan) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if err := types.StdDurationUnmarshal(&m.Duration, dAtA[iNdEx:postIndex]); err != nil { + if err := github_com_gogo_protobuf_types.StdDurationUnmarshal(&m.Duration, dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -856,7 +932,7 @@ func (m *RecordedSpan) Unmarshal(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Stats == nil { - m.Stats = &google_protobuf1.Any{} + m.Stats = &types.Any{} } if err := m.Stats.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err @@ -938,7 +1014,7 @@ func (m *RecordedSpan_LogRecord) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - if err := types.StdTimeUnmarshal(&m.Time, dAtA[iNdEx:postIndex]); err != nil { + if err := github_com_gogo_protobuf_types.StdTimeUnmarshal(&m.Time, dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex @@ -1207,9 +1283,11 @@ var ( ErrIntOverflowRecordedSpan = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("util/tracing/recorded_span.proto", fileDescriptorRecordedSpan) } +func init() { + proto.RegisterFile("util/tracing/recorded_span.proto", fileDescriptor_recorded_span_36cb8126afc70b66) +} -var fileDescriptorRecordedSpan = []byte{ +var fileDescriptor_recorded_span_36cb8126afc70b66 = []byte{ // 536 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x52, 0x4f, 0x6f, 0xd3, 0x3e, 0x18, 0x9e, 0xd7, 0xb4, 0x69, 0xde, 0x56, 0x3f, 0x4d, 0x56, 0xf5, 0x53, 0x16, 0xa1, 0xa4, 0x80, diff --git a/pkg/util/unresolved_addr.go b/pkg/util/unresolved_addr.go index 087bceca460b..6fec5f6dedfb 100644 --- a/pkg/util/unresolved_addr.go +++ b/pkg/util/unresolved_addr.go @@ -78,7 +78,7 @@ func (a UnresolvedAddr) IsEmpty() bool { } // String returns the address's string form. -func (a *UnresolvedAddr) String() string { +func (a UnresolvedAddr) String() string { return a.AddressField } diff --git a/pkg/util/unresolved_addr.pb.go b/pkg/util/unresolved_addr.pb.go index 01656d24a85b..4a2f989810c5 100644 --- a/pkg/util/unresolved_addr.pb.go +++ b/pkg/util/unresolved_addr.pb.go @@ -1,15 +1,6 @@ // Code generated by protoc-gen-gogo. DO NOT EDIT. // source: util/unresolved_addr.proto -/* - Package util is a generated protocol buffer package. - - It is generated from these files: - util/unresolved_addr.proto - - It has these top-level messages: - UnresolvedAddr -*/ package util import proto "github.com/gogo/protobuf/proto" @@ -31,13 +22,39 @@ const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package // / UnresolvedAddr is an unresolved version of net.Addr. type UnresolvedAddr struct { - NetworkField string `protobuf:"bytes,1,opt,name=network_field,json=networkField" json:"network_field"` - AddressField string `protobuf:"bytes,2,opt,name=address_field,json=addressField" json:"address_field"` + NetworkField string `protobuf:"bytes,1,opt,name=network_field,json=networkField" json:"network_field"` + AddressField string `protobuf:"bytes,2,opt,name=address_field,json=addressField" json:"address_field"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_sizecache int32 `json:"-"` } -func (m *UnresolvedAddr) Reset() { *m = UnresolvedAddr{} } -func (*UnresolvedAddr) ProtoMessage() {} -func (*UnresolvedAddr) Descriptor() ([]byte, []int) { return fileDescriptorUnresolvedAddr, []int{0} } +func (m *UnresolvedAddr) Reset() { *m = UnresolvedAddr{} } +func (*UnresolvedAddr) ProtoMessage() {} +func (*UnresolvedAddr) Descriptor() ([]byte, []int) { + return fileDescriptor_unresolved_addr_13d7133e21609618, []int{0} +} +func (m *UnresolvedAddr) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *UnresolvedAddr) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *UnresolvedAddr) XXX_Merge(src proto.Message) { + xxx_messageInfo_UnresolvedAddr.Merge(dst, src) +} +func (m *UnresolvedAddr) XXX_Size() int { + return m.Size() +} +func (m *UnresolvedAddr) XXX_DiscardUnknown() { + xxx_messageInfo_UnresolvedAddr.DiscardUnknown(m) +} + +var xxx_messageInfo_UnresolvedAddr proto.InternalMessageInfo func init() { proto.RegisterType((*UnresolvedAddr)(nil), "cockroach.util.UnresolvedAddr") @@ -78,6 +95,9 @@ func encodeVarintUnresolvedAddr(dAtA []byte, offset int, v uint64) int { return offset + 1 } func (m *UnresolvedAddr) Size() (n int) { + if m == nil { + return 0 + } var l int _ = l l = len(m.NetworkField) @@ -313,9 +333,11 @@ var ( ErrIntOverflowUnresolvedAddr = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("util/unresolved_addr.proto", fileDescriptorUnresolvedAddr) } +func init() { + proto.RegisterFile("util/unresolved_addr.proto", fileDescriptor_unresolved_addr_13d7133e21609618) +} -var fileDescriptorUnresolvedAddr = []byte{ +var fileDescriptor_unresolved_addr_13d7133e21609618 = []byte{ // 180 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x92, 0x2a, 0x2d, 0xc9, 0xcc, 0xd1, 0x2f, 0xcd, 0x2b, 0x4a, 0x2d, 0xce, 0xcf, 0x29, 0x4b, 0x4d, 0x89, 0x4f, 0x4c, 0x49, 0x29, diff --git a/vendor b/vendor index b5f87e73f96b..9ea85486cede 160000 --- a/vendor +++ b/vendor @@ -1 +1 @@ -Subproject commit b5f87e73f96be5e0bdd4d1bcbdd5a1b2d0d7270e +Subproject commit 9ea85486cede9a33357c75111339e015b882ddae