diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index 8f8b8d642f55..dfb03f778d8a 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -94,6 +94,7 @@ ALL_TESTS = [ "//pkg/kv/kvserver/closedts/storage:storage_test", "//pkg/kv/kvserver/closedts/tracker:tracker_test", "//pkg/kv/kvserver/closedts/transport:transport_test", + "//pkg/kv/kvserver/closedts:closedts_test", "//pkg/kv/kvserver/concurrency:concurrency_test", "//pkg/kv/kvserver/gc:gc_test", "//pkg/kv/kvserver/idalloc:idalloc_test", diff --git a/pkg/kv/kvserver/BUILD.bazel b/pkg/kv/kvserver/BUILD.bazel index 24e3da6833b5..f95f247da103 100644 --- a/pkg/kv/kvserver/BUILD.bazel +++ b/pkg/kv/kvserver/BUILD.bazel @@ -116,6 +116,7 @@ go_library( "//pkg/kv/kvserver/closedts", "//pkg/kv/kvserver/closedts/container", "//pkg/kv/kvserver/closedts/ctpb", + "//pkg/kv/kvserver/closedts/sidetransport", "//pkg/kv/kvserver/closedts/storage", "//pkg/kv/kvserver/closedts/tracker", "//pkg/kv/kvserver/concurrency", @@ -244,6 +245,7 @@ go_test( "replica_application_cmd_buf_test.go", "replica_application_state_machine_test.go", "replica_batch_updates_test.go", + "replica_closedts_test.go", "replica_command_test.go", "replica_consistency_test.go", "replica_evaluate_test.go", diff --git a/pkg/kv/kvserver/closedts/BUILD.bazel b/pkg/kv/kvserver/closedts/BUILD.bazel index 2cd5383b442e..4774e892be04 100644 --- a/pkg/kv/kvserver/closedts/BUILD.bazel +++ b/pkg/kv/kvserver/closedts/BUILD.bazel @@ -1,9 +1,10 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "closedts", srcs = [ "closedts.go", + "policy.go", "setting.go", ], importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts", @@ -16,3 +17,16 @@ go_library( "@com_github_cockroachdb_errors//:errors", ], ) + +go_test( + name = "closedts_test", + srcs = ["policy_test.go"], + embed = [":closedts"], + deps = [ + "//pkg/roachpb", + "//pkg/util/hlc", + "//pkg/util/leaktest", + "//pkg/util/log", + "@com_github_stretchr_testify//require", + ], +) diff --git a/pkg/kv/kvserver/closedts/ctpb/BUILD.bazel b/pkg/kv/kvserver/closedts/ctpb/BUILD.bazel index 876d2b610807..1f267fae5b48 100644 --- a/pkg/kv/kvserver/closedts/ctpb/BUILD.bazel +++ b/pkg/kv/kvserver/closedts/ctpb/BUILD.bazel @@ -8,6 +8,7 @@ go_library( "client.go", "entry.go", "server.go", + "service.go", ], embed = [":ctpb_go_proto"], importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb", @@ -24,6 +25,7 @@ proto_library( strip_import_prefix = "/pkg", visibility = ["//visibility:public"], deps = [ + "//pkg/roachpb:roachpb_proto", "//pkg/util/hlc:hlc_proto", "@com_github_gogo_protobuf//gogoproto:gogo_proto", "@go_googleapis//google/api:annotations_proto", @@ -37,6 +39,7 @@ go_proto_library( proto = ":ctpb_proto", visibility = ["//visibility:public"], deps = [ + "//pkg/roachpb", "//pkg/util/hlc", "@com_github_gogo_protobuf//gogoproto", "@org_golang_google_genproto//googleapis/api/annotations:go_default_library", diff --git a/pkg/kv/kvserver/closedts/ctpb/service.go b/pkg/kv/kvserver/closedts/ctpb/service.go new file mode 100644 index 000000000000..1dbace2b42c4 --- /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 + +// SeqNum identifies a ctpb.Update. +type SeqNum int64 diff --git a/pkg/kv/kvserver/closedts/ctpb/service.pb.go b/pkg/kv/kvserver/closedts/ctpb/service.pb.go index d634b6a317ac..e4e34d67af40 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,200 @@ var _ = math.Inf // proto package needs to be updated. const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package +// Update 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 Update 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 SeqNum `protobuf:"varint,2,opt,name=seq_num,json=seqNum,proto3,casttype=SeqNum" 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 sender is + // temporarily slowed down or if the stream experience network problems and + // some incremental messages are 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"` + ClosedTimestamps []Update_GroupUpdate `protobuf:"bytes,4,rep,name=closed_timestamps,json=closedTimestamps,proto3" json:"closed_timestamps"` + // removed contains the set of ranges that are no longer registered on the + // stream and who future updates are no longer applicable to. + // + // The field will be empty if snapshot is true, as a snapshot message implies + // that all ranges not present in the snapshot's added_or_updated list are no + // longer tracked. + Removed []github_com_cockroachdb_cockroach_pkg_roachpb.RangeID `protobuf:"varint,5,rep,packed,name=removed,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.RangeID" json:"removed,omitempty"` + AddedOrUpdated []Update_RangeUpdate `protobuf:"bytes,6,rep,name=added_or_updated,json=addedOrUpdated,proto3" json:"added_or_updated"` +} + +func (m *Update) Reset() { *m = Update{} } +func (m *Update) String() string { return proto.CompactTextString(m) } +func (*Update) ProtoMessage() {} +func (*Update) Descriptor() ([]byte, []int) { + return fileDescriptor_service_96a1a4bff833e11e, []int{0} +} +func (m *Update) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Update) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *Update) XXX_Merge(src proto.Message) { + xxx_messageInfo_Update.Merge(dst, src) +} +func (m *Update) XXX_Size() int { + return m.Size() +} +func (m *Update) XXX_DiscardUnknown() { + xxx_messageInfo_Update.DiscardUnknown(m) +} + +var xxx_messageInfo_Update proto.InternalMessageInfo + +// closed_timestamps 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, as are races between two +// side-transport streams for an outgoing and incoming leaseholder. +type Update_GroupUpdate 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 *Update_GroupUpdate) Reset() { *m = Update_GroupUpdate{} } +func (m *Update_GroupUpdate) String() string { return proto.CompactTextString(m) } +func (*Update_GroupUpdate) ProtoMessage() {} +func (*Update_GroupUpdate) Descriptor() ([]byte, []int) { + return fileDescriptor_service_96a1a4bff833e11e, []int{0, 0} +} +func (m *Update_GroupUpdate) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Update_GroupUpdate) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *Update_GroupUpdate) XXX_Merge(src proto.Message) { + xxx_messageInfo_Update_GroupUpdate.Merge(dst, src) +} +func (m *Update_GroupUpdate) XXX_Size() int { + return m.Size() +} +func (m *Update_GroupUpdate) XXX_DiscardUnknown() { + xxx_messageInfo_Update_GroupUpdate.DiscardUnknown(m) +} + +var xxx_messageInfo_Update_GroupUpdate proto.InternalMessageInfo + +// added_or_updated contains the set of ranges that are either being added to +// the tracked ranges set with a given (lai, policy) or updated within the +// tracked range set with a new (lai, policy). All future updates on the +// stream are applicable to these ranges until they are removed, either +// explicitly by being included in a future removed set or implicitly by not +// being included in the added_or_updated field of a future snapshot. +type Update_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 *Update_RangeUpdate) Reset() { *m = Update_RangeUpdate{} } +func (m *Update_RangeUpdate) String() string { return proto.CompactTextString(m) } +func (*Update_RangeUpdate) ProtoMessage() {} +func (*Update_RangeUpdate) Descriptor() ([]byte, []int) { + return fileDescriptor_service_96a1a4bff833e11e, []int{0, 1} +} +func (m *Update_RangeUpdate) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Update_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 *Update_RangeUpdate) XXX_Merge(src proto.Message) { + xxx_messageInfo_Update_RangeUpdate.Merge(dst, src) +} +func (m *Update_RangeUpdate) XXX_Size() int { + return m.Size() +} +func (m *Update_RangeUpdate) XXX_DiscardUnknown() { + xxx_messageInfo_Update_RangeUpdate.DiscardUnknown(m) +} + +var xxx_messageInfo_Update_RangeUpdate 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_96a1a4bff833e11e, []int{1} +} +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((*Update)(nil), "cockroach.kv.kvserver.ctupdate.Update") + proto.RegisterType((*Update_GroupUpdate)(nil), "cockroach.kv.kvserver.ctupdate.Update.GroupUpdate") + proto.RegisterType((*Update_RangeUpdate)(nil), "cockroach.kv.kvserver.ctupdate.Update.RangeUpdate") + 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 +327,1012 @@ var _ClosedTimestamp_serviceDesc = grpc.ServiceDesc{ Metadata: "kv/kvserver/closedts/ctpb/service.proto", } +// SideTransportClient is the client API for SideTransport service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream. +type SideTransportClient interface { + PushUpdates(ctx context.Context, opts ...grpc.CallOption) (SideTransport_PushUpdatesClient, error) +} + +type sideTransportClient struct { + cc *grpc.ClientConn +} + +func NewSideTransportClient(cc *grpc.ClientConn) SideTransportClient { + return &sideTransportClient{cc} +} + +func (c *sideTransportClient) PushUpdates(ctx context.Context, opts ...grpc.CallOption) (SideTransport_PushUpdatesClient, error) { + stream, err := c.cc.NewStream(ctx, &_SideTransport_serviceDesc.Streams[0], "/cockroach.kv.kvserver.ctupdate.SideTransport/PushUpdates", opts...) + if err != nil { + return nil, err + } + x := &sideTransportPushUpdatesClient{stream} + return x, nil +} + +type SideTransport_PushUpdatesClient interface { + Send(*Update) error + Recv() (*Response, error) + grpc.ClientStream +} + +type sideTransportPushUpdatesClient struct { + grpc.ClientStream +} + +func (x *sideTransportPushUpdatesClient) Send(m *Update) error { + return x.ClientStream.SendMsg(m) +} + +func (x *sideTransportPushUpdatesClient) Recv() (*Response, error) { + m := new(Response) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +// SideTransportServer is the server API for SideTransport service. +type SideTransportServer interface { + PushUpdates(SideTransport_PushUpdatesServer) error +} + +func RegisterSideTransportServer(s *grpc.Server, srv SideTransportServer) { + s.RegisterService(&_SideTransport_serviceDesc, srv) +} + +func _SideTransport_PushUpdates_Handler(srv interface{}, stream grpc.ServerStream) error { + return srv.(SideTransportServer).PushUpdates(&sideTransportPushUpdatesServer{stream}) +} + +type SideTransport_PushUpdatesServer interface { + Send(*Response) error + Recv() (*Update, error) + grpc.ServerStream +} + +type sideTransportPushUpdatesServer struct { + grpc.ServerStream +} + +func (x *sideTransportPushUpdatesServer) Send(m *Response) error { + return x.ServerStream.SendMsg(m) +} + +func (x *sideTransportPushUpdatesServer) Recv() (*Update, error) { + m := new(Update) + if err := x.ServerStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +var _SideTransport_serviceDesc = grpc.ServiceDesc{ + ServiceName: "cockroach.kv.kvserver.ctupdate.SideTransport", + HandlerType: (*SideTransportServer)(nil), + Methods: []grpc.MethodDesc{}, + Streams: []grpc.StreamDesc{ + { + StreamName: "PushUpdates", + Handler: _SideTransport_PushUpdates_Handler, + ServerStreams: true, + ClientStreams: true, + }, + }, + Metadata: "kv/kvserver/closedts/ctpb/service.proto", +} + +func (m *Update) 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 *Update) 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.ClosedTimestamps) > 0 { + for _, msg := range m.ClosedTimestamps { + dAtA[i] = 0x22 + 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.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] = 0x2a + 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] = 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 *Update_GroupUpdate) 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 *Update_GroupUpdate) 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 *Update_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 *Update_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 *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 *Update) 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.ClosedTimestamps) > 0 { + for _, e := range m.ClosedTimestamps { + l = e.Size() + n += 1 + l + sovService(uint64(l)) + } + } + 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)) + } + } + return n +} + +func (m *Update_GroupUpdate) 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 *Update_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 *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 *Update) 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: Update: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Update: 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 |= (SeqNum(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 != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ClosedTimestamps", 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.ClosedTimestamps = append(m.ClosedTimestamps, Update_GroupUpdate{}) + if err := m.ClosedTimestamps[len(m.ClosedTimestamps)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + 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 6: + 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, Update_RangeUpdate{}) + if err := m.AddedOrUpdated[len(m.AddedOrUpdated)-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 *Update_GroupUpdate) 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 *Update_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 *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_96a1a4bff833e11e) +} + +var fileDescriptor_service_96a1a4bff833e11e = []byte{ + // 628 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x54, 0xc1, 0x4e, 0xdb, 0x30, + 0x18, 0x6e, 0x96, 0x92, 0x56, 0xae, 0x06, 0xcc, 0xda, 0x21, 0x8a, 0xb6, 0xa4, 0x62, 0x82, 0xe5, + 0xb2, 0x78, 0x2a, 0x3b, 0xec, 0x4a, 0x19, 0x42, 0x48, 0x13, 0x43, 0x81, 0x5d, 0xd0, 0xb4, 0xca, + 0x8d, 0xad, 0x34, 0x6a, 0x1a, 0x07, 0xdb, 0xa9, 0xc4, 0x5b, 0xec, 0x21, 0xf6, 0x28, 0x3b, 0x70, + 0xe4, 0x34, 0x71, 0x8a, 0xb6, 0xf0, 0x16, 0x3d, 0x4d, 0x89, 0x43, 0x57, 0x2a, 0x6d, 0x80, 0x38, + 0xf9, 0xf7, 0x6f, 0xff, 0x9f, 0xbf, 0xef, 0xf3, 0x6f, 0x83, 0xd7, 0xe3, 0x29, 0x1a, 0x4f, 0x05, + 0xe5, 0x53, 0xca, 0x51, 0x10, 0x33, 0x41, 0x89, 0x14, 0x28, 0x90, 0xe9, 0x10, 0x95, 0xc9, 0x28, + 0xa0, 0x5e, 0xca, 0x99, 0x64, 0xd0, 0x0e, 0x58, 0x30, 0xe6, 0x0c, 0x07, 0x23, 0x6f, 0x3c, 0xf5, + 0x6e, 0x4a, 0xbc, 0x40, 0x66, 0x29, 0xc1, 0x92, 0x5a, 0x9b, 0xff, 0x06, 0xa2, 0x89, 0xe4, 0xe7, + 0x0a, 0xc6, 0x82, 0x15, 0x44, 0x3a, 0x44, 0x04, 0x4b, 0x5c, 0xe7, 0xcc, 0x4c, 0x46, 0x31, 0x1a, + 0xc5, 0x01, 0x92, 0xd1, 0x84, 0x0a, 0x89, 0x27, 0x69, 0xbd, 0xf2, 0x3c, 0x64, 0x21, 0xab, 0x42, + 0x54, 0x46, 0x75, 0xf6, 0x45, 0xc8, 0x58, 0x18, 0x53, 0x84, 0xd3, 0x08, 0xe1, 0x24, 0x61, 0x12, + 0xcb, 0x88, 0x25, 0x42, 0xad, 0x6e, 0xfc, 0x30, 0x80, 0xf1, 0xb9, 0xe2, 0x04, 0x4f, 0x41, 0x2b, + 0x61, 0x84, 0x0e, 0x22, 0x62, 0x6a, 0x5d, 0xcd, 0x5d, 0xe9, 0xef, 0x14, 0xb9, 0x63, 0x1c, 0x32, + 0x42, 0x0f, 0x3e, 0xcc, 0x72, 0x67, 0x3b, 0x8c, 0xe4, 0x28, 0x1b, 0x7a, 0x01, 0x9b, 0xa0, 0xb9, + 0x3a, 0x32, 0xfc, 0x1b, 0xa3, 0x74, 0x1c, 0xa2, 0x9a, 0xb0, 0xa7, 0xca, 0x7c, 0xa3, 0x44, 0x3c, + 0x20, 0xf0, 0x15, 0x68, 0x09, 0x7a, 0x36, 0x48, 0xb2, 0x89, 0xf9, 0xa4, 0xab, 0xb9, 0x7a, 0x1f, + 0xcc, 0x72, 0xc7, 0x38, 0xa6, 0x67, 0x87, 0xd9, 0xc4, 0x37, 0x44, 0x35, 0x42, 0x0b, 0xb4, 0x45, + 0x82, 0x53, 0x31, 0x62, 0xd2, 0xd4, 0xbb, 0x9a, 0xdb, 0xf6, 0xe7, 0x73, 0x48, 0xc1, 0x33, 0x65, + 0xd3, 0x60, 0xae, 0x5a, 0x98, 0xcd, 0xae, 0xee, 0x76, 0x7a, 0x3d, 0xef, 0xff, 0x66, 0x7b, 0x4a, + 0x9f, 0xb7, 0xcf, 0x59, 0x96, 0xaa, 0xb8, 0xdf, 0xbc, 0xc8, 0x9d, 0x86, 0xbf, 0xae, 0x20, 0x4f, + 0xe6, 0x88, 0xd0, 0x07, 0x2d, 0x4e, 0x27, 0x6c, 0x4a, 0x89, 0xb9, 0xd2, 0xd5, 0xdd, 0x95, 0xfe, + 0xfb, 0x59, 0xee, 0xbc, 0x7b, 0x90, 0x72, 0x1f, 0x27, 0x61, 0x29, 0xfd, 0x06, 0x08, 0x0e, 0xc1, + 0x3a, 0x26, 0x84, 0x92, 0x01, 0xe3, 0x03, 0xc5, 0x88, 0x98, 0xc6, 0x83, 0x98, 0x57, 0x90, 0xb7, + 0x98, 0xaf, 0x56, 0x88, 0x9f, 0xb8, 0x4a, 0x12, 0xeb, 0xbb, 0x06, 0x3a, 0x0b, 0xfa, 0xe0, 0x1e, + 0x30, 0x52, 0x16, 0x47, 0xc1, 0x79, 0x75, 0x95, 0xab, 0xbd, 0x37, 0x0b, 0x27, 0xdd, 0x22, 0xba, + 0x7b, 0xdb, 0x81, 0xa3, 0xaa, 0xc8, 0xaf, 0x8b, 0xe1, 0x21, 0x58, 0x5f, 0x76, 0xbd, 0xba, 0xbf, + 0x4e, 0xef, 0xe5, 0x02, 0x60, 0xd9, 0x90, 0xde, 0x28, 0x0e, 0xbc, 0x39, 0x4c, 0xcd, 0x72, 0x6d, + 0xc9, 0x5f, 0xeb, 0xa7, 0x06, 0x3a, 0x0b, 0x62, 0xe0, 0x57, 0xd0, 0xe6, 0xe5, 0xf4, 0xa6, 0xe7, + 0x9a, 0xfd, 0xdd, 0x22, 0x77, 0x5a, 0xb5, 0x85, 0x8f, 0xb0, 0xbe, 0x0a, 0x08, 0xec, 0x02, 0x3d, + 0xc6, 0x51, 0xdd, 0x72, 0xab, 0x45, 0xee, 0xe8, 0x1f, 0x77, 0x0e, 0x66, 0x6a, 0xf0, 0xcb, 0xa5, + 0x05, 0xa3, 0xf4, 0x47, 0x18, 0xb5, 0x01, 0x40, 0xdb, 0xa7, 0x22, 0x65, 0x89, 0xa0, 0x3d, 0x06, + 0xd6, 0x96, 0x36, 0xc3, 0x2f, 0x40, 0xdf, 0xa7, 0x12, 0xba, 0x77, 0xdd, 0xb7, 0x4f, 0x71, 0x50, + 0xbe, 0x4e, 0x6b, 0xf3, 0xae, 0x9d, 0x7b, 0xe5, 0x2f, 0xb1, 0xd1, 0x70, 0xb5, 0xb7, 0x5a, 0x6f, + 0x0a, 0x9e, 0x1e, 0x47, 0x84, 0x9e, 0x70, 0x9c, 0x88, 0x94, 0xf1, 0xf2, 0xb1, 0x74, 0x8e, 0x32, + 0x31, 0x52, 0x26, 0x0b, 0xb8, 0x75, 0xbf, 0x36, 0xb3, 0xee, 0x41, 0x4f, 0x49, 0x54, 0xe7, 0xf6, + 0xb7, 0x2e, 0x7e, 0xdb, 0x8d, 0x8b, 0xc2, 0xd6, 0x2e, 0x0b, 0x5b, 0xbb, 0x2a, 0x6c, 0xed, 0x57, + 0x61, 0x6b, 0xdf, 0xae, 0xed, 0xc6, 0xe5, 0xb5, 0xdd, 0xb8, 0xba, 0xb6, 0x1b, 0xa7, 0xcd, 0xf2, + 0x47, 0x1b, 0x1a, 0xd5, 0x57, 0xb3, 0xfd, 0x27, 0x00, 0x00, 0xff, 0xff, 0x2d, 0x8d, 0x65, 0x3b, + 0x3e, 0x05, 0x00, 0x00, } diff --git a/pkg/kv/kvserver/closedts/ctpb/service.proto b/pkg/kv/kvserver/closedts/ctpb/service.proto index cf7df9498d79..5a976d87e29f 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,80 @@ import "google/api/annotations.proto"; service ClosedTimestamp { rpc Get(stream Reaction) returns (stream Entry) { } } + +// Update 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 Update { + // 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) = "SeqNum"]; + + // 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 sender is + // temporarily slowed down or if the stream experience network problems and + // some incremental messages are dropped (although generally we expect that + // to result in a stream failing and a new one being established). + bool snapshot = 3; + + // closed_timestamps 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, as are races between two + // side-transport streams for an outgoing and incoming leaseholder. + message GroupUpdate { + roachpb.RangeClosedTimestampPolicy policy = 1; + util.hlc.Timestamp closed_timestamp = 2 [(gogoproto.nullable) = false]; + } + repeated GroupUpdate closed_timestamps = 4 [(gogoproto.nullable) = false]; + + // removed contains the set of ranges that are no longer registered on the + // stream and who future updates are no longer applicable to. + // + // The field will be empty if snapshot is true, as a snapshot message implies + // that all ranges not present in the snapshot's added_or_updated list are no + // longer tracked. + repeated int32 removed = 5 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.RangeID"]; + + // added_or_updated contains the set of ranges that are either being added to + // the tracked ranges set with a given (lai, policy) or updated within the + // tracked range set with a new (lai, policy). All future updates on the + // stream are applicable to these ranges until they are removed, either + // explicitly by being included in a future removed set or implicitly by not + // being included in the added_or_updated field of a future snapshot. + 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; + } + repeated RangeUpdate added_or_updated = 6 [(gogoproto.nullable) = false]; +} + +message Response {} + +service SideTransport { + rpc PushUpdates(stream Update) returns (stream Response) { } +} diff --git a/pkg/kv/kvserver/closedts/policy.go b/pkg/kv/kvserver/closedts/policy.go new file mode 100644 index 000000000000..509eb9ebe6a7 --- /dev/null +++ b/pkg/kv/kvserver/closedts/policy.go @@ -0,0 +1,42 @@ +// 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 closedts + +import ( + "time" + + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/util/hlc" +) + +// TargetForPolicy returns the target closed timestamp for a range with the +// given policy. +func TargetForPolicy( + now hlc.ClockTimestamp, + maxClockOffset time.Duration, + lagTargetDuration time.Duration, + policy roachpb.RangeClosedTimestampPolicy, +) hlc.Timestamp { + switch policy { + case roachpb.LAG_BY_CLUSTER_SETTING, roachpb.LEAD_FOR_GLOBAL_READS: + return hlc.Timestamp{WallTime: now.WallTime - lagTargetDuration.Nanoseconds()} + // TODO(andrei,nvanbenschoten): Resolve all the issues preventing us from closing + // timestamps in the future (which, in turn, forces future-time writes on + // global ranges), and enable the proper logic below. + //case roachpb.LEAD_FOR_GLOBAL_READS: + // closedTSTarget = hlc.Timestamp{ + // WallTime: now + 2*maxClockOffset.Nanoseconds(), + // Synthetic: true, + // } + default: + panic("unexpected RangeClosedTimestampPolicy") + } +} diff --git a/pkg/kv/kvserver/closedts/policy_test.go b/pkg/kv/kvserver/closedts/policy_test.go new file mode 100644 index 000000000000..93f133bd033e --- /dev/null +++ b/pkg/kv/kvserver/closedts/policy_test.go @@ -0,0 +1,53 @@ +// 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 closedts + +import ( + "testing" + + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/stretchr/testify/require" +) + +func TestTargetForPolicy(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + const nowNanos = 100 + const maxOffsetNanos = 20 + const lagTargetNanos = 10 + + for _, tc := range []struct { + rangePolicy roachpb.RangeClosedTimestampPolicy + expClosedTSTarget hlc.Timestamp + }{ + { + rangePolicy: roachpb.LAG_BY_CLUSTER_SETTING, + expClosedTSTarget: hlc.Timestamp{WallTime: nowNanos - lagTargetNanos}, + }, + { + rangePolicy: roachpb.LEAD_FOR_GLOBAL_READS, + expClosedTSTarget: hlc.Timestamp{WallTime: nowNanos - lagTargetNanos}, + // TODO(andrei, nvanbenschoten): What we should be expecting here is the following, once + // the propBuf starts properly implementing this timestamp closing policy: + // expClosedTSTarget: hlc.Timestamp{WallTime: nowNanos + 2*maxOffsetNanos, Synthetic: true}, + }, + } { + t.Run(tc.rangePolicy.String(), func(t *testing.T) { + now := hlc.ClockTimestamp{WallTime: nowNanos} + target := TargetForPolicy(now, maxOffsetNanos, lagTargetNanos, tc.rangePolicy) + require.Equal(t, tc.expClosedTSTarget, target) + }) + } +} diff --git a/pkg/kv/kvserver/closedts/setting.go b/pkg/kv/kvserver/closedts/setting.go index e0554ef43f4c..617bcbff014c 100644 --- a/pkg/kv/kvserver/closedts/setting.go +++ b/pkg/kv/kvserver/closedts/setting.go @@ -37,3 +37,12 @@ var CloseFraction = settings.RegisterFloatSetting( } return nil }) + +// SideTransportCloseInterval determines the ClosedTimestampSender's frequency. +var SideTransportCloseInterval = settings.RegisterDurationSetting( + "kv.closed_timestamp.side_transport_interval", + "the interval at which the closed-timestamp side-transport attempts to "+ + "advance each range's closed timestamp; set to 0 to disable the side-transport.", + 200*time.Millisecond, + settings.NonNegativeDuration, +) diff --git a/pkg/kv/kvserver/closedts/sidetransport/BUILD.bazel b/pkg/kv/kvserver/closedts/sidetransport/BUILD.bazel new file mode 100644 index 000000000000..95e711e46203 --- /dev/null +++ b/pkg/kv/kvserver/closedts/sidetransport/BUILD.bazel @@ -0,0 +1,39 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") + +go_library( + name = "sidetransport", + srcs = ["sender.go"], + importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/sidetransport", + visibility = ["//visibility:public"], + deps = [ + "//pkg/clusterversion", + "//pkg/kv/kvserver/closedts", + "//pkg/kv/kvserver/closedts/ctpb", + "//pkg/roachpb", + "//pkg/rpc", + "//pkg/rpc/nodedialer", + "//pkg/settings/cluster", + "//pkg/util", + "//pkg/util/hlc", + "//pkg/util/log", + "//pkg/util/stop", + "//pkg/util/syncutil", + "//pkg/util/timeutil", + ], +) + +go_test( + name = "sidetransport_test", + srcs = ["sender_test.go"], + embed = [":sidetransport"], + deps = [ + "//pkg/kv/kvserver/closedts", + "//pkg/kv/kvserver/closedts/ctpb", + "//pkg/roachpb", + "//pkg/settings/cluster", + "//pkg/util/hlc", + "//pkg/util/leaktest", + "//pkg/util/stop", + "@com_github_stretchr_testify//require", + ], +) diff --git a/pkg/kv/kvserver/closedts/sidetransport/sender.go b/pkg/kv/kvserver/closedts/sidetransport/sender.go new file mode 100644 index 000000000000..c82d3230efd1 --- /dev/null +++ b/pkg/kv/kvserver/closedts/sidetransport/sender.go @@ -0,0 +1,698 @@ +// 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" + "io" + "sync" + "sync/atomic" + "time" + + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "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" +) + +// Sender 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 Sender 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 Sender struct { + stopper *stop.Stopper + st *cluster.Settings + clock *hlc.Clock + nodeID roachpb.NodeID + + trackedMu struct { + syncutil.Mutex + // lastSeqNum is the sequence number of the last message published. + lastSeqNum ctpb.SeqNum + // lastClosed is the closed timestamp published for each policy in the + // last message. + lastClosed [roachpb.MAX_CLOSED_TIMESTAMP_POLICY]hlc.Timestamp + // 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 Update.Removed. + tracked map[roachpb.RangeID]trackedRange + } + + 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 + + // connFactory is used to establish new connections. + connFactory connFactory + // conns contains connections to all nodes with follower replicas of any of + // the registered leaseholder. 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. + conns map[roachpb.NodeID]conn +} + +// trackedRange contains the information that the side-transport last published +// about a particular range. +type trackedRange struct { + lai ctpb.LAI + policy roachpb.RangeClosedTimestampPolicy +} + +// leaseholder represents a leaseholder replicas that has been registered with +// the sender and can send closed timestamp updates through the side transport. +type leaseholder struct { + Replica + leaseSeq roachpb.LeaseSequence +} + +// Replica represents a *Replica object, but with only the capabilities needed +// by the closed timestamp side transport to accomplish its job. +type Replica interface { + // Accessors. + StoreID() roachpb.StoreID + GetRangeID() roachpb.RangeID + Desc() *roachpb.RangeDescriptor + + // 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. + BumpSideTransportClosed( + ctx context.Context, + now hlc.ClockTimestamp, + targetByPolicy [roachpb.MAX_CLOSED_TIMESTAMP_POLICY]hlc.Timestamp, + ) (bool, ctpb.LAI, roachpb.RangeClosedTimestampPolicy) +} + +// NewSender creates a Sender. Run must be called on it afterwards to get it to +// start publishing closed timestamps. +func NewSender( + stopper *stop.Stopper, st *cluster.Settings, clock *hlc.Clock, dialer *nodedialer.Dialer, +) *Sender { + connFactory := newRPCConnFactory(dialer) + return newSenderWithConnFactory(stopper, st, clock, connFactory) +} + +func newSenderWithConnFactory( + stopper *stop.Stopper, st *cluster.Settings, clock *hlc.Clock, connFactory connFactory, +) *Sender { + s := &Sender{ + stopper: stopper, + st: st, + clock: clock, + buf: newUpdatesBuf(), + connFactory: connFactory, + } + s.trackedMu.tracked = make(map[roachpb.RangeID]trackedRange) + s.leaseholdersMu.leaseholders = make(map[roachpb.RangeID]leaseholder) + s.conns = make(map[roachpb.NodeID]conn) + 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 *Sender) Run(ctx context.Context, nodeID roachpb.NodeID) { + s.nodeID = nodeID + + confCh := make(chan struct{}, 1) + confChanged := func() { + select { + case confCh <- struct{}{}: + default: + } + } + closedts.SideTransportCloseInterval.SetOnChange(&s.st.SV, confChanged) + + _ /* err */ = s.stopper.RunAsyncTask(ctx, "closedts side-transport publisher", + func(ctx context.Context) { + defer func() { + // Closing the buffer signals all connections to quit. + s.buf.Close() + }() + + var timer timeutil.Timer + defer timer.Stop() + for { + interval := closedts.SideTransportCloseInterval.Get(&s.st.SV) + if interval > 0 { + timer.Reset(closedts.SideTransportCloseInterval.Get(&s.st.SV)) + } else { + // Disable the side-transport. + timer.Stop() + } + select { + case <-timer.C: + timer.Read = true + if !s.st.Version.IsActive(ctx, clusterversion.ClosedTimestampsRaftTransport) { + continue + } + s.publish(ctx) + case <-confCh: + // Loop around to use the updated timer. + continue + case <-s.stopper.ShouldQuiesce(): + return + } + } + }) +} + +// RegisterLeaseholder adds a replica to the leaseholders collection. From now +// on, until the replica is unregistered, the side-transport will try to advance +// this replica's closed timestamp. +func (s *Sender) RegisterLeaseholder( + ctx context.Context, r Replica, leaseSeq roachpb.LeaseSequence, +) { + s.leaseholdersMu.Lock() + defer s.leaseholdersMu.Unlock() + + if lh, ok := s.leaseholdersMu.leaseholders[r.GetRangeID()]; ok { + // The leaseholder is already registered. If we're already aware of this + // or a newer lease, there's nothing to do. + if lh.leaseSeq >= leaseSeq { + return + } + // Otherwise, update the leaseholder, which may be different object if + // the lease moved between replicas for the same range on the same node + // but on different stores. + } + s.leaseholdersMu.leaseholders[r.GetRangeID()] = leaseholder{ + Replica: r, + leaseSeq: leaseSeq, + } +} + +// UnregisterLeaseholder removes a replica from the leaseholders collection, if +// the replica is currently tracked. +func (s *Sender) UnregisterLeaseholder( + ctx context.Context, storeID roachpb.StoreID, rangeID roachpb.RangeID, +) { + 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 *Sender) publish(ctx context.Context) hlc.ClockTimestamp { + s.trackedMu.Lock() + defer s.trackedMu.Unlock() + + msg := &ctpb.Update{ + NodeID: s.nodeID, + ClosedTimestamps: make([]ctpb.Update_GroupUpdate, len(s.trackedMu.lastClosed)), + } + + // Determine the message's sequence number. + s.trackedMu.lastSeqNum++ + msg.SeqNum = s.trackedMu.lastSeqNum + // The first message produced is essentially a snapshot, since it has no + // previous state to reference. + msg.Snapshot = msg.SeqNum == 1 + + // 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. + now := s.clock.NowAsClockTimestamp() + maxClockOffset := s.clock.MaxOffset() + lagTargetDuration := closedts.TargetDuration.Get(&s.st.SV) + for i := range s.trackedMu.lastClosed { + pol := roachpb.RangeClosedTimestampPolicy(i) + target := closedts.TargetForPolicy(now, maxClockOffset, lagTargetDuration, pol) + s.trackedMu.lastClosed[pol] = target + msg.ClosedTimestamps[pol] = ctpb.Update_GroupUpdate{ + Policy: roachpb.RangeClosedTimestampPolicy(pol), + ClosedTimestamp: target, + } + } + + // 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 Sender 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() + + // 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() + + // 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) + } + } + + // Iterate through each leaseholder and determine whether it can be part of + // this update or not. + for _, lh := range leaseholders { + lhRangeID := lh.GetRangeID() + lastMsg, tracked := s.trackedMu.tracked[lhRangeID] + + // 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 lh.Desc().Replicas().VoterFullAndNonVoterDescriptors() { + nodesWithFollowers.Add(int(repl.NodeID)) + } + + // Check whether the desired timestamp can be closed on this range. + canClose, lai, policy := lh.BumpSideTransportClosed(ctx, now, s.trackedMu.lastClosed) + 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, lhRangeID) + delete(s.trackedMu.tracked, lhRangeID) + } + 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.Update_RangeUpdate{ + RangeID: lhRangeID, + LAI: lai, + Policy: policy, + }) + s.trackedMu.tracked[lhRangeID] = trackedRange{lai: lai, policy: policy} + } + } + + // 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, c := range s.conns { + if !nodesWithFollowers.Contains(int(nodeID)) { + delete(s.conns, nodeID) + c.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're closing are written directly to the sideTransportClosedTimestamp + // fields of the local replicas in BumpSideTransportClosed. + if _, ok := s.conns[nodeID]; !ok && nodeID != s.nodeID { + c := s.connFactory.new(s, nodeID) + c.run(ctx, s.stopper) + s.conns[nodeID] = c + } + }) + + // Publish the new message to all connections. + s.buf.Push(ctx, msg) + + // Return the publication time, for tests. + return now +} + +// 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 *Sender) GetSnapshot() *ctpb.Update { + s.trackedMu.Lock() + defer s.trackedMu.Unlock() + + msg := &ctpb.Update{ + NodeID: s.nodeID, + // 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, + Snapshot: true, + ClosedTimestamps: make([]ctpb.Update_GroupUpdate, len(s.trackedMu.lastClosed)), + AddedOrUpdated: make([]ctpb.Update_RangeUpdate, 0, len(s.trackedMu.tracked)), + } + for pol, ts := range s.trackedMu.lastClosed { + msg.ClosedTimestamps[pol] = ctpb.Update_GroupUpdate{ + Policy: roachpb.RangeClosedTimestampPolicy(pol), + ClosedTimestamp: ts, + } + } + for rid, r := range s.trackedMu.tracked { + msg.AddedOrUpdated = append(msg.AddedOrUpdated, ctpb.Update_RangeUpdate{ + RangeID: rid, + LAI: r.lai, + Policy: r.policy, + }) + } + return msg +} + +// updatesBuf is a circular buffer of Updates. 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 + // updated is signaled when a new item is inserted. + updated sync.Cond + // data contains pointers to the Updates. + data []*ctpb.Update + // 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 + // closed is set by the producer to signal the consumers to exit. + closed bool + } +} + +// Size the buffer such that a stream sender goroutine can be blocked for a +// little while and not have to send a snapshot when it resumes. +const updatesBufSize = 50 + +func newUpdatesBuf() *updatesBuf { + buf := &updatesBuf{} + buf.mu.updated.L = &buf.mu + buf.mu.data = make([]*ctpb.Update, updatesBufSize) + return buf +} + +// Push adds a new update to the back of the buffer. +func (b *updatesBuf) Push(ctx context.Context, update *ctpb.Update) { + b.mu.Lock() + defer b.mu.Unlock() + + // If the buffer is not empty, sanity check the seq num. + if b.sizeLocked() != 0 { + lastIdx := b.lastIdxLocked() + 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) + } + } + + overwrite := b.fullLocked() + b.mu.data[b.mu.tail] = update + 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() +} + +func (b *updatesBuf) lastIdxLocked() int { + lastIdx := b.mu.tail - 1 + if lastIdx < 0 { + lastIdx += len(b.mu.data) + } + return lastIdx +} + +// 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. +// +// The bool retval is set to false if the producer has closed the buffer. In +// that case, the consumers should quit. +func (b *updatesBuf) GetBySeq(ctx context.Context, seqNum ctpb.SeqNum) (*ctpb.Update, bool) { + b.mu.Lock() + defer b.mu.Unlock() + + // Loop until the requested seqNum is added to the buffer. + for { + if b.mu.closed { + return nil, false + } + + var firstSeq, lastSeq ctpb.SeqNum + if b.sizeLocked() == 0 { + firstSeq, lastSeq = 0, 0 + } else { + firstSeq, lastSeq = b.mu.data[b.mu.head].SeqNum, b.mu.data[b.lastIdxLocked()].SeqNum + } + if seqNum < firstSeq { + // Requesting a message that's not in the buffer any more. + return nil, true + } + // 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], true + } +} + +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. + // Since there's no popping from the buffer, it can only be empty if nothing + // was ever pushed to it. + 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) +} + +// Close unblocks all the consumers and signals them to exit. +func (b *updatesBuf) Close() { + b.mu.Lock() + defer b.mu.Unlock() + b.mu.closed = true + b.mu.updated.Broadcast() +} + +// connFactory is capable of creating new connections to specific nodes. +type connFactory interface { + new(*Sender, roachpb.NodeID) conn +} + +// conn is a side-transport connection to a node. A conn watches an updatesBuf +// and streams all the messages to the respective node. +type conn interface { + run(context.Context, *stop.Stopper) + close() +} + +// rpcConnFactory is an implementation of connFactory that establishes +// connections to other nodes using gRPC. +type rpcConnFactory struct { + dialer *nodedialer.Dialer +} + +func newRPCConnFactory(dialer *nodedialer.Dialer) connFactory { + return &rpcConnFactory{ + dialer: dialer, + } +} + +// new implements the connFactory interface. +func (f *rpcConnFactory) new(s *Sender, nodeID roachpb.NodeID) conn { + return newRPCConn(f.dialer, s, nodeID) +} + +// rpcConn is an implementation of conn that is implemented using a gRPC stream. +// +// The connection will read messages from producer.buf. 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. +type rpcConn struct { + log.AmbientContext + dialer *nodedialer.Dialer + producer *Sender + nodeID roachpb.NodeID + stream ctpb.SideTransport_PushUpdatesClient + closed int32 // atomic +} + +func newRPCConn(dialer *nodedialer.Dialer, producer *Sender, nodeID roachpb.NodeID) conn { + r := &rpcConn{ + dialer: dialer, + producer: producer, + nodeID: nodeID, + } + r.AddLogTag("ctstream", nodeID) + return r +} + +// close makes the connection stop sending messages. The run() goroutine will +// exit asynchronously. The parent Sender is expected to remove this connection +// from its list. +func (r *rpcConn) close() { + atomic.StoreInt32(&r.closed, 1) +} + +func (r *rpcConn) sendMsg(ctx context.Context, msg *ctpb.Update) error { + if r.stream == nil { + conn, err := r.dialer.Dial(ctx, r.nodeID, rpc.SystemClass) + if err != nil { + return err + } + r.stream, err = ctpb.NewSideTransportClient(conn).PushUpdates(ctx) + if err != nil { + return err + } + } + return r.stream.Send(msg) +} + +// run implements the conn interface. +func (r *rpcConn) 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 = r.AnnotateCtx(ctx) + + defer func() { + if r.stream != nil { + _ /* err */ = r.stream.CloseSend() + r.stream = nil + } + }() + + everyN := log.Every(10 * time.Second) + + var lastSent ctpb.SeqNum + for { + var msg *ctpb.Update + + // If we've been disconnected, reset the message sequence. We'll ask the + // buffer for the very first message ever, which was a snapshot. + // Generally, the buffer is not going to have that message any more and + // so we'll generate a snapshot below. Except soon after startup when + // streams are initially established, when the initial message should + // still be in the buffer. + if r.stream == nil { + lastSent = 0 + } + + var ok bool + msg, ok = r.producer.buf.GetBySeq(ctx, lastSent+1) + // We can be signaled to stop in two ways: the buffer can be closed (in + // which case all connections must exit), or this connection was closed + // via close(). In either case, we quit. + if !ok { + return + } + closed := atomic.LoadInt32(&r.closed) > 0 + if closed { + return + } + + 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() + } + + lastSent = msg.SeqNum + if err := r.sendMsg(ctx, msg); err != nil { + if err != io.EOF && 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/closedts/sidetransport/sender_test.go b/pkg/kv/kvserver/closedts/sidetransport/sender_test.go new file mode 100644 index 000000000000..ff2dfb476a2f --- /dev/null +++ b/pkg/kv/kvserver/closedts/sidetransport/sender_test.go @@ -0,0 +1,209 @@ +// 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" + "testing" + "time" + + "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/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/stop" + "github.com/stretchr/testify/require" +) + +// mockReplica is a mock implementation of the Replica interface. +type mockReplica struct { + storeID roachpb.StoreID + rangeID roachpb.RangeID + desc roachpb.RangeDescriptor + + canBump bool + lai ctpb.LAI + policy roachpb.RangeClosedTimestampPolicy +} + +func (m *mockReplica) StoreID() roachpb.StoreID { return m.storeID } +func (m *mockReplica) GetRangeID() roachpb.RangeID { return m.rangeID } +func (m *mockReplica) Desc() *roachpb.RangeDescriptor { return &m.desc } +func (m *mockReplica) BumpSideTransportClosed( + _ context.Context, _ hlc.ClockTimestamp, _ [roachpb.MAX_CLOSED_TIMESTAMP_POLICY]hlc.Timestamp, +) (bool, ctpb.LAI, roachpb.RangeClosedTimestampPolicy) { + return m.canBump, m.lai, m.policy +} + +// mockConnFactory is a mock implementation of the connFactory interface. +type mockConnFactory struct{} + +func (f *mockConnFactory) new(_ *Sender, nodeID roachpb.NodeID) conn { + return &mockConn{nodeID: nodeID} +} + +// mockConn is a mock implementation of the conn interface. +type mockConn struct { + nodeID roachpb.NodeID + running bool + closed bool +} + +func (c *mockConn) run(context.Context, *stop.Stopper) { c.running = true } +func (c *mockConn) close() { c.closed = true } + +func newMockSender() (*Sender, *stop.Stopper) { + stopper := stop.NewStopper() + st := cluster.MakeTestingClusterSettings() + clock := hlc.NewClock(hlc.UnixNano, time.Nanosecond) + connFactory := &mockConnFactory{} + s := newSenderWithConnFactory(stopper, st, clock, connFactory) + s.nodeID = 1 // usually set in (*Sender).Run + return s, stopper +} + +func newMockReplica(id roachpb.RangeID, nodes ...roachpb.NodeID) *mockReplica { + var desc roachpb.RangeDescriptor + desc.RangeID = id + for _, nodeID := range nodes { + desc.AddReplica(nodeID, roachpb.StoreID(nodeID), roachpb.VOTER_FULL) + } + return &mockReplica{ + storeID: 1, + rangeID: id, + desc: desc, + canBump: true, + lai: 5, + policy: roachpb.LAG_BY_CLUSTER_SETTING, + } +} + +func expGroupUpdates(s *Sender, now hlc.ClockTimestamp) []ctpb.Update_GroupUpdate { + maxClockOffset := s.clock.MaxOffset() + lagTargetDuration := closedts.TargetDuration.Get(&s.st.SV) + targetForPolicy := func(pol roachpb.RangeClosedTimestampPolicy) hlc.Timestamp { + return closedts.TargetForPolicy(now, maxClockOffset, lagTargetDuration, pol) + } + return []ctpb.Update_GroupUpdate{ + {roachpb.LAG_BY_CLUSTER_SETTING, targetForPolicy(roachpb.LAG_BY_CLUSTER_SETTING)}, + {roachpb.LEAD_FOR_GLOBAL_READS, targetForPolicy(roachpb.LEAD_FOR_GLOBAL_READS)}, + } +} + +func TestSenderBasic(t *testing.T) { + defer leaktest.AfterTest(t)() + ctx := context.Background() + s, stopper := newMockSender() + defer stopper.Stop(ctx) + + // No leaseholders. + now := s.publish(ctx) + require.Len(t, s.trackedMu.tracked, 0) + require.Len(t, s.leaseholdersMu.leaseholders, 0) + require.Len(t, s.conns, 0) + + require.Equal(t, ctpb.SeqNum(1), s.trackedMu.lastSeqNum) + up, ok := s.buf.GetBySeq(ctx, 1) + require.True(t, ok) + require.Equal(t, roachpb.NodeID(1), up.NodeID) + require.Equal(t, ctpb.SeqNum(1), up.SeqNum) + require.True(t, up.Snapshot) + require.Equal(t, expGroupUpdates(s, now), up.ClosedTimestamps) + require.Nil(t, up.Removed) + require.Nil(t, up.AddedOrUpdated) + + // Add a leaseholder that can close. + r1 := newMockReplica(15, 1, 2, 3) + s.RegisterLeaseholder(ctx, r1, 1) + now = s.publish(ctx) + require.Len(t, s.trackedMu.tracked, 1) + require.Equal(t, map[roachpb.RangeID]trackedRange{ + 15: {lai: 5, policy: roachpb.LAG_BY_CLUSTER_SETTING}, + }, s.trackedMu.tracked) + require.Len(t, s.leaseholdersMu.leaseholders, 1) + require.Len(t, s.conns, 2) + + require.Equal(t, ctpb.SeqNum(2), s.trackedMu.lastSeqNum) + up, ok = s.buf.GetBySeq(ctx, 2) + require.True(t, ok) + require.Equal(t, roachpb.NodeID(1), up.NodeID) + require.Equal(t, ctpb.SeqNum(2), up.SeqNum) + require.Equal(t, false, up.Snapshot) + require.Equal(t, expGroupUpdates(s, now), up.ClosedTimestamps) + require.Nil(t, up.Removed) + require.Equal(t, []ctpb.Update_RangeUpdate{ + {RangeID: 15, LAI: 5, Policy: roachpb.LAG_BY_CLUSTER_SETTING}, + }, up.AddedOrUpdated) + + c2, ok := s.conns[2] + require.True(t, ok) + require.Equal(t, &mockConn{nodeID: 2, running: true, closed: false}, c2.(*mockConn)) + c3, ok := s.conns[3] + require.True(t, ok) + require.Equal(t, &mockConn{nodeID: 3, running: true, closed: false}, c3.(*mockConn)) + + // The leaseholder can not close the next timestamp. + r1.canBump = false + now = s.publish(ctx) + require.Len(t, s.trackedMu.tracked, 0) + require.Len(t, s.leaseholdersMu.leaseholders, 1) + require.Len(t, s.conns, 2) + + require.Equal(t, ctpb.SeqNum(3), s.trackedMu.lastSeqNum) + up, ok = s.buf.GetBySeq(ctx, 3) + require.True(t, ok) + require.Equal(t, roachpb.NodeID(1), up.NodeID) + require.Equal(t, ctpb.SeqNum(3), up.SeqNum) + require.Equal(t, false, up.Snapshot) + require.Equal(t, expGroupUpdates(s, now), up.ClosedTimestamps) + require.Equal(t, []roachpb.RangeID{15}, up.Removed) + require.Nil(t, up.AddedOrUpdated) + + // The leaseholder loses its lease. + s.UnregisterLeaseholder(ctx, 1, 15) + now = s.publish(ctx) + require.Len(t, s.trackedMu.tracked, 0) + require.Len(t, s.leaseholdersMu.leaseholders, 0) + require.Len(t, s.conns, 0) + + require.Equal(t, ctpb.SeqNum(4), s.trackedMu.lastSeqNum) + up, ok = s.buf.GetBySeq(ctx, 4) + require.True(t, ok) + require.Equal(t, roachpb.NodeID(1), up.NodeID) + require.Equal(t, ctpb.SeqNum(4), up.SeqNum) + require.Equal(t, false, up.Snapshot) + require.Equal(t, expGroupUpdates(s, now), up.ClosedTimestamps) + require.Nil(t, up.Removed) + require.Nil(t, up.AddedOrUpdated) + + require.True(t, c2.(*mockConn).closed) + require.True(t, c3.(*mockConn).closed) +} + +func TestSenderConnectionChanges(t *testing.T) { + // TODO: Two ranges. + // Add follower for range 1: 2, 3. + // - check conns to 2 and 3. + // Add follower for range 2: 3, 4. + // - check conns to 2, 3, 4. + // Remove followers for range 2, 3. + // - check conns to 3, 4. + // Remove followers for range 3. + // - check conns to 4. +} + +func TestSenderSameRangeDifferentStores(t *testing.T) { + // TODO: Two replicas, different stores, same replica. +} + +// TODO(andrei): add test for updatesBuf. diff --git a/pkg/kv/kvserver/kvserverbase/base.go b/pkg/kv/kvserver/kvserverbase/base.go index 5f9e394d3bc7..4967c27f8a25 100644 --- a/pkg/kv/kvserver/kvserverbase/base.go +++ b/pkg/kv/kvserver/kvserverbase/base.go @@ -67,6 +67,7 @@ type ApplyFilterArgs struct { CmdID CmdIDKey RangeID roachpb.RangeID StoreID roachpb.StoreID + Req *roachpb.BatchRequest // only set on the leaseholder } // InRaftCmd returns true if the filter is running in the context of a Raft diff --git a/pkg/kv/kvserver/replica.go b/pkg/kv/kvserver/replica.go index 13368fe4e44d..571048001e3d 100644 --- a/pkg/kv/kvserver/replica.go +++ b/pkg/kv/kvserver/replica.go @@ -387,13 +387,41 @@ type Replica struct { // // TODO(ajwerner): move the proposal map and ProposalData entirely under // the raftMu. - proposals map[kvserverbase.CmdIDKey]*ProposalData + proposals map[kvserverbase.CmdIDKey]*ProposalData + // Indicates that the replica is in the process of applying log entries. + // Updated to true in handleRaftReady before entries are removed from + // the proposals map and set to false after they are applied. Useful in + // conjunction with len(proposals) to check for any in-flight proposals + // whose effects have not yet taken hold without synchronizing with + // raftMu and the entire handleRaftReady loop. Not needed if raftMu is + // already held. + applyingEntries bool + // The replica's Raft group "node". internalRaftGroup *raft.RawNode // The ID of the replica within the Raft group. This value may never be 0. replicaID roachpb.ReplicaID // 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. + // + // TODO(andrei): document here and probably elsewhere the relationship + // between the sideTransportClosedTimestamp and the raftClosedTimestamp. + // Specifically that for a given LAI, the side transport closed timestamp + // will always lead the raft closed timestamp, but that across LAIs, the + // larger LAI will always include the larger closed timestamp, independent + // of the source. + sideTransportClosedTimestamp hlc.Timestamp + // sideTransportCloseTimestampLAI is the lease-applied index associated + // with sideTransportClosedTimestamp. + sideTransportCloseTimestampLAI ctpb.LAI // The ID of the leader replica within the Raft group. Used to determine // when the leadership changes. diff --git a/pkg/kv/kvserver/replica_application_result.go b/pkg/kv/kvserver/replica_application_result.go index 45fba8ea0cd3..1d212e8d9c24 100644 --- a/pkg/kv/kvserver/replica_application_result.go +++ b/pkg/kv/kvserver/replica_application_result.go @@ -119,6 +119,7 @@ func (r *Replica) prepareLocalResult(ctx context.Context, cmd *replicatedCmd) { ReplicatedEvalResult: *cmd.replicatedResult(), StoreID: r.store.StoreID(), RangeID: r.RangeID, + Req: cmd.proposal.Request, }) if cmd.proposalRetry == 0 { cmd.proposalRetry = proposalReevaluationReason(newPropRetry) diff --git a/pkg/kv/kvserver/replica_application_state_machine.go b/pkg/kv/kvserver/replica_application_state_machine.go index 5c2eb7bd8325..2aa5acb0f544 100644 --- a/pkg/kv/kvserver/replica_application_state_machine.go +++ b/pkg/kv/kvserver/replica_application_state_machine.go @@ -134,13 +134,17 @@ func (r *Replica) shouldApplyCommand( ctx, cmd.idKey, &cmd.raftCmd, cmd.IsLocal(), replicaState, ) if filter := r.store.cfg.TestingKnobs.TestingApplyFilter; cmd.forcedErr == nil && filter != nil { - var newPropRetry int - newPropRetry, cmd.forcedErr = filter(kvserverbase.ApplyFilterArgs{ + args := kvserverbase.ApplyFilterArgs{ CmdID: cmd.idKey, ReplicatedEvalResult: *cmd.replicatedResult(), StoreID: r.store.StoreID(), RangeID: r.RangeID, - }) + } + if cmd.IsLocal() { + args.Req = cmd.proposal.Request + } + var newPropRetry int + newPropRetry, cmd.forcedErr = filter(args) if cmd.proposalRetry == 0 { cmd.proposalRetry = proposalReevaluationReason(newPropRetry) } diff --git a/pkg/kv/kvserver/replica_closedts.go b/pkg/kv/kvserver/replica_closedts.go index 3d29397537b3..5bfc5619e782 100644 --- a/pkg/kv/kvserver/replica_closedts.go +++ b/pkg/kv/kvserver/replica_closedts.go @@ -12,7 +12,6 @@ package kvserver import ( "context" - "time" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" @@ -62,35 +61,92 @@ 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, +) (ok bool, _ ctpb.LAI, _ roachpb.RangeClosedTimestampPolicy) { + r.mu.Lock() + defer r.mu.Unlock() + + // This method can be called even after a Replica is destroyed and removed + // from the Store's replicas map, because unlinkReplicaByRangeIDLocked does + // not synchronize with sidetransport.Sender.publish, which maintains a + // local copy of its leaseholder map. To avoid issues resulting from this, + // we first check if the replica is destroyed. + if _, err := r.isDestroyedRLocked(); err != nil { + return false, 0, 0 + } + + lai := ctpb.LAI(r.mu.state.LeaseAppliedIndex) + 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 is merging into its left-hand neighbor, we can't close + // timestamps any more because the joint-range would not be aware of reads + // performed based on this advanced closed timestamp. + if r.mergeInProgressRLocked() { + return false, 0, 0 + } + + // If there are pending Raft proposals in-flight or committed entries that + // have yet to be applied, 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 || r.mu.applyingEntries { + return false, 0, 0 + } + + // MaybeForwardClosedLocked checks that there are no evaluating requests + // writing under target. + if !r.mu.proposalBuf.MaybeForwardClosedLocked(ctx, target) { + return false, 0, 0 + } + + // 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, 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. func (r *Replica) closedTimestampTargetRLocked() hlc.Timestamp { now := r.Clock().NowAsClockTimestamp() - policy := r.closedTimestampPolicyRLocked() + maxClockOffset := r.Clock().MaxOffset() lagTargetDuration := closedts.TargetDuration.Get(&r.ClusterSettings().SV) - return closedTimestampTargetByPolicy(now, policy, lagTargetDuration) -} - -// closedTimestampTargetByPolicy returns the target closed timestamp for a range -// with the given policy. -func closedTimestampTargetByPolicy( - now hlc.ClockTimestamp, - policy roachpb.RangeClosedTimestampPolicy, - lagTargetDuration time.Duration, -) hlc.Timestamp { - var closedTSTarget hlc.Timestamp - switch policy { - case roachpb.LAG_BY_CLUSTER_SETTING, roachpb.LEAD_FOR_GLOBAL_READS: - closedTSTarget = hlc.Timestamp{WallTime: now.WallTime - lagTargetDuration.Nanoseconds()} - // TODO(andrei,nvanbenschoten): Resolve all the issues preventing us from closing - // timestamps in the future (which, in turn, forces future-time writes on - // global ranges), and enable the proper logic below. - //case roachpb.LEAD_FOR_GLOBAL_READS: - // closedTSTarget = hlc.Timestamp{ - // WallTime: now + 2*b.clock.MaxOffset().Nanoseconds(), - // Synthetic: true, - // } - } - return closedTSTarget + policy := r.closedTimestampPolicyRLocked() + return closedts.TargetForPolicy(now, maxClockOffset, lagTargetDuration, policy) } diff --git a/pkg/kv/kvserver/replica_closedts_test.go b/pkg/kv/kvserver/replica_closedts_test.go index 7ff1d17a1e44..857459ac6118 100644 --- a/pkg/kv/kvserver/replica_closedts_test.go +++ b/pkg/kv/kvserver/replica_closedts_test.go @@ -1,43 +1,424 @@ -package kvserver +// 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_test import ( + "context" + "sync" "testing" + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/errors" "github.com/stretchr/testify/require" ) -func TestClosedTimestampTargetByPolicy(t *testing.T) { +// TestBumpSideTransportClosed tests the various states that a replica can find +// itself in when its TestBumpSideTransportClosed is called. It verifies that +// the method only returns successfully if it can bump its closed timestamp to +// the target. +func TestBumpSideTransportClosed(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) + ctx := context.Background() - const nowNanos = 100 - const maxOffsetNanos = 20 - manualClock := hlc.NewManualClock(nowNanos) - clock := hlc.NewClock(manualClock.UnixNano, maxOffsetNanos) - const lagTargetNanos = 10 - - for _, tc := range []struct { - rangePolicy roachpb.RangeClosedTimestampPolicy - expClosedTSTarget hlc.Timestamp + type setupArgs struct { + tc *testcluster.TestCluster + leftDesc, rightDesc roachpb.RangeDescriptor + repl *kvserver.Replica + now hlc.ClockTimestamp + target hlc.Timestamp + filterC chan chan struct{} + } + testCases := []struct { + name string + exp bool + // Optional, to configure testing filters. + knobs func() (_ *kvserver.StoreTestingKnobs, filterC chan chan struct{}) + // Configures the replica to test different situtations. + setup func(_ setupArgs) (unblockFilterC chan struct{}, asyncErrC chan error, _ error) }{ { - rangePolicy: roachpb.LAG_BY_CLUSTER_SETTING, - expClosedTSTarget: hlc.Timestamp{WallTime: nowNanos - lagTargetNanos}, + name: "basic", + exp: true, + setup: func(a setupArgs) (chan struct{}, chan error, error) { + // Nothing going on. + return nil, nil, nil + }, + }, + { + name: "replica destroyed", + exp: false, + setup: func(a setupArgs) (chan struct{}, chan error, error) { + // Merge the range away to destroy it. + _, err := a.tc.Server(0).MergeRanges(a.leftDesc.StartKey.AsRawKey()) + return nil, nil, err + }, + }, + { + name: "lease invalid", + exp: false, + setup: func(a setupArgs) (chan struct{}, chan error, error) { + // Revoke the range's lease to prevent it from being valid. + l, _ := a.repl.GetLease() + a.repl.RevokeLease(ctx, l.Sequence) + return nil, nil, nil + }, + }, + { + name: "lease owned elsewhere", + exp: false, + setup: func(a setupArgs) (chan struct{}, chan error, error) { + // Transfer the range's lease. + return nil, nil, a.tc.TransferRangeLease(a.rightDesc, a.tc.Target(1)) + }, + }, + { + name: "merge in progress", + exp: false, + knobs: func() (*kvserver.StoreTestingKnobs, chan chan struct{}) { + mergeC := make(chan chan struct{}) + testingResponseFilter := func(ctx context.Context, ba roachpb.BatchRequest, br *roachpb.BatchResponse) *roachpb.Error { + if ba.IsSingleSubsumeRequest() { + unblockC := make(chan struct{}) + mergeC <- unblockC + <-unblockC + } + return nil + } + return &kvserver.StoreTestingKnobs{TestingResponseFilter: testingResponseFilter}, mergeC + }, + setup: func(a setupArgs) (chan struct{}, chan error, error) { + // Initiate a range merge and pause it after subsumption. + errC := make(chan error, 1) + _ = a.tc.Stopper().RunAsyncTask(ctx, "merge", func(context.Context) { + _, err := a.tc.Server(0).MergeRanges(a.leftDesc.StartKey.AsRawKey()) + errC <- err + }) + unblockFilterC := <-a.filterC + return unblockFilterC, errC, nil + }, + }, + { + name: "raft application in progress", + exp: false, + knobs: func() (*kvserver.StoreTestingKnobs, chan chan struct{}) { + applyC := make(chan chan struct{}) + var once sync.Once // ignore reproposals + testingApplyFilter := func(filterArgs kvserverbase.ApplyFilterArgs) (int, *roachpb.Error) { + if filterArgs.Req != nil && filterArgs.Req.IsSingleRequest() { + put := filterArgs.Req.Requests[0].GetPut() + if put != nil && put.Key.Equal(roachpb.Key("key_filter")) { + once.Do(func() { + unblockC := make(chan struct{}) + applyC <- unblockC + <-unblockC + }) + } + } + return 0, nil + } + return &kvserver.StoreTestingKnobs{TestingApplyFilter: testingApplyFilter}, applyC + }, + setup: func(a setupArgs) (chan struct{}, chan error, error) { + // Initiate a Raft proposal and pause it during application. + errC := make(chan error, 1) + _ = a.tc.Stopper().RunAsyncTask(ctx, "write", func(context.Context) { + errC <- a.tc.Server(0).DB().Put(ctx, "key_filter", "val") + }) + unblockFilterC := <-a.filterC + return unblockFilterC, errC, nil + }, + }, + { + name: "evaluating request below closed timestamp target", + exp: false, + knobs: func() (*kvserver.StoreTestingKnobs, chan chan struct{}) { + proposeC := make(chan chan struct{}) + testingProposalFilter := func(args kvserverbase.ProposalFilterArgs) *roachpb.Error { + if args.Req.IsSingleRequest() { + put := args.Req.Requests[0].GetPut() + if put != nil && put.Key.Equal(roachpb.Key("key_filter")) { + unblockC := make(chan struct{}) + proposeC <- unblockC + <-unblockC + } + } + return nil + } + return &kvserver.StoreTestingKnobs{TestingProposalFilter: testingProposalFilter}, proposeC + }, + setup: func(a setupArgs) (chan struct{}, chan error, error) { + // Initiate a write and pause it during evaluation. + errC := make(chan error, 1) + _ = a.tc.Stopper().RunAsyncTask(ctx, "write", func(context.Context) { + ts := a.target.Add(-1, 0) + putArgs := putArgs(roachpb.Key("key_filter"), []byte("val")) + sender := a.tc.Server(0).DB().NonTransactionalSender() + _, pErr := kv.SendWrappedWith(ctx, sender, roachpb.Header{Timestamp: ts}, putArgs) + errC <- pErr.GoError() + }) + unblockFilterC := <-a.filterC + return unblockFilterC, errC, nil + }, }, { - rangePolicy: roachpb.LEAD_FOR_GLOBAL_READS, - expClosedTSTarget: hlc.Timestamp{WallTime: nowNanos - lagTargetNanos}, - // TODO(andrei, nvanbenschoten): What we should be expecting here is the following, once - // the propBuf starts properly implementing this timestamp closing policy: - // expClosedTSTarget: hlc.Timestamp{WallTime: nowNanos + 2*maxOffsetNanos, Synthetic: true}, + name: "evaluating request at closed timestamp target", + exp: false, + knobs: func() (*kvserver.StoreTestingKnobs, chan chan struct{}) { + proposeC := make(chan chan struct{}) + testingProposalFilter := func(args kvserverbase.ProposalFilterArgs) *roachpb.Error { + if args.Req.IsSingleRequest() { + put := args.Req.Requests[0].GetPut() + if put != nil && put.Key.Equal(roachpb.Key("key_filter")) { + unblockC := make(chan struct{}) + proposeC <- unblockC + <-unblockC + } + } + return nil + } + return &kvserver.StoreTestingKnobs{TestingProposalFilter: testingProposalFilter}, proposeC + }, + setup: func(a setupArgs) (chan struct{}, chan error, error) { + // Initiate a write and pause it during evaluation. + errC := make(chan error, 1) + _ = a.tc.Stopper().RunAsyncTask(ctx, "write", func(context.Context) { + ts := a.target + putArgs := putArgs(roachpb.Key("key_filter"), []byte("val")) + sender := a.tc.Server(0).DB().NonTransactionalSender() + _, pErr := kv.SendWrappedWith(ctx, sender, roachpb.Header{Timestamp: ts}, putArgs) + errC <- pErr.GoError() + }) + unblockFilterC := <-a.filterC + return unblockFilterC, errC, nil + }, }, - } { - t.Run(tc.rangePolicy.String(), func(t *testing.T) { - require.Equal(t, tc.expClosedTSTarget, closedTimestampTargetByPolicy(clock.NowAsClockTimestamp(), tc.rangePolicy, lagTargetNanos)) + { + name: "evaluating request above closed timestamp target", + exp: true, + knobs: func() (*kvserver.StoreTestingKnobs, chan chan struct{}) { + proposeC := make(chan chan struct{}) + testingProposalFilter := func(args kvserverbase.ProposalFilterArgs) *roachpb.Error { + if args.Req.IsSingleRequest() { + put := args.Req.Requests[0].GetPut() + if put != nil && put.Key.Equal(roachpb.Key("key_filter")) { + unblockC := make(chan struct{}) + proposeC <- unblockC + <-unblockC + } + } + return nil + } + return &kvserver.StoreTestingKnobs{TestingProposalFilter: testingProposalFilter}, proposeC + }, + setup: func(a setupArgs) (chan struct{}, chan error, error) { + // Initiate a write and pause it during evaluation. + errC := make(chan error, 1) + _ = a.tc.Stopper().RunAsyncTask(ctx, "write", func(context.Context) { + ts := a.target.Add(1, 0) + putArgs := putArgs(roachpb.Key("key_filter"), []byte("val")) + sender := a.tc.Server(0).DB().NonTransactionalSender() + _, pErr := kv.SendWrappedWith(ctx, sender, roachpb.Header{Timestamp: ts}, putArgs) + errC <- pErr.GoError() + }) + unblockFilterC := <-a.filterC + return unblockFilterC, errC, nil + }, + }, + { + name: "existing closed timestamp before", + exp: true, + setup: func(a setupArgs) (chan struct{}, chan error, error) { + // Manually bump the assigned closed timestamp to a time below + // where the test will attempt to bump it to. + var targets [roachpb.MAX_CLOSED_TIMESTAMP_POLICY]hlc.Timestamp + targets[roachpb.LAG_BY_CLUSTER_SETTING] = a.target.Add(-1, 0) + return nil, nil, testutils.SucceedsSoonError(func() error { + ok, _, _ := a.repl.BumpSideTransportClosed(ctx, a.now, targets) + if !ok { + return errors.New("bumping side-transport unexpectedly failed") + } + return nil + }) + }, + }, + { + name: "existing closed timestamp equal", + exp: false, + setup: func(a setupArgs) (chan struct{}, chan error, error) { + // Manually bump the assigned closed timestamp to a time equal + // to where the test will attempt to bump it to. + var targets [roachpb.MAX_CLOSED_TIMESTAMP_POLICY]hlc.Timestamp + targets[roachpb.LAG_BY_CLUSTER_SETTING] = a.target + return nil, nil, testutils.SucceedsSoonError(func() error { + ok, _, _ := a.repl.BumpSideTransportClosed(ctx, a.now, targets) + if !ok { + return errors.New("bumping side-transport unexpectedly failed") + } + return nil + }) + }, + }, + { + name: "existing closed timestamp above", + exp: false, + setup: func(a setupArgs) (chan struct{}, chan error, error) { + // Manually bump the assigned closed timestamp to a time above + // where the test will attempt to bump it to. + var targets [roachpb.MAX_CLOSED_TIMESTAMP_POLICY]hlc.Timestamp + targets[roachpb.LAG_BY_CLUSTER_SETTING] = a.target.Add(1, 0) + return nil, nil, testutils.SucceedsSoonError(func() error { + ok, _, _ := a.repl.BumpSideTransportClosed(ctx, a.now, targets) + if !ok { + return errors.New("bumping side-transport unexpectedly failed") + } + return nil + }) + }, + }, + } + for _, test := range testCases { + t.Run(test.name, func(t *testing.T) { + var knobs base.ModuleTestingKnobs + var filterC chan chan struct{} + if test.knobs != nil { + knobs, filterC = test.knobs() + } + + tc := testcluster.StartTestCluster(t, 2, base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, + ServerArgs: base.TestServerArgs{ + Knobs: base.TestingKnobs{ + Store: knobs, + }, + }, + }) + defer tc.Stopper().Stop(ctx) + + leftDesc, rightDesc, err := tc.SplitRange(roachpb.Key("key")) + require.NoError(t, err) + tc.AddVotersOrFatal(t, leftDesc.StartKey.AsRawKey(), tc.Target(1)) + tc.AddVotersOrFatal(t, rightDesc.StartKey.AsRawKey(), tc.Target(1)) + store := tc.GetFirstStoreFromServer(t, 0) + require.NoError(t, err) + repl := store.LookupReplica(rightDesc.StartKey) + require.NotNil(t, repl) + + now := tc.Server(0).Clock().NowAsClockTimestamp() + var targets [roachpb.MAX_CLOSED_TIMESTAMP_POLICY]hlc.Timestamp + target := now.ToTimestamp() + targets[roachpb.LAG_BY_CLUSTER_SETTING] = target + + // Run the setup function to get the replica in the desired state. + unblockFilterC, asyncErrC, err := test.setup(setupArgs{ + tc: tc, + leftDesc: leftDesc, + rightDesc: rightDesc, + repl: repl, + now: now, + target: target, + filterC: filterC, + }) + require.NoError(t, err) + + // Try to bump the closed timestamp. Use succeeds soon if we are + // expecting the call to succeed, to avoid any flakiness. Don't do + // so if we expect the call to fail, in which case any flakiness + // would be a serious bug. + if test.exp { + testutils.SucceedsSoon(t, func() error { + ok, _, _ := repl.BumpSideTransportClosed(ctx, now, targets) + if !ok { + return errors.New("bumping side-transport unexpectedly failed") + } + return nil + }) + } else { + ok, _, _ := repl.BumpSideTransportClosed(ctx, now, targets) + require.False(t, ok) + } + + // Clean up, if necessary. + if unblockFilterC != nil { + close(unblockFilterC) + require.NoError(t, <-asyncErrC) + } }) } } + +// BenchmarkBumpSideTransportClosed measures the latency of a single call to +// (*Replica).BumpSideTransportClosed. The closed timestamp side-transport was +// designed with a performance expectation of this check taking no more than +// 100ns, so that calling the method on 10,000 leaseholders on a node could be +// done in less than 1ms. +// +// TODO(nvanbenschoten,andrei): Currently, the benchmark indicates that a call +// takes about 130ns. This exceeds the latency budget we've allocated to the +// call. However, it looks like there is some low-hanging fruit. 70% of the time +// is spent in leaseStatusForRequestRLocked, within which 24% of the total time +// is spent zeroing and copying memory and 30% of the total time is spent in +// (*NodeLiveness).GetLiveness, grabbing the current node's liveness record. If +// we eliminate some memory copying and pass the node liveness record in to the +// function so that we only have to grab it once instead of on each call to +// BumpSideTransportClosed, we should be able to reach our target latency. +func BenchmarkBumpSideTransportClosed(b *testing.B) { + defer leaktest.AfterTest(b)() + defer log.Scope(b).Close(b) + + ctx := context.Background() + manual := hlc.NewHybridManualClock() + s, _, _ := serverutils.StartServer(b, base.TestServerArgs{ + Knobs: base.TestingKnobs{ + Server: &server.TestingKnobs{ + ClockSource: manual.UnixNano, + }, + }, + }) + defer s.Stopper().Stop(ctx) + + key, err := s.ScratchRange() + require.NoError(b, err) + store, err := s.GetStores().(*kvserver.Stores).GetStore(s.GetFirstStoreID()) + require.NoError(b, err) + rkey := keys.MustAddr(key) + r := store.LookupReplica(rkey) + require.NotNil(b, r) + + manual.Pause() + now := s.Clock().NowAsClockTimestamp() + var targets [roachpb.MAX_CLOSED_TIMESTAMP_POLICY]hlc.Timestamp + + b.ResetTimer() + for i := 0; i < b.N; i++ { + // Advance time and the closed timestamp target. + now = now.ToTimestamp().Add(1, 0).UnsafeToClockTimestamp() + targets[roachpb.LAG_BY_CLUSTER_SETTING] = now.ToTimestamp() + + // Perform the call. + ok, _, _ := r.BumpSideTransportClosed(ctx, now, targets) + if !ok { + b.Fatal("BumpSideTransportClosed unexpectedly failed") + } + } +} diff --git a/pkg/kv/kvserver/replica_proposal.go b/pkg/kv/kvserver/replica_proposal.go index 0a37a1ae560c..9a84295451cb 100644 --- a/pkg/kv/kvserver/replica_proposal.go +++ b/pkg/kv/kvserver/replica_proposal.go @@ -520,6 +520,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) + } + // 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..48eb82d036d6 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 @@ -472,7 +473,7 @@ func (b *propBuf) FlushLockedWithRaftGroup( // lease applied index advances outside of this proposer's control (i.e. // other leaseholders commit some stuff and then we get the lease back), // future proposals will be given sufficiently high max lease indexes. - defer b.forwardLeaseIndexBase(b.p.leaseAppliedIndex()) + defer b.forwardLeaseIndexBaseLocked(b.p.leaseAppliedIndex()) // We hold the write lock while reading from and flushing the proposal // buffer. This ensures that we synchronize with all producers and other @@ -494,7 +495,7 @@ func (b *propBuf) FlushLockedWithRaftGroup( // Update the maximum lease index base value, based on the maximum lease // index assigned since the last flush. - b.forwardLeaseIndexBase(b.liBase + res.leaseIndexOffset()) + b.forwardLeaseIndexBaseLocked(b.liBase + res.leaseIndexOffset()) // Iterate through the proposals in the buffer and propose them to Raft. // While doing so, build up batches of entries and submit them to Raft all @@ -768,12 +769,17 @@ func (b *propBuf) assignClosedTimestampToProposalLocked( return err } -func (b *propBuf) forwardLeaseIndexBase(v uint64) { +func (b *propBuf) forwardLeaseIndexBaseLocked(v uint64) { if b.liBase < v { b.liBase = v } } +// forwardClosedTimestamp forwards the closed timestamp tracked by the propBuf. +func (b *propBuf) forwardClosedTimestampLocked(closedTS hlc.Timestamp) bool { + return b.assignedClosedTimestamp.Forward(closedTS) +} + func proposeBatch(raftGroup proposerRaft, replID roachpb.ReplicaID, ents []raftpb.Entry) error { if len(ents) == 0 { return nil @@ -820,11 +826,6 @@ 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) -} - // EvaluatingRequestsCount returns the count of requests currently tracked by // the propBuf. func (b *propBuf) EvaluatingRequestsCount() int { @@ -911,6 +912,25 @@ func (b *propBuf) TrackEvaluatingRequest( return minTS, TrackedRequestToken{tok: tok, b: b} } +// MaybeForwardClosedLocked checks whether the closed timestamp can be advanced +// to target. If so, the assigned closed timestamp is forwarded to the target, +// ensuring that no future writes ever write below it. +// +// 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 equal to or below target (as +// tracked by the evalTracker). We can't close timestamps at or above these +// requests' write timestamps. +func (b *propBuf) MaybeForwardClosedLocked(ctx context.Context, target hlc.Timestamp) bool { + if lb := b.evalTracker.LowerBound(ctx); !lb.IsEmpty() && lb.LessEq(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..efe0a6a9c91a 100644 --- a/pkg/kv/kvserver/replica_proposal_buf_test.go +++ b/pkg/kv/kvserver/replica_proposal_buf_test.go @@ -39,6 +39,7 @@ import ( // testProposer is a testing implementation of proposer. type testProposer struct { syncutil.RWMutex + clock *hlc.Clock ds destroyStatus lai uint64 enqueued int @@ -60,7 +61,6 @@ type testProposer struct { // is. Some types of replicas are not eligible to get a lease. leaderReplicaType roachpb.ReplicaType rangePolicy roachpb.RangeClosedTimestampPolicy - clock *hlc.Clock } var _ proposer = &testProposer{} @@ -125,7 +125,9 @@ func (t *testProposer) closedTimestampTarget() hlc.Timestamp { if t.clock == nil { return hlc.Timestamp{} } - return closedTimestampTargetByPolicy(t.clock.NowAsClockTimestamp(), t.rangePolicy, time.Second) + return closedts.TargetForPolicy( + t.clock.NowAsClockTimestamp(), t.clock.MaxOffset(), time.Second, t.rangePolicy, + ) } func (t *testProposer) raftTransportClosedTimestampEnabled() bool { @@ -825,10 +827,10 @@ func TestProposalBufferClosedTimestamp(t *testing.T) { t.Run(tc.name, func(t *testing.T) { r := &testProposerRaft{} p := testProposer{ + clock: clock, lai: 10, raftGroup: r, rangePolicy: tc.rangePolicy, - clock: clock, } tracker := mockTracker{ lowerBound: tc.trackerLowerBound, diff --git a/pkg/kv/kvserver/replica_raft.go b/pkg/kv/kvserver/replica_raft.go index e1ffbafea801..6f906c05d405 100644 --- a/pkg/kv/kvserver/replica_raft.go +++ b/pkg/kv/kvserver/replica_raft.go @@ -507,6 +507,7 @@ func (r *Replica) handleRaftReadyRaftMuLocked( unquiesceAndWakeLeader := hasReady || numFlushed > 0 || len(r.mu.proposals) > 0 return unquiesceAndWakeLeader, nil }) + r.mu.applyingEntries = len(rd.CommittedEntries) > 0 r.mu.Unlock() if errors.Is(err, errRemoved) { // If we've been removed then just return. @@ -864,6 +865,7 @@ func (r *Replica) handleRaftReadyRaftMuLocked( } return true, nil }) + r.mu.applyingEntries = false r.mu.Unlock() if err != nil { return stats, expl, errors.Wrap(err, expl) diff --git a/pkg/kv/kvserver/stateloader/stateloader.go b/pkg/kv/kvserver/stateloader/stateloader.go index 4e4060848c62..3c81dfe57e5f 100644 --- a/pkg/kv/kvserver/stateloader/stateloader.go +++ b/pkg/kv/kvserver/stateloader/stateloader.go @@ -299,9 +299,9 @@ func (rsl StateLoader) LoadMVCCStats( // by the range applied state key. // // TODO(andrei): raftClosedTimestamp is a pointer to avoid an allocation when -// putting it in RangeAppliedState. RangeAppliedState.RaftClosedTimestamp is -// made non-nullable (see comments on the field), this argument should be taken -// by value. +// putting it in RangeAppliedState. Once RangeAppliedState.RaftClosedTimestamp +// is made non-nullable (see comments on the field), this argument should be +// taken by value. func (rsl StateLoader) SetRangeAppliedState( ctx context.Context, readWriter storage.ReadWriter, diff --git a/pkg/kv/kvserver/store.go b/pkg/kv/kvserver/store.go index fee2ca5a6a89..63ad654cf40c 100644 --- a/pkg/kv/kvserver/store.go +++ b/pkg/kv/kvserver/store.go @@ -35,6 +35,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/container" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/sidetransport" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/idalloc" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/intentresolver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" @@ -421,6 +422,7 @@ type Store struct { txnWaitMetrics *txnwait.Metrics sstSnapshotStorage SSTSnapshotStorage protectedtsCache protectedts.Cache + ctSender *sidetransport.Sender // 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 + ClosedTimestampSender *sidetransport.Sender // SQLExecutor is used by the store to execute SQL statements. SQLExecutor sqlutil.InternalExecutor @@ -801,6 +804,7 @@ func NewStore( engine: eng, nodeDesc: nodeDesc, metrics: newStoreMetrics(cfg.HistogramWindowInterval), + ctSender: cfg.ClosedTimestampSender, } if cfg.RPCContext != nil { s.allocator = MakeAllocator(cfg.StorePool, cfg.RPCContext.RemoteClocks.Latency) @@ -2762,6 +2766,31 @@ func (s *Store) PurgeOutdatedReplicas(ctx context.Context, version roachpb.Versi return g.Wait() } +// registerLeaseholder registers the provided replica as a leaseholder in the +// node's closed timestamp side transport. +func (s *Store) registerLeaseholder( + ctx context.Context, r *Replica, leaseSeq roachpb.LeaseSequence, +) { + if s.ctSender != nil { + s.ctSender.RegisterLeaseholder(ctx, r, leaseSeq) + } +} + +// unregisterLeaseholder unregisters the provided replica from node's closed +// timestamp side transport if it had been previously registered as a +// leaseholder. +func (s *Store) unregisterLeaseholder(ctx context.Context, r *Replica) { + s.unregisterLeaseholderByID(ctx, r.RangeID) +} + +// unregisterLeaseholderByID is like unregisterLeaseholder, but it accepts a +// range ID instead of a replica. +func (s *Store) unregisterLeaseholderByID(ctx context.Context, rangeID roachpb.RangeID) { + if s.ctSender != nil { + s.ctSender.UnregisterLeaseholder(ctx, s.StoreID(), rangeID) + } +} + // 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. diff --git a/pkg/kv/kvserver/store_remove_replica.go b/pkg/kv/kvserver/store_remove_replica.go index 653378af2f4d..7458449cf464 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.unregisterLeaseholderByID(ctx, rangeID) } // 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/BUILD.bazel b/pkg/server/BUILD.bazel index 2ac143451c10..695c45507ce3 100644 --- a/pkg/server/BUILD.bazel +++ b/pkg/server/BUILD.bazel @@ -71,6 +71,7 @@ go_library( "//pkg/kv/kvprober", "//pkg/kv/kvserver", "//pkg/kv/kvserver/closedts/container", + "//pkg/kv/kvserver/closedts/sidetransport", "//pkg/kv/kvserver/kvserverbase", "//pkg/kv/kvserver/kvserverpb", "//pkg/kv/kvserver/liveness", diff --git a/pkg/server/server.go b/pkg/server/server.go index 3e739e31a07e..dededd356b61 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -43,6 +43,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvprober" "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" @@ -145,6 +146,7 @@ type Server struct { recorder *status.MetricsRecorder runtime *status.RuntimeStatSampler updates *diagnostics.UpdateChecker + ctSender *sidetransport.Sender admin *adminServer status *statusServer @@ -478,6 +480,8 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { } sTS := ts.MakeServer(cfg.AmbientCtx, tsDB, nodeCountFn, cfg.TimeSeriesServerConfig, stopper) + ctSender := sidetransport.NewSender(stopper, st, clock, nodeDialer) + // The InternalExecutor will be further initialized later, as we create more // of the server's components. There's a circular dependency - many things // need an InternalExecutor, but the InternalExecutor needs an ExecutorConfig, @@ -531,6 +535,7 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { LogRangeEvents: cfg.EventLogEnabled, RangeDescriptorCache: distSender.RangeDescriptorCache(), TimeSeriesDataStore: tsDB, + ClosedTimestampSender: ctSender, // Initialize the closed timestamp subsystem. Note that it won't // be ready until it is .Start()ed, but the grpc server can be @@ -706,6 +711,7 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { registry: registry, recorder: recorder, updates: updates, + ctSender: ctSender, runtime: runtimeSampler, admin: sAdmin, status: sStatus, @@ -1817,6 +1823,8 @@ func (s *Server) PreStart(ctx context.Context) error { return errors.Wrapf(err, "failed to register engines with debug server") } + s.ctSender.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.