From 47fbe890f23cf0d7c04f170ce252d085c69276de Mon Sep 17 00:00:00 2001 From: Andrei Matei Date: Wed, 24 Feb 2021 14:32:16 -0500 Subject: [PATCH] kvserver: implement closed ts side-transport publisher The side-transport is a component running on each node and periodically publishing closed timestamps on ranges with the lease on the respective node. This complements the closing of timestamps through Raft commands such that inactive ranges still have their timestamp advanced. This commit introduces only the publishing side (the consumer is coming separately) - the guy opening streaming connections to all other nodes with follower replicas for some ranges with local leases and periodically publishing closed timestamp updates on a bunch of ranges at once. Care has been taken to make the communication protocol efficient. Each stream is stateful and the information in every message is nicely compressed. See [the RFC](https://github.com/cockroachdb/cockroach/pull/56675) for details. Release justifaction: Needed for global tables. Release note: None --- pkg/kv/kvserver/closedts/ctpb/service.go | 14 + pkg/kv/kvserver/closedts/ctpb/service.pb.go | 1216 ++++++++++++++++- pkg/kv/kvserver/closedts/ctpb/service.proto | 64 + .../sidetransport/closed_timestamp_sender.go | 563 ++++++++ pkg/kv/kvserver/replica.go | 12 + pkg/kv/kvserver/replica_closedts.go | 72 +- pkg/kv/kvserver/replica_closedts_test.go | 12 +- pkg/kv/kvserver/replica_proposal.go | 7 + pkg/kv/kvserver/replica_proposal_buf.go | 24 +- pkg/kv/kvserver/replica_proposal_buf_test.go | 2 +- pkg/kv/kvserver/store.go | 123 +- pkg/kv/kvserver/store_remove_replica.go | 1 + pkg/roachpb/data.pb.go | 406 +++--- pkg/roachpb/data.proto | 3 + pkg/server/server.go | 139 +- 15 files changed, 2321 insertions(+), 337 deletions(-) create mode 100644 pkg/kv/kvserver/closedts/ctpb/service.go create mode 100644 pkg/kv/kvserver/closedts/sidetransport/closed_timestamp_sender.go diff --git a/pkg/kv/kvserver/closedts/ctpb/service.go b/pkg/kv/kvserver/closedts/ctpb/service.go new file mode 100644 index 000000000000..de53cad08319 --- /dev/null +++ b/pkg/kv/kvserver/closedts/ctpb/service.go @@ -0,0 +1,14 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package ctpb + +// UpdateSequenceNumber identifies a ClosedTimestampUpdate. +type UpdateSequenceNumber int64 diff --git a/pkg/kv/kvserver/closedts/ctpb/service.pb.go b/pkg/kv/kvserver/closedts/ctpb/service.pb.go index d634b6a317ac..5b366268ac71 100644 --- a/pkg/kv/kvserver/closedts/ctpb/service.pb.go +++ b/pkg/kv/kvserver/closedts/ctpb/service.pb.go @@ -6,12 +6,18 @@ package ctpb import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" +import roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" +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" grpc "google.golang.org/grpc" ) +import io "io" + // Reference imports to suppress errors if they are not otherwise used. var _ = proto.Marshal var _ = fmt.Errorf @@ -23,6 +29,188 @@ var _ = math.Inf // proto package needs to be updated. const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package +type RangeUpdate 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"` + LAI LAI `protobuf:"varint,2,opt,name=lai,proto3,casttype=LAI" json:"lai,omitempty"` + Policy roachpb.RangeClosedTimestampPolicy `protobuf:"varint,3,opt,name=policy,proto3,enum=cockroach.roachpb.RangeClosedTimestampPolicy" json:"policy,omitempty"` +} + +func (m *RangeUpdate) Reset() { *m = RangeUpdate{} } +func (m *RangeUpdate) String() string { return proto.CompactTextString(m) } +func (*RangeUpdate) ProtoMessage() {} +func (*RangeUpdate) Descriptor() ([]byte, []int) { + return fileDescriptor_service_123b88c5897d6128, []int{0} +} +func (m *RangeUpdate) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *RangeUpdate) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *RangeUpdate) XXX_Merge(src proto.Message) { + xxx_messageInfo_RangeUpdate.Merge(dst, src) +} +func (m *RangeUpdate) XXX_Size() int { + return m.Size() +} +func (m *RangeUpdate) XXX_DiscardUnknown() { + xxx_messageInfo_RangeUpdate.DiscardUnknown(m) +} + +var xxx_messageInfo_RangeUpdate proto.InternalMessageInfo + +// ClosedTimestampUpdate contains information about (the advancement of) closed +// timestamps for ranges with leases on the sender node. Updates are of two +// types: snapshots and incrementals. Snapshots are stand-alone messages, +// explicitly containing state about a bunch of ranges. Incrementals are deltas +// since the previous message (the previous message can be a snapshot or another +// incremental); they contain info about which new ranges are included in the +// info provided, which ranges are removed, and how the closed timestamps for +// different categories of ranges advanced. Ranges communicated by a previous +// message and not touched by an incremental are "implicitly" referenced by the +// incremental. In order to properly handle incrementals, the recipient +// maintains a "stream's state": the group of ranges that can be implicitly +// referenced by the next message. +type ClosedTimestampUpdate struct { + // node_id identifies the sending node. + 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"` + // seq_num identifies this update across all updates produced by a node. The + // sequence is reset when the node restarts, so a recipient can only count on + // it increasing within a single PushUpdates stream. + // + // All messages have sequence numbers, including snapshots. A snapshot can be + // applied on top of any state (i.e. it can be applied after having skipped + // messages); its sequence number tells the recipient what incremental message + // it should expect afterwards. + SeqNum UpdateSequenceNumber `protobuf:"varint,2,opt,name=seq_num,json=seqNum,proto3,casttype=UpdateSequenceNumber" json:"seq_num,omitempty"` + // snapshot indicates whether this message is standalone, or whether it's just + // a delta since the messages with the previous seq_num. A snapshot + // re-initializes all of the recipient's state. The first message on a stream + // is always a snapshot. Afterwards, there could be others if the stream + // experience network problems and some incremental messages were dropped + // (although generally we expect that to result in a stream failing and a new + // one being established). + Snapshot bool `protobuf:"varint,3,opt,name=snapshot,proto3" json:"snapshot,omitempty"` + Removed []github_com_cockroachdb_cockroach_pkg_roachpb.RangeID `protobuf:"varint,4,rep,packed,name=removed,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" json:"removed,omitempty"` + AddedOrUpdated []RangeUpdate `protobuf:"bytes,5,rep,name=addedOrUpdated,proto3" json:"addedOrUpdated"` + // closed_timestamp represents the timestamps that are being closed for each + // group of ranges, with a group being represented by its policy. + // + // The recipient is supposed to forward the closed timestamps of the affected + // ranges to these values. Upon receiving one of these updates, the recipient + // should generally not assume that it hasn't been informed of a higher closed + // timestamp for any range in particular - races between this side-transport + // and the regular Raft transport are possible. + ClosedTimestamp []ClosedTimestampUpdateGroupUpdate `protobuf:"bytes,6,rep,name=closed_timestamp,json=closedTimestamp,proto3" json:"closed_timestamp"` +} + +func (m *ClosedTimestampUpdate) Reset() { *m = ClosedTimestampUpdate{} } +func (m *ClosedTimestampUpdate) String() string { return proto.CompactTextString(m) } +func (*ClosedTimestampUpdate) ProtoMessage() {} +func (*ClosedTimestampUpdate) Descriptor() ([]byte, []int) { + return fileDescriptor_service_123b88c5897d6128, []int{1} +} +func (m *ClosedTimestampUpdate) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ClosedTimestampUpdate) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ClosedTimestampUpdate) XXX_Merge(src proto.Message) { + xxx_messageInfo_ClosedTimestampUpdate.Merge(dst, src) +} +func (m *ClosedTimestampUpdate) XXX_Size() int { + return m.Size() +} +func (m *ClosedTimestampUpdate) XXX_DiscardUnknown() { + xxx_messageInfo_ClosedTimestampUpdate.DiscardUnknown(m) +} + +var xxx_messageInfo_ClosedTimestampUpdate proto.InternalMessageInfo + +type ClosedTimestampUpdateGroupUpdate struct { + Policy roachpb.RangeClosedTimestampPolicy `protobuf:"varint,1,opt,name=policy,proto3,enum=cockroach.roachpb.RangeClosedTimestampPolicy" json:"policy,omitempty"` + ClosedTimestamp hlc.Timestamp `protobuf:"bytes,2,opt,name=closed_timestamp,json=closedTimestamp,proto3" json:"closed_timestamp"` +} + +func (m *ClosedTimestampUpdateGroupUpdate) Reset() { *m = ClosedTimestampUpdateGroupUpdate{} } +func (m *ClosedTimestampUpdateGroupUpdate) String() string { return proto.CompactTextString(m) } +func (*ClosedTimestampUpdateGroupUpdate) ProtoMessage() {} +func (*ClosedTimestampUpdateGroupUpdate) Descriptor() ([]byte, []int) { + return fileDescriptor_service_123b88c5897d6128, []int{1, 0} +} +func (m *ClosedTimestampUpdateGroupUpdate) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ClosedTimestampUpdateGroupUpdate) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ClosedTimestampUpdateGroupUpdate) XXX_Merge(src proto.Message) { + xxx_messageInfo_ClosedTimestampUpdateGroupUpdate.Merge(dst, src) +} +func (m *ClosedTimestampUpdateGroupUpdate) XXX_Size() int { + return m.Size() +} +func (m *ClosedTimestampUpdateGroupUpdate) XXX_DiscardUnknown() { + xxx_messageInfo_ClosedTimestampUpdateGroupUpdate.DiscardUnknown(m) +} + +var xxx_messageInfo_ClosedTimestampUpdateGroupUpdate proto.InternalMessageInfo + +type Response struct { +} + +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_service_123b88c5897d6128, []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) +} + +var xxx_messageInfo_Response proto.InternalMessageInfo + +func init() { + proto.RegisterType((*RangeUpdate)(nil), "cockroach.kv.kvserver.ctupdate.RangeUpdate") + proto.RegisterType((*ClosedTimestampUpdate)(nil), "cockroach.kv.kvserver.ctupdate.ClosedTimestampUpdate") + proto.RegisterType((*ClosedTimestampUpdateGroupUpdate)(nil), "cockroach.kv.kvserver.ctupdate.ClosedTimestampUpdate.groupUpdate") + proto.RegisterType((*Response)(nil), "cockroach.kv.kvserver.ctupdate.Response") +} + // Reference imports to suppress errors if they are not otherwise used. var _ context.Context var _ grpc.ClientConn @@ -127,25 +315,1013 @@ var _ClosedTimestamp_serviceDesc = grpc.ServiceDesc{ Metadata: "kv/kvserver/closedts/ctpb/service.proto", } +// ClosedTimestampSideTransportClient is the client API for ClosedTimestampSideTransport service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream. +type ClosedTimestampSideTransportClient interface { + PushUpdates(ctx context.Context, opts ...grpc.CallOption) (ClosedTimestampSideTransport_PushUpdatesClient, error) +} + +type closedTimestampSideTransportClient struct { + cc *grpc.ClientConn +} + +func NewClosedTimestampSideTransportClient(cc *grpc.ClientConn) ClosedTimestampSideTransportClient { + return &closedTimestampSideTransportClient{cc} +} + +func (c *closedTimestampSideTransportClient) PushUpdates(ctx context.Context, opts ...grpc.CallOption) (ClosedTimestampSideTransport_PushUpdatesClient, error) { + stream, err := c.cc.NewStream(ctx, &_ClosedTimestampSideTransport_serviceDesc.Streams[0], "/cockroach.kv.kvserver.ctupdate.ClosedTimestampSideTransport/PushUpdates", opts...) + if err != nil { + return nil, err + } + x := &closedTimestampSideTransportPushUpdatesClient{stream} + return x, nil +} + +type ClosedTimestampSideTransport_PushUpdatesClient interface { + Send(*ClosedTimestampUpdate) error + Recv() (*Response, error) + grpc.ClientStream +} + +type closedTimestampSideTransportPushUpdatesClient struct { + grpc.ClientStream +} + +func (x *closedTimestampSideTransportPushUpdatesClient) Send(m *ClosedTimestampUpdate) error { + return x.ClientStream.SendMsg(m) +} + +func (x *closedTimestampSideTransportPushUpdatesClient) Recv() (*Response, error) { + m := new(Response) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +// ClosedTimestampSideTransportServer is the server API for ClosedTimestampSideTransport service. +type ClosedTimestampSideTransportServer interface { + PushUpdates(ClosedTimestampSideTransport_PushUpdatesServer) error +} + +func RegisterClosedTimestampSideTransportServer(s *grpc.Server, srv ClosedTimestampSideTransportServer) { + s.RegisterService(&_ClosedTimestampSideTransport_serviceDesc, srv) +} + +func _ClosedTimestampSideTransport_PushUpdates_Handler(srv interface{}, stream grpc.ServerStream) error { + return srv.(ClosedTimestampSideTransportServer).PushUpdates(&closedTimestampSideTransportPushUpdatesServer{stream}) +} + +type ClosedTimestampSideTransport_PushUpdatesServer interface { + Send(*Response) error + Recv() (*ClosedTimestampUpdate, error) + grpc.ServerStream +} + +type closedTimestampSideTransportPushUpdatesServer struct { + grpc.ServerStream +} + +func (x *closedTimestampSideTransportPushUpdatesServer) Send(m *Response) error { + return x.ServerStream.SendMsg(m) +} + +func (x *closedTimestampSideTransportPushUpdatesServer) Recv() (*ClosedTimestampUpdate, error) { + m := new(ClosedTimestampUpdate) + if err := x.ServerStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +var _ClosedTimestampSideTransport_serviceDesc = grpc.ServiceDesc{ + ServiceName: "cockroach.kv.kvserver.ctupdate.ClosedTimestampSideTransport", + HandlerType: (*ClosedTimestampSideTransportServer)(nil), + Methods: []grpc.MethodDesc{}, + Streams: []grpc.StreamDesc{ + { + StreamName: "PushUpdates", + Handler: _ClosedTimestampSideTransport_PushUpdates_Handler, + ServerStreams: true, + ClientStreams: true, + }, + }, + Metadata: "kv/kvserver/closedts/ctpb/service.proto", +} + +func (m *RangeUpdate) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *RangeUpdate) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if m.RangeID != 0 { + dAtA[i] = 0x8 + i++ + i = encodeVarintService(dAtA, i, uint64(m.RangeID)) + } + if m.LAI != 0 { + dAtA[i] = 0x10 + i++ + i = encodeVarintService(dAtA, i, uint64(m.LAI)) + } + if m.Policy != 0 { + dAtA[i] = 0x18 + i++ + i = encodeVarintService(dAtA, i, uint64(m.Policy)) + } + return i, nil +} + +func (m *ClosedTimestampUpdate) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClosedTimestampUpdate) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if m.NodeID != 0 { + dAtA[i] = 0x8 + i++ + i = encodeVarintService(dAtA, i, uint64(m.NodeID)) + } + if m.SeqNum != 0 { + dAtA[i] = 0x10 + i++ + i = encodeVarintService(dAtA, i, uint64(m.SeqNum)) + } + if m.Snapshot { + dAtA[i] = 0x18 + i++ + if m.Snapshot { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i++ + } + if len(m.Removed) > 0 { + dAtA2 := make([]byte, len(m.Removed)*10) + var j1 int + for _, num1 := range m.Removed { + num := uint64(num1) + for num >= 1<<7 { + dAtA2[j1] = uint8(uint64(num)&0x7f | 0x80) + num >>= 7 + j1++ + } + dAtA2[j1] = uint8(num) + j1++ + } + dAtA[i] = 0x22 + i++ + i = encodeVarintService(dAtA, i, uint64(j1)) + i += copy(dAtA[i:], dAtA2[:j1]) + } + if len(m.AddedOrUpdated) > 0 { + for _, msg := range m.AddedOrUpdated { + dAtA[i] = 0x2a + i++ + i = encodeVarintService(dAtA, i, uint64(msg.Size())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } + if len(m.ClosedTimestamp) > 0 { + for _, msg := range m.ClosedTimestamp { + dAtA[i] = 0x32 + i++ + i = encodeVarintService(dAtA, i, uint64(msg.Size())) + n, err := msg.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n + } + } + return i, nil +} + +func (m *ClosedTimestampUpdateGroupUpdate) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClosedTimestampUpdateGroupUpdate) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if m.Policy != 0 { + dAtA[i] = 0x8 + i++ + i = encodeVarintService(dAtA, i, uint64(m.Policy)) + } + dAtA[i] = 0x12 + i++ + i = encodeVarintService(dAtA, i, uint64(m.ClosedTimestamp.Size())) + n3, err := m.ClosedTimestamp.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n3 + return i, nil +} + +func (m *Response) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Response) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + return i, nil +} + +func encodeVarintService(dAtA []byte, offset int, v uint64) int { + for v >= 1<<7 { + dAtA[offset] = uint8(v&0x7f | 0x80) + v >>= 7 + offset++ + } + dAtA[offset] = uint8(v) + return offset + 1 +} +func (m *RangeUpdate) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.RangeID != 0 { + n += 1 + sovService(uint64(m.RangeID)) + } + if m.LAI != 0 { + n += 1 + sovService(uint64(m.LAI)) + } + if m.Policy != 0 { + n += 1 + sovService(uint64(m.Policy)) + } + return n +} + +func (m *ClosedTimestampUpdate) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.NodeID != 0 { + n += 1 + sovService(uint64(m.NodeID)) + } + if m.SeqNum != 0 { + n += 1 + sovService(uint64(m.SeqNum)) + } + if m.Snapshot { + n += 2 + } + if len(m.Removed) > 0 { + l = 0 + for _, e := range m.Removed { + l += sovService(uint64(e)) + } + n += 1 + sovService(uint64(l)) + l + } + if len(m.AddedOrUpdated) > 0 { + for _, e := range m.AddedOrUpdated { + l = e.Size() + n += 1 + l + sovService(uint64(l)) + } + } + if len(m.ClosedTimestamp) > 0 { + for _, e := range m.ClosedTimestamp { + l = e.Size() + n += 1 + l + sovService(uint64(l)) + } + } + return n +} + +func (m *ClosedTimestampUpdateGroupUpdate) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Policy != 0 { + n += 1 + sovService(uint64(m.Policy)) + } + l = m.ClosedTimestamp.Size() + n += 1 + l + sovService(uint64(l)) + return n +} + +func (m *Response) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + return n +} + +func sovService(x uint64) (n int) { + for { + n++ + x >>= 7 + if x == 0 { + break + } + } + return n +} +func sozService(x uint64) (n int) { + return sovService(uint64((x << 1) ^ uint64((int64(x) >> 63)))) +} +func (m *RangeUpdate) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowService + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: RangeUpdate: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: RangeUpdate: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field RangeID", wireType) + } + m.RangeID = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowService + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.RangeID |= (github_com_cockroachdb_cockroach_pkg_roachpb.RangeID(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field LAI", wireType) + } + m.LAI = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowService + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.LAI |= (LAI(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Policy", wireType) + } + m.Policy = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowService + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Policy |= (roachpb.RangeClosedTimestampPolicy(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipService(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthService + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClosedTimestampUpdate) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowService + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClosedTimestampUpdate: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClosedTimestampUpdate: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field NodeID", wireType) + } + m.NodeID = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowService + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.NodeID |= (github_com_cockroachdb_cockroach_pkg_roachpb.NodeID(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field SeqNum", wireType) + } + m.SeqNum = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowService + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.SeqNum |= (UpdateSequenceNumber(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Snapshot", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowService + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.Snapshot = bool(v != 0) + case 4: + if wireType == 0 { + var v github_com_cockroachdb_cockroach_pkg_roachpb.RangeID + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowService + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (github_com_cockroachdb_cockroach_pkg_roachpb.RangeID(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.Removed = append(m.Removed, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowService + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthService + } + postIndex := iNdEx + packedLen + 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.Removed) == 0 { + m.Removed = 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 { + if shift >= 64 { + return ErrIntOverflowService + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= (github_com_cockroachdb_cockroach_pkg_roachpb.RangeID(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + m.Removed = append(m.Removed, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field Removed", wireType) + } + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field AddedOrUpdated", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowService + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthService + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.AddedOrUpdated = append(m.AddedOrUpdated, RangeUpdate{}) + if err := m.AddedOrUpdated[len(m.AddedOrUpdated)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ClosedTimestamp", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowService + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthService + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.ClosedTimestamp = append(m.ClosedTimestamp, ClosedTimestampUpdateGroupUpdate{}) + if err := m.ClosedTimestamp[len(m.ClosedTimestamp)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipService(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthService + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClosedTimestampUpdateGroupUpdate) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowService + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: groupUpdate: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: groupUpdate: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Policy", wireType) + } + m.Policy = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowService + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Policy |= (roachpb.RangeClosedTimestampPolicy(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ClosedTimestamp", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowService + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthService + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.ClosedTimestamp.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipService(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthService + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Response) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowService + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Response: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Response: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + default: + iNdEx = preIndex + skippy, err := skipService(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthService + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func skipService(dAtA []byte) (n int, err error) { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowService + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + wireType := int(wire & 0x7) + switch wireType { + case 0: + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowService + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + iNdEx++ + if dAtA[iNdEx-1] < 0x80 { + break + } + } + return iNdEx, nil + case 1: + iNdEx += 8 + return iNdEx, nil + case 2: + var length int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowService + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + length |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + iNdEx += length + if length < 0 { + return 0, ErrInvalidLengthService + } + return iNdEx, nil + case 3: + for { + var innerWire uint64 + var start int = iNdEx + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowService + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + innerWire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + innerWireType := int(innerWire & 0x7) + if innerWireType == 4 { + break + } + next, err := skipService(dAtA[start:]) + if err != nil { + return 0, err + } + iNdEx = start + next + } + return iNdEx, nil + case 4: + return iNdEx, nil + case 5: + iNdEx += 4 + return iNdEx, nil + default: + return 0, fmt.Errorf("proto: illegal wireType %d", wireType) + } + } + panic("unreachable") +} + +var ( + ErrInvalidLengthService = fmt.Errorf("proto: negative length found during unmarshaling") + ErrIntOverflowService = fmt.Errorf("proto: integer overflow") +) + func init() { - proto.RegisterFile("kv/kvserver/closedts/ctpb/service.proto", fileDescriptor_service_3096204571c3b264) -} - -var fileDescriptor_service_3096204571c3b264 = []byte{ - // 230 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x84, 0x8f, 0xb1, 0x4a, 0x04, 0x31, - 0x10, 0x86, 0x13, 0x14, 0x8b, 0x6d, 0x84, 0xc5, 0x6a, 0x91, 0x29, 0x84, 0xd3, 0xab, 0x12, 0xd1, - 0x37, 0x50, 0xc4, 0x5e, 0xac, 0xc4, 0x26, 0x37, 0x37, 0xac, 0xcb, 0xde, 0xed, 0x84, 0x64, 0x0c, - 0xf8, 0x16, 0x3e, 0xd6, 0x95, 0x57, 0x5e, 0xa9, 0xd9, 0x17, 0x91, 0xec, 0x69, 0xa9, 0x76, 0x3f, - 0xff, 0xcc, 0x7c, 0x7c, 0x53, 0x5d, 0xf4, 0xc9, 0xf6, 0x29, 0x52, 0x48, 0x14, 0x2c, 0xae, 0x38, - 0xd2, 0x52, 0xa2, 0x45, 0xf1, 0x0b, 0x5b, 0xca, 0x0e, 0xc9, 0xf8, 0xc0, 0xc2, 0x35, 0x20, 0x63, - 0x1f, 0xd8, 0xe1, 0x8b, 0xe9, 0x93, 0xf9, 0x39, 0x31, 0x28, 0xaf, 0x7e, 0xe9, 0x84, 0x9a, 0xd9, - 0xef, 0x20, 0x1a, 0x24, 0xbc, 0xed, 0x31, 0xcd, 0x49, 0xcb, 0x2d, 0x4f, 0xd1, 0x96, 0xf4, 0xdd, - 0x9e, 0xb6, 0xcc, 0xed, 0x8a, 0xac, 0xf3, 0x9d, 0x75, 0xc3, 0xc0, 0xe2, 0xa4, 0xe3, 0x21, 0xee, - 0xa7, 0x57, 0x5c, 0x1d, 0xdf, 0x4e, 0xc0, 0xc7, 0x6e, 0x4d, 0x51, 0xdc, 0xda, 0xd7, 0xcf, 0xd5, - 0xc1, 0x3d, 0x49, 0x3d, 0x37, 0x7f, 0x5b, 0x99, 0x07, 0x72, 0x58, 0x50, 0xcd, 0xec, 0xbf, 0xcd, - 0xbb, 0x22, 0x79, 0xa6, 0xe6, 0xfa, 0x52, 0xdf, 0x9c, 0x6f, 0x3e, 0x41, 0x6d, 0x32, 0xe8, 0x6d, - 0x06, 0xbd, 0xcb, 0xa0, 0x3f, 0x32, 0xe8, 0xf7, 0x11, 0xd4, 0x76, 0x04, 0xb5, 0x1b, 0x41, 0x3d, - 0x1d, 0x96, 0xc7, 0x16, 0x47, 0x93, 0xdf, 0xf5, 0x57, 0x00, 0x00, 0x00, 0xff, 0xff, 0x7e, 0xcb, - 0xa0, 0xcc, 0x45, 0x01, 0x00, 0x00, + proto.RegisterFile("kv/kvserver/closedts/ctpb/service.proto", fileDescriptor_service_123b88c5897d6128) +} + +var fileDescriptor_service_123b88c5897d6128 = []byte{ + // 643 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x94, 0xdf, 0x6a, 0xdb, 0x3e, + 0x14, 0xc7, 0xa3, 0x5f, 0xfe, 0xa2, 0x40, 0xfb, 0x43, 0x74, 0x60, 0x42, 0xe7, 0x84, 0x40, 0x37, + 0xc3, 0x98, 0xb5, 0xa5, 0x1b, 0xec, 0xb6, 0xe9, 0xca, 0x08, 0x8c, 0xac, 0xb8, 0xdd, 0xc5, 0xca, + 0x58, 0x51, 0x24, 0xe1, 0x98, 0xd8, 0x96, 0x2b, 0xc9, 0x81, 0xbe, 0xc3, 0x2e, 0x76, 0xb1, 0x47, + 0xd8, 0xc3, 0xf4, 0xb2, 0x57, 0xa3, 0x57, 0x61, 0x73, 0xdf, 0xa2, 0x57, 0xc3, 0x7f, 0x9a, 0xa6, + 0xa1, 0x5d, 0xb7, 0xf5, 0xca, 0xb2, 0x74, 0xce, 0xe7, 0x9c, 0xef, 0x39, 0x47, 0x82, 0x8f, 0x27, + 0x53, 0x3c, 0x99, 0x2a, 0x2e, 0xa7, 0x5c, 0x62, 0xea, 0x0b, 0xc5, 0x99, 0x56, 0x98, 0xea, 0x68, + 0x84, 0xd3, 0x4d, 0x8f, 0x72, 0x3b, 0x92, 0x42, 0x0b, 0x64, 0x52, 0x41, 0x27, 0x52, 0x10, 0x3a, + 0xb6, 0x27, 0x53, 0xfb, 0xd2, 0xc5, 0xa6, 0x3a, 0x8e, 0x18, 0xd1, 0xbc, 0xb5, 0x71, 0x3b, 0x88, + 0x87, 0x5a, 0x1e, 0xe7, 0x98, 0x16, 0xca, 0x10, 0xd1, 0x08, 0x33, 0xa2, 0x49, 0xb1, 0x67, 0xc4, + 0xda, 0xf3, 0xf1, 0xd8, 0xa7, 0x58, 0x7b, 0x01, 0x57, 0x9a, 0x04, 0x51, 0x71, 0xb2, 0xe6, 0x0a, + 0x57, 0x64, 0x4b, 0x9c, 0xae, 0x8a, 0xdd, 0x75, 0x57, 0x08, 0xd7, 0xe7, 0x98, 0x44, 0x1e, 0x26, + 0x61, 0x28, 0x34, 0xd1, 0x9e, 0x08, 0x55, 0x7e, 0xda, 0xfd, 0x0e, 0x60, 0xd3, 0x21, 0xa1, 0xcb, + 0xdf, 0x67, 0x89, 0xa1, 0x4f, 0xb0, 0x21, 0xd3, 0xdf, 0x43, 0x8f, 0x19, 0xa0, 0x03, 0xac, 0x4a, + 0x7f, 0x3b, 0x99, 0xb5, 0xeb, 0x99, 0xc9, 0xe0, 0xf5, 0xc5, 0xac, 0xfd, 0xc2, 0xf5, 0xf4, 0x38, + 0x1e, 0xd9, 0x54, 0x04, 0x78, 0x2e, 0x92, 0x8d, 0xae, 0xd6, 0x38, 0x9a, 0xb8, 0xb8, 0xc8, 0xdb, + 0x2e, 0xfc, 0x9c, 0x7a, 0x06, 0x1d, 0x30, 0xd4, 0x81, 0x65, 0x9f, 0x78, 0xc6, 0x7f, 0x1d, 0x60, + 0x95, 0xfb, 0x2b, 0xc9, 0xac, 0x5d, 0x7e, 0xbb, 0x35, 0xb8, 0xc8, 0x3f, 0x4e, 0x7a, 0x84, 0x76, + 0x60, 0x2d, 0x12, 0xbe, 0x47, 0x8f, 0x8d, 0x72, 0x07, 0x58, 0x2b, 0xbd, 0xa7, 0xf6, 0x55, 0x2d, + 0xaf, 0x61, 0xb7, 0xb3, 0xb2, 0xed, 0x5f, 0x16, 0x61, 0x37, 0x73, 0x72, 0x0a, 0xe7, 0xee, 0xe7, + 0x2a, 0x7c, 0xb0, 0x64, 0x51, 0x48, 0x3c, 0x80, 0xf5, 0x50, 0xb0, 0xb9, 0xc2, 0x6a, 0x7f, 0x2b, + 0x99, 0xb5, 0x6b, 0x43, 0xc1, 0x72, 0x81, 0x9b, 0x7f, 0x25, 0x30, 0x77, 0x73, 0x6a, 0x29, 0x71, + 0xc0, 0xd0, 0x73, 0x58, 0x57, 0xfc, 0xe8, 0x30, 0x8c, 0x83, 0x42, 0xa2, 0x71, 0x31, 0x6b, 0xaf, + 0xe5, 0x81, 0xf7, 0xf8, 0x51, 0xcc, 0x43, 0xca, 0x87, 0x71, 0x30, 0xe2, 0xd2, 0xa9, 0x29, 0x7e, + 0x34, 0x8c, 0x03, 0xd4, 0x82, 0x0d, 0x15, 0x92, 0x48, 0x8d, 0x85, 0xce, 0x14, 0x37, 0x9c, 0xf9, + 0x3f, 0x72, 0x60, 0x5d, 0xf2, 0x40, 0x4c, 0x39, 0x33, 0x2a, 0x9d, 0xb2, 0x55, 0xed, 0xbf, 0xba, + 0x47, 0x07, 0x72, 0x10, 0xfa, 0x00, 0x57, 0x08, 0x63, 0x9c, 0xbd, 0x93, 0x79, 0x5a, 0xcc, 0xa8, + 0x76, 0xca, 0x56, 0xb3, 0xf7, 0xc4, 0xfe, 0xfd, 0xcc, 0xda, 0x0b, 0x63, 0xd2, 0xaf, 0x9c, 0xcc, + 0xda, 0x25, 0x67, 0x09, 0x84, 0x24, 0xfc, 0x3f, 0x9f, 0xe5, 0xc3, 0xf9, 0x68, 0x1a, 0xb5, 0x0c, + 0xbe, 0x75, 0x17, 0xfc, 0xc6, 0x56, 0xd9, 0xae, 0x14, 0x71, 0x74, 0x2d, 0xe4, 0x2a, 0xbd, 0x6e, + 0xd8, 0xfa, 0x06, 0x60, 0x73, 0xc1, 0x6c, 0x61, 0x7c, 0xc0, 0x3d, 0xc6, 0x07, 0x0d, 0x6f, 0x90, + 0x92, 0x76, 0xb4, 0xd9, 0x7b, 0xb8, 0x00, 0x4c, 0xaf, 0xa2, 0x3d, 0xf6, 0xa9, 0x3d, 0xc7, 0xdc, + 0x92, 0x66, 0x17, 0xc2, 0x86, 0xc3, 0x55, 0x24, 0x42, 0xc5, 0x7b, 0x02, 0xae, 0x2e, 0x05, 0x47, + 0x1f, 0x61, 0xf9, 0x0d, 0xd7, 0xc8, 0xba, 0xb3, 0x07, 0x9c, 0xd0, 0xf4, 0xfa, 0xb6, 0x36, 0xee, + 0xb2, 0xdc, 0x49, 0x9f, 0x91, 0x6e, 0xc9, 0x02, 0xcf, 0x40, 0xef, 0x2b, 0x80, 0xeb, 0x4b, 0x11, + 0xf7, 0x3c, 0xc6, 0xf7, 0x25, 0x09, 0x55, 0x24, 0xa4, 0x46, 0x1a, 0x36, 0x77, 0x63, 0x35, 0xce, + 0x4b, 0xa8, 0xd0, 0xcb, 0x7f, 0xea, 0x56, 0xeb, 0x0f, 0xb2, 0xcf, 0x2b, 0x90, 0xa7, 0xd5, 0x7f, + 0x74, 0xf2, 0xd3, 0x2c, 0x9d, 0x24, 0x26, 0x38, 0x4d, 0x4c, 0x70, 0x96, 0x98, 0xe0, 0x47, 0x62, + 0x82, 0x2f, 0xe7, 0x66, 0xe9, 0xf4, 0xdc, 0x2c, 0x9d, 0x9d, 0x9b, 0xa5, 0x83, 0x4a, 0xfa, 0x22, + 0x8e, 0x6a, 0xd9, 0x53, 0xb5, 0xf9, 0x2b, 0x00, 0x00, 0xff, 0xff, 0x62, 0xcc, 0xff, 0x9f, 0x7e, + 0x05, 0x00, 0x00, } diff --git a/pkg/kv/kvserver/closedts/ctpb/service.proto b/pkg/kv/kvserver/closedts/ctpb/service.proto index cf7df9498d79..3c5e77134833 100644 --- a/pkg/kv/kvserver/closedts/ctpb/service.proto +++ b/pkg/kv/kvserver/closedts/ctpb/service.proto @@ -13,6 +13,8 @@ package cockroach.kv.kvserver.ctupdate; option go_package = "ctpb"; import "kv/kvserver/closedts/ctpb/entry.proto"; +import "roachpb/data.proto"; +import "util/hlc/timestamp.proto"; import "gogoproto/gogo.proto"; import "google/api/annotations.proto"; @@ -20,3 +22,65 @@ import "google/api/annotations.proto"; service ClosedTimestamp { rpc Get(stream Reaction) returns (stream Entry) { } } + +message RangeUpdate { + uint64 range_id = 1 [(gogoproto.customname) = "RangeID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.RangeID"]; + int64 lai = 2 [(gogoproto.customname) = "LAI", (gogoproto.casttype) = "LAI"]; + roachpb.RangeClosedTimestampPolicy policy = 3; +} + +// ClosedTimestampUpdate contains information about (the advancement of) closed +// timestamps for ranges with leases on the sender node. Updates are of two +// types: snapshots and incrementals. Snapshots are stand-alone messages, +// explicitly containing state about a bunch of ranges. Incrementals are deltas +// since the previous message (the previous message can be a snapshot or another +// incremental); they contain info about which new ranges are included in the +// info provided, which ranges are removed, and how the closed timestamps for +// different categories of ranges advanced. Ranges communicated by a previous +// message and not touched by an incremental are "implicitly" referenced by the +// incremental. In order to properly handle incrementals, the recipient +// maintains a "stream's state": the group of ranges that can be implicitly +// referenced by the next message. +message ClosedTimestampUpdate { + // node_id identifies the sending node. + int32 node_id = 1 [(gogoproto.customname) = "NodeID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.NodeID"]; + // seq_num identifies this update across all updates produced by a node. The + // sequence is reset when the node restarts, so a recipient can only count on + // it increasing within a single PushUpdates stream. + // + // All messages have sequence numbers, including snapshots. A snapshot can be + // applied on top of any state (i.e. it can be applied after having skipped + // messages); its sequence number tells the recipient what incremental message + // it should expect afterwards. + int64 seq_num = 2 [(gogoproto.casttype) = "UpdateSequenceNumber"]; + // snapshot indicates whether this message is standalone, or whether it's just + // a delta since the messages with the previous seq_num. A snapshot + // re-initializes all of the recipient's state. The first message on a stream + // is always a snapshot. Afterwards, there could be others if the stream + // experience network problems and some incremental messages were dropped + // (although generally we expect that to result in a stream failing and a new + // one being established). + bool snapshot = 3; + repeated int32 removed = 4 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.RangeID"]; + repeated RangeUpdate addedOrUpdated = 5 [(gogoproto.nullable) = false]; + + message groupUpdate { + roachpb.RangeClosedTimestampPolicy policy = 1; + util.hlc.Timestamp closed_timestamp = 2 [(gogoproto.nullable) = false]; + } + // closed_timestamp represents the timestamps that are being closed for each + // group of ranges, with a group being represented by its policy. + // + // The recipient is supposed to forward the closed timestamps of the affected + // ranges to these values. Upon receiving one of these updates, the recipient + // should generally not assume that it hasn't been informed of a higher closed + // timestamp for any range in particular - races between this side-transport + // and the regular Raft transport are possible. + repeated groupUpdate closed_timestamp = 6 [(gogoproto.nullable) = false]; +} + +message Response {} + +service ClosedTimestampSideTransport { + rpc PushUpdates(stream ClosedTimestampUpdate) returns (stream Response) { } +} diff --git a/pkg/kv/kvserver/closedts/sidetransport/closed_timestamp_sender.go b/pkg/kv/kvserver/closedts/sidetransport/closed_timestamp_sender.go new file mode 100644 index 000000000000..3b3cd1857456 --- /dev/null +++ b/pkg/kv/kvserver/closedts/sidetransport/closed_timestamp_sender.go @@ -0,0 +1,563 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package sidetransport + +import ( + "context" + "fmt" + "sync" + "sync/atomic" + "time" + + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/rpc" + "github.com/cockroachdb/cockroach/pkg/rpc/nodedialer" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/util" + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/stop" + "github.com/cockroachdb/cockroach/pkg/util/syncutil" + "github.com/cockroachdb/cockroach/pkg/util/timeutil" +) + +// closingPeriod dictates how often the ClosedTimestampSender will close +// timestamps. +const closingPeriod = 200 * time.Millisecond + +// ClosedTimestampSender represents the sending-side of the closed timestamps +// "side-transport". Its role is to periodically advance the closed timestamps +// of all the ranges with leases on the current node and to communicate these +// closed timestamps to all other nodes that have replicas for any of these +// ranges. +// +// This side-transport is particularly important for range that are not seeing +// frequent writes; in the absence of Raft proposals, this is the only way for +// the closed timestamps to advance. +// +// The ClosedTimestampSender is notified when leases are acquired or lost by the +// current node. The sender periodically loops through all the ranges with local +// leases, tries to advance the closed timestamp of each range according to its +// policy, and then publishes a message with the update to all other nodes with +// non-leaseholder replicas. Every node receives the same messages; for +// efficiency the sender does not keep track of which follower node is +// interested in which ranges. On the receiver side the closed timestamp updates +// are processed lazily, so it doesn't particularly matter that each receiver is +// told about ranges that it doesn't care about. +type ClosedTimestampSender struct { + stopper *stop.Stopper + dialer *nodedialer.Dialer + st *cluster.Settings + clock *hlc.Clock + nodeID roachpb.NodeID + + trackedMu struct { + syncutil.Mutex + // tracked maintains the information that was communicated to connections in + // the last sent message (implicitly or explicitly). A range enters this + // structure as soon as it's included in a message, and exits it when it's + // removed through ClosedTimestampUpdate.Removed. + tracked map[roachpb.RangeID]trackedRange + lastTimestamps map[roachpb.RangeClosedTimestampPolicy]hlc.Timestamp + // lastSeqNum is the sequence number of the last message published. + lastSeqNum ctpb.UpdateSequenceNumber + } + + leaseholdersMu struct { + syncutil.Mutex + leaseholders map[roachpb.RangeID]leaseholder + } + + // buf contains recent messages published to connections. Adding a message to + // this buffer signals the connections to send it on their streams. + buf *updatesBuf + + // connections contains connections to all nodes with follower replicas. + // connections are added as nodes get replicas for ranges with local leases + // and removed when the respective node no longer has any replicas with + // local leases. + connections map[roachpb.NodeID]*connection +} + +// trackedRange contains the information that the side-transport last published +// about a particular range. +type trackedRange struct { + lai ctpb.LAI + policy roachpb.RangeClosedTimestampPolicy +} + +type leaseholder struct { + r *kvserver.Replica + storeID roachpb.StoreID + leaseSeq roachpb.LeaseSequence +} + +// NewSideTransportSender creates a ClosedTimestampSender. Run must be called on +// it afterwards to get it to start publishing closed timestamps. +func NewSideTransportSender( + stopper *stop.Stopper, dialer *nodedialer.Dialer, clock *hlc.Clock, st *cluster.Settings, +) *ClosedTimestampSender { + bufSize := 3 * time.Second.Nanoseconds() / closingPeriod.Nanoseconds() + if bufSize < 2 { + bufSize = 2 + } + s := &ClosedTimestampSender{ + stopper: stopper, + dialer: dialer, + buf: newUpdatesBuf(int(bufSize)), + clock: clock, + st: st, + } + s.trackedMu.tracked = make(map[roachpb.RangeID]trackedRange) + s.trackedMu.lastTimestamps = make(map[roachpb.RangeClosedTimestampPolicy]hlc.Timestamp) + s.leaseholdersMu.leaseholders = make(map[roachpb.RangeID]leaseholder) + s.connections = make(map[roachpb.NodeID]*connection) + return s +} + +// Run starts a goroutine that periodically closes new timestamps for all the +// ranges where the leaseholder is on this node. +// +// nodeID is the id of the local node. Used to avoid connecting to ourselves. +// This is not know at construction time. +func (s *ClosedTimestampSender) Run(ctx context.Context, nodeID roachpb.NodeID) { + s.nodeID = nodeID + + _ /* err */ = s.stopper.RunAsyncTask(ctx, "closedts side-transport publisher", + func(ctx context.Context) { + defer func() { + // Cleanup all connections. + s.trackedMu.Lock() + defer s.trackedMu.Unlock() + for _, r := range s.connections { + r.close() + } + }() + + var timer timeutil.Timer + defer timer.Stop() + for { + timer.Reset(closingPeriod) + select { + case <-timer.C: + timer.Read = true + if !s.st.Version.IsActive(ctx, clusterversion.ClosedTimestampsRaftTransport) { + continue + } + s.publish(ctx) + case <-s.stopper.ShouldQuiesce(): + return + } + } + }) +} + +// RegisterLeaseholder adds a range to the leaseholders collection. From now on, +// the side-transport will try to advance this range's closed timestamp. +func (s *ClosedTimestampSender) RegisterLeaseholder( + ctx context.Context, r *kvserver.Replica, storeID roachpb.StoreID, leaseSeq roachpb.LeaseSequence, +) { + s.leaseholdersMu.Lock() + defer s.leaseholdersMu.Unlock() + + if lh, ok := s.leaseholdersMu.leaseholders[r.RangeID]; ok { + // The leaseholder is already registered. If we're already aware of a newer lease, + // there's nothing to do. + if lh.leaseSeq >= leaseSeq { + return + } + } + s.leaseholdersMu.leaseholders[r.RangeID] = leaseholder{ + r: r, + storeID: storeID, + leaseSeq: leaseSeq, + } +} + +// UnregisterLeaseholder removes a range from the leaseholders collection. +func (s *ClosedTimestampSender) UnregisterLeaseholder( + ctx context.Context, rangeID roachpb.RangeID, storeID roachpb.StoreID, +) { + s.leaseholdersMu.Lock() + defer s.leaseholdersMu.Unlock() + + if lh, ok := s.leaseholdersMu.leaseholders[rangeID]; ok && lh.storeID == storeID { + delete(s.leaseholdersMu.leaseholders, rangeID) + } +} + +func (s *ClosedTimestampSender) publish(ctx context.Context) { + s.trackedMu.Lock() + defer s.trackedMu.Unlock() + + // We'll accumulate all the nodes we need to connect to in order to check if + // we need to open new connections or close existing ones. + nodesWithFollowers := util.MakeFastIntSet() + + // Fix the closed timestamps that will be communicated to by this message. + // These timestamps (one per range policy) will apply to all the ranges + // included in message. + var targetByPolicy [roachpb.MAX_CLOSED_TIMESTAMP_POLICY]hlc.Timestamp + now := s.clock.NowAsClockTimestamp() + lagTargetDuration := closedts.TargetDuration.Get(&s.st.SV) + targetByPolicy[roachpb.LAG_BY_CLUSTER_SETTING] = + kvserver.ClosedTimestampTargetByPolicy(now, roachpb.LAG_BY_CLUSTER_SETTING, lagTargetDuration) + targetByPolicy[roachpb.LEAD_FOR_GLOBAL_READS] = + kvserver.ClosedTimestampTargetByPolicy(now, roachpb.LEAD_FOR_GLOBAL_READS, lagTargetDuration) + + s.trackedMu.lastSeqNum++ + msg := &ctpb.ClosedTimestampUpdate{ + NodeID: s.nodeID, + SeqNum: s.trackedMu.lastSeqNum, + // The first message produced is essentially a snapshot, since it has no + // previous state to reference. + Snapshot: s.trackedMu.lastSeqNum == 1, + } + + // Make a copy of the leaseholders map, in order to release its mutex quickly. We can't hold + // this mutex while calling into any replicas (and locking the replica) because replicas call + // into the ClosedTimestampSender and take leaseholdersMu through Register/UnregisterLeaseholder. + s.leaseholdersMu.Lock() + leaseholders := make(map[roachpb.RangeID]leaseholder, len(s.leaseholdersMu.leaseholders)) + for k, v := range s.leaseholdersMu.leaseholders { + leaseholders[k] = v + } + s.leaseholdersMu.Unlock() + + for _, lh := range leaseholders { + r := lh.r + // Make sure that we're communicating with all of the range's followers. + // Note that we're including this range's followers before deciding below if + // this message will include this range. This is because we don't want + // dynamic conditions about the activity of this range to dictate the + // opening and closing of connections to the other nodes. + for _, repl := range r.Desc().Replicas().VoterFullAndNonVoterDescriptors() { + nodesWithFollowers.Add(int(repl.NodeID)) + } + + // Check whether the desired timestamp can be closed on this range. + canClose, lai, policy := r.BumpSideTransportClosed(ctx, now, targetByPolicy) + lastMsg, tracked := s.trackedMu.tracked[r.RangeID] + if !canClose { + // We can't close the desired timestamp. If this range was tracked, we + // need to un-track it. + if tracked { + msg.Removed = append(msg.Removed, r.RangeID) + delete(s.trackedMu.tracked, r.RangeID) + } + continue + } + + // Check whether the range needs to be explicitly updated through the + // current message, or if its update can be implicit. + needExplicit := false + if !tracked { + // If the range was not included in the last message, we need to include + // it now to start "tracking" it in the side-transport. + needExplicit = true + } else if lastMsg.lai < lai { + // If the range's LAI has changed, we need to explicitly publish the new + // LAI. + needExplicit = true + } else if lastMsg.policy != policy { + // If the policy changed, we need to explicitly publish that; the + // receiver will updates its bookkeeping to indicate that this range is + // updated through implicit updates for the new policy. + needExplicit = true + } + if needExplicit { + msg.AddedOrUpdated = append(msg.AddedOrUpdated, ctpb.RangeUpdate{RangeID: r.RangeID, LAI: lai, Policy: policy}) + s.trackedMu.tracked[r.RangeID] = trackedRange{lai: lai, policy: policy} + } + + } + + // If there's any tracked ranges for which we're not the leaseholder any more, + // we need to untrack them and tell the connections about it. + for rid := range s.trackedMu.tracked { + if _, ok := leaseholders[rid]; !ok { + msg.Removed = append(msg.Removed, rid) + delete(s.trackedMu.tracked, rid) + } + } + + // Close connections to the nodes that no longer need any info from us + // (because they don't have replicas for any of the ranges with leases on this + // node). + for nodeID, conn := range s.connections { + if !nodesWithFollowers.Contains(int(nodeID)) { + delete(s.connections, nodeID) + conn.close() + } + } + + // Open connections to any node that needs info from us and is missing a conn. + nodesWithFollowers.ForEach(func(nid int) { + nodeID := roachpb.NodeID(nid) + // Note that we don't open a connection to ourselves. The timestamps that + // we'we closing are written directly to the sideTransportClosedTimestamp + // fields of the local replicas. + if _, ok := s.connections[nodeID]; !ok && nodeID != s.nodeID { + c := newConnection(s, nodeID, s.dialer, s.buf) + s.connections[nodeID] = c + c.run(ctx, s.stopper) + } + }) + + // Publish the new message to all connections. + s.buf.Push(ctx, msg) +} + +// GetSnapshot generates an update that contains all the sender's state (as +// opposed to being an incremental delta since a previous message). The returned +// msg will have the `snapshot` field set, and a sequence number indicating +// where to resume sending incremental updates. +func (s *ClosedTimestampSender) GetSnapshot() *ctpb.ClosedTimestampUpdate { + s.trackedMu.Lock() + defer s.trackedMu.Unlock() + + msg := &ctpb.ClosedTimestampUpdate{ + Snapshot: true, + // Assigning this SeqNum means that the next incremental sent needs to be + // lastSeqNum+1. Notice that GetSnapshot synchronizes with the publishing of + // of incremental messages. + SeqNum: s.trackedMu.lastSeqNum, + } + for rid, r := range s.trackedMu.tracked { + msg.AddedOrUpdated = append(msg.AddedOrUpdated, ctpb.RangeUpdate{ + RangeID: rid, + LAI: r.lai, + Policy: r.policy, + }) + } + for policy, ts := range s.trackedMu.lastTimestamps { + msg.ClosedTimestamp = append(msg.ClosedTimestamp, ctpb.ClosedTimestampUpdateGroupUpdate{ + Policy: policy, + ClosedTimestamp: ts, + }) + } + return msg +} + +// updatesBuf is a circular buffer of ClosedTimestampUpdates. It's created with +// a given capacity and, once it fills up, new items overwrite the oldest ones. +// It lets consumers query for the update with a particular sequence number and +// it lets queries block until the next update is produced. +type updatesBuf struct { + mu struct { + syncutil.Mutex + data []*ctpb.ClosedTimestampUpdate + // head points to the earliest update in the buffer. If the buffer is empty, + // head is 0 and the respective slot is nil. + // + // tail points to the next slot to be written to. When the buffer is full, + // tail == head meaning that the head will be overwritten by the next + // insertion. + head, tail int + // updated is signaled when a new item is inserted. + updated *sync.Cond + } +} + +func newUpdatesBuf(size int) *updatesBuf { + buf := &updatesBuf{} + buf.mu.data = make([]*ctpb.ClosedTimestampUpdate, size) + buf.mu.updated = sync.NewCond(&buf.mu) + return buf +} + +func (b *updatesBuf) Push(ctx context.Context, update *ctpb.ClosedTimestampUpdate) { + b.mu.Lock() + defer b.mu.Unlock() + + // If the buffer is not empty, sanity check the seq num. + if b.sizeLocked() != 0 { + lastIdx := (b.mu.tail - 1) + if lastIdx < 0 { + lastIdx += len(b.mu.data) + } + if prevSeq := b.mu.data[lastIdx].SeqNum; prevSeq != update.SeqNum-1 { + log.Fatalf(ctx, "bad sequence number; expected %d, got %d", prevSeq+1, update.SeqNum) + } + } + + b.mu.data[b.mu.tail] = update + overwrite := b.fullLocked() + b.mu.tail = (b.mu.tail + 1) % len(b.mu.data) + // If the tail just overwrote the head, move the head. + if overwrite { + b.mu.head = (b.mu.head + 1) % len(b.mu.data) + } + + // Notify everybody who might have been waiting for this message - we expect + // all the connections to be blocked waiting. + b.mu.updated.Broadcast() +} + +// GetBySeq looks through the buffer and returns the update with the requested +// sequence number. It's OK to request a seqNum one higher than the highest +// produced; the call will block until the message is produced. +// +// If the requested message is too old and is no longer in the buffer, returns nil. +func (b *updatesBuf) GetBySeq( + ctx context.Context, seqNum ctpb.UpdateSequenceNumber, +) *ctpb.ClosedTimestampUpdate { + b.mu.Lock() + defer b.mu.Unlock() + + // Loop until the requested seqNum is added to the buffer. + for { + var firstSeq, lastSeq ctpb.UpdateSequenceNumber + if b.sizeLocked() == 0 { + firstSeq, lastSeq = 0, 0 + } else { + firstSeq, lastSeq = b.mu.data[b.mu.head].SeqNum, b.mu.data[b.mu.tail].SeqNum + } + if seqNum < firstSeq { + // Requesting a message that's not in the buffer any more. + return nil + } + // If the requested msg has not been produced yet, block. + if seqNum == lastSeq+1 { + b.mu.updated.Wait() + continue + } + if seqNum > lastSeq+1 { + log.Fatalf(ctx, "skipping sequence numbers; requested: %d, last: %d", seqNum, lastSeq) + } + idx := (b.mu.head + (int)(seqNum-firstSeq)) % len(b.mu.data) + return b.mu.data[idx] + } +} + +func (b *updatesBuf) sizeLocked() int { + if b.mu.head < b.mu.tail { + return b.mu.tail - b.mu.head + } else if b.mu.head == b.mu.tail { + // The buffer is either empty or full. + if b.mu.head == 0 && b.mu.data[0] == nil { + return 0 + } + return len(b.mu.data) + } else { + return len(b.mu.data) + b.mu.tail - b.mu.head + } +} + +func (b *updatesBuf) fullLocked() bool { + return b.sizeLocked() == len(b.mu.data) +} + +// connection represents a connection to one particular node. The connection +// watches an updatesBuf and streams all the messages to the respective node. +type connection struct { + log.AmbientContext + producer *ClosedTimestampSender + nodeID roachpb.NodeID + dialer *nodedialer.Dialer + // buf accumulates messages to be sent to the connection. If the buffer + // overflows (because this stream is disconnected for long enough), we'll have + // to send a snapshot before we can resume sending regular messages. + buf *updatesBuf + stream ctpb.ClosedTimestampSideTransport_PushUpdatesClient + closed int32 // atomic +} + +func newConnection( + p *ClosedTimestampSender, nodeID roachpb.NodeID, dialer *nodedialer.Dialer, buf *updatesBuf, +) *connection { + r := &connection{ + producer: p, + nodeID: nodeID, + dialer: dialer, + buf: buf, + } + r.AddLogTag("ctstream", nodeID) + return r +} + +// close makes the connection stop sending messages. The run() goroutine will +// exit asynchronously. The parent ClosedTimestampSender is expected to remove +// this connection from its list. +func (r *connection) close() { + atomic.StoreInt32(&r.closed, 1) +} + +func (r *connection) sendMsg(ctx context.Context, msg *ctpb.ClosedTimestampUpdate) error { + if r.stream == nil { + conn, err := r.dialer.Dial(ctx, r.nodeID, rpc.SystemClass) + if err != nil { + return err + } + r.stream, err = ctpb.NewClosedTimestampSideTransportClient(conn).PushUpdates(ctx) + if err != nil { + return err + } + } + return r.stream.Send(msg) +} + +func (r *connection) run(ctx context.Context, stopper *stop.Stopper) { + _ /* err */ = stopper.RunAsyncTask(ctx, fmt.Sprintf("closedts publisher for n%d", r.nodeID), + func(ctx context.Context) { + ctx, cancel := stopper.WithCancelOnQuiesce(r.AnnotateCtx(ctx)) + defer cancel() + everyN := log.Every(10 * time.Second) + + var lastSent ctpb.UpdateSequenceNumber + for { + var msg *ctpb.ClosedTimestampUpdate + + if r.stream == nil { + // The stream is not established; the first message needs to be a + // snapshot. + msg = r.producer.GetSnapshot() + } else { + msg = r.buf.GetBySeq(ctx, lastSent+1) + if msg == nil { + // The sequence number we've requested is no longer in the buffer. + // We need to generate a snapshot in order to re-initialize the + // stream. + msg = r.producer.GetSnapshot() + } + } + + // See if we've been signaled to stop. + closed := atomic.LoadInt32(&r.closed) > 0 + if closed || ctx.Err() != nil { + if r.stream != nil { + _ /* err */ = r.stream.CloseSend() + } + return + } + + lastSent = msg.SeqNum + if err := r.sendMsg(ctx, msg); err != nil { + if everyN.ShouldLog() { + log.Warningf(ctx, "failed to send closed timestamp message %d to n%d: %s", + lastSent, r.nodeID, err) + } + // Keep track of the fact that we need a new connection. + // + // TODO(andrei): Instead of simply trying to establish a connection + // again when the next message needs to be sent and get rejected by + // the circuit breaker if the remote node is still unreachable, we + // should have a blocking version of Dial() that we just leave hanging + // and get a notification when it succeeds. + r.stream = nil + } + } + }) +} diff --git a/pkg/kv/kvserver/replica.go b/pkg/kv/kvserver/replica.go index d1b483773ac3..e8d5833dceb9 100644 --- a/pkg/kv/kvserver/replica.go +++ b/pkg/kv/kvserver/replica.go @@ -402,6 +402,18 @@ type Replica struct { // The minimum allowed ID for this replica. Initialized from // RangeTombstone.NextReplicaID. tombstoneMinReplicaID roachpb.ReplicaID + // sideTransportClosedTimestamp stores the closed timestamp that was + // communicated by the side transport. The replica can use it if it has + // applied all the commands with indexes <= sideTransportCloseTimestampLAI. + // Note that there's also state.RaftClosedTimestamp, which might be higher + // than this closed timestamp. The maximum across the two can be used. + // + // TODO(andrei): actually implement and reference also the global storage + // for side-transport closed timestamps. + sideTransportClosedTimestamp hlc.Timestamp + // sideTransportCloseTimestampLAI is the LAI associated with + // sideTransportClosedTimestamp. + sideTransportCloseTimestampLAI uint64 // The ID of the leader replica within the Raft group. Used to determine // when the leadership changes. diff --git a/pkg/kv/kvserver/replica_closedts.go b/pkg/kv/kvserver/replica_closedts.go index 3d29397537b3..ba8767376335 100644 --- a/pkg/kv/kvserver/replica_closedts.go +++ b/pkg/kv/kvserver/replica_closedts.go @@ -62,6 +62,72 @@ func (r *Replica) EmitMLAI() { } } +// BumpSideTransportClosed advances the range's closed timestamp if it can. If +// the closed timestamp is advanced, the function synchronizes with incoming +// requests, making sure that future requests are not allowed to write below the +// new closed timestamp. +// +// Returns false is the desired timestamp could not be closed. This can happen if the +// lease is no longer valid, if the range has proposals in-flight, if there are +// requests evaluating above the desired closed timestamp, or if the range has already +// closed a higher timestamp. +// +// If the closed timestamp was advanced, the function returns a LAI to be +// attached to the newly closed timestamp. +// +// This is called by the closed timestamp side-transport. The desired closed timestamp +// is passed as a map from range policy to timestamp; this function looks up the entry +// for this range. +func (r *Replica) BumpSideTransportClosed( + ctx context.Context, + now hlc.ClockTimestamp, + targetByPolicy [roachpb.MAX_CLOSED_TIMESTAMP_POLICY]hlc.Timestamp, +) (bool, ctpb.LAI, roachpb.RangeClosedTimestampPolicy) { + r.mu.Lock() + defer r.mu.Unlock() + + policy := r.closedTimestampPolicyRLocked() + target := targetByPolicy[policy] + // We can't close timestamps outside our lease. + st := r.leaseStatusForRequestRLocked(ctx, now, target) + if !st.IsValid() && !st.OwnedBy(r.StoreID()) { + return false, 0, 0 + } + + // If the range has been subsumed we can only close up to the subsumption + // time. + if freezeStart := r.getFreezeStartRLocked(); !freezeStart.IsEmpty() { + if freezeStart.Less(target) { + return false, 0, 0 + } + } + + // If there are pending proposals in-flight, the side-transport doesn't + // advance the closed timestamp. The side-transport can't publish a closed + // timestamp with an LAI that takes the in-flight LAIs into consideration, + // because the in-flight proposals might not actually end up applying. In + // order to publish a closed timestamp with an LAI that doesn't consider these + // in-flight proposals we'd have to check that they're all trying to write + // above `target`; that's too expensive. + // + // Note that the proposals in the proposalBuf don't matter here; these + // proposals and their timestamps are still tracked in proposal buffer's + // tracker, and they'll be considered below. + if len(r.mu.proposals) > 0 { + return false, 0, 0 + } + + if !r.mu.proposalBuf.MaybeForwardClosedLocked(ctx, target) { + return false, 0, 0 + } + lai := r.mu.state.LeaseAppliedIndex + // Update the replica directly since there's no side-transport connection to + // the local node. + r.mu.sideTransportClosedTimestamp = target + r.mu.sideTransportCloseTimestampLAI = lai + return true, ctpb.LAI(lai), policy +} + // closedTimestampTargetRLocked computes the timestamp we'd like to close for // this range. Note that we might not be able to ultimately close this timestamp // if there are requests in flight. @@ -69,12 +135,12 @@ func (r *Replica) closedTimestampTargetRLocked() hlc.Timestamp { now := r.Clock().NowAsClockTimestamp() policy := r.closedTimestampPolicyRLocked() lagTargetDuration := closedts.TargetDuration.Get(&r.ClusterSettings().SV) - return closedTimestampTargetByPolicy(now, policy, lagTargetDuration) + return ClosedTimestampTargetByPolicy(now, policy, lagTargetDuration) } -// closedTimestampTargetByPolicy returns the target closed timestamp for a range +// ClosedTimestampTargetByPolicy returns the target closed timestamp for a range // with the given policy. -func closedTimestampTargetByPolicy( +func ClosedTimestampTargetByPolicy( now hlc.ClockTimestamp, policy roachpb.RangeClosedTimestampPolicy, lagTargetDuration time.Duration, diff --git a/pkg/kv/kvserver/replica_closedts_test.go b/pkg/kv/kvserver/replica_closedts_test.go index 7ff1d17a1e44..e8232bf88a4d 100644 --- a/pkg/kv/kvserver/replica_closedts_test.go +++ b/pkg/kv/kvserver/replica_closedts_test.go @@ -1,3 +1,13 @@ +// Copyright 2021 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + package kvserver import ( @@ -37,7 +47,7 @@ func TestClosedTimestampTargetByPolicy(t *testing.T) { }, } { t.Run(tc.rangePolicy.String(), func(t *testing.T) { - require.Equal(t, tc.expClosedTSTarget, closedTimestampTargetByPolicy(clock.NowAsClockTimestamp(), tc.rangePolicy, lagTargetNanos)) + require.Equal(t, tc.expClosedTSTarget, ClosedTimestampTargetByPolicy(clock.NowAsClockTimestamp(), tc.rangePolicy, lagTargetNanos)) }) } } diff --git a/pkg/kv/kvserver/replica_proposal.go b/pkg/kv/kvserver/replica_proposal.go index edfee4173b8c..0e8e3f1aa033 100644 --- a/pkg/kv/kvserver/replica_proposal.go +++ b/pkg/kv/kvserver/replica_proposal.go @@ -526,6 +526,13 @@ func (r *Replica) leasePostApplyLocked( } } + // Inform the store of this lease. + if iAmTheLeaseHolder { + r.store.registerLeaseholder(ctx, r, newLease.Sequence) + } else { + r.store.unregisterLeaseholder(ctx, r.RangeID) + } + // Mark the new lease in the replica's lease history. if r.leaseHistory != nil { r.leaseHistory.add(*newLease) diff --git a/pkg/kv/kvserver/replica_proposal_buf.go b/pkg/kv/kvserver/replica_proposal_buf.go index 37558e97222b..b34eb9af8c8a 100644 --- a/pkg/kv/kvserver/replica_proposal_buf.go +++ b/pkg/kv/kvserver/replica_proposal_buf.go @@ -153,7 +153,8 @@ type propBuf struct { // assignedClosedTimestamp is the largest "closed timestamp" - i.e. the largest // timestamp that was communicated to other replicas as closed, representing a // promise that this leaseholder will not evaluate writes below this timestamp - // any more. + // any more. It is set when proposals are flushed from the buffer, and also + // by the side-transport which closes timestamps out of band. // // Note that this field is not used by the local replica (or by anybody) // directly to decide whether follower reads can be served. See @@ -821,8 +822,8 @@ func (b *propBuf) OnLeaseChangeLocked(leaseOwned bool, closedTS hlc.Timestamp) { } // forwardClosedTimestamp forwards the closed timestamp tracked by the propBuf. -func (b *propBuf) forwardClosedTimestampLocked(closedTS hlc.Timestamp) { - b.assignedClosedTimestamp.Forward(closedTS) +func (b *propBuf) forwardClosedTimestampLocked(closedTS hlc.Timestamp) bool { + return b.assignedClosedTimestamp.Forward(closedTS) } // EvaluatingRequestsCount returns the count of requests currently tracked by @@ -911,6 +912,23 @@ func (b *propBuf) TrackEvaluatingRequest( return minTS, TrackedRequestToken{tok: tok, b: b} } +// MaybeForwardClosedLocked checks whether the closed timestamp can be advanced +// to target. +// +// Returns false in the following cases: +// 1) target is below the propBuf's closed timestamp. This ensures that the +// side-transport (the caller) is prevented from publishing closed timestamp +// regressions. In other words, for a given LAI, the side-transport only +// publishes closed timestamps higher than what Raft published. +// 2) There are requests evaluating at timestamps below target (as tracked by +// the evalTracker). We can't close timestamps below these requests. +func (b *propBuf) MaybeForwardClosedLocked(ctx context.Context, target hlc.Timestamp) bool { + if lb := b.evalTracker.LowerBound(ctx); lb.Less(target) { + return false + } + return b.forwardClosedTimestampLocked(target) +} + const propBufArrayMinSize = 4 const propBufArrayMaxSize = 256 const propBufArrayShrinkDelay = 16 diff --git a/pkg/kv/kvserver/replica_proposal_buf_test.go b/pkg/kv/kvserver/replica_proposal_buf_test.go index e6e092124115..30fa14890361 100644 --- a/pkg/kv/kvserver/replica_proposal_buf_test.go +++ b/pkg/kv/kvserver/replica_proposal_buf_test.go @@ -125,7 +125,7 @@ func (t *testProposer) closedTimestampTarget() hlc.Timestamp { if t.clock == nil { return hlc.Timestamp{} } - return closedTimestampTargetByPolicy(t.clock.NowAsClockTimestamp(), t.rangePolicy, time.Second) + return ClosedTimestampTargetByPolicy(t.clock.NowAsClockTimestamp(), t.rangePolicy, time.Second) } func (t *testProposer) raftTransportClosedTimestampEnabled() bool { diff --git a/pkg/kv/kvserver/store.go b/pkg/kv/kvserver/store.go index fee2ca5a6a89..09491d5f3506 100644 --- a/pkg/kv/kvserver/store.go +++ b/pkg/kv/kvserver/store.go @@ -206,16 +206,17 @@ func TestStoreConfig(clock *hlc.Clock) StoreConfig { } st := cluster.MakeTestingClusterSettings() sc := StoreConfig{ - DefaultZoneConfig: zonepb.DefaultZoneConfigRef(), - DefaultSystemZoneConfig: zonepb.DefaultSystemZoneConfigRef(), - Settings: st, - AmbientCtx: log.AmbientContext{Tracer: st.Tracer}, - Clock: clock, - CoalescedHeartbeatsInterval: 50 * time.Millisecond, - ScanInterval: 10 * time.Minute, - HistogramWindowInterval: metric.TestSampleInterval, - ClosedTimestamp: container.NoopContainer(), - ProtectedTimestampCache: protectedts.EmptyCache(clock), + DefaultZoneConfig: zonepb.DefaultZoneConfigRef(), + DefaultSystemZoneConfig: zonepb.DefaultSystemZoneConfigRef(), + Settings: st, + AmbientCtx: log.AmbientContext{Tracer: st.Tracer}, + Clock: clock, + CoalescedHeartbeatsInterval: 50 * time.Millisecond, + ScanInterval: 10 * time.Minute, + HistogramWindowInterval: metric.TestSampleInterval, + ClosedTimestamp: container.NoopContainer(), + ClosedTimestampSideTransport: dummyClosedTimestampSideTransport{}, + ProtectedTimestampCache: protectedts.EmptyCache(clock), } // Use shorter Raft tick settings in order to minimize start up and failover @@ -393,34 +394,35 @@ func (rs *storeReplicaVisitor) EstimatedCount() int { // A Store maintains a map of ranges by start key. A Store corresponds // to one physical device. type Store struct { - Ident *roachpb.StoreIdent // pointer to catch access before Start() is called - cfg StoreConfig - db *kv.DB - engine storage.Engine // The underlying key-value store - tsCache tscache.Cache // Most recent timestamps for keys / key ranges - allocator Allocator // Makes allocation decisions - replRankings *replicaRankings - storeRebalancer *StoreRebalancer - rangeIDAlloc *idalloc.Allocator // Range ID allocator - gcQueue *gcQueue // Garbage collection queue - mergeQueue *mergeQueue // Range merging queue - splitQueue *splitQueue // Range splitting queue - replicateQueue *replicateQueue // Replication queue - replicaGCQueue *replicaGCQueue // Replica GC queue - raftLogQueue *raftLogQueue // Raft log truncation queue - raftSnapshotQueue *raftSnapshotQueue // Raft repair queue - tsMaintenanceQueue *timeSeriesMaintenanceQueue // Time series maintenance queue - scanner *replicaScanner // Replica scanner - consistencyQueue *consistencyQueue // Replica consistency check queue - consistencyLimiter *quotapool.RateLimiter // Rate limits consistency checks - metrics *StoreMetrics - intentResolver *intentresolver.IntentResolver - recoveryMgr txnrecovery.Manager - raftEntryCache *raftentry.Cache - limiters batcheval.Limiters - txnWaitMetrics *txnwait.Metrics - sstSnapshotStorage SSTSnapshotStorage - protectedtsCache protectedts.Cache + Ident *roachpb.StoreIdent // pointer to catch access before Start() is called + cfg StoreConfig + db *kv.DB + engine storage.Engine // The underlying key-value store + tsCache tscache.Cache // Most recent timestamps for keys / key ranges + allocator Allocator // Makes allocation decisions + replRankings *replicaRankings + storeRebalancer *StoreRebalancer + rangeIDAlloc *idalloc.Allocator // Range ID allocator + gcQueue *gcQueue // Garbage collection queue + mergeQueue *mergeQueue // Range merging queue + splitQueue *splitQueue // Range splitting queue + replicateQueue *replicateQueue // Replication queue + replicaGCQueue *replicaGCQueue // Replica GC queue + raftLogQueue *raftLogQueue // Raft log truncation queue + raftSnapshotQueue *raftSnapshotQueue // Raft repair queue + tsMaintenanceQueue *timeSeriesMaintenanceQueue // Time series maintenance queue + scanner *replicaScanner // Replica scanner + consistencyQueue *consistencyQueue // Replica consistency check queue + consistencyLimiter *quotapool.RateLimiter // Rate limits consistency checks + metrics *StoreMetrics + intentResolver *intentresolver.IntentResolver + recoveryMgr txnrecovery.Manager + raftEntryCache *raftentry.Cache + limiters batcheval.Limiters + txnWaitMetrics *txnwait.Metrics + sstSnapshotStorage SSTSnapshotStorage + protectedtsCache protectedts.Cache + closedTimestampSideTransport closedTimestampSideTransport // gossipRangeCountdown and leaseRangeCountdown are countdowns of // changes to range and leaseholder counts, after which the store @@ -648,7 +650,8 @@ type StoreConfig struct { RPCContext *rpc.Context RangeDescriptorCache *rangecache.RangeCache - ClosedTimestamp *container.Container + ClosedTimestamp *container.Container + ClosedTimestampSideTransport closedTimestampSideTransport // SQLExecutor is used by the store to execute SQL statements. SQLExecutor sqlutil.InternalExecutor @@ -796,11 +799,12 @@ func NewStore( log.Fatalf(ctx, "invalid store configuration: %+v", &cfg) } s := &Store{ - cfg: cfg, - db: cfg.DB, // TODO(tschottdorf): remove redundancy. - engine: eng, - nodeDesc: nodeDesc, - metrics: newStoreMetrics(cfg.HistogramWindowInterval), + cfg: cfg, + db: cfg.DB, // TODO(tschottdorf): remove redundancy. + engine: eng, + nodeDesc: nodeDesc, + metrics: newStoreMetrics(cfg.HistogramWindowInterval), + closedTimestampSideTransport: cfg.ClosedTimestampSideTransport, } if cfg.RPCContext != nil { s.allocator = MakeAllocator(cfg.StorePool, cfg.RPCContext.RemoteClocks.Latency) @@ -2762,6 +2766,16 @@ func (s *Store) PurgeOutdatedReplicas(ctx context.Context, version roachpb.Versi return g.Wait() } +func (s *Store) registerLeaseholder( + ctx context.Context, r *Replica, leaseSeq roachpb.LeaseSequence, +) { + s.closedTimestampSideTransport.RegisterLeaseholder(ctx, r, s.StoreID(), leaseSeq) +} + +func (s *Store) unregisterLeaseholder(ctx context.Context, rangeID roachpb.RangeID) { + s.closedTimestampSideTransport.UnregisterLeaseholder(ctx, rangeID, s.StoreID()) +} + // WriteClusterVersion writes the given cluster version to the store-local // cluster version key. We only accept a raw engine to ensure we're persisting // the write durably. @@ -2792,3 +2806,24 @@ func min(a, b int) int { } return b } + +// closedTimestampSideTransport is implemented by SideTransportSender. The +// interface serves to avoid a circular dependency. +type closedTimestampSideTransport interface { + RegisterLeaseholder(context.Context, *Replica, roachpb.StoreID, roachpb.LeaseSequence) + UnregisterLeaseholder(context.Context, roachpb.RangeID, roachpb.StoreID) +} + +type dummyClosedTimestampSideTransport struct{} + +func (d dummyClosedTimestampSideTransport) RegisterLeaseholder( + context.Context, *Replica, roachpb.StoreID, roachpb.LeaseSequence, +) { +} + +func (d dummyClosedTimestampSideTransport) UnregisterLeaseholder( + context.Context, roachpb.RangeID, roachpb.StoreID, +) { +} + +var _ closedTimestampSideTransport = dummyClosedTimestampSideTransport{} diff --git a/pkg/kv/kvserver/store_remove_replica.go b/pkg/kv/kvserver/store_remove_replica.go index 653378af2f4d..92fdbc9c79df 100644 --- a/pkg/kv/kvserver/store_remove_replica.go +++ b/pkg/kv/kvserver/store_remove_replica.go @@ -255,6 +255,7 @@ func (s *Store) unlinkReplicaByRangeIDLocked(ctx context.Context, rangeID roachp delete(s.mu.uninitReplicas, rangeID) s.replicaQueues.Delete(int64(rangeID)) s.mu.replicas.Delete(int64(rangeID)) + s.closedTimestampSideTransport.UnregisterLeaseholder(ctx, rangeID, s.StoreID()) } // removePlaceholder removes a placeholder for the specified range if it diff --git a/pkg/roachpb/data.pb.go b/pkg/roachpb/data.pb.go index 2309cba13ff6..d44298a41cbe 100644 --- a/pkg/roachpb/data.pb.go +++ b/pkg/roachpb/data.pb.go @@ -99,7 +99,7 @@ func (x ValueType) String() string { return proto.EnumName(ValueType_name, int32(x)) } func (ValueType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_data_73b4def784a0a137, []int{0} + return fileDescriptor_data_55a3230e612c48bc, []int{0} } // ReplicaChangeType is a parameter of ChangeReplicasTrigger. @@ -129,7 +129,7 @@ func (x ReplicaChangeType) String() string { return proto.EnumName(ReplicaChangeType_name, int32(x)) } func (ReplicaChangeType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_data_73b4def784a0a137, []int{1} + return fileDescriptor_data_55a3230e612c48bc, []int{1} } // TransactionStatus specifies possible states for a transaction. @@ -181,7 +181,7 @@ func (x TransactionStatus) String() string { return proto.EnumName(TransactionStatus_name, int32(x)) } func (TransactionStatus) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_data_73b4def784a0a137, []int{2} + return fileDescriptor_data_55a3230e612c48bc, []int{2} } // RangeClosedTimestampPolicy represents the policy used by the leaseholder of a @@ -202,22 +202,26 @@ const ( // (which itself is a function of leaseholder -> follower network latency and // closed timestamp update periodicity). LEAD_FOR_GLOBAL_READS RangeClosedTimestampPolicy = 1 + // Keep this sentinel value higher than the rest. + MAX_CLOSED_TIMESTAMP_POLICY RangeClosedTimestampPolicy = 2 ) var RangeClosedTimestampPolicy_name = map[int32]string{ 0: "LAG_BY_CLUSTER_SETTING", 1: "LEAD_FOR_GLOBAL_READS", + 2: "MAX_CLOSED_TIMESTAMP_POLICY", } var RangeClosedTimestampPolicy_value = map[string]int32{ - "LAG_BY_CLUSTER_SETTING": 0, - "LEAD_FOR_GLOBAL_READS": 1, + "LAG_BY_CLUSTER_SETTING": 0, + "LEAD_FOR_GLOBAL_READS": 1, + "MAX_CLOSED_TIMESTAMP_POLICY": 2, } func (x RangeClosedTimestampPolicy) String() string { return proto.EnumName(RangeClosedTimestampPolicy_name, int32(x)) } func (RangeClosedTimestampPolicy) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_data_73b4def784a0a137, []int{3} + return fileDescriptor_data_55a3230e612c48bc, []int{3} } // Span is a key range with an inclusive start Key and an exclusive end Key. @@ -234,7 +238,7 @@ type Span struct { func (m *Span) Reset() { *m = Span{} } func (*Span) ProtoMessage() {} func (*Span) Descriptor() ([]byte, []int) { - return fileDescriptor_data_73b4def784a0a137, []int{0} + return fileDescriptor_data_55a3230e612c48bc, []int{0} } func (m *Span) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -286,7 +290,7 @@ 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_73b4def784a0a137, []int{1} + return fileDescriptor_data_55a3230e612c48bc, []int{1} } func (m *Value) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -322,7 +326,7 @@ 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_73b4def784a0a137, []int{2} + return fileDescriptor_data_55a3230e612c48bc, []int{2} } func (m *KeyValue) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -360,7 +364,7 @@ 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_73b4def784a0a137, []int{3} + return fileDescriptor_data_55a3230e612c48bc, []int{3} } func (m *StoreIdent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -400,7 +404,7 @@ 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_73b4def784a0a137, []int{4} + return fileDescriptor_data_55a3230e612c48bc, []int{4} } func (m *SplitTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -491,7 +495,7 @@ 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_73b4def784a0a137, []int{5} + return fileDescriptor_data_55a3230e612c48bc, []int{5} } func (m *MergeTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -561,7 +565,7 @@ type ChangeReplicasTrigger struct { func (m *ChangeReplicasTrigger) Reset() { *m = ChangeReplicasTrigger{} } func (*ChangeReplicasTrigger) ProtoMessage() {} func (*ChangeReplicasTrigger) Descriptor() ([]byte, []int) { - return fileDescriptor_data_73b4def784a0a137, []int{6} + return fileDescriptor_data_55a3230e612c48bc, []int{6} } func (m *ChangeReplicasTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -603,7 +607,7 @@ 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_73b4def784a0a137, []int{7} + return fileDescriptor_data_55a3230e612c48bc, []int{7} } func (m *ModifiedSpanTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -649,7 +653,7 @@ func (m *StickyBitTrigger) Reset() { *m = StickyBitTrigger{} } func (m *StickyBitTrigger) String() string { return proto.CompactTextString(m) } func (*StickyBitTrigger) ProtoMessage() {} func (*StickyBitTrigger) Descriptor() ([]byte, []int) { - return fileDescriptor_data_73b4def784a0a137, []int{8} + return fileDescriptor_data_55a3230e612c48bc, []int{8} } func (m *StickyBitTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -688,7 +692,7 @@ 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_73b4def784a0a137, []int{9} + return fileDescriptor_data_55a3230e612c48bc, []int{9} } func (m *InternalCommitTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -757,7 +761,7 @@ 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_73b4def784a0a137, []int{10} + return fileDescriptor_data_55a3230e612c48bc, []int{10} } func (m *ObservedTimestamp) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -932,7 +936,7 @@ type Transaction struct { func (m *Transaction) Reset() { *m = Transaction{} } func (*Transaction) ProtoMessage() {} func (*Transaction) Descriptor() ([]byte, []int) { - return fileDescriptor_data_73b4def784a0a137, []int{11} + return fileDescriptor_data_55a3230e612c48bc, []int{11} } func (m *Transaction) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -983,7 +987,7 @@ func (m *TransactionRecord) Reset() { *m = TransactionRecord{} } func (m *TransactionRecord) String() string { return proto.CompactTextString(m) } func (*TransactionRecord) ProtoMessage() {} func (*TransactionRecord) Descriptor() ([]byte, []int) { - return fileDescriptor_data_73b4def784a0a137, []int{12} + return fileDescriptor_data_55a3230e612c48bc, []int{12} } func (m *TransactionRecord) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1023,7 +1027,7 @@ 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_73b4def784a0a137, []int{13} + return fileDescriptor_data_55a3230e612c48bc, []int{13} } func (m *Intent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1060,7 +1064,7 @@ func (m *Intent_SingleKeySpan) Reset() { *m = Intent_SingleKeySpan{} } func (m *Intent_SingleKeySpan) String() string { return proto.CompactTextString(m) } func (*Intent_SingleKeySpan) ProtoMessage() {} func (*Intent_SingleKeySpan) Descriptor() ([]byte, []int) { - return fileDescriptor_data_73b4def784a0a137, []int{13, 0} + return fileDescriptor_data_55a3230e612c48bc, []int{13, 0} } func (m *Intent_SingleKeySpan) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1097,7 +1101,7 @@ func (m *LockAcquisition) Reset() { *m = LockAcquisition{} } func (m *LockAcquisition) String() string { return proto.CompactTextString(m) } func (*LockAcquisition) ProtoMessage() {} func (*LockAcquisition) Descriptor() ([]byte, []int) { - return fileDescriptor_data_73b4def784a0a137, []int{14} + return fileDescriptor_data_55a3230e612c48bc, []int{14} } func (m *LockAcquisition) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1138,7 +1142,7 @@ func (m *LockUpdate) Reset() { *m = LockUpdate{} } func (m *LockUpdate) String() string { return proto.CompactTextString(m) } func (*LockUpdate) ProtoMessage() {} func (*LockUpdate) Descriptor() ([]byte, []int) { - return fileDescriptor_data_73b4def784a0a137, []int{15} + return fileDescriptor_data_55a3230e612c48bc, []int{15} } func (m *LockUpdate) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1175,7 +1179,7 @@ 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_73b4def784a0a137, []int{16} + return fileDescriptor_data_55a3230e612c48bc, []int{16} } func (m *SequencedWrite) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1240,7 +1244,7 @@ type Lease struct { func (m *Lease) Reset() { *m = Lease{} } func (*Lease) ProtoMessage() {} func (*Lease) Descriptor() ([]byte, []int) { - return fileDescriptor_data_73b4def784a0a137, []int{17} + return fileDescriptor_data_55a3230e612c48bc, []int{17} } func (m *Lease) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1285,7 +1289,7 @@ 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_73b4def784a0a137, []int{18} + return fileDescriptor_data_55a3230e612c48bc, []int{18} } func (m *AbortSpanEntry) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1342,7 +1346,7 @@ func (m *LeafTxnInputState) Reset() { *m = LeafTxnInputState{} } func (m *LeafTxnInputState) String() string { return proto.CompactTextString(m) } func (*LeafTxnInputState) ProtoMessage() {} func (*LeafTxnInputState) Descriptor() ([]byte, []int) { - return fileDescriptor_data_73b4def784a0a137, []int{19} + return fileDescriptor_data_55a3230e612c48bc, []int{19} } func (m *LeafTxnInputState) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1395,7 +1399,7 @@ func (m *LeafTxnFinalState) Reset() { *m = LeafTxnFinalState{} } func (m *LeafTxnFinalState) String() string { return proto.CompactTextString(m) } func (*LeafTxnFinalState) ProtoMessage() {} func (*LeafTxnFinalState) Descriptor() ([]byte, []int) { - return fileDescriptor_data_73b4def784a0a137, []int{20} + return fileDescriptor_data_55a3230e612c48bc, []int{20} } func (m *LeafTxnFinalState) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1434,7 +1438,7 @@ func (m *ClientRangeInfo) Reset() { *m = ClientRangeInfo{} } func (m *ClientRangeInfo) String() string { return proto.CompactTextString(m) } func (*ClientRangeInfo) ProtoMessage() {} func (*ClientRangeInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_data_73b4def784a0a137, []int{21} + return fileDescriptor_data_55a3230e612c48bc, []int{21} } func (m *ClientRangeInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1471,7 +1475,7 @@ type RangeInfo struct { func (m *RangeInfo) Reset() { *m = RangeInfo{} } func (*RangeInfo) ProtoMessage() {} func (*RangeInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_data_73b4def784a0a137, []int{22} + return fileDescriptor_data_55a3230e612c48bc, []int{22} } func (m *RangeInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7329,174 +7333,176 @@ var ( ErrIntOverflowData = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/data.proto", fileDescriptor_data_73b4def784a0a137) } - -var fileDescriptor_data_73b4def784a0a137 = []byte{ - // 2654 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xdc, 0x59, 0xcb, 0x73, 0x23, 0x57, - 0xd5, 0x77, 0x5b, 0x2d, 0xa9, 0x75, 0x2c, 0xc9, 0xed, 0x3b, 0xb6, 0x47, 0xe3, 0x7c, 0x9f, 0x35, - 0x28, 0x40, 0x86, 0x29, 0x22, 0x15, 0x4e, 0x48, 0xa5, 0x86, 0xa1, 0x0a, 0xbd, 0xc6, 0x91, 0x46, - 0x96, 0x26, 0x2d, 0x79, 0x82, 0x93, 0x50, 0x4d, 0xab, 0xfb, 0x5a, 0x6e, 0xdc, 0xea, 0xd6, 0x74, - 0xb7, 0x1c, 0x8b, 0x1d, 0x1b, 0x2a, 0x95, 0x05, 0xb0, 0x64, 0x99, 0x2a, 0x76, 0xb0, 0x80, 0x25, - 0xc5, 0x82, 0x75, 0x16, 0x2c, 0xb2, 0xa1, 0x12, 0xa0, 0x4a, 0x05, 0xce, 0x26, 0x0b, 0xfe, 0x02, - 0x17, 0x54, 0x51, 0xf7, 0xde, 0x7e, 0xd9, 0x23, 0x1b, 0x39, 0x1e, 0x42, 0x8a, 0x8d, 0xdd, 0x7d, - 0x1e, 0xbf, 0x73, 0x1f, 0xe7, 0xd5, 0x47, 0x80, 0x6c, 0x4b, 0x51, 0x0f, 0x46, 0xfd, 0x92, 0xa6, - 0xb8, 0x4a, 0x71, 0x64, 0x5b, 0xae, 0x85, 0x56, 0x54, 0x4b, 0x3d, 0xa4, 0xf4, 0xa2, 0xc7, 0xdd, - 0xb8, 0x7b, 0x78, 0x54, 0x3a, 0x3c, 0x72, 0xb0, 0x7d, 0x84, 0xed, 0x92, 0x6a, 0x99, 0xea, 0xd8, - 0xb6, 0xb1, 0xa9, 0x4e, 0x4a, 0x86, 0xa5, 0x1e, 0xd2, 0x3f, 0xba, 0x39, 0x60, 0xea, 0x67, 0x65, - 0x6d, 0xac, 0x68, 0xce, 0x78, 0x38, 0x54, 0xec, 0x49, 0xc9, 0x76, 0x46, 0xfd, 0x92, 0xf7, 0xe2, - 0xc9, 0xae, 0xfb, 0xe6, 0x87, 0xd8, 0x55, 0xc2, 0x25, 0x6c, 0x3c, 0xe7, 0xb8, 0x96, 0xad, 0x0c, - 0x70, 0x09, 0x9b, 0x03, 0xdd, 0xc4, 0x44, 0xe0, 0x48, 0x55, 0x3d, 0xe6, 0xff, 0xcd, 0x64, 0xbe, - 0xe4, 0x71, 0x73, 0x63, 0x57, 0x37, 0x4a, 0x07, 0x86, 0x5a, 0x72, 0xf5, 0x21, 0x76, 0x5c, 0x65, - 0x38, 0xf2, 0x38, 0xab, 0x03, 0x6b, 0x60, 0xd1, 0xc7, 0x12, 0x79, 0x62, 0xd4, 0xc2, 0x1e, 0xf0, - 0xdd, 0x91, 0x62, 0xa2, 0x5b, 0x10, 0x3b, 0xc4, 0x93, 0x5c, 0xec, 0x36, 0x77, 0x27, 0x5d, 0x49, - 0x9e, 0x4e, 0xf3, 0xb1, 0x87, 0x78, 0x22, 0x11, 0x1a, 0xba, 0x0d, 0x49, 0x6c, 0x6a, 0x32, 0x61, - 0xf3, 0x67, 0xd9, 0x09, 0x6c, 0x6a, 0x0f, 0xf1, 0xe4, 0x9e, 0xf0, 0xf3, 0xf7, 0xf3, 0x0b, 0xbf, - 0x7d, 0x3f, 0xcf, 0x35, 0x79, 0x81, 0x13, 0x17, 0x9b, 0xbc, 0xb0, 0x28, 0xc6, 0x0a, 0x03, 0x88, - 0x3f, 0x56, 0x8c, 0x31, 0x46, 0xcf, 0x41, 0xca, 0x56, 0xde, 0x91, 0xfb, 0x13, 0x17, 0x3b, 0x39, - 0x8e, 0x40, 0x48, 0x82, 0xad, 0xbc, 0x53, 0x21, 0xef, 0xa8, 0x0c, 0xa9, 0x60, 0xa5, 0xb9, 0xc5, - 0xdb, 0xdc, 0x9d, 0xa5, 0xad, 0xff, 0x2f, 0x86, 0x57, 0x40, 0xb6, 0x53, 0x3c, 0x30, 0xd4, 0x62, - 0xcf, 0x17, 0xaa, 0xf0, 0x1f, 0x4c, 0xf3, 0x0b, 0x52, 0xa8, 0x55, 0x78, 0x0b, 0x84, 0x87, 0x78, - 0xc2, 0x6c, 0x79, 0xfb, 0xe0, 0x66, 0xec, 0xe3, 0x65, 0x88, 0x1f, 0x11, 0x19, 0xcf, 0x4a, 0xae, - 0xf8, 0xd4, 0x45, 0x17, 0x29, 0x86, 0x67, 0x80, 0x09, 0x17, 0x3e, 0xe2, 0x00, 0xba, 0xae, 0x65, - 0xe3, 0x86, 0x86, 0x4d, 0x17, 0x0d, 0x00, 0x54, 0x63, 0xec, 0xb8, 0xd8, 0x96, 0x75, 0xcd, 0x33, - 0xf3, 0x1a, 0x91, 0xff, 0xf3, 0x34, 0xff, 0xd2, 0x40, 0x77, 0x0f, 0xc6, 0xfd, 0xa2, 0x6a, 0x0d, - 0x4b, 0x01, 0xb6, 0xd6, 0x0f, 0x9f, 0x4b, 0xa3, 0xc3, 0x41, 0x89, 0x5e, 0xd0, 0x78, 0xac, 0x6b, - 0xc5, 0xdd, 0xdd, 0x46, 0xed, 0x64, 0x9a, 0x4f, 0x55, 0x19, 0x60, 0xa3, 0x26, 0xa5, 0x3c, 0xec, - 0x86, 0x86, 0x5e, 0x84, 0xa4, 0x69, 0x69, 0x98, 0x58, 0x21, 0xeb, 0x8d, 0x57, 0x56, 0x4f, 0xa6, - 0xf9, 0x44, 0xdb, 0xd2, 0x70, 0xa3, 0x76, 0x1a, 0x3c, 0x49, 0x09, 0x22, 0xd4, 0xd0, 0xd0, 0x37, - 0x40, 0x20, 0x7e, 0x41, 0xe5, 0x63, 0x54, 0x7e, 0xfd, 0x64, 0x9a, 0x4f, 0xb2, 0x95, 0x13, 0x05, - 0xff, 0x51, 0x4a, 0x3a, 0x6c, 0x37, 0x85, 0x5f, 0x72, 0x90, 0xee, 0x8e, 0x0c, 0xdd, 0xed, 0xd9, - 0xfa, 0x60, 0x80, 0x6d, 0x54, 0x87, 0x94, 0x81, 0xf7, 0x5d, 0x59, 0xc3, 0x8e, 0x4a, 0xb7, 0xb6, - 0xb4, 0x55, 0x98, 0x71, 0x48, 0x92, 0x62, 0x0e, 0x70, 0x0d, 0x3b, 0xaa, 0xad, 0x8f, 0x5c, 0xcb, - 0xf6, 0x8e, 0x4b, 0x20, 0xaa, 0x84, 0x8a, 0xb6, 0x01, 0x6c, 0x7d, 0x70, 0xe0, 0xe1, 0x2c, 0x5e, - 0x11, 0x27, 0x45, 0x75, 0x09, 0xf9, 0x1e, 0xff, 0x29, 0x73, 0xa9, 0x98, 0xc8, 0x17, 0xfe, 0xc8, - 0x43, 0x7a, 0x07, 0xdb, 0x03, 0xfc, 0x05, 0x5d, 0x2c, 0x1a, 0x80, 0xc8, 0x80, 0x48, 0x34, 0xca, - 0x8e, 0xab, 0xb8, 0x0e, 0x0d, 0x97, 0xa5, 0xad, 0xaf, 0x44, 0xe0, 0xbc, 0xd8, 0x2d, 0xfa, 0xb1, - 0x5b, 0xdc, 0x79, 0x5c, 0xad, 0x76, 0x89, 0x70, 0x65, 0x9d, 0x20, 0x9e, 0x4c, 0xf3, 0x59, 0x89, - 0xc0, 0x04, 0x74, 0x29, 0x4b, 0x61, 0x77, 0x8e, 0x54, 0x95, 0xbe, 0xa3, 0x1f, 0x73, 0x90, 0xde, - 0xb7, 0x31, 0xfe, 0x21, 0x26, 0x56, 0x6c, 0x37, 0x17, 0x9f, 0x27, 0x68, 0x6a, 0x04, 0xfd, 0x74, - 0x9a, 0xbf, 0x3f, 0xbf, 0x8f, 0x12, 0x80, 0x2a, 0x49, 0x6e, 0x01, 0x8a, 0xb4, 0xc4, 0x0c, 0x77, - 0x89, 0x5d, 0xb4, 0x07, 0xeb, 0x6c, 0xc7, 0xaa, 0x61, 0x39, 0x58, 0x93, 0xc3, 0x30, 0x4e, 0xcc, - 0x1f, 0xc6, 0xab, 0x14, 0xa2, 0x4a, 0x11, 0x02, 0x1e, 0x7a, 0x1b, 0x10, 0x83, 0x26, 0x09, 0x54, - 0xf6, 0x92, 0x66, 0x2e, 0x49, 0x61, 0x8b, 0x11, 0xd8, 0xc3, 0xa3, 0xa2, 0x9f, 0x6b, 0x8b, 0x91, - 0x5c, 0x5b, 0x94, 0xb0, 0xa2, 0x75, 0xd9, 0xb3, 0xc4, 0xae, 0x25, 0x42, 0x39, 0xe3, 0x57, 0xbf, - 0x8a, 0xc3, 0x5a, 0xf5, 0x80, 0x5c, 0xae, 0x84, 0x47, 0x86, 0xae, 0x2a, 0x8e, 0xef, 0x60, 0x6f, - 0xc2, 0xba, 0x86, 0x47, 0x36, 0x56, 0x15, 0x17, 0x6b, 0xb2, 0x4a, 0x65, 0x64, 0x77, 0x32, 0xc2, - 0xd4, 0xdb, 0xb2, 0x5b, 0x5f, 0x9e, 0xe5, 0x25, 0x0c, 0x83, 0x01, 0xf6, 0x26, 0x23, 0x2c, 0xad, - 0x86, 0x18, 0x21, 0x15, 0xed, 0x01, 0x8a, 0x60, 0xdb, 0x4c, 0xcb, 0xf3, 0xbe, 0x4b, 0x70, 0x9f, - 0xf2, 0xbf, 0x95, 0x10, 0xc5, 0x13, 0x41, 0x3f, 0x80, 0xe7, 0x22, 0xd0, 0xe3, 0x91, 0x16, 0x35, - 0xe1, 0xe4, 0x62, 0xb7, 0x63, 0x57, 0xb4, 0x71, 0x2b, 0x84, 0xdb, 0x65, 0x68, 0xfe, 0x49, 0x21, - 0x0c, 0x1b, 0x11, 0x5b, 0x26, 0x3e, 0x76, 0x7d, 0x43, 0x24, 0x0d, 0xf1, 0x34, 0x0d, 0xdd, 0x39, - 0x99, 0xe6, 0x6f, 0xd6, 0x02, 0xa9, 0x36, 0x3e, 0x76, 0x3d, 0x7d, 0x9a, 0x96, 0x52, 0xc1, 0x8b, - 0x74, 0x53, 0x9b, 0x29, 0xa5, 0xa1, 0x57, 0x80, 0xa7, 0xd1, 0x19, 0x9f, 0x37, 0x3a, 0x25, 0x2a, - 0x8f, 0xfa, 0x70, 0x53, 0x37, 0x5d, 0x6c, 0x9b, 0x8a, 0x21, 0x2b, 0x9a, 0x16, 0x3d, 0x86, 0xc4, - 0x95, 0x8f, 0x61, 0xcd, 0x87, 0x2a, 0x13, 0xa4, 0xe0, 0x08, 0xf6, 0xe1, 0x56, 0x60, 0xc3, 0xc6, - 0x43, 0xeb, 0x28, 0x6a, 0x25, 0x79, 0x65, 0x2b, 0xc1, 0x82, 0x25, 0x86, 0xe5, 0xdb, 0xb9, 0xc7, - 0x93, 0x12, 0x5b, 0x78, 0x8f, 0x83, 0x1b, 0x3b, 0x96, 0xa6, 0xef, 0xeb, 0x58, 0x23, 0x65, 0xdb, - 0xf7, 0xd5, 0xaf, 0x03, 0x72, 0x26, 0x8e, 0x8b, 0x87, 0xb2, 0x6a, 0x99, 0xfb, 0xfa, 0x40, 0x76, - 0x46, 0x8a, 0x49, 0xfd, 0x54, 0x90, 0x44, 0xc6, 0xa9, 0x52, 0x06, 0xad, 0xf5, 0x75, 0x40, 0xb4, - 0xb4, 0x18, 0xfa, 0x11, 0x36, 0xb1, 0xe3, 0x30, 0x69, 0xe6, 0x7d, 0x37, 0x67, 0x2c, 0x96, 0x28, - 0x49, 0x22, 0x51, 0x69, 0x79, 0x1a, 0x84, 0x52, 0x78, 0x0c, 0x62, 0xd7, 0xd5, 0xd5, 0xc3, 0x49, - 0x25, 0x2c, 0x21, 0x15, 0x00, 0x87, 0xd2, 0xe4, 0xbe, 0xee, 0x7a, 0x69, 0x79, 0xbe, 0x72, 0xee, - 0xf8, 0x50, 0x85, 0xdf, 0xc5, 0x60, 0xad, 0xe1, 0x1d, 0x43, 0xd5, 0x1a, 0x0e, 0x43, 0xf4, 0x1a, - 0x64, 0x1c, 0x52, 0xb0, 0x64, 0x97, 0x11, 0x3c, 0x03, 0xf9, 0x99, 0x6b, 0x0e, 0x0b, 0x9b, 0x94, - 0x76, 0xa2, 0x65, 0xae, 0x06, 0x99, 0x21, 0xa9, 0x24, 0x01, 0xca, 0xe2, 0x85, 0x28, 0xd1, 0x8a, - 0x23, 0xa5, 0x87, 0xd1, 0xfa, 0xf3, 0x7d, 0xb8, 0xe9, 0xe5, 0x04, 0xff, 0xba, 0x03, 0xbc, 0x18, - 0xc5, 0xbb, 0x33, 0x03, 0x6f, 0x66, 0xa6, 0x91, 0xd6, 0xd4, 0x0b, 0x12, 0xd0, 0xda, 0xd0, 0xbb, - 0x6b, 0x7a, 0x43, 0x01, 0x3e, 0x2b, 0x2b, 0x5f, 0x9d, 0xb5, 0xde, 0xa7, 0x7d, 0x43, 0xba, 0x31, - 0x9c, 0xe1, 0x30, 0xaf, 0x03, 0x0a, 0xef, 0x29, 0x00, 0x66, 0x01, 0xf6, 0xfc, 0xac, 0xe3, 0x3c, - 0x77, 0xd1, 0x92, 0xe8, 0x9c, 0xa3, 0xdc, 0xe3, 0xdf, 0x7d, 0x3f, 0xcf, 0x15, 0xfe, 0xc0, 0xc1, - 0x4a, 0xa7, 0x4f, 0x53, 0x72, 0x24, 0x9f, 0x47, 0x9a, 0x19, 0x6e, 0x8e, 0x66, 0xe6, 0x47, 0xdc, - 0x95, 0x9b, 0xc2, 0x67, 0x53, 0xdf, 0x42, 0xab, 0xf7, 0x78, 0xd2, 0xcf, 0x16, 0x4e, 0x13, 0xb0, - 0xd4, 0xb3, 0x15, 0xd3, 0x51, 0x54, 0x57, 0xb7, 0x4c, 0x54, 0x06, 0x9e, 0xf4, 0xea, 0x9e, 0xe3, - 0x3d, 0x7f, 0x59, 0x65, 0xef, 0x1d, 0x9b, 0x3b, 0xd8, 0x55, 0x2a, 0x02, 0x59, 0xd9, 0x87, 0xd3, - 0x3c, 0x27, 0x51, 0x55, 0x84, 0x80, 0x37, 0x95, 0x21, 0xeb, 0x42, 0x53, 0x12, 0x7d, 0x46, 0xf7, - 0x21, 0x41, 0x3a, 0x86, 0x31, 0x6b, 0x19, 0x66, 0xd7, 0x96, 0xc8, 0x32, 0xba, 0x54, 0x56, 0xf2, - 0x74, 0x50, 0x13, 0xb2, 0x86, 0xe2, 0xb8, 0xf2, 0x01, 0x56, 0x6c, 0xb7, 0x8f, 0x95, 0x39, 0x5b, - 0x02, 0x16, 0x78, 0x19, 0xa2, 0xfa, 0x9a, 0xaf, 0x89, 0xbe, 0x07, 0xb9, 0x81, 0x61, 0xf5, 0x15, - 0x43, 0x1e, 0x9b, 0x2a, 0xb6, 0x5d, 0x45, 0x37, 0xdd, 0x89, 0x6c, 0xe8, 0x43, 0xdd, 0xf5, 0xea, - 0xef, 0x5c, 0xa8, 0xeb, 0x0c, 0x64, 0x37, 0xc4, 0x68, 0x11, 0x08, 0xf4, 0x16, 0xdc, 0xb0, 0x3c, - 0xef, 0x08, 0xfb, 0x05, 0x27, 0x27, 0x5c, 0x98, 0x28, 0x9f, 0xf2, 0x25, 0xcf, 0x00, 0xb2, 0xce, - 0x33, 0x1c, 0x74, 0x1f, 0x80, 0x5c, 0x27, 0x0d, 0x16, 0x27, 0xb7, 0x44, 0x31, 0x2f, 0xca, 0x67, - 0x7e, 0xda, 0x21, 0x0a, 0xe4, 0xdd, 0x41, 0x05, 0xc8, 0xbc, 0x63, 0xeb, 0x2e, 0x96, 0x5d, 0xcb, - 0x92, 0x2d, 0x43, 0xcb, 0xa5, 0x69, 0xfa, 0x5c, 0xa2, 0xc4, 0x9e, 0x65, 0x75, 0x0c, 0x8d, 0x9c, - 0x34, 0xed, 0x48, 0x42, 0xe7, 0x5c, 0xbe, 0xc2, 0x49, 0x13, 0xd5, 0x30, 0x26, 0x5e, 0x86, 0x75, - 0x95, 0x66, 0xb7, 0x10, 0x4d, 0xde, 0xd7, 0x8f, 0xb1, 0x96, 0x13, 0xa9, 0xe1, 0x55, 0xc6, 0x0d, - 0x14, 0x1e, 0x10, 0x1e, 0x7a, 0x1d, 0x44, 0xdd, 0x94, 0xf7, 0x0d, 0xda, 0x1d, 0xd1, 0xa5, 0x39, - 0xb9, 0x15, 0xba, 0xd3, 0x2f, 0xcd, 0xda, 0x29, 0x7e, 0x32, 0xc6, 0xa6, 0x8a, 0xb5, 0x37, 0x88, - 0xa4, 0xb7, 0x8e, 0xac, 0x6e, 0x3e, 0xa0, 0xfa, 0x94, 0xe8, 0x20, 0x0b, 0x96, 0xf5, 0x81, 0x69, - 0xd9, 0x24, 0xcd, 0xe0, 0x27, 0xe6, 0x78, 0xe8, 0xe4, 0x10, 0x45, 0x2c, 0x5e, 0xe6, 0xde, 0x0d, - 0xa6, 0xd2, 0xc5, 0x4f, 0xda, 0xe3, 0x21, 0xad, 0xbf, 0x61, 0x07, 0x7b, 0x86, 0xe7, 0x48, 0x59, - 0x3d, 0x78, 0x27, 0xe8, 0x67, 0x3e, 0x17, 0x63, 0x22, 0xdf, 0xe4, 0x85, 0x84, 0x98, 0x6c, 0xf2, - 0x42, 0x4a, 0x84, 0x26, 0x2f, 0x64, 0xc4, 0x6c, 0x93, 0x17, 0xb2, 0xe2, 0x72, 0xe1, 0x37, 0x3c, - 0xac, 0x44, 0xbc, 0x5e, 0xc2, 0xaa, 0x65, 0x6b, 0xcf, 0x22, 0x04, 0xbf, 0x38, 0xe1, 0x76, 0x3d, - 0x97, 0xfd, 0x5f, 0x70, 0x06, 0xde, 0x73, 0x84, 0x45, 0x31, 0x76, 0xce, 0x1d, 0x92, 0xa2, 0xd0, - 0xe4, 0x05, 0x41, 0x4c, 0x05, 0xae, 0x01, 0xe2, 0x52, 0x93, 0x17, 0xd2, 0x62, 0x26, 0xea, 0x26, - 0x4d, 0x5e, 0x58, 0x16, 0xc5, 0x26, 0x2f, 0x88, 0xe2, 0x4a, 0x61, 0xca, 0x41, 0x82, 0xf4, 0x0e, - 0x26, 0xf9, 0x3c, 0x59, 0x76, 0x74, 0x73, 0x60, 0x60, 0xf9, 0x10, 0x4f, 0xc2, 0x86, 0x68, 0x69, - 0xeb, 0x85, 0x19, 0x67, 0xc3, 0x74, 0x8a, 0x5d, 0xaa, 0xf0, 0x10, 0x4f, 0xe8, 0x79, 0x87, 0x6e, - 0x93, 0x71, 0xa2, 0x0c, 0xf4, 0x2d, 0x88, 0xb9, 0xc7, 0x7e, 0xc7, 0x34, 0x97, 0x07, 0xb2, 0xc3, - 0x26, 0x5a, 0x1b, 0xf7, 0x21, 0x73, 0xc6, 0xcc, 0x25, 0xa3, 0x97, 0xe8, 0x38, 0xa5, 0xc9, 0x0b, - 0xbc, 0x18, 0x2f, 0xfc, 0x89, 0x83, 0xe5, 0x96, 0xa5, 0x1e, 0x96, 0xd5, 0x27, 0x63, 0xdd, 0xd1, - 0x69, 0x51, 0xfa, 0x26, 0xf0, 0x91, 0xed, 0x5d, 0xe8, 0x3e, 0x91, 0x28, 0x70, 0xae, 0xbb, 0x0b, - 0x24, 0x01, 0x68, 0x63, 0x5b, 0xe9, 0xeb, 0x86, 0xee, 0xb2, 0xb5, 0x67, 0xb7, 0xb6, 0x2e, 0xf8, - 0x32, 0x8b, 0x4c, 0xcc, 0x8a, 0xc4, 0x79, 0x8b, 0xb5, 0x40, 0x53, 0x8a, 0xa0, 0x14, 0x7e, 0xbd, - 0x08, 0x40, 0xf6, 0xc6, 0x3e, 0x33, 0xfe, 0x2b, 0xdb, 0x0a, 0x33, 0x43, 0xec, 0x33, 0x64, 0x86, - 0x19, 0xc1, 0xc3, 0xff, 0x27, 0x83, 0xa7, 0xf0, 0x53, 0x0e, 0xb2, 0x67, 0xc3, 0xfa, 0xb2, 0x01, - 0xd8, 0xdb, 0x20, 0x38, 0x9e, 0xb0, 0x37, 0x53, 0xfa, 0xce, 0xdc, 0x1d, 0xd3, 0xf9, 0xa1, 0x23, - 0x39, 0xbc, 0x2e, 0x7e, 0x22, 0x05, 0x88, 0x5e, 0xc3, 0xf4, 0x7b, 0x1e, 0xe2, 0x2d, 0xac, 0x38, - 0x18, 0x4d, 0x20, 0xce, 0xe6, 0x13, 0xdc, 0xe7, 0xd7, 0xbf, 0x31, 0x8b, 0xe8, 0xdb, 0x00, 0xf8, - 0x78, 0xa4, 0xdb, 0x0a, 0xb9, 0xa3, 0xb9, 0xfa, 0x47, 0x29, 0xa2, 0x80, 0x6a, 0x90, 0xf4, 0x3f, - 0xc9, 0x63, 0x57, 0xfe, 0x24, 0xf7, 0x55, 0xd1, 0x2e, 0x44, 0x3e, 0x68, 0xd9, 0xa8, 0x86, 0xfc, - 0x75, 0x74, 0x7f, 0x2e, 0xf4, 0x6f, 0x56, 0xb4, 0x16, 0x6a, 0xd3, 0x79, 0x4b, 0x97, 0xea, 0xa2, - 0x9f, 0x70, 0xb0, 0x34, 0xb2, 0xad, 0x11, 0x1b, 0xb9, 0x38, 0xf3, 0xd5, 0x9e, 0xf6, 0xc9, 0x34, - 0x0f, 0x8f, 0x3c, 0xad, 0x5e, 0xf7, 0xda, 0xe7, 0x0c, 0xfe, 0x0a, 0x7a, 0x0e, 0x5a, 0x85, 0x38, - 0x1e, 0x59, 0xea, 0x01, 0x9d, 0xfa, 0xc4, 0x24, 0xf6, 0x82, 0x5e, 0x8c, 0xf8, 0x1a, 0xe9, 0x1b, - 0x63, 0x95, 0x95, 0xd3, 0x69, 0x3e, 0x43, 0x5d, 0xc3, 0xf7, 0xd8, 0x88, 0xf3, 0x04, 0x2d, 0x41, - 0xe1, 0x2f, 0x1c, 0x64, 0xcb, 0x7d, 0xcb, 0x76, 0x49, 0x70, 0xd7, 0x4d, 0xd7, 0x9e, 0x5c, 0xe6, - 0xd2, 0xd7, 0x9f, 0x1e, 0x23, 0x05, 0x84, 0x91, 0xad, 0x5b, 0xb6, 0x9f, 0xc7, 0xe2, 0x95, 0xfa, - 0xe9, 0x34, 0x5f, 0xfe, 0xcc, 0x51, 0xf1, 0xc8, 0x03, 0x93, 0x02, 0xd8, 0x7b, 0x02, 0xd9, 0xd9, - 0xa7, 0x64, 0x77, 0xff, 0x5c, 0x84, 0x95, 0x16, 0x56, 0xf6, 0x7b, 0xc7, 0x66, 0xc3, 0x1c, 0x8d, - 0xc9, 0xa5, 0xba, 0x18, 0xbd, 0xc2, 0x52, 0x16, 0x0b, 0x94, 0xcd, 0xcb, 0x53, 0x4e, 0x34, 0x5b, - 0xbd, 0x00, 0xcb, 0x36, 0xde, 0xb7, 0xb1, 0x73, 0x20, 0xeb, 0xe6, 0x91, 0x62, 0xe8, 0x1a, 0x3d, - 0x6b, 0x41, 0xca, 0x7a, 0xe4, 0x06, 0xa3, 0xce, 0x6c, 0x14, 0x84, 0xeb, 0x35, 0x0a, 0x5b, 0xb0, - 0xe6, 0xb8, 0x78, 0x34, 0xd2, 0xcd, 0x81, 0x3c, 0x24, 0xdf, 0x76, 0xd8, 0x54, 0xfa, 0x06, 0xd6, - 0x72, 0x29, 0xba, 0x82, 0x1b, 0x3e, 0x73, 0xc7, 0xd2, 0x70, 0x9d, 0xb1, 0x50, 0x1f, 0xd2, 0x6c, - 0xa0, 0x87, 0x9f, 0xc8, 0xe6, 0x78, 0x98, 0x83, 0x67, 0x94, 0x84, 0x80, 0xa0, 0xb2, 0x1c, 0x79, - 0xae, 0x93, 0xe0, 0xc5, 0x78, 0x93, 0x17, 0xe2, 0x62, 0x82, 0x75, 0x15, 0x85, 0xf7, 0xc2, 0xf3, - 0x7f, 0xa0, 0x9b, 0x8a, 0x71, 0xbd, 0xf3, 0x7f, 0x15, 0x72, 0xd1, 0x11, 0xa1, 0x35, 0x1c, 0x2a, - 0x26, 0xf9, 0x3f, 0x36, 0x5d, 0xe6, 0x4a, 0x52, 0x64, 0x84, 0x58, 0x65, 0xec, 0x2a, 0xe1, 0xa2, - 0x0a, 0x64, 0xfc, 0x9b, 0x63, 0xad, 0x1f, 0x3f, 0x4f, 0xeb, 0x97, 0xf6, 0x74, 0x58, 0xf7, 0x37, - 0xef, 0xed, 0x07, 0x47, 0x12, 0x1c, 0x03, 0x6b, 0xab, 0x0a, 0xff, 0xe0, 0x60, 0xb9, 0x6a, 0xe8, - 0xd8, 0x74, 0x69, 0xd5, 0x69, 0x98, 0xfb, 0x16, 0x7a, 0x0d, 0xd6, 0xb4, 0x20, 0xa5, 0xc9, 0x03, - 0x6c, 0x62, 0x2f, 0x8b, 0x72, 0x34, 0x88, 0x6f, 0x9c, 0x4e, 0xf3, 0xcb, 0x54, 0x7a, 0x3b, 0x60, - 0x49, 0xab, 0xa1, 0x46, 0x48, 0x45, 0xaf, 0x42, 0xd6, 0x20, 0xd1, 0x2e, 0x9f, 0xa9, 0x39, 0x33, - 0xf3, 0x40, 0xc6, 0x88, 0xbe, 0x22, 0x0c, 0x37, 0xcf, 0x8f, 0x94, 0xe5, 0x91, 0x65, 0xe8, 0xaa, - 0xdf, 0x68, 0xbc, 0x78, 0xd1, 0x08, 0xf0, 0xdc, 0x1c, 0xf9, 0x11, 0x55, 0x92, 0xd6, 0xd4, 0x59, - 0xe4, 0xc2, 0xdf, 0x39, 0x48, 0x85, 0x1b, 0xbf, 0xef, 0x0d, 0x19, 0xaf, 0xfa, 0x53, 0x02, 0x1b, - 0x35, 0xbe, 0x0c, 0x71, 0xba, 0x87, 0x4b, 0x7e, 0x5b, 0xa2, 0x5b, 0xf6, 0x7f, 0x5b, 0xa2, 0xc2, - 0x9f, 0xd3, 0x46, 0xd9, 0xec, 0xf0, 0xee, 0x47, 0x1c, 0xa4, 0xe8, 0xef, 0x5b, 0x74, 0x02, 0xbd, - 0x04, 0xc9, 0xdd, 0xf6, 0xc3, 0x76, 0xe7, 0x8d, 0xb6, 0xb8, 0x80, 0x92, 0x10, 0x6b, 0xb4, 0x7b, - 0x22, 0x87, 0x52, 0x10, 0x7f, 0xd0, 0xea, 0x94, 0x7b, 0xe2, 0x22, 0x79, 0xac, 0xec, 0xf5, 0xea, - 0x5d, 0x31, 0x86, 0x6e, 0xc0, 0x72, 0xad, 0xde, 0x6a, 0xec, 0x34, 0x7a, 0xf5, 0x9a, 0xcc, 0x88, - 0x02, 0x12, 0x80, 0xef, 0x35, 0x76, 0xea, 0x22, 0x4f, 0xa0, 0x6a, 0xf5, 0x6a, 0x63, 0xa7, 0xdc, - 0x12, 0xe3, 0x68, 0x0d, 0x56, 0x42, 0x59, 0x9f, 0x9c, 0x42, 0x69, 0x10, 0x6a, 0xbb, 0x52, 0xb9, - 0xd7, 0xe8, 0xb4, 0xc5, 0x04, 0x02, 0x48, 0x10, 0xdd, 0xde, 0x9b, 0x62, 0x9a, 0xd8, 0xde, 0xae, - 0x77, 0xc4, 0x0c, 0x35, 0xd8, 0xf9, 0xee, 0x56, 0x4d, 0xcc, 0x92, 0xc7, 0xde, 0xee, 0xa3, 0x56, - 0x5d, 0x04, 0xa2, 0x58, 0x69, 0xf4, 0xca, 0x92, 0x54, 0xde, 0x13, 0x97, 0x50, 0x16, 0x80, 0x28, - 0x76, 0xeb, 0x52, 0xa3, 0xde, 0x15, 0xb5, 0x02, 0xf9, 0x48, 0x48, 0xde, 0x55, 0x61, 0xe5, 0xa9, - 0xc1, 0x3b, 0xca, 0x40, 0xaa, 0x5c, 0xab, 0xc9, 0x8f, 0x3b, 0xbd, 0xba, 0x24, 0x2e, 0x20, 0x11, - 0xd2, 0x52, 0x7d, 0xa7, 0xf3, 0xb8, 0xee, 0x51, 0x38, 0xb4, 0x02, 0x19, 0x22, 0xd0, 0xee, 0xb4, - 0x3d, 0xd2, 0x22, 0x5a, 0x05, 0xd1, 0x13, 0x0a, 0xa9, 0xb1, 0x0d, 0xfe, 0xdd, 0x5f, 0x6c, 0x2e, - 0xdc, 0xed, 0x9c, 0xf9, 0x16, 0x65, 0x8d, 0x1f, 0xd9, 0xfa, 0xa3, 0x7a, 0xbb, 0xd6, 0x68, 0x6f, - 0x8b, 0x0b, 0xe4, 0xa5, 0xdb, 0x2b, 0x6f, 0x93, 0x97, 0x18, 0x31, 0x5f, 0xed, 0xec, 0xec, 0x34, - 0x7a, 0xbd, 0x7a, 0x4d, 0xe4, 0x08, 0xaf, 0x5c, 0xe9, 0x48, 0xe4, 0x65, 0xd1, 0x03, 0xdc, 0x83, - 0x8d, 0x8b, 0xaf, 0x12, 0x6d, 0xc0, 0x7a, 0xab, 0xbc, 0x2d, 0x57, 0xf6, 0xe4, 0x6a, 0x6b, 0xb7, - 0xdb, 0xab, 0x4b, 0x72, 0xb7, 0xde, 0xeb, 0x31, 0x43, 0xb7, 0x60, 0xad, 0x55, 0x2f, 0xd7, 0xe4, - 0x07, 0x1d, 0x49, 0xde, 0x6e, 0x75, 0x2a, 0xe5, 0x96, 0x2c, 0xd5, 0xcb, 0xb5, 0xae, 0xc8, 0x31, - 0xe8, 0xca, 0xd7, 0x3e, 0xf8, 0xdb, 0xe6, 0xc2, 0x07, 0x27, 0x9b, 0xdc, 0x87, 0x27, 0x9b, 0xdc, - 0xc7, 0x27, 0x9b, 0xdc, 0x5f, 0x4f, 0x36, 0xb9, 0x9f, 0x7d, 0xb2, 0xb9, 0xf0, 0xe1, 0x27, 0x9b, - 0x0b, 0x1f, 0x7f, 0xb2, 0xb9, 0xf0, 0x66, 0xd2, 0xf3, 0xa6, 0x7e, 0x82, 0xfe, 0x0c, 0xfc, 0xd2, - 0xbf, 0x02, 0x00, 0x00, 0xff, 0xff, 0xd5, 0x23, 0x4d, 0x2f, 0x0a, 0x1f, 0x00, 0x00, +func init() { proto.RegisterFile("roachpb/data.proto", fileDescriptor_data_55a3230e612c48bc) } + +var fileDescriptor_data_55a3230e612c48bc = []byte{ + // 2683 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xdc, 0x59, 0xcd, 0x6f, 0xe3, 0xd6, + 0xb5, 0x37, 0x2d, 0xca, 0xa2, 0x8e, 0x2d, 0x99, 0xbe, 0x63, 0x7b, 0x34, 0x9e, 0xf7, 0xac, 0x79, + 0xca, 0x7b, 0x2f, 0xd3, 0x41, 0x23, 0xa3, 0x4e, 0x1a, 0x04, 0xd3, 0x29, 0x50, 0x7d, 0x8d, 0x23, + 0x8d, 0x3e, 0x1c, 0x8a, 0x9e, 0xc4, 0x49, 0x0a, 0x96, 0x22, 0xaf, 0x65, 0xd6, 0x14, 0xa9, 0x21, + 0x29, 0xc7, 0xca, 0xae, 0x9b, 0x22, 0xc8, 0xa2, 0xed, 0xb2, 0xcb, 0x00, 0xdd, 0xb5, 0x8b, 0x76, + 0x59, 0x74, 0xd1, 0x75, 0x16, 0x5d, 0x64, 0x53, 0x24, 0x6d, 0x01, 0xa1, 0x75, 0x36, 0x59, 0xf4, + 0x2f, 0x30, 0x5a, 0xa0, 0xb8, 0xf7, 0xf2, 0xcb, 0x1e, 0xd9, 0x95, 0x33, 0x69, 0x1a, 0x74, 0x63, + 0x93, 0xe7, 0x9e, 0xf3, 0x3b, 0xf7, 0x9e, 0x7b, 0xbe, 0x78, 0x04, 0xc8, 0xb1, 0x55, 0xed, 0x70, + 0xd8, 0xdb, 0xd2, 0x55, 0x4f, 0x2d, 0x0e, 0x1d, 0xdb, 0xb3, 0xd1, 0x8a, 0x66, 0x6b, 0x47, 0x94, + 0x5e, 0xf4, 0x57, 0x37, 0xee, 0x1d, 0x1d, 0x6f, 0x1d, 0x1d, 0xbb, 0xd8, 0x39, 0xc6, 0xce, 0x96, + 0x66, 0x5b, 0xda, 0xc8, 0x71, 0xb0, 0xa5, 0x8d, 0xb7, 0x4c, 0x5b, 0x3b, 0xa2, 0x7f, 0x0c, 0xab, + 0xcf, 0xc4, 0xcf, 0xf3, 0x3a, 0x58, 0xd5, 0xdd, 0xd1, 0x60, 0xa0, 0x3a, 0xe3, 0x2d, 0xc7, 0x1d, + 0xf6, 0xb6, 0xfc, 0x17, 0x9f, 0x77, 0x3d, 0x50, 0x3f, 0xc0, 0x9e, 0x1a, 0x6d, 0x61, 0xe3, 0xb6, + 0xeb, 0xd9, 0x8e, 0xda, 0xc7, 0x5b, 0xd8, 0xea, 0x1b, 0x16, 0x26, 0x0c, 0xc7, 0x9a, 0xe6, 0x2f, + 0xfe, 0xd7, 0xd4, 0xc5, 0x17, 0xfd, 0xd5, 0xdc, 0xc8, 0x33, 0xcc, 0xad, 0x43, 0x53, 0xdb, 0xf2, + 0x8c, 0x01, 0x76, 0x3d, 0x75, 0x30, 0xf4, 0x57, 0x56, 0xfb, 0x76, 0xdf, 0xa6, 0x8f, 0x5b, 0xe4, + 0x89, 0x51, 0x0b, 0xfb, 0xc0, 0x77, 0x87, 0xaa, 0x85, 0x6e, 0x41, 0xe2, 0x08, 0x8f, 0x73, 0x89, + 0x3b, 0xdc, 0xdd, 0xa5, 0x72, 0xea, 0x6c, 0x92, 0x4f, 0x3c, 0xc2, 0x63, 0x89, 0xd0, 0xd0, 0x1d, + 0x48, 0x61, 0x4b, 0x57, 0xc8, 0x32, 0x7f, 0x7e, 0x79, 0x01, 0x5b, 0xfa, 0x23, 0x3c, 0xbe, 0x2f, + 0xfc, 0xf4, 0x83, 0xfc, 0xdc, 0xaf, 0x3f, 0xc8, 0x73, 0x0d, 0x5e, 0xe0, 0xc4, 0xf9, 0x06, 0x2f, + 0xcc, 0x8b, 0x89, 0x42, 0x1f, 0x92, 0x8f, 0x55, 0x73, 0x84, 0xd1, 0x6d, 0x48, 0x3b, 0xea, 0x3b, + 0x4a, 0x6f, 0xec, 0x61, 0x37, 0xc7, 0x11, 0x08, 0x49, 0x70, 0xd4, 0x77, 0xca, 0xe4, 0x1d, 0x95, + 0x20, 0x1d, 0xee, 0x34, 0x37, 0x7f, 0x87, 0xbb, 0xbb, 0xb8, 0xfd, 0xdf, 0xc5, 0xe8, 0x0a, 0xc8, + 0x71, 0x8a, 0x87, 0xa6, 0x56, 0x94, 0x03, 0xa6, 0x32, 0xff, 0xe1, 0x24, 0x3f, 0x27, 0x45, 0x52, + 0x85, 0xb7, 0x40, 0x78, 0x84, 0xc7, 0x4c, 0x97, 0x7f, 0x0e, 0x6e, 0xca, 0x39, 0x5e, 0x82, 0xe4, + 0x31, 0xe1, 0xf1, 0xb5, 0xe4, 0x8a, 0x4f, 0x5d, 0x74, 0x91, 0x62, 0xf8, 0x0a, 0x18, 0x73, 0xe1, + 0x63, 0x0e, 0xa0, 0xeb, 0xd9, 0x0e, 0xae, 0xeb, 0xd8, 0xf2, 0x50, 0x1f, 0x40, 0x33, 0x47, 0xae, + 0x87, 0x1d, 0xc5, 0xd0, 0x7d, 0x35, 0xaf, 0x12, 0xfe, 0x3f, 0x4e, 0xf2, 0x2f, 0xf6, 0x0d, 0xef, + 0x70, 0xd4, 0x2b, 0x6a, 0xf6, 0x60, 0x2b, 0xc4, 0xd6, 0x7b, 0xd1, 0xf3, 0xd6, 0xf0, 0xa8, 0xbf, + 0x45, 0x2f, 0x68, 0x34, 0x32, 0xf4, 0xe2, 0xde, 0x5e, 0xbd, 0x7a, 0x3a, 0xc9, 0xa7, 0x2b, 0x0c, + 0xb0, 0x5e, 0x95, 0xd2, 0x3e, 0x76, 0x5d, 0x47, 0x2f, 0x40, 0xca, 0xb2, 0x75, 0x4c, 0xb4, 0x90, + 0xfd, 0x26, 0xcb, 0xab, 0xa7, 0x93, 0xfc, 0x42, 0xdb, 0xd6, 0x71, 0xbd, 0x7a, 0x16, 0x3e, 0x49, + 0x0b, 0x84, 0xa9, 0xae, 0xa3, 0x6f, 0x80, 0x40, 0xfc, 0x82, 0xf2, 0x27, 0x28, 0xff, 0xfa, 0xe9, + 0x24, 0x9f, 0x62, 0x3b, 0x27, 0x02, 0xc1, 0xa3, 0x94, 0x72, 0xd9, 0x69, 0x0a, 0x3f, 0xe7, 0x60, + 0xa9, 0x3b, 0x34, 0x0d, 0x4f, 0x76, 0x8c, 0x7e, 0x1f, 0x3b, 0xa8, 0x06, 0x69, 0x13, 0x1f, 0x78, + 0x8a, 0x8e, 0x5d, 0x8d, 0x1e, 0x6d, 0x71, 0xbb, 0x30, 0xc5, 0x48, 0x92, 0x6a, 0xf5, 0x71, 0x15, + 0xbb, 0x9a, 0x63, 0x0c, 0x3d, 0xdb, 0xf1, 0xcd, 0x25, 0x10, 0x51, 0x42, 0x45, 0x3b, 0x00, 0x8e, + 0xd1, 0x3f, 0xf4, 0x71, 0xe6, 0xaf, 0x89, 0x93, 0xa6, 0xb2, 0x84, 0x7c, 0x9f, 0xff, 0x8c, 0xb9, + 0x54, 0x42, 0xe4, 0x0b, 0xbf, 0xe7, 0x61, 0xa9, 0x85, 0x9d, 0x3e, 0xfe, 0x8a, 0x6e, 0x16, 0xf5, + 0x41, 0x64, 0x40, 0x24, 0x1a, 0x15, 0xd7, 0x53, 0x3d, 0x97, 0x86, 0xcb, 0xe2, 0xf6, 0xff, 0xc5, + 0xe0, 0xfc, 0xd8, 0x2d, 0x06, 0xb1, 0x5b, 0x6c, 0x3d, 0xae, 0x54, 0xba, 0x84, 0xb9, 0xbc, 0x4e, + 0x10, 0x4f, 0x27, 0xf9, 0xac, 0x44, 0x60, 0x42, 0xba, 0x94, 0xa5, 0xb0, 0xad, 0x63, 0x4d, 0xa3, + 0xef, 0xe8, 0x87, 0x1c, 0x2c, 0x1d, 0x38, 0x18, 0xbf, 0x8b, 0x89, 0x16, 0xc7, 0xcb, 0x25, 0x67, + 0x09, 0x9a, 0x2a, 0x41, 0x3f, 0x9b, 0xe4, 0x1f, 0xcc, 0xee, 0xa3, 0x04, 0xa0, 0x42, 0x92, 0x5b, + 0x88, 0x22, 0x2d, 0x32, 0xc5, 0x5d, 0xa2, 0x17, 0xed, 0xc3, 0x3a, 0x3b, 0xb1, 0x66, 0xda, 0x2e, + 0xd6, 0x95, 0x28, 0x8c, 0x17, 0x66, 0x0f, 0xe3, 0x55, 0x0a, 0x51, 0xa1, 0x08, 0xe1, 0x1a, 0x7a, + 0x1b, 0x10, 0x83, 0x26, 0x09, 0x54, 0xf1, 0x93, 0x66, 0x2e, 0x45, 0x61, 0x8b, 0x31, 0xd8, 0xa3, + 0xe3, 0x62, 0x90, 0x6b, 0x8b, 0xb1, 0x5c, 0x5b, 0x94, 0xb0, 0xaa, 0x77, 0xd9, 0xb3, 0xc4, 0xae, + 0x25, 0x46, 0x39, 0xe7, 0x57, 0xbf, 0x48, 0xc2, 0x5a, 0xe5, 0x90, 0x5c, 0xae, 0x84, 0x87, 0xa6, + 0xa1, 0xa9, 0x6e, 0xe0, 0x60, 0x6f, 0xc2, 0xba, 0x8e, 0x87, 0x0e, 0xd6, 0x54, 0x0f, 0xeb, 0x8a, + 0x46, 0x79, 0x14, 0x6f, 0x3c, 0xc4, 0xd4, 0xdb, 0xb2, 0xdb, 0xff, 0x3b, 0xcd, 0x4b, 0x18, 0x06, + 0x03, 0x94, 0xc7, 0x43, 0x2c, 0xad, 0x46, 0x18, 0x11, 0x15, 0xed, 0x03, 0x8a, 0x61, 0x3b, 0x4c, + 0xca, 0xf7, 0xbe, 0x2b, 0x70, 0x9f, 0xf2, 0xbf, 0x95, 0x08, 0xc5, 0x67, 0x41, 0xdf, 0x87, 0xdb, + 0x31, 0xe8, 0xd1, 0x50, 0x8f, 0xab, 0x70, 0x73, 0x89, 0x3b, 0x89, 0x6b, 0xea, 0xb8, 0x15, 0xc1, + 0xed, 0x31, 0xb4, 0xc0, 0x52, 0x08, 0xc3, 0x46, 0x4c, 0x97, 0x85, 0x4f, 0xbc, 0x40, 0x11, 0x49, + 0x43, 0x3c, 0x4d, 0x43, 0x77, 0x4f, 0x27, 0xf9, 0x9b, 0xd5, 0x90, 0xab, 0x8d, 0x4f, 0x3c, 0x5f, + 0x9e, 0xa6, 0xa5, 0x74, 0xf8, 0x22, 0xdd, 0xd4, 0xa7, 0x72, 0xe9, 0xe8, 0x65, 0xe0, 0x69, 0x74, + 0x26, 0x67, 0x8d, 0x4e, 0x89, 0xf2, 0xa3, 0x1e, 0xdc, 0x34, 0x2c, 0x0f, 0x3b, 0x96, 0x6a, 0x2a, + 0xaa, 0xae, 0xc7, 0xcd, 0xb0, 0x70, 0x6d, 0x33, 0xac, 0x05, 0x50, 0x25, 0x82, 0x14, 0x9a, 0xe0, + 0x00, 0x6e, 0x85, 0x3a, 0x1c, 0x3c, 0xb0, 0x8f, 0xe3, 0x5a, 0x52, 0xd7, 0xd6, 0x12, 0x6e, 0x58, + 0x62, 0x58, 0x81, 0x9e, 0xfb, 0x3c, 0x29, 0xb1, 0x85, 0xf7, 0x39, 0xb8, 0xd1, 0xb2, 0x75, 0xe3, + 0xc0, 0xc0, 0x3a, 0x29, 0xdb, 0x81, 0xaf, 0x7e, 0x1d, 0x90, 0x3b, 0x76, 0x3d, 0x3c, 0x50, 0x34, + 0xdb, 0x3a, 0x30, 0xfa, 0x8a, 0x3b, 0x54, 0x2d, 0xea, 0xa7, 0x82, 0x24, 0xb2, 0x95, 0x0a, 0x5d, + 0xa0, 0xb5, 0xbe, 0x06, 0x88, 0x96, 0x16, 0xd3, 0x38, 0xc6, 0x16, 0x76, 0x5d, 0xc6, 0xcd, 0xbc, + 0xef, 0xe6, 0x94, 0xcd, 0x12, 0x21, 0x49, 0x24, 0x22, 0x4d, 0x5f, 0x82, 0x50, 0x0a, 0x8f, 0x41, + 0xec, 0x7a, 0x86, 0x76, 0x34, 0x2e, 0x47, 0x25, 0xa4, 0x0c, 0xe0, 0x52, 0x9a, 0xd2, 0x33, 0x3c, + 0x3f, 0x2d, 0xcf, 0x56, 0xce, 0xdd, 0x00, 0xaa, 0xf0, 0x9b, 0x04, 0xac, 0xd5, 0x7d, 0x33, 0x54, + 0xec, 0xc1, 0x20, 0x42, 0xaf, 0x42, 0xc6, 0x25, 0x05, 0x4b, 0xf1, 0x18, 0xc1, 0x57, 0x90, 0x9f, + 0xba, 0xe7, 0xa8, 0xb0, 0x49, 0x4b, 0x6e, 0xbc, 0xcc, 0x55, 0x21, 0x33, 0x20, 0x95, 0x24, 0x44, + 0x99, 0xbf, 0x14, 0x25, 0x5e, 0x71, 0xa4, 0xa5, 0x41, 0xbc, 0xfe, 0x7c, 0x0f, 0x6e, 0xfa, 0x39, + 0x21, 0xb8, 0xee, 0x10, 0x2f, 0x41, 0xf1, 0xee, 0x4e, 0xc1, 0x9b, 0x9a, 0x69, 0xa4, 0x35, 0xed, + 0x92, 0x04, 0xb4, 0x36, 0xf0, 0xef, 0x9a, 0xde, 0x50, 0x88, 0xcf, 0xca, 0xca, 0xff, 0x4f, 0xdb, + 0xef, 0xd3, 0xbe, 0x21, 0xdd, 0x18, 0x4c, 0x71, 0x98, 0xd7, 0x00, 0x45, 0xf7, 0x14, 0x02, 0xb3, + 0x00, 0x7b, 0x6e, 0x9a, 0x39, 0x2f, 0x5c, 0xb4, 0x24, 0xba, 0x17, 0x28, 0xf7, 0xf9, 0xf7, 0x3e, + 0xc8, 0x73, 0x85, 0xdf, 0x71, 0xb0, 0xd2, 0xe9, 0xd1, 0x94, 0x1c, 0xcb, 0xe7, 0xb1, 0x66, 0x86, + 0x9b, 0xa1, 0x99, 0xf9, 0x01, 0x77, 0xed, 0xa6, 0xf0, 0x8b, 0xa9, 0x6f, 0x91, 0xd6, 0xfb, 0x3c, + 0xe9, 0x67, 0x0b, 0x67, 0x0b, 0xb0, 0x28, 0x3b, 0xaa, 0xe5, 0xaa, 0x9a, 0x67, 0xd8, 0x16, 0x2a, + 0x01, 0x4f, 0x7a, 0x75, 0xdf, 0xf1, 0x9e, 0xbb, 0xaa, 0xb2, 0xcb, 0x27, 0x56, 0x0b, 0x7b, 0x6a, + 0x59, 0x20, 0x3b, 0xfb, 0x68, 0x92, 0xe7, 0x24, 0x2a, 0x8a, 0x10, 0xf0, 0x96, 0x3a, 0x60, 0x5d, + 0x68, 0x5a, 0xa2, 0xcf, 0xe8, 0x01, 0x2c, 0x90, 0x8e, 0x61, 0xc4, 0x5a, 0x86, 0xe9, 0xb5, 0x25, + 0xb6, 0x8d, 0x2e, 0xe5, 0x95, 0x7c, 0x19, 0xd4, 0x80, 0xac, 0xa9, 0xba, 0x9e, 0x72, 0x88, 0x55, + 0xc7, 0xeb, 0x61, 0x75, 0xc6, 0x96, 0x80, 0x05, 0x5e, 0x86, 0x88, 0xbe, 0x1a, 0x48, 0xa2, 0xef, + 0x42, 0xae, 0x6f, 0xda, 0x3d, 0xd5, 0x54, 0x46, 0x96, 0x86, 0x1d, 0x4f, 0x35, 0x2c, 0x6f, 0xac, + 0x98, 0xc6, 0xc0, 0xf0, 0xfc, 0xfa, 0x3b, 0x13, 0xea, 0x3a, 0x03, 0xd9, 0x8b, 0x30, 0x9a, 0x04, + 0x02, 0xbd, 0x05, 0x37, 0x6c, 0xdf, 0x3b, 0xa2, 0x7e, 0xc1, 0xcd, 0x09, 0x97, 0x26, 0xca, 0xa7, + 0x7c, 0xc9, 0x57, 0x80, 0xec, 0x8b, 0x0b, 0x2e, 0x7a, 0x00, 0x40, 0xae, 0x93, 0x06, 0x8b, 0x9b, + 0x5b, 0xa4, 0x98, 0x97, 0xe5, 0xb3, 0x20, 0xed, 0x10, 0x01, 0xf2, 0xee, 0xa2, 0x02, 0x64, 0xde, + 0x71, 0x0c, 0x0f, 0x2b, 0x9e, 0x6d, 0x2b, 0xb6, 0xa9, 0xe7, 0x96, 0x68, 0xfa, 0x5c, 0xa4, 0x44, + 0xd9, 0xb6, 0x3b, 0xa6, 0x4e, 0x2c, 0x4d, 0x3b, 0x92, 0xc8, 0x39, 0x97, 0xaf, 0x61, 0x69, 0x22, + 0x1a, 0xc5, 0xc4, 0x4b, 0xb0, 0xae, 0xd1, 0xec, 0x16, 0xa1, 0x29, 0x07, 0xc6, 0x09, 0xd6, 0x73, + 0x22, 0x55, 0xbc, 0xca, 0x56, 0x43, 0x81, 0x87, 0x64, 0x0d, 0xbd, 0x06, 0xa2, 0x61, 0x29, 0x07, + 0x26, 0xed, 0x8e, 0xe8, 0xd6, 0xdc, 0xdc, 0x0a, 0x3d, 0xe9, 0xff, 0x4c, 0x3b, 0x29, 0x7e, 0x32, + 0xc2, 0x96, 0x86, 0xf5, 0xd7, 0x09, 0xa7, 0xbf, 0x8f, 0xac, 0x61, 0x3d, 0xa4, 0xf2, 0x94, 0xe8, + 0x22, 0x1b, 0x96, 0x8d, 0xbe, 0x65, 0x3b, 0x24, 0xcd, 0xe0, 0x27, 0xd6, 0x68, 0xe0, 0xe6, 0x10, + 0x45, 0x2c, 0x5e, 0xe5, 0xde, 0x75, 0x26, 0xd2, 0xc5, 0x4f, 0xda, 0xa3, 0x01, 0xad, 0xbf, 0x51, + 0x07, 0x7b, 0x6e, 0xcd, 0x95, 0xb2, 0x46, 0xf8, 0x4e, 0xd0, 0xcf, 0x7d, 0x2e, 0x26, 0x44, 0xbe, + 0xc1, 0x0b, 0x0b, 0x62, 0xaa, 0xc1, 0x0b, 0x69, 0x11, 0x1a, 0xbc, 0x90, 0x11, 0xb3, 0x0d, 0x5e, + 0xc8, 0x8a, 0xcb, 0x85, 0x5f, 0xf1, 0xb0, 0x12, 0xf3, 0x7a, 0x09, 0x6b, 0xb6, 0xa3, 0x7f, 0x11, + 0x21, 0xf8, 0xd5, 0x09, 0xb7, 0x67, 0x73, 0xd9, 0xff, 0x04, 0x67, 0xe0, 0x7d, 0x47, 0x98, 0x17, + 0x13, 0x17, 0xdc, 0x21, 0x25, 0x0a, 0x0d, 0x5e, 0x10, 0xc4, 0x74, 0xe8, 0x1a, 0x20, 0x2e, 0x36, + 0x78, 0x61, 0x49, 0xcc, 0xc4, 0xdd, 0xa4, 0xc1, 0x0b, 0xcb, 0xa2, 0xd8, 0xe0, 0x05, 0x51, 0x5c, + 0x29, 0x4c, 0x38, 0x58, 0x20, 0xbd, 0x83, 0x45, 0x3e, 0x4f, 0x96, 0x5d, 0xc3, 0xea, 0x9b, 0x58, + 0x39, 0xc2, 0xe3, 0xa8, 0x21, 0x5a, 0xdc, 0x7e, 0x7e, 0x8a, 0x6d, 0x98, 0x4c, 0xb1, 0x4b, 0x05, + 0x1e, 0xe1, 0x31, 0xb5, 0x77, 0xe4, 0x36, 0x19, 0x37, 0xbe, 0x80, 0xbe, 0x05, 0x09, 0xef, 0x24, + 0xe8, 0x98, 0x66, 0xf2, 0x40, 0x66, 0x6c, 0x22, 0xb5, 0xf1, 0x00, 0x32, 0xe7, 0xd4, 0x5c, 0x31, + 0x7a, 0x89, 0x8f, 0x53, 0x1a, 0xbc, 0xc0, 0x8b, 0xc9, 0xc2, 0x1f, 0x38, 0x58, 0x6e, 0xda, 0xda, + 0x51, 0x49, 0x7b, 0x32, 0x32, 0x5c, 0x83, 0x16, 0xa5, 0x6f, 0x02, 0x1f, 0x3b, 0xde, 0xa5, 0xee, + 0x13, 0x8b, 0x02, 0xf7, 0x59, 0x4f, 0x81, 0x24, 0x00, 0x7d, 0xe4, 0xa8, 0x3d, 0xc3, 0x34, 0x3c, + 0xb6, 0xf7, 0xec, 0xf6, 0xf6, 0x25, 0x5f, 0x66, 0xb1, 0x89, 0x59, 0x91, 0x38, 0x6f, 0xb1, 0x1a, + 0x4a, 0x4a, 0x31, 0x94, 0xc2, 0x2f, 0xe7, 0x01, 0xc8, 0xd9, 0xd8, 0x67, 0xc6, 0xbf, 0xe5, 0x58, + 0x51, 0x66, 0x48, 0x7c, 0x8e, 0xcc, 0x30, 0x25, 0x78, 0xf8, 0x7f, 0x65, 0xf0, 0x14, 0x7e, 0xcc, + 0x41, 0xf6, 0x7c, 0x58, 0x5f, 0x35, 0x00, 0x7b, 0x1b, 0x04, 0xd7, 0x67, 0xf6, 0x67, 0x4a, 0xdf, + 0x99, 0xb9, 0x63, 0xba, 0x38, 0x74, 0x24, 0xc6, 0xeb, 0xe2, 0x27, 0x52, 0x88, 0xe8, 0x37, 0x4c, + 0xbf, 0xe5, 0x21, 0xd9, 0xc4, 0xaa, 0x8b, 0xd1, 0x18, 0x92, 0x6c, 0x3e, 0xc1, 0x7d, 0x79, 0xfd, + 0x1b, 0xd3, 0x88, 0xbe, 0x0d, 0x80, 0x4f, 0x86, 0x86, 0xa3, 0x92, 0x3b, 0x9a, 0xa9, 0x7f, 0x94, + 0x62, 0x02, 0xa8, 0x0a, 0xa9, 0xe0, 0x93, 0x3c, 0x71, 0xed, 0x4f, 0xf2, 0x40, 0x14, 0xed, 0x41, + 0xec, 0x83, 0x96, 0x8d, 0x6a, 0xc8, 0x5f, 0xd7, 0x08, 0xe6, 0x42, 0xff, 0x64, 0x47, 0x6b, 0x91, + 0x34, 0x9d, 0xb7, 0x74, 0xa9, 0x2c, 0xfa, 0x11, 0x07, 0x8b, 0x43, 0xc7, 0x1e, 0xb2, 0x91, 0x8b, + 0x3b, 0x5b, 0xed, 0x69, 0x9f, 0x4e, 0xf2, 0xb0, 0xeb, 0x4b, 0xc9, 0xdd, 0x67, 0xb6, 0x33, 0x04, + 0x3b, 0x90, 0x5d, 0xb4, 0x0a, 0x49, 0x3c, 0xb4, 0xb5, 0x43, 0x3a, 0xf5, 0x49, 0x48, 0xec, 0x05, + 0xbd, 0x10, 0xf3, 0x35, 0xd2, 0x37, 0x26, 0xca, 0x2b, 0x67, 0x93, 0x7c, 0x86, 0xba, 0x46, 0xe0, + 0xb1, 0x31, 0xe7, 0x09, 0x5b, 0x82, 0xc2, 0x9f, 0x38, 0xc8, 0x96, 0x7a, 0xb6, 0xe3, 0x91, 0xe0, + 0xae, 0x59, 0x9e, 0x33, 0xbe, 0xca, 0xa5, 0x9f, 0x7d, 0x7a, 0x8c, 0x54, 0x10, 0x86, 0x8e, 0x61, + 0x3b, 0x41, 0x1e, 0x4b, 0x96, 0x6b, 0x67, 0x93, 0x7c, 0xe9, 0x73, 0x47, 0xc5, 0xae, 0x0f, 0x26, + 0x85, 0xb0, 0xf7, 0x05, 0x72, 0xb2, 0xcf, 0xc8, 0xe9, 0xfe, 0x3e, 0x0f, 0x2b, 0x4d, 0xac, 0x1e, + 0xc8, 0x27, 0x56, 0xdd, 0x1a, 0x8e, 0xc8, 0xa5, 0x7a, 0x18, 0xbd, 0xcc, 0x52, 0x16, 0x0b, 0x94, + 0xcd, 0xab, 0x53, 0x4e, 0x3c, 0x5b, 0x3d, 0x0f, 0xcb, 0x0e, 0x3e, 0x70, 0xb0, 0x7b, 0xa8, 0x18, + 0xd6, 0xb1, 0x6a, 0x1a, 0x3a, 0xb5, 0xb5, 0x20, 0x65, 0x7d, 0x72, 0x9d, 0x51, 0xa7, 0x36, 0x0a, + 0xc2, 0xb3, 0x35, 0x0a, 0xdb, 0xb0, 0xe6, 0x7a, 0x78, 0x38, 0x34, 0xac, 0xbe, 0x32, 0x20, 0xdf, + 0x76, 0xd8, 0x52, 0x7b, 0x26, 0xd6, 0x73, 0x69, 0xba, 0x83, 0x1b, 0xc1, 0x62, 0xcb, 0xd6, 0x71, + 0x8d, 0x2d, 0xa1, 0x1e, 0x2c, 0xb1, 0x81, 0x1e, 0x7e, 0xa2, 0x58, 0xa3, 0x41, 0x0e, 0xbe, 0xa0, + 0x24, 0x04, 0x04, 0x95, 0xe5, 0xc8, 0x0b, 0x9d, 0x04, 0x2f, 0x26, 0x1b, 0xbc, 0x90, 0x14, 0x17, + 0x58, 0x57, 0x51, 0x78, 0x3f, 0xb2, 0xff, 0x43, 0xc3, 0x52, 0xcd, 0x67, 0xb3, 0xff, 0x2b, 0x90, + 0x8b, 0x8f, 0x08, 0xed, 0xc1, 0x40, 0xb5, 0xc8, 0xff, 0x91, 0xe5, 0x31, 0x57, 0x92, 0x62, 0x23, + 0xc4, 0x0a, 0x5b, 0xae, 0x90, 0x55, 0x54, 0x86, 0x4c, 0x70, 0x73, 0xac, 0xf5, 0xe3, 0x67, 0x69, + 0xfd, 0x96, 0x7c, 0x19, 0xd6, 0xfd, 0xcd, 0x7a, 0xfb, 0xa1, 0x49, 0x42, 0x33, 0xb0, 0xb6, 0xaa, + 0xf0, 0x37, 0x0e, 0x96, 0x2b, 0xa6, 0x81, 0x2d, 0x8f, 0x56, 0x9d, 0xba, 0x75, 0x60, 0xa3, 0x57, + 0x61, 0x4d, 0x0f, 0x53, 0x9a, 0xd2, 0xc7, 0x16, 0xf6, 0xb3, 0x28, 0x47, 0x83, 0xf8, 0xc6, 0xd9, + 0x24, 0xbf, 0x4c, 0xb9, 0x77, 0xc2, 0x25, 0x69, 0x35, 0x92, 0x88, 0xa8, 0xe8, 0x15, 0xc8, 0x9a, + 0x24, 0xda, 0x95, 0x73, 0x35, 0x67, 0x6a, 0x1e, 0xc8, 0x98, 0xf1, 0x57, 0x84, 0xe1, 0xe6, 0xc5, + 0x91, 0xb2, 0x32, 0xb4, 0x4d, 0x43, 0x0b, 0x1a, 0x8d, 0x17, 0x2e, 0x1b, 0x01, 0x5e, 0x98, 0x23, + 0xef, 0x52, 0x21, 0x69, 0x4d, 0x9b, 0x46, 0x2e, 0xfc, 0x95, 0x83, 0x74, 0x74, 0xf0, 0x07, 0xfe, + 0x90, 0xf1, 0xba, 0x3f, 0x25, 0xb0, 0x51, 0xe3, 0x4b, 0x90, 0xa4, 0x67, 0xb8, 0xe2, 0xb7, 0x25, + 0x7a, 0xe4, 0xe0, 0xb7, 0x25, 0xca, 0xfc, 0x25, 0x1d, 0x94, 0xcd, 0x0e, 0xef, 0x7d, 0xcc, 0x41, + 0x9a, 0xfe, 0xbe, 0x45, 0x27, 0xd0, 0x8b, 0x90, 0xda, 0x6b, 0x3f, 0x6a, 0x77, 0x5e, 0x6f, 0x8b, + 0x73, 0x28, 0x05, 0x89, 0x7a, 0x5b, 0x16, 0x39, 0x94, 0x86, 0xe4, 0xc3, 0x66, 0xa7, 0x24, 0x8b, + 0xf3, 0xe4, 0xb1, 0xbc, 0x2f, 0xd7, 0xba, 0x62, 0x02, 0xdd, 0x80, 0xe5, 0x6a, 0xad, 0x59, 0x6f, + 0xd5, 0xe5, 0x5a, 0x55, 0x61, 0x44, 0x01, 0x09, 0xc0, 0xcb, 0xf5, 0x56, 0x4d, 0xe4, 0x09, 0x54, + 0xb5, 0x56, 0xa9, 0xb7, 0x4a, 0x4d, 0x31, 0x89, 0xd6, 0x60, 0x25, 0xe2, 0x0d, 0xc8, 0x69, 0xb4, + 0x04, 0x42, 0x75, 0x4f, 0x2a, 0xc9, 0xf5, 0x4e, 0x5b, 0x5c, 0x40, 0x00, 0x0b, 0x44, 0x56, 0x7e, + 0x53, 0x5c, 0x22, 0xba, 0x77, 0x6a, 0x1d, 0x31, 0x43, 0x15, 0x76, 0xde, 0xd8, 0xae, 0x8a, 0x59, + 0xf2, 0x28, 0xef, 0xed, 0x36, 0x6b, 0x22, 0x10, 0xc1, 0x72, 0x5d, 0x2e, 0x49, 0x52, 0x69, 0x5f, + 0x5c, 0x44, 0x59, 0x00, 0x22, 0xd8, 0xad, 0x49, 0xf5, 0x5a, 0x57, 0xd4, 0x0b, 0xe4, 0x23, 0x21, + 0x75, 0x4f, 0x83, 0x95, 0xa7, 0x06, 0xef, 0x28, 0x03, 0xe9, 0x52, 0xb5, 0xaa, 0x3c, 0xee, 0xc8, + 0x35, 0x49, 0x9c, 0x43, 0x22, 0x2c, 0x49, 0xb5, 0x56, 0xe7, 0x71, 0xcd, 0xa7, 0x70, 0x68, 0x05, + 0x32, 0x84, 0xa1, 0xdd, 0x69, 0xfb, 0xa4, 0x79, 0xb4, 0x0a, 0xa2, 0xcf, 0x14, 0x51, 0x13, 0x1b, + 0xfc, 0x7b, 0x3f, 0xdb, 0x9c, 0xbb, 0xd7, 0x39, 0xf7, 0x2d, 0xca, 0x1a, 0x3f, 0x72, 0xf4, 0xdd, + 0x5a, 0xbb, 0x5a, 0x6f, 0xef, 0x88, 0x73, 0xe4, 0xa5, 0x2b, 0x97, 0x76, 0xc8, 0x4b, 0x82, 0xa8, + 0xaf, 0x74, 0x5a, 0xad, 0xba, 0x2c, 0xd7, 0xaa, 0x22, 0x47, 0xd6, 0x4a, 0xe5, 0x8e, 0x44, 0x5e, + 0xe6, 0x7d, 0xc0, 0x77, 0x61, 0xe3, 0xf2, 0xab, 0x44, 0x1b, 0xb0, 0xde, 0x2c, 0xed, 0x28, 0xe5, + 0x7d, 0xa5, 0xd2, 0xdc, 0xeb, 0xca, 0x35, 0x49, 0xe9, 0xd6, 0x64, 0x99, 0x29, 0xba, 0x05, 0x6b, + 0xcd, 0x5a, 0xa9, 0xaa, 0x3c, 0xec, 0x48, 0xca, 0x4e, 0xb3, 0x53, 0x2e, 0x35, 0x15, 0xa9, 0x56, + 0xaa, 0x76, 0x45, 0x0e, 0xe5, 0xe1, 0x76, 0xab, 0xf4, 0x86, 0x52, 0x69, 0x76, 0xba, 0xb5, 0xaa, + 0x42, 0x6d, 0x25, 0x97, 0x5a, 0xbb, 0xca, 0x6e, 0xa7, 0x59, 0xaf, 0xec, 0x07, 0xba, 0xcb, 0x5f, + 0xfb, 0xf0, 0x2f, 0x9b, 0x73, 0x1f, 0x9e, 0x6e, 0x72, 0x1f, 0x9d, 0x6e, 0x72, 0x9f, 0x9c, 0x6e, + 0x72, 0x7f, 0x3e, 0xdd, 0xe4, 0x7e, 0xf2, 0xe9, 0xe6, 0xdc, 0x47, 0x9f, 0x6e, 0xce, 0x7d, 0xf2, + 0xe9, 0xe6, 0xdc, 0x9b, 0x29, 0xdf, 0xdd, 0x7a, 0x0b, 0xf4, 0x77, 0xe2, 0x17, 0xff, 0x11, 0x00, + 0x00, 0xff, 0xff, 0x5b, 0x28, 0x10, 0xe8, 0x2b, 0x1f, 0x00, 0x00, } diff --git a/pkg/roachpb/data.proto b/pkg/roachpb/data.proto index e55eb7379a6a..b9cf60151026 100644 --- a/pkg/roachpb/data.proto +++ b/pkg/roachpb/data.proto @@ -712,6 +712,9 @@ enum RangeClosedTimestampPolicy { // (which itself is a function of leaseholder -> follower network latency and // closed timestamp update periodicity). LEAD_FOR_GLOBAL_READS = 1; + + // Keep this sentinel value higher than the rest. + MAX_CLOSED_TIMESTAMP_POLICY = 2; } // ClientRangeInfo represents the kvclient's knowledge about the state of the diff --git a/pkg/server/server.go b/pkg/server/server.go index ab56929573ec..1181a8cbf5f2 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -42,6 +42,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/container" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/sidetransport" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts" @@ -131,19 +132,20 @@ type Server struct { rpcContext *rpc.Context engines Engines // The gRPC server on which the different RPC handlers will be registered. - grpc *grpcServer - gossip *gossip.Gossip - nodeDialer *nodedialer.Dialer - nodeLiveness *liveness.NodeLiveness - storePool *kvserver.StorePool - tcsFactory *kvcoord.TxnCoordSenderFactory - distSender *kvcoord.DistSender - db *kv.DB - node *Node - registry *metric.Registry - recorder *status.MetricsRecorder - runtime *status.RuntimeStatSampler - updates *diagnostics.UpdateChecker + grpc *grpcServer + gossip *gossip.Gossip + nodeDialer *nodedialer.Dialer + nodeLiveness *liveness.NodeLiveness + storePool *kvserver.StorePool + tcsFactory *kvcoord.TxnCoordSenderFactory + distSender *kvcoord.DistSender + db *kv.DB + node *Node + registry *metric.Registry + recorder *status.MetricsRecorder + runtime *status.RuntimeStatSampler + updates *diagnostics.UpdateChecker + closedTimestampSideTransportSender *sidetransport.ClosedTimestampSender admin *adminServer status *statusServer @@ -508,27 +510,31 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { // ClosedTimestamp), but the Node needs a StoreConfig to be made. var lateBoundNode *Node + closedTimestampSideTransportSender := sidetransport.NewSideTransportSender( + stopper, nodeDialer, clock, st) + storeCfg := kvserver.StoreConfig{ - DefaultZoneConfig: &cfg.DefaultZoneConfig, - Settings: st, - AmbientCtx: cfg.AmbientCtx, - RaftConfig: cfg.RaftConfig, - Clock: clock, - DB: db, - Gossip: g, - NodeLiveness: nodeLiveness, - Transport: raftTransport, - NodeDialer: nodeDialer, - RPCContext: rpcContext, - ScanInterval: cfg.ScanInterval, - ScanMinIdleTime: cfg.ScanMinIdleTime, - ScanMaxIdleTime: cfg.ScanMaxIdleTime, - HistogramWindowInterval: cfg.HistogramWindowInterval(), - StorePool: storePool, - SQLExecutor: internalExecutor, - LogRangeEvents: cfg.EventLogEnabled, - RangeDescriptorCache: distSender.RangeDescriptorCache(), - TimeSeriesDataStore: tsDB, + DefaultZoneConfig: &cfg.DefaultZoneConfig, + Settings: st, + AmbientCtx: cfg.AmbientCtx, + RaftConfig: cfg.RaftConfig, + Clock: clock, + DB: db, + Gossip: g, + NodeLiveness: nodeLiveness, + Transport: raftTransport, + NodeDialer: nodeDialer, + RPCContext: rpcContext, + ScanInterval: cfg.ScanInterval, + ScanMinIdleTime: cfg.ScanMinIdleTime, + ScanMaxIdleTime: cfg.ScanMaxIdleTime, + HistogramWindowInterval: cfg.HistogramWindowInterval(), + StorePool: storePool, + SQLExecutor: internalExecutor, + LogRangeEvents: cfg.EventLogEnabled, + RangeDescriptorCache: distSender.RangeDescriptorCache(), + TimeSeriesDataStore: tsDB, + ClosedTimestampSideTransport: closedTimestampSideTransportSender, // Initialize the closed timestamp subsystem. Note that it won't // be ready until it is .Start()ed, but the grpc server can be @@ -679,38 +685,39 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { node.InitLogger(sqlServer.execCfg) *lateBoundServer = Server{ - nodeIDContainer: nodeIDContainer, - cfg: cfg, - st: st, - clock: clock, - rpcContext: rpcContext, - engines: engines, - grpc: grpcServer, - gossip: g, - nodeDialer: nodeDialer, - nodeLiveness: nodeLiveness, - storePool: storePool, - tcsFactory: tcsFactory, - distSender: distSender, - db: db, - node: node, - registry: registry, - recorder: recorder, - updates: updates, - runtime: runtimeSampler, - admin: sAdmin, - status: sStatus, - authentication: sAuth, - tsDB: tsDB, - tsServer: &sTS, - raftTransport: raftTransport, - stopper: stopper, - debug: debugServer, - replicationReporter: replicationReporter, - protectedtsProvider: protectedtsProvider, - protectedtsReconciler: protectedtsReconciler, - sqlServer: sqlServer, - externalStorageBuilder: externalStorageBuilder, + nodeIDContainer: nodeIDContainer, + cfg: cfg, + st: st, + clock: clock, + rpcContext: rpcContext, + engines: engines, + grpc: grpcServer, + gossip: g, + nodeDialer: nodeDialer, + nodeLiveness: nodeLiveness, + storePool: storePool, + tcsFactory: tcsFactory, + distSender: distSender, + db: db, + node: node, + registry: registry, + recorder: recorder, + updates: updates, + closedTimestampSideTransportSender: closedTimestampSideTransportSender, + runtime: runtimeSampler, + admin: sAdmin, + status: sStatus, + authentication: sAuth, + tsDB: tsDB, + tsServer: &sTS, + raftTransport: raftTransport, + stopper: stopper, + debug: debugServer, + replicationReporter: replicationReporter, + protectedtsProvider: protectedtsProvider, + protectedtsReconciler: protectedtsReconciler, + sqlServer: sqlServer, + externalStorageBuilder: externalStorageBuilder, } return lateBoundServer, err } @@ -1810,6 +1817,8 @@ func (s *Server) PreStart(ctx context.Context) error { return errors.Wrapf(err, "failed to register engines with debug server") } + s.closedTimestampSideTransportSender.Run(ctx, state.nodeID) + // Attempt to upgrade cluster version now that the sql server has been // started. At this point we know that all sqlmigrations have successfully // been run so it is safe to upgrade to the binary's current version.