diff --git a/docs/generated/redact_safe.md b/docs/generated/redact_safe.md index 2697ee106691..52b50fe0f9ed 100644 --- a/docs/generated/redact_safe.md +++ b/docs/generated/redact_safe.md @@ -31,6 +31,7 @@ pkg/sql/sem/tree/table_ref.go | `ColumnID` pkg/storage/enginepb/mvcc3.go | `MVCCStatsDelta` pkg/storage/enginepb/mvcc3.go | `*MVCCStats` pkg/util/hlc/timestamp.go | `Timestamp` +pkg/util/hlc/timestamp.go | `ClockTimestamp` pkg/util/log/redact.go | `reflect.TypeOf(true)` pkg/util/log/redact.go | `reflect.TypeOf(123)` pkg/util/log/redact.go | `reflect.TypeOf(int8(0))` diff --git a/pkg/kv/client_test.go b/pkg/kv/client_test.go index 621e2fa86ee5..d2a6052377b9 100644 --- a/pkg/kv/client_test.go +++ b/pkg/kv/client_test.go @@ -936,8 +936,8 @@ func TestNodeIDAndObservedTimestamps(t *testing.T) { for i, test := range directCases { t.Run(fmt.Sprintf("direct-txn-%d", i), func(t *testing.T) { db := setup(test.nodeID) - now := db.Clock().Now() - kvTxn := roachpb.MakeTransaction("unnamed", nil /*baseKey*/, roachpb.NormalUserPriority, now, db.Clock().MaxOffset().Nanoseconds()) + now := db.Clock().NowAsClockTimestamp() + kvTxn := roachpb.MakeTransaction("unnamed", nil /*baseKey*/, roachpb.NormalUserPriority, now.ToTimestamp(), db.Clock().MaxOffset().Nanoseconds()) txn := kv.NewTxnFromProto(ctx, db, test.nodeID, now, test.typ, &kvTxn) ots := txn.TestingCloneTxn().ObservedTimestamps if (len(ots) == 1 && ots[0].NodeID == test.nodeID) != test.expObserved { diff --git a/pkg/kv/kvclient/kvcoord/dist_sender_server_test.go b/pkg/kv/kvclient/kvcoord/dist_sender_server_test.go index e5e309251814..6a0c893098bb 100644 --- a/pkg/kv/kvclient/kvcoord/dist_sender_server_test.go +++ b/pkg/kv/kvclient/kvcoord/dist_sender_server_test.go @@ -1535,8 +1535,8 @@ func TestPropagateTxnOnError(t *testing.T) { // response that does not result in an error. Even though the batch as a // whole results in an error, the transaction should still propagate this // information. - ot1 := roachpb.ObservedTimestamp{NodeID: 7, Timestamp: hlc.Timestamp{WallTime: 15}} - ot2 := roachpb.ObservedTimestamp{NodeID: 8, Timestamp: hlc.Timestamp{WallTime: 16}} + ot1 := roachpb.ObservedTimestamp{NodeID: 7, Timestamp: hlc.ClockTimestamp{WallTime: 15}} + ot2 := roachpb.ObservedTimestamp{NodeID: 8, Timestamp: hlc.ClockTimestamp{WallTime: 16}} containsObservedTSs := func(txn *roachpb.Transaction) bool { contains := func(ot roachpb.ObservedTimestamp) bool { for _, ts := range txn.ObservedTimestamps { diff --git a/pkg/kv/kvclient/kvcoord/dist_sender_test.go b/pkg/kv/kvclient/kvcoord/dist_sender_test.go index c370c602af4f..6a0cf1f2e955 100644 --- a/pkg/kv/kvclient/kvcoord/dist_sender_test.go +++ b/pkg/kv/kvclient/kvcoord/dist_sender_test.go @@ -564,7 +564,7 @@ func TestImmutableBatchArgs(t *testing.T) { // An optimization does copy-on-write if we haven't observed anything, // so make sure we're not in that case. - txn.UpdateObservedTimestamp(1, hlc.MaxTimestamp) + txn.UpdateObservedTimestamp(1, hlc.MaxClockTimestamp) put := roachpb.NewPut(roachpb.Key("don't"), roachpb.Value{}) if _, pErr := kv.SendWrappedWith(context.Background(), ds, roachpb.Header{ @@ -2180,20 +2180,20 @@ func TestClockUpdateOnResponse(t *testing.T) { // Prepare the test function put := roachpb.NewPut(roachpb.Key("a"), roachpb.MakeValueFromString("value")) - doCheck := func(sender kv.Sender, fakeTime hlc.Timestamp) { + doCheck := func(sender kv.Sender, fakeTime hlc.ClockTimestamp) { ds.transportFactory = SenderTransportFactory(tracing.NewTracer(), sender) _, err := kv.SendWrapped(context.Background(), ds, put) if err != nil && err != expectedErr { t.Fatal(err) } - newTime := ds.clock.Now() + newTime := ds.clock.NowAsClockTimestamp() if newTime.Less(fakeTime) { t.Fatalf("clock was not advanced: expected >= %s; got %s", fakeTime, newTime) } } // Test timestamp propagation on valid BatchResults. - fakeTime := ds.clock.Now().Add(10000000000 /*10s*/, 0) + fakeTime := ds.clock.Now().Add(10000000000 /*10s*/, 0).UnsafeToClockTimestamp() replyNormal := kv.SenderFunc( func(_ context.Context, args roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { rb := args.CreateReply() @@ -2203,7 +2203,7 @@ func TestClockUpdateOnResponse(t *testing.T) { doCheck(replyNormal, fakeTime) // Test timestamp propagation on errors. - fakeTime = ds.clock.Now().Add(10000000000 /*10s*/, 0) + fakeTime = ds.clock.Now().Add(10000000000 /*10s*/, 0).UnsafeToClockTimestamp() replyError := kv.SenderFunc( func(_ context.Context, _ roachpb.BatchRequest) (*roachpb.BatchResponse, *roachpb.Error) { pErr := expectedErr diff --git a/pkg/kv/kvclient/kvcoord/txn_coord_sender_test.go b/pkg/kv/kvclient/kvcoord/txn_coord_sender_test.go index 0324777d7112..0eff49ec39fe 100644 --- a/pkg/kv/kvclient/kvcoord/txn_coord_sender_test.go +++ b/pkg/kv/kvclient/kvcoord/txn_coord_sender_test.go @@ -726,7 +726,7 @@ func TestTxnCoordSenderTxnUpdatedOnError(t *testing.T) { name: "ReadWithinUncertaintyIntervalError", pErrGen: func(txn *roachpb.Transaction) *roachpb.Error { const nodeID = 1 - txn.UpdateObservedTimestamp(nodeID, plus10) + txn.UpdateObservedTimestamp(nodeID, plus10.UnsafeToClockTimestamp()) pErr := roachpb.NewErrorWithTxn( roachpb.NewReadWithinUncertaintyIntervalError( hlc.Timestamp{}, hlc.Timestamp{}, nil), @@ -818,12 +818,12 @@ func TestTxnCoordSenderTxnUpdatedOnError(t *testing.T) { ) db := kv.NewDB(ambient, tsf, clock, stopper) key := roachpb.Key("test-key") - now := clock.Now() + now := clock.NowAsClockTimestamp() origTxnProto := roachpb.MakeTransaction( "test txn", key, roachpb.UserPriority(0), - now, + now.ToTimestamp(), clock.MaxOffset().Nanoseconds(), ) // TODO(andrei): I've monkeyed with the priorities on this initial @@ -1270,7 +1270,7 @@ func TestAbortTransactionOnCommitErrors(t *testing.T) { const nodeID = 0 // ReadWithinUncertaintyIntervalErrors need a clock to have been // recorded on the origin. - txn.UpdateObservedTimestamp(nodeID, makeTS(123, 0)) + txn.UpdateObservedTimestamp(nodeID, makeTS(123, 0).UnsafeToClockTimestamp()) return roachpb.NewErrorWithTxn( roachpb.NewReadWithinUncertaintyIntervalError(hlc.Timestamp{}, hlc.Timestamp{}, nil), &txn) diff --git a/pkg/kv/kvclient/kvcoord/txn_interceptor_span_refresher_test.go b/pkg/kv/kvclient/kvcoord/txn_interceptor_span_refresher_test.go index 3639c01cbe45..f851b1d39ff5 100644 --- a/pkg/kv/kvclient/kvcoord/txn_interceptor_span_refresher_test.go +++ b/pkg/kv/kvclient/kvcoord/txn_interceptor_span_refresher_test.go @@ -120,7 +120,7 @@ func TestTxnSpanRefresherRefreshesTransactions(t *testing.T) { defer log.Scope(t).Close(t) txn := makeTxnProto() - txn.UpdateObservedTimestamp(1, txn.WriteTimestamp.Add(20, 0)) + txn.UpdateObservedTimestamp(1, txn.WriteTimestamp.Add(20, 0).UnsafeToClockTimestamp()) keyA, keyB := roachpb.Key("a"), roachpb.Key("b") cases := []struct { diff --git a/pkg/kv/kvserver/batcheval/cmd_lease_request.go b/pkg/kv/kvserver/batcheval/cmd_lease_request.go index 2e91a00b4ed9..332c5f112fe8 100644 --- a/pkg/kv/kvserver/batcheval/cmd_lease_request.go +++ b/pkg/kv/kvserver/batcheval/cmd_lease_request.go @@ -118,7 +118,7 @@ func RequestLease( // The bug prevented with this is unlikely to occur in practice // since earlier commands usually apply before this lease will. if ts := args.MinProposedTS; isExtension && ts != nil { - effectiveStart.Forward(*ts) + effectiveStart.Forward(ts.ToTimestamp()) } } else if prevLease.Type() == roachpb.LeaseExpiration { diff --git a/pkg/kv/kvserver/batcheval/cmd_subsume.go b/pkg/kv/kvserver/batcheval/cmd_subsume.go index 1d8f2daaf963..ea78a0295690 100644 --- a/pkg/kv/kvserver/batcheval/cmd_subsume.go +++ b/pkg/kv/kvserver/batcheval/cmd_subsume.go @@ -170,9 +170,9 @@ func Subsume( reply.MVCCStats = cArgs.EvalCtx.GetMVCCStats() reply.LeaseAppliedIndex = lai - reply.FreezeStart = cArgs.EvalCtx.Clock().Now() + reply.FreezeStart = cArgs.EvalCtx.Clock().NowAsClockTimestamp() return result.Result{ - Local: result.LocalResult{FreezeStart: reply.FreezeStart}, + Local: result.LocalResult{FreezeStart: reply.FreezeStart.ToTimestamp()}, }, nil } diff --git a/pkg/kv/kvserver/client_replica_test.go b/pkg/kv/kvserver/client_replica_test.go index c62813bccbcd..44b89fd8b469 100644 --- a/pkg/kv/kvserver/client_replica_test.go +++ b/pkg/kv/kvserver/client_replica_test.go @@ -951,7 +951,7 @@ func TestRangeLimitTxnMaxTimestamp(t *testing.T) { // Start a transaction using node2 as a gateway. txn := roachpb.MakeTransaction("test", keyA, 1, clock2.Now(), 250 /* maxOffsetNs */) // Simulate a read to another range on node2 by setting the observed timestamp. - txn.UpdateObservedTimestamp(2, clock2.Now()) + txn.UpdateObservedTimestamp(2, clock2.NowAsClockTimestamp()) defer mtc.Stop() mtc.Start(t, 2) diff --git a/pkg/kv/kvserver/closed_timestamp_test.go b/pkg/kv/kvserver/closed_timestamp_test.go index faf5c88d47ac..ad2d306fb613 100644 --- a/pkg/kv/kvserver/closed_timestamp_test.go +++ b/pkg/kv/kvserver/closed_timestamp_test.go @@ -842,7 +842,7 @@ func (filter *mergeFilter) SuspendMergeTrigger( // We block the LHS leaseholder from applying the merge trigger. Note // that RHS followers will have already caught up to the leaseholder // well before this point. - blocker.signal(freezeStart) + blocker.signal(freezeStart.ToTimestamp()) // Wait for the merge to be unblocked. <-blocker.unblockCh } diff --git a/pkg/kv/kvserver/concurrency/concurrency_manager_test.go b/pkg/kv/kvserver/concurrency/concurrency_manager_test.go index aa9bf5388e0a..71af16437848 100644 --- a/pkg/kv/kvserver/concurrency/concurrency_manager_test.go +++ b/pkg/kv/kvserver/concurrency/concurrency_manager_test.go @@ -119,7 +119,7 @@ func TestConcurrencyManagerBasic(t *testing.T) { ReadTimestamp: ts, MaxTimestamp: maxTS, } - txn.UpdateObservedTimestamp(c.nodeDesc.NodeID, ts) + txn.UpdateObservedTimestamp(c.nodeDesc.NodeID, ts.UnsafeToClockTimestamp()) c.registerTxn(txnName, txn) return "" diff --git a/pkg/kv/kvserver/intentresolver/intent_resolver_test.go b/pkg/kv/kvserver/intentresolver/intent_resolver_test.go index 0bb07e945936..0e1b153f9f7a 100644 --- a/pkg/kv/kvserver/intentresolver/intent_resolver_test.go +++ b/pkg/kv/kvserver/intentresolver/intent_resolver_test.go @@ -72,7 +72,7 @@ func TestCleanupTxnIntentsOnGCAsync(t *testing.T) { // Txn1 is in the pending state but is expired. txn1 := newTransaction("txn1", key, 1, clock) txn1.ReadTimestamp.WallTime -= int64(100 * time.Second) - txn1.LastHeartbeat = txn1.ReadTimestamp + txn1.LastHeartbeat = txn1.ReadTimestamp.UnsafeToClockTimestamp().ToTimestamp() // Txn2 is in the staging state and is not old enough to have expired so the // code ought to send nothing. txn2 := newTransaction("txn2", key, 1, clock) diff --git a/pkg/kv/kvserver/kvserverpb/proposer_kv.pb.go b/pkg/kv/kvserver/kvserverpb/proposer_kv.pb.go index d0ed4bce4e06..208fe0c6e23b 100644 --- a/pkg/kv/kvserver/kvserverpb/proposer_kv.pb.go +++ b/pkg/kv/kvserver/kvserverpb/proposer_kv.pb.go @@ -12,6 +12,7 @@ import hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" import github_com_cockroachdb_cockroach_pkg_util_uuid "github.com/cockroachdb/cockroach/pkg/util/uuid" import github_com_cockroachdb_cockroach_pkg_roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" +import github_com_cockroachdb_cockroach_pkg_util_hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" import bytes "bytes" @@ -47,7 +48,7 @@ func (m *Split) Reset() { *m = Split{} } func (m *Split) String() string { return proto.CompactTextString(m) } func (*Split) ProtoMessage() {} func (*Split) Descriptor() ([]byte, []int) { - return fileDescriptor_proposer_kv_ed289abcb229e5c7, []int{0} + return fileDescriptor_proposer_kv_eb94ca29d884b99a, []int{0} } func (m *Split) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -82,7 +83,7 @@ func (m *Merge) Reset() { *m = Merge{} } func (m *Merge) String() string { return proto.CompactTextString(m) } func (*Merge) ProtoMessage() {} func (*Merge) Descriptor() ([]byte, []int) { - return fileDescriptor_proposer_kv_ed289abcb229e5c7, []int{1} + return fileDescriptor_proposer_kv_eb94ca29d884b99a, []int{1} } func (m *Merge) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -116,7 +117,7 @@ type ChangeReplicas struct { func (m *ChangeReplicas) Reset() { *m = ChangeReplicas{} } func (*ChangeReplicas) ProtoMessage() {} func (*ChangeReplicas) Descriptor() ([]byte, []int) { - return fileDescriptor_proposer_kv_ed289abcb229e5c7, []int{2} + return fileDescriptor_proposer_kv_eb94ca29d884b99a, []int{2} } func (m *ChangeReplicas) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -168,7 +169,7 @@ func (m *ComputeChecksum) Reset() { *m = ComputeChecksum{} } func (m *ComputeChecksum) String() string { return proto.CompactTextString(m) } func (*ComputeChecksum) ProtoMessage() {} func (*ComputeChecksum) Descriptor() ([]byte, []int) { - return fileDescriptor_proposer_kv_ed289abcb229e5c7, []int{3} + return fileDescriptor_proposer_kv_eb94ca29d884b99a, []int{3} } func (m *ComputeChecksum) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -205,7 +206,7 @@ func (m *Compaction) Reset() { *m = Compaction{} } func (m *Compaction) String() string { return proto.CompactTextString(m) } func (*Compaction) ProtoMessage() {} func (*Compaction) Descriptor() ([]byte, []int) { - return fileDescriptor_proposer_kv_ed289abcb229e5c7, []int{4} + return fileDescriptor_proposer_kv_eb94ca29d884b99a, []int{4} } func (m *Compaction) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -242,7 +243,7 @@ func (m *SuggestedCompaction) Reset() { *m = SuggestedCompaction{} } func (m *SuggestedCompaction) String() string { return proto.CompactTextString(m) } func (*SuggestedCompaction) ProtoMessage() {} func (*SuggestedCompaction) Descriptor() ([]byte, []int) { - return fileDescriptor_proposer_kv_ed289abcb229e5c7, []int{5} + return fileDescriptor_proposer_kv_eb94ca29d884b99a, []int{5} } func (m *SuggestedCompaction) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -297,14 +298,14 @@ type ReplicatedEvalResult struct { // This is the proposal timestamp for the active lease while evaluating a lease request. // It will be used to make sure we know if a lease was extended after we sent out the request // but before we tried to apply it. - PrevLeaseProposal *hlc.Timestamp `protobuf:"bytes,20,opt,name=prev_lease_proposal,json=prevLeaseProposal,proto3" json:"prev_lease_proposal,omitempty"` + PrevLeaseProposal *github_com_cockroachdb_cockroach_pkg_util_hlc.ClockTimestamp `protobuf:"bytes,20,opt,name=prev_lease_proposal,json=prevLeaseProposal,proto3,casttype=github.com/cockroachdb/cockroach/pkg/util/hlc.ClockTimestamp" json:"prev_lease_proposal,omitempty"` } func (m *ReplicatedEvalResult) Reset() { *m = ReplicatedEvalResult{} } func (m *ReplicatedEvalResult) String() string { return proto.CompactTextString(m) } func (*ReplicatedEvalResult) ProtoMessage() {} func (*ReplicatedEvalResult) Descriptor() ([]byte, []int) { - return fileDescriptor_proposer_kv_ed289abcb229e5c7, []int{6} + return fileDescriptor_proposer_kv_eb94ca29d884b99a, []int{6} } func (m *ReplicatedEvalResult) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -348,7 +349,7 @@ func (m *ReplicatedEvalResult_AddSSTable) Reset() { *m = ReplicatedEvalR func (m *ReplicatedEvalResult_AddSSTable) String() string { return proto.CompactTextString(m) } func (*ReplicatedEvalResult_AddSSTable) ProtoMessage() {} func (*ReplicatedEvalResult_AddSSTable) Descriptor() ([]byte, []int) { - return fileDescriptor_proposer_kv_ed289abcb229e5c7, []int{6, 0} + return fileDescriptor_proposer_kv_eb94ca29d884b99a, []int{6, 0} } func (m *ReplicatedEvalResult_AddSSTable) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -385,7 +386,7 @@ func (m *WriteBatch) Reset() { *m = WriteBatch{} } func (m *WriteBatch) String() string { return proto.CompactTextString(m) } func (*WriteBatch) ProtoMessage() {} func (*WriteBatch) Descriptor() ([]byte, []int) { - return fileDescriptor_proposer_kv_ed289abcb229e5c7, []int{7} + return fileDescriptor_proposer_kv_eb94ca29d884b99a, []int{7} } func (m *WriteBatch) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -422,7 +423,7 @@ func (m *LogicalOpLog) Reset() { *m = LogicalOpLog{} } func (m *LogicalOpLog) String() string { return proto.CompactTextString(m) } func (*LogicalOpLog) ProtoMessage() {} func (*LogicalOpLog) Descriptor() ([]byte, []int) { - return fileDescriptor_proposer_kv_ed289abcb229e5c7, []int{8} + return fileDescriptor_proposer_kv_eb94ca29d884b99a, []int{8} } func (m *LogicalOpLog) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -527,7 +528,7 @@ func (m *RaftCommand) Reset() { *m = RaftCommand{} } func (m *RaftCommand) String() string { return proto.CompactTextString(m) } func (*RaftCommand) ProtoMessage() {} func (*RaftCommand) Descriptor() ([]byte, []int) { - return fileDescriptor_proposer_kv_ed289abcb229e5c7, []int{9} + return fileDescriptor_proposer_kv_eb94ca29d884b99a, []int{9} } func (m *RaftCommand) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -566,7 +567,7 @@ func (m *RaftCommandFooter) Reset() { *m = RaftCommandFooter{} } func (m *RaftCommandFooter) String() string { return proto.CompactTextString(m) } func (*RaftCommandFooter) ProtoMessage() {} func (*RaftCommandFooter) Descriptor() ([]byte, []int) { - return fileDescriptor_proposer_kv_ed289abcb229e5c7, []int{10} + return fileDescriptor_proposer_kv_eb94ca29d884b99a, []int{10} } func (m *RaftCommandFooter) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2644,7 +2645,7 @@ func (m *ReplicatedEvalResult) Unmarshal(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.PrevLeaseProposal == nil { - m.PrevLeaseProposal = &hlc.Timestamp{} + m.PrevLeaseProposal = &github_com_cockroachdb_cockroach_pkg_util_hlc.ClockTimestamp{} } if err := m.PrevLeaseProposal.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err @@ -3476,97 +3477,98 @@ var ( ) func init() { - proto.RegisterFile("kv/kvserver/kvserverpb/proposer_kv.proto", fileDescriptor_proposer_kv_ed289abcb229e5c7) + proto.RegisterFile("kv/kvserver/kvserverpb/proposer_kv.proto", fileDescriptor_proposer_kv_eb94ca29d884b99a) } -var fileDescriptor_proposer_kv_ed289abcb229e5c7 = []byte{ - // 1404 bytes of a gzipped FileDescriptorProto +var fileDescriptor_proposer_kv_eb94ca29d884b99a = []byte{ + // 1424 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x57, 0x4f, 0x6f, 0xdb, 0x46, 0x16, 0xb7, 0x2c, 0xc9, 0xa6, 0x9e, 0x6c, 0x89, 0x9e, 0x38, 0x09, 0xd7, 0xbb, 0x2b, 0x19, 0xda, - 0x6c, 0xe0, 0xdd, 0xa6, 0x54, 0x60, 0xb7, 0x40, 0x91, 0x04, 0x45, 0x2d, 0x39, 0x69, 0xac, 0xd8, - 0x6e, 0x32, 0x72, 0xd2, 0x22, 0x3d, 0x10, 0x23, 0x72, 0x42, 0xb1, 0xa2, 0x48, 0x66, 0x38, 0x52, - 0xe2, 0x6f, 0xd1, 0x02, 0x3d, 0xf4, 0xd4, 0xe6, 0xd8, 0x8f, 0x92, 0x4b, 0x81, 0x1c, 0x83, 0x1e, - 0x84, 0xc6, 0xb9, 0xf4, 0x33, 0xe4, 0x50, 0x14, 0x33, 0x1c, 0x4a, 0x72, 0xe1, 0xd4, 0x4a, 0x7b, - 0x1b, 0xbe, 0x99, 0xdf, 0xef, 0xcd, 0xbc, 0x3f, 0xbf, 0x19, 0xc2, 0x46, 0x6f, 0x58, 0xef, 0x0d, - 0x63, 0xca, 0x86, 0x94, 0x8d, 0x07, 0x51, 0xa7, 0x1e, 0xb1, 0x30, 0x0a, 0x63, 0xca, 0xac, 0xde, - 0xd0, 0x8c, 0x58, 0xc8, 0x43, 0x54, 0xb5, 0x43, 0xbb, 0xc7, 0x42, 0x62, 0x77, 0xcd, 0xde, 0xd0, - 0x4c, 0x97, 0x9a, 0x31, 0x0f, 0x19, 0x71, 0x69, 0xd4, 0x59, 0x5b, 0x91, 0x93, 0x51, 0xa7, 0x4e, - 0x22, 0x2f, 0xc1, 0xac, 0xa1, 0xd4, 0xe4, 0x10, 0x4e, 0x94, 0xed, 0x42, 0x6a, 0xeb, 0x53, 0x4e, - 0xa6, 0xec, 0xff, 0x54, 0x4c, 0x75, 0x1a, 0xb8, 0x5e, 0x40, 0xc5, 0x82, 0xa1, 0x6d, 0xab, 0xc9, - 0x7f, 0x9d, 0x3a, 0xb9, 0xa5, 0x66, 0x6b, 0x6f, 0x39, 0x44, 0xcc, 0x09, 0xa7, 0x6a, 0x8d, 0x31, - 0xe0, 0x9e, 0x5f, 0xef, 0xfa, 0x76, 0x9d, 0x7b, 0x7d, 0x1a, 0x73, 0xd2, 0x8f, 0xd4, 0xcc, 0xaa, - 0x1b, 0xba, 0xa1, 0x1c, 0xd6, 0xc5, 0x28, 0xb1, 0xd6, 0x7e, 0xcc, 0x40, 0xbe, 0x1d, 0xf9, 0x1e, - 0x47, 0x4d, 0x58, 0xe4, 0xcc, 0x73, 0x5d, 0xca, 0x8c, 0xcc, 0x7a, 0x66, 0xa3, 0xb8, 0x59, 0x35, - 0x27, 0xa1, 0x50, 0x87, 0x31, 0xe5, 0xd2, 0xc3, 0x64, 0x59, 0x43, 0x7b, 0x3e, 0xaa, 0xce, 0xbd, - 0x18, 0x55, 0x33, 0x38, 0x45, 0xa2, 0x43, 0x28, 0xb0, 0x6e, 0x6c, 0x39, 0xd4, 0xe7, 0xc4, 0x98, - 0x97, 0x34, 0xff, 0x9d, 0xa2, 0x51, 0xc7, 0x33, 0xd3, 0xe3, 0x99, 0xfb, 0x0f, 0x9a, 0xcd, 0x36, - 0x27, 0x3c, 0x6e, 0xe8, 0x82, 0xec, 0x78, 0x54, 0xd5, 0xf0, 0xed, 0xf6, 0x8e, 0x80, 0x63, 0x8d, - 0x75, 0x63, 0x39, 0xba, 0x96, 0xfb, 0xf5, 0x59, 0x35, 0x53, 0xc3, 0x90, 0xdf, 0xa7, 0xcc, 0xa5, - 0xb3, 0xed, 0x54, 0x2e, 0x7d, 0xfb, 0x4e, 0x15, 0xa7, 0x03, 0xa5, 0x66, 0x97, 0x04, 0x2e, 0xc5, - 0x34, 0xf2, 0x3d, 0x9b, 0xc4, 0x68, 0xef, 0x8f, 0xe4, 0x1b, 0xa7, 0x90, 0x9f, 0xc4, 0xfc, 0x99, - 0x97, 0xef, 0x9e, 0x55, 0xe7, 0x6a, 0xaf, 0xe6, 0xa1, 0xdc, 0x0c, 0xfb, 0xd1, 0x80, 0xd3, 0x66, - 0x97, 0xda, 0xbd, 0x78, 0xd0, 0x47, 0x5f, 0x41, 0xd1, 0x56, 0x63, 0xcb, 0x73, 0xa4, 0xaf, 0xa5, - 0xc6, 0xae, 0x60, 0xf8, 0x79, 0x54, 0xdd, 0x72, 0x3d, 0xde, 0x1d, 0x74, 0x4c, 0x3b, 0xec, 0xd7, - 0xc7, 0xde, 0x9d, 0xce, 0x64, 0x5c, 0x8f, 0x7a, 0x6e, 0x5d, 0xa6, 0x7a, 0x30, 0xf0, 0x1c, 0xf3, - 0xfe, 0xfd, 0xdd, 0x9d, 0xe3, 0x51, 0x15, 0x52, 0xf6, 0xdd, 0x1d, 0x0c, 0x29, 0xfb, 0xae, 0x83, - 0xfe, 0x03, 0xcb, 0x31, 0x19, 0x52, 0x2b, 0x0e, 0x48, 0x14, 0x77, 0x43, 0x2e, 0x33, 0xa3, 0xe1, - 0x25, 0x61, 0x6c, 0x2b, 0x1b, 0xda, 0x82, 0x5c, 0x3f, 0x74, 0xa8, 0x91, 0x5d, 0xcf, 0x6c, 0x94, - 0x4e, 0x0d, 0x69, 0xca, 0xbe, 0x1f, 0x3a, 0x14, 0xcb, 0xc5, 0xa8, 0x02, 0x89, 0x9f, 0x28, 0xf4, - 0x02, 0x6e, 0xe4, 0x24, 0xed, 0x94, 0x05, 0x19, 0xb0, 0x38, 0xa4, 0x2c, 0xf6, 0xc2, 0xc0, 0xc8, - 0xaf, 0x67, 0x36, 0x96, 0x71, 0xfa, 0x89, 0x6e, 0x43, 0x81, 0x53, 0xd6, 0xf7, 0x02, 0xc2, 0xa9, - 0xb1, 0xb0, 0x9e, 0xdd, 0x28, 0x6e, 0x5e, 0x3a, 0xc5, 0xa7, 0x8a, 0xf1, 0x0e, 0x8d, 0x6d, 0xe6, - 0x45, 0x3c, 0x64, 0x8d, 0x9c, 0x88, 0x11, 0x9e, 0x80, 0x55, 0x26, 0x1f, 0x00, 0x88, 0x10, 0x13, - 0x9b, 0x0b, 0xf6, 0x55, 0xc8, 0x77, 0x8e, 0x38, 0x8d, 0x65, 0x5c, 0xb3, 0x38, 0xf9, 0x40, 0x57, - 0x00, 0xc5, 0x03, 0xd7, 0xa5, 0x31, 0xa7, 0x8e, 0x45, 0xb8, 0x15, 0x90, 0x20, 0x8c, 0x65, 0x30, - 0xb2, 0x58, 0x1f, 0xcf, 0x6c, 0xf3, 0x03, 0x61, 0x57, 0xbc, 0xdf, 0xce, 0xc3, 0xb9, 0x76, 0x3a, - 0x35, 0xe5, 0xe1, 0x1e, 0x14, 0x62, 0x4e, 0x18, 0xb7, 0x7a, 0xf4, 0x48, 0x65, 0xef, 0x83, 0x37, - 0xa3, 0xea, 0xd5, 0x99, 0x32, 0x97, 0x9e, 0xee, 0x0e, 0x3d, 0xc2, 0x9a, 0xa4, 0xb9, 0x43, 0x8f, - 0xd0, 0x3e, 0x2c, 0xd2, 0xc0, 0x91, 0x84, 0xf3, 0x7f, 0x83, 0x70, 0x81, 0x06, 0x8e, 0xa0, 0xbb, - 0x0f, 0x60, 0x8f, 0xf7, 0x2b, 0xd3, 0x5a, 0xdc, 0x7c, 0xcf, 0x3c, 0x43, 0xde, 0xcc, 0xc9, 0x11, - 0xa7, 0xea, 0x79, 0x8a, 0x48, 0x85, 0xe5, 0xb7, 0x45, 0x58, 0x55, 0xb9, 0xe1, 0xd4, 0xb9, 0x39, - 0x24, 0x3e, 0xa6, 0xf1, 0xc0, 0x17, 0x32, 0x92, 0x97, 0x7a, 0xa4, 0xba, 0xff, 0xfd, 0x33, 0x1d, - 0x2a, 0x16, 0xa1, 0x02, 0x14, 0x27, 0x58, 0x74, 0x03, 0xf2, 0xb1, 0x50, 0x1a, 0xb5, 0xeb, 0xcb, - 0x67, 0x92, 0x48, 0x5d, 0xc2, 0x09, 0x48, 0xa0, 0xfb, 0xa2, 0xfb, 0x65, 0x3d, 0xce, 0x82, 0x96, - 0x5a, 0x81, 0x13, 0x10, 0xda, 0x00, 0xdd, 0x8b, 0x2d, 0x9f, 0x92, 0x98, 0x5a, 0x8c, 0x3e, 0x1e, - 0xd0, 0x98, 0x1b, 0x0b, 0xb2, 0xb0, 0x4b, 0x5e, 0xbc, 0x27, 0xcc, 0x38, 0xb1, 0xa2, 0x6d, 0x28, - 0x8c, 0x45, 0xd6, 0xd0, 0xa4, 0xaf, 0x7f, 0x4f, 0xf9, 0x12, 0xed, 0x69, 0x76, 0x7d, 0xdb, 0x3c, - 0x4c, 0x17, 0x8d, 0x6b, 0x37, 0x35, 0xa0, 0xbb, 0xa0, 0x3b, 0x34, 0x62, 0x54, 0x46, 0x51, 0xc9, - 0x26, 0xbc, 0x83, 0x6c, 0xe2, 0xf2, 0x04, 0x2e, 0xb5, 0x12, 0x7d, 0x01, 0x65, 0x5b, 0xaa, 0x93, - 0xc5, 0x94, 0x3c, 0x19, 0x4b, 0x92, 0xb0, 0x7e, 0x76, 0xea, 0x4f, 0xa8, 0x1a, 0x2e, 0xd9, 0x27, - 0x95, 0xf1, 0x12, 0x94, 0x18, 0x79, 0xc4, 0x2d, 0x3f, 0x74, 0xd5, 0x4e, 0x97, 0x65, 0xe7, 0x2c, - 0x09, 0xeb, 0x5e, 0xe8, 0x26, 0xfe, 0x1f, 0x43, 0x91, 0x38, 0x8e, 0x15, 0xc7, 0x9c, 0x74, 0x7c, - 0x6a, 0xac, 0x48, 0xdf, 0x9f, 0xcc, 0x5a, 0x05, 0x27, 0x6a, 0xc9, 0xdc, 0x76, 0x9c, 0x76, 0xfb, - 0x50, 0xf0, 0x34, 0x4a, 0x42, 0xde, 0x26, 0xdf, 0x18, 0x88, 0xe3, 0xb4, 0x13, 0x1f, 0xe8, 0x16, - 0xe4, 0x93, 0xfd, 0x20, 0xe9, 0xec, 0xff, 0x33, 0x45, 0x4e, 0xee, 0x56, 0x25, 0x24, 0x81, 0xa3, - 0x7d, 0x38, 0x17, 0x31, 0x3a, 0x54, 0xb9, 0x4f, 0x9e, 0x06, 0xc4, 0x37, 0x56, 0x67, 0xc8, 0x2c, - 0x5e, 0x11, 0x48, 0x59, 0x1d, 0x77, 0x15, 0x0e, 0x7d, 0x09, 0xba, 0x9d, 0x88, 0xbe, 0x95, 0x6a, - 0xb1, 0x71, 0x5e, 0x72, 0x5d, 0x9d, 0xa9, 0x0b, 0xa7, 0x6e, 0x0b, 0x5c, 0xb6, 0x4f, 0x1a, 0xd6, - 0x3e, 0x85, 0xa9, 0x68, 0x20, 0x04, 0x39, 0xf1, 0xc4, 0x48, 0x74, 0x08, 0xcb, 0x31, 0xaa, 0x42, - 0xde, 0x66, 0xf6, 0xd6, 0xa6, 0x6c, 0xc4, 0xe5, 0x46, 0xe1, 0x78, 0x54, 0xcd, 0x37, 0x71, 0x73, - 0x6b, 0x13, 0x27, 0xf6, 0xa4, 0x91, 0x5b, 0x39, 0x2d, 0xa3, 0xcf, 0xb7, 0x72, 0x5a, 0x5e, 0x5f, - 0x68, 0xe5, 0xb4, 0x45, 0x5d, 0x6b, 0xe5, 0xb4, 0x82, 0x0e, 0xad, 0x9c, 0x56, 0xd2, 0xcb, 0xad, - 0x9c, 0x56, 0xd6, 0xf5, 0x56, 0x4e, 0xd3, 0xf5, 0x95, 0x56, 0x4e, 0x3b, 0xa7, 0xaf, 0xb6, 0x16, - 0xb4, 0x6f, 0x0e, 0xf4, 0xef, 0x0f, 0x6a, 0xeb, 0x00, 0x9f, 0x33, 0x8f, 0xd3, 0x06, 0xe1, 0x76, - 0xf7, 0xb4, 0x0d, 0xd4, 0xee, 0xc1, 0xd2, 0x5e, 0xe8, 0x7a, 0x36, 0xf1, 0x3f, 0x8b, 0xf6, 0x42, - 0x17, 0x6d, 0x43, 0x36, 0x8c, 0x84, 0x22, 0x0b, 0xad, 0xff, 0xdf, 0x59, 0x49, 0x1a, 0x43, 0x55, - 0x8e, 0x04, 0xb6, 0xf6, 0x53, 0x1e, 0x8a, 0x98, 0x3c, 0xe2, 0xcd, 0xb0, 0xdf, 0x27, 0x81, 0x83, - 0x2e, 0x43, 0xb9, 0x4f, 0x9e, 0xaa, 0x84, 0x79, 0x81, 0x43, 0x9f, 0xca, 0x9e, 0xcf, 0xe1, 0xe5, - 0x3e, 0x79, 0x2a, 0xb3, 0xb1, 0x2b, 0x8c, 0xe8, 0x10, 0xfe, 0x31, 0xd5, 0x66, 0xe3, 0x47, 0x9f, - 0xc4, 0xc9, 0x8b, 0xa9, 0xb8, 0x69, 0x9c, 0x72, 0xf9, 0x24, 0xdd, 0x7e, 0x71, 0x02, 0xbd, 0xab, - 0x90, 0x72, 0x02, 0x0d, 0xe1, 0xe2, 0x49, 0x2a, 0x2b, 0x16, 0xca, 0x10, 0xd8, 0x54, 0x0a, 0x46, - 0xb6, 0xf1, 0xf1, 0x9b, 0x51, 0xf5, 0xda, 0x3b, 0xe9, 0xb7, 0x24, 0x6e, 0x2b, 0x16, 0x7c, 0x3e, - 0x9a, 0xf6, 0x97, 0x9a, 0xd1, 0x63, 0xb8, 0xc0, 0xc6, 0xed, 0x62, 0xd1, 0x21, 0xf1, 0x2d, 0x26, - 0x1b, 0x46, 0x36, 0x64, 0x71, 0xf3, 0xc3, 0xbf, 0xd4, 0x6d, 0x2a, 0xce, 0xab, 0xec, 0x34, 0x55, - 0xdf, 0x83, 0xe2, 0x13, 0x91, 0x6d, 0xab, 0x23, 0xd2, 0x6d, 0x94, 0x66, 0xbc, 0x4c, 0x26, 0x15, - 0x82, 0xe1, 0xc9, 0xa4, 0x5a, 0xda, 0x50, 0xf2, 0x93, 0xf4, 0x5a, 0x61, 0x24, 0xf4, 0xc4, 0x28, - 0xcf, 0x78, 0x59, 0x4c, 0x17, 0x14, 0x5e, 0xf2, 0xa7, 0xcb, 0xeb, 0x21, 0x00, 0x67, 0xc4, 0xa6, - 0x96, 0x2c, 0x44, 0x5d, 0x56, 0xd9, 0xf5, 0xb3, 0x23, 0x31, 0xa9, 0x26, 0xf3, 0x50, 0xc0, 0x77, - 0x08, 0x27, 0x37, 0x03, 0xce, 0x8e, 0x70, 0x81, 0xa7, 0xdf, 0x6b, 0x37, 0xa0, 0x74, 0x72, 0x12, - 0xe9, 0x90, 0x4d, 0x2f, 0xfe, 0x02, 0x16, 0x43, 0xf1, 0xe4, 0x18, 0x12, 0x7f, 0x90, 0x5c, 0x7c, - 0x05, 0x9c, 0x7c, 0x5c, 0x9b, 0xff, 0x48, 0xb4, 0x58, 0x56, 0xcf, 0x8d, 0x1b, 0x6d, 0x5e, 0xcf, - 0x26, 0x4d, 0xf4, 0xc3, 0x41, 0xed, 0x3a, 0xac, 0x4c, 0x6d, 0xe0, 0x56, 0x18, 0x72, 0xca, 0x66, - 0x2d, 0xea, 0xc6, 0x95, 0xe7, 0xaf, 0x2a, 0x73, 0xcf, 0x8f, 0x2b, 0x99, 0x17, 0xc7, 0x95, 0xcc, - 0xcb, 0xe3, 0x4a, 0xe6, 0x97, 0xe3, 0x4a, 0xe6, 0xeb, 0xd7, 0x95, 0xb9, 0x17, 0xaf, 0x2b, 0x73, - 0x2f, 0x5f, 0x57, 0xe6, 0x1e, 0xc2, 0xe4, 0x27, 0xa1, 0xb3, 0x20, 0xdf, 0xfb, 0x5b, 0xbf, 0x07, - 0x00, 0x00, 0xff, 0xff, 0x09, 0xb2, 0xa4, 0xe6, 0x0a, 0x0d, 0x00, 0x00, + 0x6c, 0xe0, 0xdd, 0xcd, 0x52, 0x81, 0xbd, 0x0b, 0x14, 0x49, 0x50, 0xc4, 0x92, 0x93, 0xc6, 0x8a, + 0xed, 0x26, 0x23, 0x27, 0x2d, 0xd2, 0x03, 0x31, 0x22, 0x27, 0x14, 0x2b, 0x8a, 0x64, 0x86, 0x23, + 0x25, 0xfe, 0x14, 0x6d, 0x81, 0x1e, 0x7a, 0x6a, 0x73, 0xec, 0xd7, 0xe8, 0x2d, 0x97, 0x02, 0x39, + 0x06, 0x3d, 0x08, 0x8d, 0x73, 0xe9, 0x67, 0xc8, 0xa9, 0x98, 0xe1, 0x50, 0x92, 0x0b, 0xa7, 0x56, + 0xda, 0xdb, 0xf0, 0xcd, 0xbc, 0xdf, 0x7b, 0xf3, 0xfe, 0xfc, 0xde, 0x10, 0x36, 0x7a, 0xc3, 0x7a, + 0x6f, 0x18, 0x53, 0x36, 0xa4, 0x6c, 0xbc, 0x88, 0x3a, 0xf5, 0x88, 0x85, 0x51, 0x18, 0x53, 0x66, + 0xf5, 0x86, 0x66, 0xc4, 0x42, 0x1e, 0xa2, 0xaa, 0x1d, 0xda, 0x3d, 0x16, 0x12, 0xbb, 0x6b, 0xf6, + 0x86, 0x66, 0x7a, 0xd4, 0x8c, 0x79, 0xc8, 0x88, 0x4b, 0xa3, 0xce, 0xda, 0x8a, 0xdc, 0x8c, 0x3a, + 0x75, 0x12, 0x79, 0x89, 0xce, 0x1a, 0x4a, 0x45, 0x0e, 0xe1, 0x44, 0xc9, 0x2e, 0xa4, 0xb2, 0x3e, + 0xe5, 0x64, 0x4a, 0xfe, 0x57, 0x85, 0x54, 0xa7, 0x81, 0xeb, 0x05, 0x54, 0x1c, 0x18, 0xda, 0xb6, + 0xda, 0xfc, 0xdb, 0xa9, 0x9b, 0x5b, 0x6a, 0xb7, 0xf6, 0x8e, 0x4b, 0xc4, 0x9c, 0x70, 0xaa, 0xce, + 0x18, 0x03, 0xee, 0xf9, 0xf5, 0xae, 0x6f, 0xd7, 0xb9, 0xd7, 0xa7, 0x31, 0x27, 0xfd, 0x48, 0xed, + 0xac, 0xba, 0xa1, 0x1b, 0xca, 0x65, 0x5d, 0xac, 0x12, 0x69, 0xed, 0xfb, 0x0c, 0xe4, 0xdb, 0x91, + 0xef, 0x71, 0xd4, 0x84, 0x45, 0xce, 0x3c, 0xd7, 0xa5, 0xcc, 0xc8, 0xac, 0x67, 0x36, 0x8a, 0x9b, + 0x55, 0x73, 0x12, 0x0a, 0x75, 0x19, 0x53, 0x1e, 0x3d, 0x4c, 0x8e, 0x35, 0xb4, 0x17, 0xa3, 0xea, + 0xdc, 0xcb, 0x51, 0x35, 0x83, 0x53, 0x4d, 0x74, 0x08, 0x05, 0xd6, 0x8d, 0x2d, 0x87, 0xfa, 0x9c, + 0x18, 0xf3, 0x12, 0xe6, 0x9f, 0x53, 0x30, 0xea, 0x7a, 0x66, 0x7a, 0x3d, 0x73, 0xff, 0x61, 0xb3, + 0xd9, 0xe6, 0x84, 0xc7, 0x0d, 0x5d, 0x80, 0x1d, 0x8f, 0xaa, 0x1a, 0xbe, 0xd3, 0xde, 0x11, 0xea, + 0x58, 0x63, 0xdd, 0x58, 0xae, 0xae, 0xe5, 0x7e, 0x79, 0x5e, 0xcd, 0xd4, 0x30, 0xe4, 0xf7, 0x29, + 0x73, 0xe9, 0x6c, 0x9e, 0xca, 0xa3, 0xef, 0xf6, 0x54, 0x61, 0x3a, 0x50, 0x6a, 0x76, 0x49, 0xe0, + 0x52, 0x4c, 0x23, 0xdf, 0xb3, 0x49, 0x8c, 0xf6, 0x7e, 0x0b, 0xbe, 0x71, 0x0a, 0xf8, 0x49, 0x9d, + 0xdf, 0xb3, 0xf2, 0xcd, 0xf3, 0xea, 0x5c, 0xed, 0xf5, 0x3c, 0x94, 0x9b, 0x61, 0x3f, 0x1a, 0x70, + 0xda, 0xec, 0x52, 0xbb, 0x17, 0x0f, 0xfa, 0xe8, 0x73, 0x28, 0xda, 0x6a, 0x6d, 0x79, 0x8e, 0xb4, + 0xb5, 0xd4, 0xd8, 0x15, 0x08, 0x3f, 0x8d, 0xaa, 0x5b, 0xae, 0xc7, 0xbb, 0x83, 0x8e, 0x69, 0x87, + 0xfd, 0xfa, 0xd8, 0xba, 0xd3, 0x99, 0xac, 0xeb, 0x51, 0xcf, 0xad, 0xcb, 0x54, 0x0f, 0x06, 0x9e, + 0x63, 0x3e, 0x78, 0xb0, 0xbb, 0x73, 0x3c, 0xaa, 0x42, 0x8a, 0xbe, 0xbb, 0x83, 0x21, 0x45, 0xdf, + 0x75, 0xd0, 0x3f, 0x60, 0x39, 0x26, 0x43, 0x6a, 0xc5, 0x01, 0x89, 0xe2, 0x6e, 0xc8, 0x65, 0x66, + 0x34, 0xbc, 0x24, 0x84, 0x6d, 0x25, 0x43, 0x5b, 0x90, 0xeb, 0x87, 0x0e, 0x35, 0xb2, 0xeb, 0x99, + 0x8d, 0xd2, 0xa9, 0x21, 0x4d, 0xd1, 0xf7, 0x43, 0x87, 0x62, 0x79, 0x18, 0x55, 0x20, 0xb1, 0x13, + 0x85, 0x5e, 0xc0, 0x8d, 0x9c, 0x84, 0x9d, 0x92, 0x20, 0x03, 0x16, 0x87, 0x94, 0xc5, 0x5e, 0x18, + 0x18, 0xf9, 0xf5, 0xcc, 0xc6, 0x32, 0x4e, 0x3f, 0xd1, 0x1d, 0x28, 0x70, 0xca, 0xfa, 0x5e, 0x40, + 0x38, 0x35, 0x16, 0xd6, 0xb3, 0x1b, 0xc5, 0xcd, 0x4b, 0xa7, 0xd8, 0x54, 0x31, 0xde, 0xa1, 0xb1, + 0xcd, 0xbc, 0x88, 0x87, 0xac, 0x91, 0x13, 0x31, 0xc2, 0x13, 0x65, 0x95, 0xc9, 0x87, 0x00, 0x22, + 0xc4, 0xc4, 0xe6, 0x02, 0x7d, 0x15, 0xf2, 0x9d, 0x23, 0x4e, 0x63, 0x19, 0xd7, 0x2c, 0x4e, 0x3e, + 0xd0, 0x15, 0x40, 0xf1, 0xc0, 0x75, 0x69, 0xcc, 0xa9, 0x63, 0x11, 0x6e, 0x05, 0x24, 0x08, 0x63, + 0x19, 0x8c, 0x2c, 0xd6, 0xc7, 0x3b, 0xdb, 0xfc, 0x40, 0xc8, 0x15, 0xee, 0xd7, 0xf3, 0x70, 0xae, + 0x9d, 0x6e, 0x4d, 0x59, 0xb8, 0x0f, 0x85, 0x98, 0x13, 0xc6, 0xad, 0x1e, 0x3d, 0x52, 0xd9, 0xfb, + 0xdf, 0xdb, 0x51, 0xf5, 0xea, 0x4c, 0x99, 0x4b, 0x6f, 0x77, 0x97, 0x1e, 0x61, 0x4d, 0xc2, 0xdc, + 0xa5, 0x47, 0x68, 0x1f, 0x16, 0x69, 0xe0, 0x48, 0xc0, 0xf9, 0x3f, 0x01, 0xb8, 0x40, 0x03, 0x47, + 0xc0, 0x3d, 0x00, 0xb0, 0xc7, 0xfe, 0xca, 0xb4, 0x16, 0x37, 0xff, 0x63, 0x9e, 0x41, 0x6f, 0xe6, + 0xe4, 0x8a, 0x53, 0xf5, 0x3c, 0x05, 0xa4, 0xc2, 0xf2, 0x83, 0x06, 0xab, 0x2a, 0x37, 0x9c, 0x3a, + 0xb7, 0x86, 0xc4, 0xc7, 0x34, 0x1e, 0xf8, 0x82, 0x46, 0xf2, 0x92, 0x8f, 0x54, 0xf7, 0xff, 0xf7, + 0x4c, 0x83, 0x0a, 0x45, 0xb0, 0x00, 0xc5, 0x89, 0x2e, 0xba, 0x01, 0xf9, 0x58, 0x30, 0x8d, 0xf2, + 0xfa, 0xf2, 0x99, 0x20, 0x92, 0x97, 0x70, 0xa2, 0x24, 0xb4, 0xfb, 0xa2, 0xfb, 0x65, 0x3d, 0xce, + 0xa2, 0x2d, 0xb9, 0x02, 0x27, 0x4a, 0x68, 0x03, 0x74, 0x2f, 0xb6, 0x7c, 0x4a, 0x62, 0x6a, 0x31, + 0xfa, 0x64, 0x40, 0x63, 0x6e, 0x2c, 0xc8, 0xc2, 0x2e, 0x79, 0xf1, 0x9e, 0x10, 0xe3, 0x44, 0x8a, + 0xb6, 0xa1, 0x30, 0x26, 0x59, 0x43, 0x93, 0xb6, 0xfe, 0x3e, 0x65, 0x4b, 0xb4, 0xa7, 0xd9, 0xf5, + 0x6d, 0xf3, 0x30, 0x3d, 0x34, 0xae, 0xdd, 0x54, 0x80, 0xee, 0x81, 0xee, 0xd0, 0x88, 0x51, 0x19, + 0x45, 0x45, 0x9b, 0xf0, 0x1e, 0xb4, 0x89, 0xcb, 0x13, 0x75, 0xc9, 0x95, 0xe8, 0x53, 0x28, 0xdb, + 0x92, 0x9d, 0x2c, 0xa6, 0xe8, 0xc9, 0x58, 0x92, 0x80, 0xf5, 0xb3, 0x53, 0x7f, 0x82, 0xd5, 0x70, + 0xc9, 0x3e, 0xc9, 0x8c, 0x97, 0xa0, 0xc4, 0xc8, 0x63, 0x6e, 0xf9, 0xa1, 0xab, 0x3c, 0x5d, 0x96, + 0x9d, 0xb3, 0x24, 0xa4, 0x7b, 0xa1, 0x9b, 0xd8, 0x7f, 0x02, 0x45, 0xe2, 0x38, 0x56, 0x1c, 0x73, + 0xd2, 0xf1, 0xa9, 0xb1, 0x22, 0x6d, 0xdf, 0x9c, 0xb5, 0x0a, 0x4e, 0xd4, 0x92, 0xb9, 0xed, 0x38, + 0xed, 0xf6, 0xa1, 0xc0, 0x69, 0x94, 0x04, 0xbd, 0x4d, 0xbe, 0x31, 0x10, 0xc7, 0x69, 0x27, 0x36, + 0xd0, 0x6d, 0xc8, 0x27, 0xfe, 0x20, 0x69, 0xec, 0xdf, 0x33, 0x45, 0x4e, 0x7a, 0xab, 0x12, 0x92, + 0xa8, 0xa3, 0x2f, 0x32, 0x70, 0x2e, 0x62, 0x74, 0xa8, 0x92, 0x9f, 0xbc, 0x0d, 0x88, 0x6f, 0xac, + 0xce, 0x92, 0xda, 0x9b, 0x6f, 0x47, 0xd5, 0x1b, 0xb3, 0xd3, 0xb6, 0x50, 0x6e, 0xfa, 0xa1, 0xdd, + 0x1b, 0x23, 0xe0, 0x15, 0x61, 0x5b, 0x16, 0xd8, 0x3d, 0x65, 0x19, 0x7d, 0x06, 0xba, 0x9d, 0xcc, + 0x0d, 0x2b, 0xa5, 0x73, 0xe3, 0xbc, 0xf4, 0xe6, 0xea, 0x4c, 0x8d, 0x3c, 0x35, 0x70, 0x70, 0xd9, + 0x3e, 0x29, 0x58, 0xfb, 0x08, 0xa6, 0x02, 0x8a, 0x10, 0xe4, 0xc4, 0x2b, 0x25, 0xa1, 0x32, 0x2c, + 0xd7, 0xa8, 0x0a, 0x79, 0x9b, 0xd9, 0x5b, 0x9b, 0xb2, 0x97, 0x97, 0x1b, 0x85, 0xe3, 0x51, 0x35, + 0xdf, 0xc4, 0xcd, 0xad, 0x4d, 0x9c, 0xc8, 0x13, 0x2e, 0x68, 0xe5, 0xb4, 0x8c, 0x3e, 0xdf, 0xca, + 0x69, 0x79, 0x7d, 0xa1, 0x95, 0xd3, 0x16, 0x75, 0xad, 0x95, 0xd3, 0x0a, 0x3a, 0xb4, 0x72, 0x5a, + 0x49, 0x2f, 0xb7, 0x72, 0x5a, 0x59, 0xd7, 0x5b, 0x39, 0x4d, 0xd7, 0x57, 0x5a, 0x39, 0xed, 0x9c, + 0xbe, 0xda, 0x5a, 0xd0, 0xbe, 0x3a, 0xd0, 0xbf, 0x3d, 0xa8, 0xad, 0x03, 0x7c, 0xc2, 0x3c, 0x4e, + 0x1b, 0x84, 0xdb, 0xdd, 0xd3, 0x1c, 0xa8, 0xdd, 0x87, 0xa5, 0xbd, 0xd0, 0xf5, 0x6c, 0xe2, 0x7f, + 0x1c, 0xed, 0x85, 0x2e, 0xda, 0x86, 0x6c, 0x18, 0x09, 0x52, 0x17, 0xe3, 0xe2, 0x5f, 0x67, 0xe5, + 0x79, 0xac, 0xaa, 0xd2, 0x2c, 0x74, 0x6b, 0x3f, 0xe6, 0xa1, 0x88, 0xc9, 0x63, 0xde, 0x0c, 0xfb, + 0x7d, 0x12, 0x38, 0xe8, 0x32, 0x94, 0xfb, 0xe4, 0x99, 0x4a, 0xb9, 0x17, 0x38, 0xf4, 0x99, 0xa4, + 0x8d, 0x1c, 0x5e, 0xee, 0x93, 0x67, 0x32, 0x1b, 0xbb, 0x42, 0x88, 0x0e, 0xe1, 0x2f, 0x53, 0x9d, + 0x3a, 0x7e, 0x37, 0x4a, 0x3d, 0x39, 0xdb, 0x8a, 0x9b, 0xc6, 0x29, 0xf3, 0x2b, 0x21, 0x8c, 0x8b, + 0x13, 0xd5, 0x7b, 0x4a, 0x53, 0x6e, 0xa0, 0x21, 0x5c, 0x3c, 0x09, 0x65, 0xc5, 0x82, 0x5c, 0x02, + 0x9b, 0x4a, 0xce, 0xc9, 0x36, 0x3e, 0x7c, 0x3b, 0xaa, 0x5e, 0x7b, 0xaf, 0x11, 0x20, 0x81, 0xdb, + 0x0a, 0x05, 0x9f, 0x8f, 0xa6, 0xed, 0xa5, 0x62, 0xf4, 0x04, 0x2e, 0xb0, 0x71, 0xc7, 0x59, 0x74, + 0x48, 0x7c, 0x8b, 0xc9, 0x9e, 0x93, 0x3d, 0x5d, 0xdc, 0xfc, 0xff, 0x1f, 0x6a, 0x58, 0x15, 0xe7, + 0x55, 0x76, 0xda, 0x60, 0xd8, 0x83, 0xe2, 0x53, 0x91, 0x6d, 0xab, 0x23, 0xd2, 0x6d, 0x94, 0x66, + 0x9c, 0x47, 0x93, 0x0a, 0xc1, 0xf0, 0x74, 0x52, 0x2d, 0x6d, 0x28, 0xf9, 0x49, 0x7a, 0xad, 0x30, + 0x12, 0x94, 0x64, 0x94, 0x67, 0x9c, 0x37, 0xd3, 0x05, 0x85, 0x97, 0xfc, 0xe9, 0xf2, 0x7a, 0x04, + 0xc0, 0x19, 0xb1, 0xa9, 0x25, 0x0b, 0x51, 0x97, 0x55, 0x76, 0xfd, 0xec, 0x48, 0x4c, 0xaa, 0xc9, + 0x3c, 0x14, 0xea, 0x3b, 0x84, 0x93, 0x5b, 0x01, 0x67, 0x47, 0xb8, 0xc0, 0xd3, 0xef, 0xb5, 0x1b, + 0x50, 0x3a, 0xb9, 0x89, 0x74, 0xc8, 0xa6, 0x6f, 0x87, 0x02, 0x16, 0x4b, 0xf1, 0x6a, 0x19, 0x12, + 0x7f, 0x90, 0xcc, 0xce, 0x02, 0x4e, 0x3e, 0xae, 0xcd, 0x7f, 0x20, 0x5a, 0x2c, 0xab, 0xe7, 0xc6, + 0x8d, 0x36, 0xaf, 0x67, 0x93, 0x26, 0xfa, 0xee, 0xa0, 0x76, 0x1d, 0x56, 0xa6, 0x1c, 0xb8, 0x1d, + 0x86, 0x9c, 0xb2, 0x59, 0x8b, 0xba, 0x71, 0xe5, 0xc5, 0xeb, 0xca, 0xdc, 0x8b, 0xe3, 0x4a, 0xe6, + 0xe5, 0x71, 0x25, 0xf3, 0xea, 0xb8, 0x92, 0xf9, 0xf9, 0xb8, 0x92, 0xf9, 0xf2, 0x4d, 0x65, 0xee, + 0xe5, 0x9b, 0xca, 0xdc, 0xab, 0x37, 0x95, 0xb9, 0x47, 0x30, 0xf9, 0xcf, 0xe8, 0x2c, 0xc8, 0x5f, + 0x86, 0xad, 0x5f, 0x03, 0x00, 0x00, 0xff, 0xff, 0xf1, 0xe4, 0x48, 0xb2, 0x4d, 0x0d, 0x00, 0x00, } diff --git a/pkg/kv/kvserver/kvserverpb/proposer_kv.proto b/pkg/kv/kvserver/kvserverpb/proposer_kv.proto index d7ccd83e4e93..5bc6530d459a 100644 --- a/pkg/kv/kvserver/kvserverpb/proposer_kv.proto +++ b/pkg/kv/kvserver/kvserverpb/proposer_kv.proto @@ -156,7 +156,7 @@ message ReplicatedEvalResult { // This is the proposal timestamp for the active lease while evaluating a lease request. // It will be used to make sure we know if a lease was extended after we sent out the request // but before we tried to apply it. - util.hlc.Timestamp prev_lease_proposal = 20; + util.hlc.Timestamp prev_lease_proposal = 20 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/util/hlc.ClockTimestamp"]; reserved 1, 5, 7, 9, 14, 15, 16, 19, 10001 to 10013; } diff --git a/pkg/kv/kvserver/observedts/limit.go b/pkg/kv/kvserver/observedts/limit.go index 484d52807acd..d93dd7ecf098 100644 --- a/pkg/kv/kvserver/observedts/limit.go +++ b/pkg/kv/kvserver/observedts/limit.go @@ -54,10 +54,11 @@ func LimitTxnMaxTimestamp( // Note that we care about an observed timestamp from the leaseholder's // node, even if this is a follower read on a different node. See the // comment in doc.go about "Follower Reads" for more. - obsTS, ok := txn.GetObservedTimestamp(status.Lease.Replica.NodeID) + obsClockTS, ok := txn.GetObservedTimestamp(status.Lease.Replica.NodeID) if !ok { return txn } + obsTS := obsClockTS.ToTimestamp() // If the lease is valid, we use the greater of the observed timestamp and // the lease start time, up to the max timestamp. This ensures we avoid // incorrect assumptions about when data was written, in absolute time on a diff --git a/pkg/kv/kvserver/observedts/limit_test.go b/pkg/kv/kvserver/observedts/limit_test.go index e169645b28a8..8d16d78f6ef1 100644 --- a/pkg/kv/kvserver/observedts/limit_test.go +++ b/pkg/kv/kvserver/observedts/limit_test.go @@ -28,7 +28,7 @@ func TestLimitTxnMaxTimestamp(t *testing.T) { ReadTimestamp: hlc.Timestamp{WallTime: 10}, MaxTimestamp: hlc.Timestamp{WallTime: 20}, } - txn.UpdateObservedTimestamp(1, hlc.Timestamp{WallTime: 15}) + txn.UpdateObservedTimestamp(1, hlc.ClockTimestamp{WallTime: 15}) txnWithMaxTimestamp := func(ts hlc.Timestamp) *roachpb.Transaction { txnClone := txn.Clone() txnClone.MaxTimestamp = ts diff --git a/pkg/kv/kvserver/replica.go b/pkg/kv/kvserver/replica.go index 1f6b68237a53..3c616411f92d 100644 --- a/pkg/kv/kvserver/replica.go +++ b/pkg/kv/kvserver/replica.go @@ -336,7 +336,7 @@ type Replica struct { // - a lease cannot be used after a transfer is initiated. Moreover, even // lease extension that were in flight at the time of the transfer cannot be // used, if they eventually apply. - minLeaseProposedTS hlc.Timestamp + minLeaseProposedTS hlc.ClockTimestamp // A pointer to the zone config for this replica. zone *zonepb.ZoneConfig // proposalBuf buffers Raft commands as they are passed to the Raft diff --git a/pkg/kv/kvserver/replica_application_state_machine.go b/pkg/kv/kvserver/replica_application_state_machine.go index ad02664c0565..f9f8644b9b94 100644 --- a/pkg/kv/kvserver/replica_application_state_machine.go +++ b/pkg/kv/kvserver/replica_application_state_machine.go @@ -375,9 +375,9 @@ type replicaAppBatch struct { // replicaState other than Stats are overwritten completely rather than // updated in-place. stats enginepb.MVCCStats - // maxTS is the maximum timestamp that any command that was staged in this - // batch was evaluated at. - maxTS hlc.Timestamp + // maxTS is the maximum clock timestamp that any command that was staged in + // this batch was evaluated at. + maxTS hlc.ClockTimestamp // migrateToAppliedStateKey tracks whether any command in the batch // triggered a migration to the replica applied state key. If so, this // migration will be performed when the application batch is committed. @@ -463,7 +463,9 @@ func (b *replicaAppBatch) Stage(cmdI apply.Command) (apply.CheckedCommand, error } // Update the batch's max timestamp. - b.maxTS.Forward(cmd.replicatedResult().Timestamp) + if clockTS, ok := cmd.replicatedResult().Timestamp.TryToClockTimestamp(); ok { + b.maxTS.Forward(clockTS) + } // Normalize the command, accounting for past migrations. b.migrateReplicatedResult(ctx, cmd) diff --git a/pkg/kv/kvserver/replica_gc_queue.go b/pkg/kv/kvserver/replica_gc_queue.go index 9e24ab5a4832..75b70d13c167 100644 --- a/pkg/kv/kvserver/replica_gc_queue.go +++ b/pkg/kv/kvserver/replica_gc_queue.go @@ -141,7 +141,7 @@ func (rgcq *replicaGCQueue) shouldQueue( } if lease, _ := repl.GetLease(); lease.ProposedTS != nil { - lastActivity.Forward(*lease.ProposedTS) + lastActivity.Forward(lease.ProposedTS.ToTimestamp()) } // It is critical to think of the replica as suspect if it is a learner as diff --git a/pkg/kv/kvserver/replica_init.go b/pkg/kv/kvserver/replica_init.go index 9a9058563921..ed73f806bbbf 100644 --- a/pkg/kv/kvserver/replica_init.go +++ b/pkg/kv/kvserver/replica_init.go @@ -187,7 +187,7 @@ func (r *Replica) loadRaftMuLockedReplicaMuLocked(desc *roachpb.RangeDescriptor) // Instead, we make the first lease special (which is OK) and the problem // disappears. if r.mu.state.Lease.Sequence > 0 { - r.mu.minLeaseProposedTS = r.Clock().Now() + r.mu.minLeaseProposedTS = r.Clock().NowAsClockTimestamp() } } diff --git a/pkg/kv/kvserver/replica_range_lease.go b/pkg/kv/kvserver/replica_range_lease.go index 612dab9215db..e488e65fb389 100644 --- a/pkg/kv/kvserver/replica_range_lease.go +++ b/pkg/kv/kvserver/replica_range_lease.go @@ -202,7 +202,13 @@ func (p *pendingLeaseRequest) InitOrJoinRequest( Key: startKey, } var leaseReq roachpb.Request - now := p.repl.store.Clock().Now() + now := p.repl.store.Clock().NowAsClockTimestamp() + // TODO(nvanbenschoten): what if status.Timestamp > now? Is that ok? + // Probably not, because the existing lease may end in the future, + // but we may be trying to evaluate a request even further into the + // future. + // Currently, this is the interaction that prevents us from making + // Lease.Start a ClockTimestamp. reqLease := roachpb.Lease{ // It's up to us to ensure that Lease.Start is greater than the // end time of the previous lease. This means that if status @@ -538,7 +544,10 @@ func (p *pendingLeaseRequest) newResolvedHandle(pErr *roachpb.Error) *leaseReque // processed the change in lease holdership). // * the client fails to read their own write. func (r *Replica) leaseStatus( - ctx context.Context, lease roachpb.Lease, timestamp, minProposedTS hlc.Timestamp, + ctx context.Context, + lease roachpb.Lease, + timestamp hlc.Timestamp, + minProposedTS hlc.ClockTimestamp, ) kvserverpb.LeaseStatus { status := kvserverpb.LeaseStatus{Timestamp: timestamp, Lease: lease} var expiration hlc.Timestamp @@ -671,7 +680,8 @@ func (r *Replica) AdminTransferLease(ctx context.Context, target roachpb.StoreID r.mu.Lock() defer r.mu.Unlock() - status := r.leaseStatus(ctx, *r.mu.state.Lease, r.store.Clock().Now(), r.mu.minLeaseProposedTS) + now := r.store.Clock().NowAsClockTimestamp() + status := r.leaseStatus(ctx, *r.mu.state.Lease, now.ToTimestamp(), r.mu.minLeaseProposedTS) if status.Lease.OwnedBy(target) { // The target is already the lease holder. Nothing to do. return nil, nil, nil @@ -716,7 +726,7 @@ func (r *Replica) AdminTransferLease(ctx context.Context, target roachpb.StoreID "another transfer to a different store is in progress") } // Stop using the current lease. - r.mu.minLeaseProposedTS = status.Timestamp + r.mu.minLeaseProposedTS = now transfer = r.mu.pendingLeaseRequest.InitOrJoinRequest( ctx, nextLeaseHolder, status, desc.StartKey.AsRawKey(), true, /* transfer */ ) @@ -860,6 +870,9 @@ func newNotLeaseHolderError( // existing lease is valid and owned by the current store. This method should // not be called directly. Use redirectOnOrAcquireLease instead. func (r *Replica) leaseGoodToGo(ctx context.Context) (kvserverpb.LeaseStatus, bool) { + // TODO(nvanbenschoten): this should take the request timestamp and forward + // now by that timestamp. Should we limit how far in the future this timestamp + // can lead clock.Now()? Something to do with < now + RangeLeaseRenewalDuration()? timestamp := r.store.Clock().Now() r.mu.RLock() defer r.mu.RUnlock() @@ -897,6 +910,8 @@ func (r *Replica) leaseGoodToGo(ctx context.Context) (kvserverpb.LeaseStatus, bo // // TODO(rangeLeaseRenewalDuration): what is rangeLeaseRenewalDuration // referring to? It appears to have rotted. +// +// TODO(nvanbenschoten): reword comment to account for request timestamp. func (r *Replica) redirectOnOrAcquireLease( ctx context.Context, ) (kvserverpb.LeaseStatus, *roachpb.Error) { @@ -908,6 +923,8 @@ func (r *Replica) redirectOnOrAcquireLease( // lease holder. Returns also on context.Done() (timeout or cancellation). var status kvserverpb.LeaseStatus for attempt := 1; ; attempt++ { + // TODO(nvanbenschoten): this should take the request timestamp and + // forward now by that timestamp. See TODO in leaseGoodToGo. timestamp := r.store.Clock().Now() llHandle, pErr := func() (*leaseRequestHandle, *roachpb.Error) { r.mu.Lock() diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index 4f6d43585c7f..5f151994064e 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -320,7 +320,7 @@ func (tc *testContext) Sender() kv.Sender { tc.Fatal(err) } } - tc.Clock().Update(ba.Timestamp) + tc.Clock().Update(ba.Timestamp.UnsafeToClockTimestamp()) return ba }) } @@ -820,7 +820,7 @@ func TestBehaviorDuringLeaseTransfer(t *testing.T) { minLeaseProposedTS := tc.repl.mu.minLeaseProposedTS leaseStartTS := tc.repl.mu.state.Lease.Start tc.repl.mu.Unlock() - if minLeaseProposedTS.LessEq(leaseStartTS) { + if minLeaseProposedTS.ToTimestamp().LessEq(leaseStartTS) { t.Fatalf("expected minLeaseProposedTS > lease start. minLeaseProposedTS: %s, "+ "leas start: %s", minLeaseProposedTS, leaseStartTS) } @@ -1650,7 +1650,7 @@ func TestReplicaNoGossipFromNonLeader(t *testing.T) { tc.manualClock.Set(leaseExpiry(tc.repl)) lease, _ := tc.repl.GetLease() if tc.repl.leaseStatus(context.Background(), - lease, tc.Clock().Now(), hlc.Timestamp{}).State != kvserverpb.LeaseState_EXPIRED { + lease, tc.Clock().Now(), hlc.ClockTimestamp{}).State != kvserverpb.LeaseState_EXPIRED { t.Fatal("range lease should have been expired") } @@ -2158,10 +2158,10 @@ func TestAcquireLease(t *testing.T) { tc.repl.mu.Lock() if !withMinLeaseProposedTS { - tc.repl.mu.minLeaseProposedTS = hlc.Timestamp{} + tc.repl.mu.minLeaseProposedTS = hlc.ClockTimestamp{} expStart = lease.Start } else { - expStart = tc.repl.mu.minLeaseProposedTS + expStart = tc.repl.mu.minLeaseProposedTS.ToTimestamp() } tc.repl.mu.Unlock() @@ -2258,7 +2258,7 @@ func TestLeaseConcurrent(t *testing.T) { for i := 0; i < num; i++ { if err := stopper.RunAsyncTask(context.Background(), "test", func(ctx context.Context) { tc.repl.mu.Lock() - status := tc.repl.leaseStatus(ctx, *tc.repl.mu.state.Lease, ts, hlc.Timestamp{}) + status := tc.repl.leaseStatus(ctx, *tc.repl.mu.state.Lease, ts, hlc.ClockTimestamp{}) llHandle := tc.repl.requestLeaseLocked(ctx, status) tc.repl.mu.Unlock() wg.Done() diff --git a/pkg/kv/kvserver/store_merge.go b/pkg/kv/kvserver/store_merge.go index de2c90694deb..0b8c9c03f24c 100644 --- a/pkg/kv/kvserver/store_merge.go +++ b/pkg/kv/kvserver/store_merge.go @@ -28,7 +28,7 @@ func (s *Store) MergeRange( ctx context.Context, leftRepl *Replica, newLeftDesc, rightDesc roachpb.RangeDescriptor, - freezeStart hlc.Timestamp, + freezeStart hlc.ClockTimestamp, ) error { if oldLeftDesc := leftRepl.Desc(); !oldLeftDesc.EndKey.Less(newLeftDesc.EndKey) { return errors.Errorf("the new end key is not greater than the current one: %+v <= %+v", @@ -87,7 +87,7 @@ func (s *Store) MergeRange( // timestamps in the timestamp cache. For a full discussion, see the comment // on TestStoreRangeMergeTimestampCacheCausality. s.Clock().Update(freezeStart) - setTimestampCacheLowWaterMark(s.tsCache, &rightDesc, freezeStart) + setTimestampCacheLowWaterMark(s.tsCache, &rightDesc, freezeStart.ToTimestamp()) } // Update the subsuming range's descriptor. diff --git a/pkg/kv/kvserver/store_send.go b/pkg/kv/kvserver/store_send.go index 1ee43455b2dc..16110e7acfe3 100644 --- a/pkg/kv/kvserver/store_send.go +++ b/pkg/kv/kvserver/store_send.go @@ -85,14 +85,16 @@ func (s *Store) Send( // Update our clock with the incoming request timestamp. This advances the // local node's clock to a high water mark from all nodes with which it has // interacted. - if s.cfg.TestingKnobs.DisableMaxOffsetCheck { - s.cfg.Clock.Update(ba.Timestamp) - } else { - // If the command appears to come from a node with a bad clock, - // reject it now before we reach that point. - var err error - if err = s.cfg.Clock.UpdateAndCheckMaxOffset(ctx, ba.Timestamp); err != nil { - return nil, roachpb.NewError(err) + if baClockTS, ok := ba.Timestamp.TryToClockTimestamp(); ok { + if s.cfg.TestingKnobs.DisableMaxOffsetCheck { + s.cfg.Clock.Update(baClockTS) + } else { + // If the command appears to come from a node with a bad clock, + // reject it now before we reach that point. + var err error + if err = s.cfg.Clock.UpdateAndCheckMaxOffset(ctx, baClockTS); err != nil { + return nil, roachpb.NewError(err) + } } } @@ -119,7 +121,9 @@ func (s *Store) Send( // Update our clock with the outgoing response txn timestamp // (if timestamp has been forwarded). if ba.Timestamp.Less(br.Txn.WriteTimestamp) { - s.cfg.Clock.Update(br.Txn.WriteTimestamp) + if clockTS, ok := br.Txn.WriteTimestamp.TryToClockTimestamp(); ok { + s.cfg.Clock.Update(clockTS) + } } } } else { @@ -127,7 +131,9 @@ func (s *Store) Send( // Update our clock with the outgoing response timestamp. // (if timestamp has been forwarded). if ba.Timestamp.Less(br.Timestamp) { - s.cfg.Clock.Update(br.Timestamp) + if clockTS, ok := br.Timestamp.TryToClockTimestamp(); ok { + s.cfg.Clock.Update(clockTS) + } } } } @@ -135,7 +141,7 @@ func (s *Store) Send( // We get the latest timestamp - we know that any // write with a higher timestamp we run into later must // have started after this point in (absolute) time. - now := s.cfg.Clock.Now() + now := s.cfg.Clock.NowAsClockTimestamp() if pErr != nil { pErr.Now = now } else { @@ -152,7 +158,7 @@ func (s *Store) Send( // this node, in which case the following is a no-op). if _, ok := ba.Txn.GetObservedTimestamp(s.NodeID()); !ok { txnClone := ba.Txn.Clone() - txnClone.UpdateObservedTimestamp(s.NodeID(), s.Clock().Now()) + txnClone.UpdateObservedTimestamp(s.NodeID(), s.Clock().NowAsClockTimestamp()) ba.Txn = txnClone } } diff --git a/pkg/kv/kvserver/store_test.go b/pkg/kv/kvserver/store_test.go index dfaaa735c021..b7c74665bb68 100644 --- a/pkg/kv/kvserver/store_test.go +++ b/pkg/kv/kvserver/store_test.go @@ -1854,11 +1854,12 @@ func TestStoreResolveWriteIntentPushOnRead(t *testing.T) { } // Determine the timestamp to read at. - readTs := store.cfg.Clock.Now() + clockTs := store.cfg.Clock.NowAsClockTimestamp() + readTs := clockTs.ToTimestamp() // Give the pusher a previous observed timestamp equal to this read // timestamp. This ensures that the pusher doesn't need to push the // intent any higher just to push it out of its uncertainty window. - pusher.UpdateObservedTimestamp(store.Ident.NodeID, readTs) + pusher.UpdateObservedTimestamp(store.Ident.NodeID, clockTs) // If the pushee is already pushed, update the transaction record. if tc.pusheeAlreadyPushed { diff --git a/pkg/kv/txn.go b/pkg/kv/txn.go index 460d44a125c3..b245bfeea795 100644 --- a/pkg/kv/txn.go +++ b/pkg/kv/txn.go @@ -98,12 +98,12 @@ func NewTxn(ctx context.Context, db *DB, gatewayNodeID roachpb.NodeID) *Txn { errors.AssertionFailedf("attempting to create txn with nil db"), ctx)) } - now := db.clock.Now() + now := db.clock.NowAsClockTimestamp() kvTxn := roachpb.MakeTransaction( "unnamed", nil, // baseKey roachpb.NormalUserPriority, - now, + now.ToTimestamp(), db.clock.MaxOffset().Nanoseconds(), ) @@ -124,7 +124,7 @@ func NewTxnFromProto( ctx context.Context, db *DB, gatewayNodeID roachpb.NodeID, - now hlc.Timestamp, + now hlc.ClockTimestamp, typ TxnType, proto *roachpb.Transaction, ) *Txn { @@ -1137,8 +1137,8 @@ func (txn *Txn) SetFixedTimestamp(ctx context.Context, ts hlc.Timestamp) { func (txn *Txn) GenerateForcedRetryableError(ctx context.Context, msg string) error { txn.mu.Lock() defer txn.mu.Unlock() - now := txn.db.clock.Now() - txn.mu.sender.ManualRestart(ctx, txn.mu.userPriority, now) + now := txn.db.clock.NowAsClockTimestamp() + txn.mu.sender.ManualRestart(ctx, txn.mu.userPriority, now.ToTimestamp()) txn.resetDeadlineLocked() return roachpb.NewTransactionRetryWithProtoRefreshError( msg, @@ -1147,7 +1147,7 @@ func (txn *Txn) GenerateForcedRetryableError(ctx context.Context, msg string) er txn.debugNameLocked(), nil, // baseKey txn.mu.userPriority, - now, + now.ToTimestamp(), txn.db.clock.MaxOffset().Nanoseconds(), )) } diff --git a/pkg/kv/txn_test.go b/pkg/kv/txn_test.go index 40b189072770..e0b3813e4308 100644 --- a/pkg/kv/txn_test.go +++ b/pkg/kv/txn_test.go @@ -280,7 +280,7 @@ func TestRunTransactionRetryOnErrors(t *testing.T) { if errors.HasType(test.err, (*roachpb.ReadWithinUncertaintyIntervalError)(nil)) { // This error requires an observed timestamp to have been // recorded on the origin node. - ba.Txn.UpdateObservedTimestamp(1, hlc.Timestamp{WallTime: 1, Logical: 1}) + ba.Txn.UpdateObservedTimestamp(1, hlc.ClockTimestamp{WallTime: 1, Logical: 1}) pErr = roachpb.NewErrorWithTxn(test.err, ba.Txn) pErr.OriginNode = 1 } else { diff --git a/pkg/roachpb/api.pb.go b/pkg/roachpb/api.pb.go index a25ebd0cf11f..57a127477414 100644 --- a/pkg/roachpb/api.pb.go +++ b/pkg/roachpb/api.pb.go @@ -13,6 +13,7 @@ import tracingpb "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" import github_com_cockroachdb_cockroach_pkg_storage_enginepb "github.com/cockroachdb/cockroach/pkg/storage/enginepb" import github_com_cockroachdb_cockroach_pkg_util_uuid "github.com/cockroachdb/cockroach/pkg/util/uuid" +import github_com_cockroachdb_cockroach_pkg_util_hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" import time "time" import ( @@ -73,7 +74,7 @@ func (x ReadConsistencyType) String() string { return proto.EnumName(ReadConsistencyType_name, int32(x)) } func (ReadConsistencyType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{0} + return fileDescriptor_api_9819b93932ee2bad, []int{0} } // ScanFormat is an enumeration of the available response formats for MVCCScan @@ -101,7 +102,7 @@ func (x ScanFormat) String() string { return proto.EnumName(ScanFormat_name, int32(x)) } func (ScanFormat) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{1} + return fileDescriptor_api_9819b93932ee2bad, []int{1} } type ChecksumMode int32 @@ -148,7 +149,7 @@ func (x ChecksumMode) String() string { return proto.EnumName(ChecksumMode_name, int32(x)) } func (ChecksumMode) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{2} + return fileDescriptor_api_9819b93932ee2bad, []int{2} } // PushTxnType determines what action to take when pushing a transaction. @@ -179,7 +180,7 @@ func (x PushTxnType) String() string { return proto.EnumName(PushTxnType_name, int32(x)) } func (PushTxnType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{3} + return fileDescriptor_api_9819b93932ee2bad, []int{3} } type ExternalStorageProvider int32 @@ -220,7 +221,7 @@ func (x ExternalStorageProvider) String() string { return proto.EnumName(ExternalStorageProvider_name, int32(x)) } func (ExternalStorageProvider) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{4} + return fileDescriptor_api_9819b93932ee2bad, []int{4} } type MVCCFilter int32 @@ -243,7 +244,7 @@ func (x MVCCFilter) String() string { return proto.EnumName(MVCCFilter_name, int32(x)) } func (MVCCFilter) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{5} + return fileDescriptor_api_9819b93932ee2bad, []int{5} } type ResponseHeader_ResumeReason int32 @@ -269,7 +270,7 @@ func (x ResponseHeader_ResumeReason) String() string { return proto.EnumName(ResponseHeader_ResumeReason_name, int32(x)) } func (ResponseHeader_ResumeReason) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{1, 0} + return fileDescriptor_api_9819b93932ee2bad, []int{1, 0} } type CheckConsistencyResponse_Status int32 @@ -311,7 +312,7 @@ func (x CheckConsistencyResponse_Status) String() string { return proto.EnumName(CheckConsistencyResponse_Status_name, int32(x)) } func (CheckConsistencyResponse_Status) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{25, 0} + return fileDescriptor_api_9819b93932ee2bad, []int{25, 0} } // RequestHeader is supplied with every storage node request. @@ -332,7 +333,7 @@ func (m *RequestHeader) Reset() { *m = RequestHeader{} } func (m *RequestHeader) String() string { return proto.CompactTextString(m) } func (*RequestHeader) ProtoMessage() {} func (*RequestHeader) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{0} + return fileDescriptor_api_9819b93932ee2bad, []int{0} } func (m *RequestHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -403,7 +404,7 @@ func (m *ResponseHeader) Reset() { *m = ResponseHeader{} } func (m *ResponseHeader) String() string { return proto.CompactTextString(m) } func (*ResponseHeader) ProtoMessage() {} func (*ResponseHeader) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{1} + return fileDescriptor_api_9819b93932ee2bad, []int{1} } func (m *ResponseHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -437,7 +438,7 @@ func (m *GetRequest) Reset() { *m = GetRequest{} } func (m *GetRequest) String() string { return proto.CompactTextString(m) } func (*GetRequest) ProtoMessage() {} func (*GetRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{2} + return fileDescriptor_api_9819b93932ee2bad, []int{2} } func (m *GetRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -480,7 +481,7 @@ func (m *GetResponse) Reset() { *m = GetResponse{} } func (m *GetResponse) String() string { return proto.CompactTextString(m) } func (*GetResponse) ProtoMessage() {} func (*GetResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{3} + return fileDescriptor_api_9819b93932ee2bad, []int{3} } func (m *GetResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -523,7 +524,7 @@ func (m *PutRequest) Reset() { *m = PutRequest{} } func (m *PutRequest) String() string { return proto.CompactTextString(m) } func (*PutRequest) ProtoMessage() {} func (*PutRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{4} + return fileDescriptor_api_9819b93932ee2bad, []int{4} } func (m *PutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -557,7 +558,7 @@ func (m *PutResponse) Reset() { *m = PutResponse{} } func (m *PutResponse) String() string { return proto.CompactTextString(m) } func (*PutResponse) ProtoMessage() {} func (*PutResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{5} + return fileDescriptor_api_9819b93932ee2bad, []int{5} } func (m *PutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -646,7 +647,7 @@ func (m *ConditionalPutRequest) Reset() { *m = ConditionalPutRequest{} } func (m *ConditionalPutRequest) String() string { return proto.CompactTextString(m) } func (*ConditionalPutRequest) ProtoMessage() {} func (*ConditionalPutRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{6} + return fileDescriptor_api_9819b93932ee2bad, []int{6} } func (m *ConditionalPutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -681,7 +682,7 @@ func (m *ConditionalPutResponse) Reset() { *m = ConditionalPutResponse{} func (m *ConditionalPutResponse) String() string { return proto.CompactTextString(m) } func (*ConditionalPutResponse) ProtoMessage() {} func (*ConditionalPutResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{7} + return fileDescriptor_api_9819b93932ee2bad, []int{7} } func (m *ConditionalPutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -727,7 +728,7 @@ func (m *InitPutRequest) Reset() { *m = InitPutRequest{} } func (m *InitPutRequest) String() string { return proto.CompactTextString(m) } func (*InitPutRequest) ProtoMessage() {} func (*InitPutRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{8} + return fileDescriptor_api_9819b93932ee2bad, []int{8} } func (m *InitPutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -761,7 +762,7 @@ func (m *InitPutResponse) Reset() { *m = InitPutResponse{} } func (m *InitPutResponse) String() string { return proto.CompactTextString(m) } func (*InitPutResponse) ProtoMessage() {} func (*InitPutResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{9} + return fileDescriptor_api_9819b93932ee2bad, []int{9} } func (m *InitPutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -801,7 +802,7 @@ func (m *IncrementRequest) Reset() { *m = IncrementRequest{} } func (m *IncrementRequest) String() string { return proto.CompactTextString(m) } func (*IncrementRequest) ProtoMessage() {} func (*IncrementRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{10} + return fileDescriptor_api_9819b93932ee2bad, []int{10} } func (m *IncrementRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -838,7 +839,7 @@ func (m *IncrementResponse) Reset() { *m = IncrementResponse{} } func (m *IncrementResponse) String() string { return proto.CompactTextString(m) } func (*IncrementResponse) ProtoMessage() {} func (*IncrementResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{11} + return fileDescriptor_api_9819b93932ee2bad, []int{11} } func (m *IncrementResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -872,7 +873,7 @@ func (m *DeleteRequest) Reset() { *m = DeleteRequest{} } func (m *DeleteRequest) String() string { return proto.CompactTextString(m) } func (*DeleteRequest) ProtoMessage() {} func (*DeleteRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{12} + return fileDescriptor_api_9819b93932ee2bad, []int{12} } func (m *DeleteRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -906,7 +907,7 @@ func (m *DeleteResponse) Reset() { *m = DeleteResponse{} } func (m *DeleteResponse) String() string { return proto.CompactTextString(m) } func (*DeleteResponse) ProtoMessage() {} func (*DeleteResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{13} + return fileDescriptor_api_9819b93932ee2bad, []int{13} } func (m *DeleteResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -958,7 +959,7 @@ func (m *DeleteRangeRequest) Reset() { *m = DeleteRangeRequest{} } func (m *DeleteRangeRequest) String() string { return proto.CompactTextString(m) } func (*DeleteRangeRequest) ProtoMessage() {} func (*DeleteRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{14} + return fileDescriptor_api_9819b93932ee2bad, []int{14} } func (m *DeleteRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -995,7 +996,7 @@ func (m *DeleteRangeResponse) Reset() { *m = DeleteRangeResponse{} } func (m *DeleteRangeResponse) String() string { return proto.CompactTextString(m) } func (*DeleteRangeResponse) ProtoMessage() {} func (*DeleteRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{15} + return fileDescriptor_api_9819b93932ee2bad, []int{15} } func (m *DeleteRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1050,7 +1051,7 @@ func (m *ClearRangeRequest) Reset() { *m = ClearRangeRequest{} } func (m *ClearRangeRequest) String() string { return proto.CompactTextString(m) } func (*ClearRangeRequest) ProtoMessage() {} func (*ClearRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{16} + return fileDescriptor_api_9819b93932ee2bad, []int{16} } func (m *ClearRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1084,7 +1085,7 @@ func (m *ClearRangeResponse) Reset() { *m = ClearRangeResponse{} } func (m *ClearRangeResponse) String() string { return proto.CompactTextString(m) } func (*ClearRangeResponse) ProtoMessage() {} func (*ClearRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{17} + return fileDescriptor_api_9819b93932ee2bad, []int{17} } func (m *ClearRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1125,7 +1126,7 @@ func (m *RevertRangeRequest) Reset() { *m = RevertRangeRequest{} } func (m *RevertRangeRequest) String() string { return proto.CompactTextString(m) } func (*RevertRangeRequest) ProtoMessage() {} func (*RevertRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{18} + return fileDescriptor_api_9819b93932ee2bad, []int{18} } func (m *RevertRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1159,7 +1160,7 @@ func (m *RevertRangeResponse) Reset() { *m = RevertRangeResponse{} } func (m *RevertRangeResponse) String() string { return proto.CompactTextString(m) } func (*RevertRangeResponse) ProtoMessage() {} func (*RevertRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{19} + return fileDescriptor_api_9819b93932ee2bad, []int{19} } func (m *RevertRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1210,7 +1211,7 @@ func (m *ScanRequest) Reset() { *m = ScanRequest{} } func (m *ScanRequest) String() string { return proto.CompactTextString(m) } func (*ScanRequest) ProtoMessage() {} func (*ScanRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{20} + return fileDescriptor_api_9819b93932ee2bad, []int{20} } func (m *ScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1262,7 +1263,7 @@ func (m *ScanResponse) Reset() { *m = ScanResponse{} } func (m *ScanResponse) String() string { return proto.CompactTextString(m) } func (*ScanResponse) ProtoMessage() {} func (*ScanResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{21} + return fileDescriptor_api_9819b93932ee2bad, []int{21} } func (m *ScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1313,7 +1314,7 @@ func (m *ReverseScanRequest) Reset() { *m = ReverseScanRequest{} } func (m *ReverseScanRequest) String() string { return proto.CompactTextString(m) } func (*ReverseScanRequest) ProtoMessage() {} func (*ReverseScanRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{22} + return fileDescriptor_api_9819b93932ee2bad, []int{22} } func (m *ReverseScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1365,7 +1366,7 @@ func (m *ReverseScanResponse) Reset() { *m = ReverseScanResponse{} } func (m *ReverseScanResponse) String() string { return proto.CompactTextString(m) } func (*ReverseScanResponse) ProtoMessage() {} func (*ReverseScanResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{23} + return fileDescriptor_api_9819b93932ee2bad, []int{23} } func (m *ReverseScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1418,7 +1419,7 @@ func (m *CheckConsistencyRequest) Reset() { *m = CheckConsistencyRequest func (m *CheckConsistencyRequest) String() string { return proto.CompactTextString(m) } func (*CheckConsistencyRequest) ProtoMessage() {} func (*CheckConsistencyRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{24} + return fileDescriptor_api_9819b93932ee2bad, []int{24} } func (m *CheckConsistencyRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1455,7 +1456,7 @@ func (m *CheckConsistencyResponse) Reset() { *m = CheckConsistencyRespon func (m *CheckConsistencyResponse) String() string { return proto.CompactTextString(m) } func (*CheckConsistencyResponse) ProtoMessage() {} func (*CheckConsistencyResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{25} + return fileDescriptor_api_9819b93932ee2bad, []int{25} } func (m *CheckConsistencyResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1499,7 +1500,7 @@ func (m *CheckConsistencyResponse_Result) Reset() { *m = CheckConsistenc func (m *CheckConsistencyResponse_Result) String() string { return proto.CompactTextString(m) } func (*CheckConsistencyResponse_Result) ProtoMessage() {} func (*CheckConsistencyResponse_Result) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{25, 0} + return fileDescriptor_api_9819b93932ee2bad, []int{25, 0} } func (m *CheckConsistencyResponse_Result) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1547,7 +1548,7 @@ func (m *RecomputeStatsRequest) Reset() { *m = RecomputeStatsRequest{} } func (m *RecomputeStatsRequest) String() string { return proto.CompactTextString(m) } func (*RecomputeStatsRequest) ProtoMessage() {} func (*RecomputeStatsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{26} + return fileDescriptor_api_9819b93932ee2bad, []int{26} } func (m *RecomputeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1583,7 +1584,7 @@ func (m *RecomputeStatsResponse) Reset() { *m = RecomputeStatsResponse{} func (m *RecomputeStatsResponse) String() string { return proto.CompactTextString(m) } func (*RecomputeStatsResponse) ProtoMessage() {} func (*RecomputeStatsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{27} + return fileDescriptor_api_9819b93932ee2bad, []int{27} } func (m *RecomputeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1694,7 +1695,7 @@ func (m *EndTxnRequest) Reset() { *m = EndTxnRequest{} } func (m *EndTxnRequest) String() string { return proto.CompactTextString(m) } func (*EndTxnRequest) ProtoMessage() {} func (*EndTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{28} + return fileDescriptor_api_9819b93932ee2bad, []int{28} } func (m *EndTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1740,7 +1741,7 @@ func (m *EndTxnResponse) Reset() { *m = EndTxnResponse{} } func (m *EndTxnResponse) String() string { return proto.CompactTextString(m) } func (*EndTxnResponse) ProtoMessage() {} func (*EndTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{29} + return fileDescriptor_api_9819b93932ee2bad, []int{29} } func (m *EndTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1801,7 +1802,7 @@ func (m *AdminSplitRequest) Reset() { *m = AdminSplitRequest{} } func (m *AdminSplitRequest) String() string { return proto.CompactTextString(m) } func (*AdminSplitRequest) ProtoMessage() {} func (*AdminSplitRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{30} + return fileDescriptor_api_9819b93932ee2bad, []int{30} } func (m *AdminSplitRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1836,7 +1837,7 @@ func (m *AdminSplitResponse) Reset() { *m = AdminSplitResponse{} } func (m *AdminSplitResponse) String() string { return proto.CompactTextString(m) } func (*AdminSplitResponse) ProtoMessage() {} func (*AdminSplitResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{31} + return fileDescriptor_api_9819b93932ee2bad, []int{31} } func (m *AdminSplitResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1875,7 +1876,7 @@ func (m *AdminUnsplitRequest) Reset() { *m = AdminUnsplitRequest{} } func (m *AdminUnsplitRequest) String() string { return proto.CompactTextString(m) } func (*AdminUnsplitRequest) ProtoMessage() {} func (*AdminUnsplitRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{32} + return fileDescriptor_api_9819b93932ee2bad, []int{32} } func (m *AdminUnsplitRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1910,7 +1911,7 @@ func (m *AdminUnsplitResponse) Reset() { *m = AdminUnsplitResponse{} } func (m *AdminUnsplitResponse) String() string { return proto.CompactTextString(m) } func (*AdminUnsplitResponse) ProtoMessage() {} func (*AdminUnsplitResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{33} + return fileDescriptor_api_9819b93932ee2bad, []int{33} } func (m *AdminUnsplitResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1953,7 +1954,7 @@ func (m *AdminMergeRequest) Reset() { *m = AdminMergeRequest{} } func (m *AdminMergeRequest) String() string { return proto.CompactTextString(m) } func (*AdminMergeRequest) ProtoMessage() {} func (*AdminMergeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{34} + return fileDescriptor_api_9819b93932ee2bad, []int{34} } func (m *AdminMergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1988,7 +1989,7 @@ func (m *AdminMergeResponse) Reset() { *m = AdminMergeResponse{} } func (m *AdminMergeResponse) String() string { return proto.CompactTextString(m) } func (*AdminMergeResponse) ProtoMessage() {} func (*AdminMergeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{35} + return fileDescriptor_api_9819b93932ee2bad, []int{35} } func (m *AdminMergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2026,7 +2027,7 @@ func (m *AdminTransferLeaseRequest) Reset() { *m = AdminTransferLeaseReq func (m *AdminTransferLeaseRequest) String() string { return proto.CompactTextString(m) } func (*AdminTransferLeaseRequest) ProtoMessage() {} func (*AdminTransferLeaseRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{36} + return fileDescriptor_api_9819b93932ee2bad, []int{36} } func (m *AdminTransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2059,7 +2060,7 @@ func (m *AdminTransferLeaseResponse) Reset() { *m = AdminTransferLeaseRe func (m *AdminTransferLeaseResponse) String() string { return proto.CompactTextString(m) } func (*AdminTransferLeaseResponse) ProtoMessage() {} func (*AdminTransferLeaseResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{37} + return fileDescriptor_api_9819b93932ee2bad, []int{37} } func (m *AdminTransferLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2094,7 +2095,7 @@ func (m *ReplicationChange) Reset() { *m = ReplicationChange{} } func (m *ReplicationChange) String() string { return proto.CompactTextString(m) } func (*ReplicationChange) ProtoMessage() {} func (*ReplicationChange) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{38} + return fileDescriptor_api_9819b93932ee2bad, []int{38} } func (m *ReplicationChange) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2152,7 +2153,7 @@ func (m *AdminChangeReplicasRequest) Reset() { *m = AdminChangeReplicasR func (m *AdminChangeReplicasRequest) String() string { return proto.CompactTextString(m) } func (*AdminChangeReplicasRequest) ProtoMessage() {} func (*AdminChangeReplicasRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{39} + return fileDescriptor_api_9819b93932ee2bad, []int{39} } func (m *AdminChangeReplicasRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2187,7 +2188,7 @@ func (m *AdminChangeReplicasResponse) Reset() { *m = AdminChangeReplicas func (m *AdminChangeReplicasResponse) String() string { return proto.CompactTextString(m) } func (*AdminChangeReplicasResponse) ProtoMessage() {} func (*AdminChangeReplicasResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{40} + return fileDescriptor_api_9819b93932ee2bad, []int{40} } func (m *AdminChangeReplicasResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2224,7 +2225,7 @@ func (m *AdminRelocateRangeRequest) Reset() { *m = AdminRelocateRangeReq func (m *AdminRelocateRangeRequest) String() string { return proto.CompactTextString(m) } func (*AdminRelocateRangeRequest) ProtoMessage() {} func (*AdminRelocateRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{41} + return fileDescriptor_api_9819b93932ee2bad, []int{41} } func (m *AdminRelocateRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2257,7 +2258,7 @@ func (m *AdminRelocateRangeResponse) Reset() { *m = AdminRelocateRangeRe func (m *AdminRelocateRangeResponse) String() string { return proto.CompactTextString(m) } func (*AdminRelocateRangeResponse) ProtoMessage() {} func (*AdminRelocateRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{42} + return fileDescriptor_api_9819b93932ee2bad, []int{42} } func (m *AdminRelocateRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2289,14 +2290,17 @@ var xxx_messageInfo_AdminRelocateRangeResponse proto.InternalMessageInfo // gossip protocol. type HeartbeatTxnRequest struct { RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` - Now hlc.Timestamp `protobuf:"bytes,2,opt,name=now,proto3" json:"now"` + // NOTE: this could use a ClockTimestamp type, but doing so results in a + // large diff that doesn't seem worth it, given that we never feed this + // timestamp back into a clock. + Now hlc.Timestamp `protobuf:"bytes,2,opt,name=now,proto3" json:"now"` } func (m *HeartbeatTxnRequest) Reset() { *m = HeartbeatTxnRequest{} } func (m *HeartbeatTxnRequest) String() string { return proto.CompactTextString(m) } func (*HeartbeatTxnRequest) ProtoMessage() {} func (*HeartbeatTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{43} + return fileDescriptor_api_9819b93932ee2bad, []int{43} } func (m *HeartbeatTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2333,7 +2337,7 @@ func (m *HeartbeatTxnResponse) Reset() { *m = HeartbeatTxnResponse{} } func (m *HeartbeatTxnResponse) String() string { return proto.CompactTextString(m) } func (*HeartbeatTxnResponse) ProtoMessage() {} func (*HeartbeatTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{44} + return fileDescriptor_api_9819b93932ee2bad, []int{44} } func (m *HeartbeatTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2371,7 +2375,7 @@ func (m *GCRequest) Reset() { *m = GCRequest{} } func (m *GCRequest) String() string { return proto.CompactTextString(m) } func (*GCRequest) ProtoMessage() {} func (*GCRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{45} + return fileDescriptor_api_9819b93932ee2bad, []int{45} } func (m *GCRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2405,7 +2409,7 @@ func (m *GCRequest_GCKey) Reset() { *m = GCRequest_GCKey{} } func (m *GCRequest_GCKey) String() string { return proto.CompactTextString(m) } func (*GCRequest_GCKey) ProtoMessage() {} func (*GCRequest_GCKey) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{45, 0} + return fileDescriptor_api_9819b93932ee2bad, []int{45, 0} } func (m *GCRequest_GCKey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2439,7 +2443,7 @@ func (m *GCResponse) Reset() { *m = GCResponse{} } func (m *GCResponse) String() string { return proto.CompactTextString(m) } func (*GCResponse) ProtoMessage() {} func (*GCResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{46} + return fileDescriptor_api_9819b93932ee2bad, []int{46} } func (m *GCResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2508,7 +2512,7 @@ func (m *PushTxnRequest) Reset() { *m = PushTxnRequest{} } func (m *PushTxnRequest) String() string { return proto.CompactTextString(m) } func (*PushTxnRequest) ProtoMessage() {} func (*PushTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{47} + return fileDescriptor_api_9819b93932ee2bad, []int{47} } func (m *PushTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2551,7 +2555,7 @@ func (m *PushTxnResponse) Reset() { *m = PushTxnResponse{} } func (m *PushTxnResponse) String() string { return proto.CompactTextString(m) } func (*PushTxnResponse) ProtoMessage() {} func (*PushTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{48} + return fileDescriptor_api_9819b93932ee2bad, []int{48} } func (m *PushTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2598,7 +2602,7 @@ func (m *RecoverTxnRequest) Reset() { *m = RecoverTxnRequest{} } func (m *RecoverTxnRequest) String() string { return proto.CompactTextString(m) } func (*RecoverTxnRequest) ProtoMessage() {} func (*RecoverTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{49} + return fileDescriptor_api_9819b93932ee2bad, []int{49} } func (m *RecoverTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2634,7 +2638,7 @@ func (m *RecoverTxnResponse) Reset() { *m = RecoverTxnResponse{} } func (m *RecoverTxnResponse) String() string { return proto.CompactTextString(m) } func (*RecoverTxnResponse) ProtoMessage() {} func (*RecoverTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{50} + return fileDescriptor_api_9819b93932ee2bad, []int{50} } func (m *RecoverTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2678,7 +2682,7 @@ func (m *QueryTxnRequest) Reset() { *m = QueryTxnRequest{} } func (m *QueryTxnRequest) String() string { return proto.CompactTextString(m) } func (*QueryTxnRequest) ProtoMessage() {} func (*QueryTxnRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{51} + return fileDescriptor_api_9819b93932ee2bad, []int{51} } func (m *QueryTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2722,7 +2726,7 @@ func (m *QueryTxnResponse) Reset() { *m = QueryTxnResponse{} } func (m *QueryTxnResponse) String() string { return proto.CompactTextString(m) } func (*QueryTxnResponse) ProtoMessage() {} func (*QueryTxnResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{52} + return fileDescriptor_api_9819b93932ee2bad, []int{52} } func (m *QueryTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2782,7 +2786,7 @@ func (m *QueryIntentRequest) Reset() { *m = QueryIntentRequest{} } func (m *QueryIntentRequest) String() string { return proto.CompactTextString(m) } func (*QueryIntentRequest) ProtoMessage() {} func (*QueryIntentRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{53} + return fileDescriptor_api_9819b93932ee2bad, []int{53} } func (m *QueryIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2818,7 +2822,7 @@ func (m *QueryIntentResponse) Reset() { *m = QueryIntentResponse{} } func (m *QueryIntentResponse) String() string { return proto.CompactTextString(m) } func (*QueryIntentResponse) ProtoMessage() {} func (*QueryIntentResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{54} + return fileDescriptor_api_9819b93932ee2bad, []int{54} } func (m *QueryIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2865,7 +2869,7 @@ func (m *ResolveIntentRequest) Reset() { *m = ResolveIntentRequest{} } func (m *ResolveIntentRequest) String() string { return proto.CompactTextString(m) } func (*ResolveIntentRequest) ProtoMessage() {} func (*ResolveIntentRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{55} + return fileDescriptor_api_9819b93932ee2bad, []int{55} } func (m *ResolveIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2900,7 +2904,7 @@ func (m *ResolveIntentResponse) Reset() { *m = ResolveIntentResponse{} } func (m *ResolveIntentResponse) String() string { return proto.CompactTextString(m) } func (*ResolveIntentResponse) ProtoMessage() {} func (*ResolveIntentResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{56} + return fileDescriptor_api_9819b93932ee2bad, []int{56} } func (m *ResolveIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2951,7 +2955,7 @@ func (m *ResolveIntentRangeRequest) Reset() { *m = ResolveIntentRangeReq func (m *ResolveIntentRangeRequest) String() string { return proto.CompactTextString(m) } func (*ResolveIntentRangeRequest) ProtoMessage() {} func (*ResolveIntentRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{57} + return fileDescriptor_api_9819b93932ee2bad, []int{57} } func (m *ResolveIntentRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2986,7 +2990,7 @@ func (m *ResolveIntentRangeResponse) Reset() { *m = ResolveIntentRangeRe func (m *ResolveIntentRangeResponse) String() string { return proto.CompactTextString(m) } func (*ResolveIntentRangeResponse) ProtoMessage() {} func (*ResolveIntentRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{58} + return fileDescriptor_api_9819b93932ee2bad, []int{58} } func (m *ResolveIntentRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3023,7 +3027,7 @@ func (m *MergeRequest) Reset() { *m = MergeRequest{} } func (m *MergeRequest) String() string { return proto.CompactTextString(m) } func (*MergeRequest) ProtoMessage() {} func (*MergeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{59} + return fileDescriptor_api_9819b93932ee2bad, []int{59} } func (m *MergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3057,7 +3061,7 @@ func (m *MergeResponse) Reset() { *m = MergeResponse{} } func (m *MergeResponse) String() string { return proto.CompactTextString(m) } func (*MergeResponse) ProtoMessage() {} func (*MergeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{60} + return fileDescriptor_api_9819b93932ee2bad, []int{60} } func (m *MergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3102,7 +3106,7 @@ func (m *TruncateLogRequest) Reset() { *m = TruncateLogRequest{} } func (m *TruncateLogRequest) String() string { return proto.CompactTextString(m) } func (*TruncateLogRequest) ProtoMessage() {} func (*TruncateLogRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{61} + return fileDescriptor_api_9819b93932ee2bad, []int{61} } func (m *TruncateLogRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3136,7 +3140,7 @@ func (m *TruncateLogResponse) Reset() { *m = TruncateLogResponse{} } func (m *TruncateLogResponse) String() string { return proto.CompactTextString(m) } func (*TruncateLogResponse) ProtoMessage() {} func (*TruncateLogResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{62} + return fileDescriptor_api_9819b93932ee2bad, []int{62} } func (m *TruncateLogResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3173,14 +3177,14 @@ type RequestLeaseRequest struct { // The MinLeaseProposedTS of the proposing replica to make sure that leases // issued after a node restart receive a new sequence number (instead of // counting as a lease extension). See #23204. - MinProposedTS *hlc.Timestamp `protobuf:"bytes,4,opt,name=min_proposed_ts,json=minProposedTs,proto3" json:"min_proposed_ts,omitempty"` + MinProposedTS *github_com_cockroachdb_cockroach_pkg_util_hlc.ClockTimestamp `protobuf:"bytes,4,opt,name=min_proposed_ts,json=minProposedTs,proto3,casttype=github.com/cockroachdb/cockroach/pkg/util/hlc.ClockTimestamp" json:"min_proposed_ts,omitempty"` } func (m *RequestLeaseRequest) Reset() { *m = RequestLeaseRequest{} } func (m *RequestLeaseRequest) String() string { return proto.CompactTextString(m) } func (*RequestLeaseRequest) ProtoMessage() {} func (*RequestLeaseRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{63} + return fileDescriptor_api_9819b93932ee2bad, []int{63} } func (m *RequestLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3229,7 +3233,7 @@ func (m *TransferLeaseRequest) Reset() { *m = TransferLeaseRequest{} } func (m *TransferLeaseRequest) String() string { return proto.CompactTextString(m) } func (*TransferLeaseRequest) ProtoMessage() {} func (*TransferLeaseRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{64} + return fileDescriptor_api_9819b93932ee2bad, []int{64} } func (m *TransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3266,7 +3270,7 @@ func (m *LeaseInfoRequest) Reset() { *m = LeaseInfoRequest{} } func (m *LeaseInfoRequest) String() string { return proto.CompactTextString(m) } func (*LeaseInfoRequest) ProtoMessage() {} func (*LeaseInfoRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{65} + return fileDescriptor_api_9819b93932ee2bad, []int{65} } func (m *LeaseInfoRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3303,7 +3307,7 @@ func (m *LeaseInfoResponse) Reset() { *m = LeaseInfoResponse{} } func (m *LeaseInfoResponse) String() string { return proto.CompactTextString(m) } func (*LeaseInfoResponse) ProtoMessage() {} func (*LeaseInfoResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{66} + return fileDescriptor_api_9819b93932ee2bad, []int{66} } func (m *LeaseInfoResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3338,7 +3342,7 @@ func (m *RequestLeaseResponse) Reset() { *m = RequestLeaseResponse{} } func (m *RequestLeaseResponse) String() string { return proto.CompactTextString(m) } func (*RequestLeaseResponse) ProtoMessage() {} func (*RequestLeaseResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{67} + return fileDescriptor_api_9819b93932ee2bad, []int{67} } func (m *RequestLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3393,7 +3397,7 @@ func (m *ComputeChecksumRequest) Reset() { *m = ComputeChecksumRequest{} func (m *ComputeChecksumRequest) String() string { return proto.CompactTextString(m) } func (*ComputeChecksumRequest) ProtoMessage() {} func (*ComputeChecksumRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{68} + return fileDescriptor_api_9819b93932ee2bad, []int{68} } func (m *ComputeChecksumRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3430,7 +3434,7 @@ func (m *ComputeChecksumResponse) Reset() { *m = ComputeChecksumResponse func (m *ComputeChecksumResponse) String() string { return proto.CompactTextString(m) } func (*ComputeChecksumResponse) ProtoMessage() {} func (*ComputeChecksumResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{69} + return fileDescriptor_api_9819b93932ee2bad, []int{69} } func (m *ComputeChecksumResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3470,7 +3474,7 @@ func (m *ExternalStorage) Reset() { *m = ExternalStorage{} } func (m *ExternalStorage) String() string { return proto.CompactTextString(m) } func (*ExternalStorage) ProtoMessage() {} func (*ExternalStorage) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{70} + return fileDescriptor_api_9819b93932ee2bad, []int{70} } func (m *ExternalStorage) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3504,7 +3508,7 @@ func (m *ExternalStorage_LocalFilePath) Reset() { *m = ExternalStorage_L func (m *ExternalStorage_LocalFilePath) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_LocalFilePath) ProtoMessage() {} func (*ExternalStorage_LocalFilePath) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{70, 0} + return fileDescriptor_api_9819b93932ee2bad, []int{70, 0} } func (m *ExternalStorage_LocalFilePath) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3537,7 +3541,7 @@ func (m *ExternalStorage_Http) Reset() { *m = ExternalStorage_Http{} } func (m *ExternalStorage_Http) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_Http) ProtoMessage() {} func (*ExternalStorage_Http) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{70, 1} + return fileDescriptor_api_9819b93932ee2bad, []int{70, 1} } func (m *ExternalStorage_Http) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3579,7 +3583,7 @@ func (m *ExternalStorage_S3) Reset() { *m = ExternalStorage_S3{} } func (m *ExternalStorage_S3) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_S3) ProtoMessage() {} func (*ExternalStorage_S3) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{70, 2} + return fileDescriptor_api_9819b93932ee2bad, []int{70, 2} } func (m *ExternalStorage_S3) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3618,7 +3622,7 @@ func (m *ExternalStorage_GCS) Reset() { *m = ExternalStorage_GCS{} } func (m *ExternalStorage_GCS) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_GCS) ProtoMessage() {} func (*ExternalStorage_GCS) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{70, 3} + return fileDescriptor_api_9819b93932ee2bad, []int{70, 3} } func (m *ExternalStorage_GCS) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3654,7 +3658,7 @@ func (m *ExternalStorage_Azure) Reset() { *m = ExternalStorage_Azure{} } func (m *ExternalStorage_Azure) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_Azure) ProtoMessage() {} func (*ExternalStorage_Azure) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{70, 4} + return fileDescriptor_api_9819b93932ee2bad, []int{70, 4} } func (m *ExternalStorage_Azure) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3693,7 +3697,7 @@ func (m *ExternalStorage_Workload) Reset() { *m = ExternalStorage_Worklo func (m *ExternalStorage_Workload) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_Workload) ProtoMessage() {} func (*ExternalStorage_Workload) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{70, 5} + return fileDescriptor_api_9819b93932ee2bad, []int{70, 5} } func (m *ExternalStorage_Workload) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3735,7 +3739,7 @@ func (m *ExternalStorage_FileTable) Reset() { *m = ExternalStorage_FileT func (m *ExternalStorage_FileTable) String() string { return proto.CompactTextString(m) } func (*ExternalStorage_FileTable) ProtoMessage() {} func (*ExternalStorage_FileTable) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{70, 6} + return fileDescriptor_api_9819b93932ee2bad, []int{70, 6} } func (m *ExternalStorage_FileTable) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3775,7 +3779,7 @@ func (m *WriteBatchRequest) Reset() { *m = WriteBatchRequest{} } func (m *WriteBatchRequest) String() string { return proto.CompactTextString(m) } func (*WriteBatchRequest) ProtoMessage() {} func (*WriteBatchRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{71} + return fileDescriptor_api_9819b93932ee2bad, []int{71} } func (m *WriteBatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3809,7 +3813,7 @@ func (m *WriteBatchResponse) Reset() { *m = WriteBatchResponse{} } func (m *WriteBatchResponse) String() string { return proto.CompactTextString(m) } func (*WriteBatchResponse) ProtoMessage() {} func (*WriteBatchResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{72} + return fileDescriptor_api_9819b93932ee2bad, []int{72} } func (m *WriteBatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3845,7 +3849,7 @@ func (m *FileEncryptionOptions) Reset() { *m = FileEncryptionOptions{} } func (m *FileEncryptionOptions) String() string { return proto.CompactTextString(m) } func (*FileEncryptionOptions) ProtoMessage() {} func (*FileEncryptionOptions) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{73} + return fileDescriptor_api_9819b93932ee2bad, []int{73} } func (m *FileEncryptionOptions) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3915,7 +3919,7 @@ func (m *ExportRequest) Reset() { *m = ExportRequest{} } func (m *ExportRequest) String() string { return proto.CompactTextString(m) } func (*ExportRequest) ProtoMessage() {} func (*ExportRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{74} + return fileDescriptor_api_9819b93932ee2bad, []int{74} } func (m *ExportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3967,7 +3971,7 @@ func (m *BulkOpSummary) Reset() { *m = BulkOpSummary{} } func (m *BulkOpSummary) String() string { return proto.CompactTextString(m) } func (*BulkOpSummary) ProtoMessage() {} func (*BulkOpSummary) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{75} + return fileDescriptor_api_9819b93932ee2bad, []int{75} } func (m *BulkOpSummary) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4003,7 +4007,7 @@ func (m *ExportResponse) Reset() { *m = ExportResponse{} } func (m *ExportResponse) String() string { return proto.CompactTextString(m) } func (*ExportResponse) ProtoMessage() {} func (*ExportResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{76} + return fileDescriptor_api_9819b93932ee2bad, []int{76} } func (m *ExportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4043,7 +4047,7 @@ func (m *ExportResponse_File) Reset() { *m = ExportResponse_File{} } func (m *ExportResponse_File) String() string { return proto.CompactTextString(m) } func (*ExportResponse_File) ProtoMessage() {} func (*ExportResponse_File) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{76, 0} + return fileDescriptor_api_9819b93932ee2bad, []int{76, 0} } func (m *ExportResponse_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4094,7 +4098,7 @@ func (m *ImportRequest) Reset() { *m = ImportRequest{} } func (m *ImportRequest) String() string { return proto.CompactTextString(m) } func (*ImportRequest) ProtoMessage() {} func (*ImportRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{77} + return fileDescriptor_api_9819b93932ee2bad, []int{77} } func (m *ImportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4129,7 +4133,7 @@ func (m *ImportRequest_File) Reset() { *m = ImportRequest_File{} } func (m *ImportRequest_File) String() string { return proto.CompactTextString(m) } func (*ImportRequest_File) ProtoMessage() {} func (*ImportRequest_File) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{77, 0} + return fileDescriptor_api_9819b93932ee2bad, []int{77, 0} } func (m *ImportRequest_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4165,7 +4169,7 @@ func (m *ImportRequest_TableRekey) Reset() { *m = ImportRequest_TableRek func (m *ImportRequest_TableRekey) String() string { return proto.CompactTextString(m) } func (*ImportRequest_TableRekey) ProtoMessage() {} func (*ImportRequest_TableRekey) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{77, 1} + return fileDescriptor_api_9819b93932ee2bad, []int{77, 1} } func (m *ImportRequest_TableRekey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4200,7 +4204,7 @@ func (m *ImportResponse) Reset() { *m = ImportResponse{} } func (m *ImportResponse) String() string { return proto.CompactTextString(m) } func (*ImportResponse) ProtoMessage() {} func (*ImportResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{78} + return fileDescriptor_api_9819b93932ee2bad, []int{78} } func (m *ImportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4238,7 +4242,7 @@ func (m *AdminScatterRequest) Reset() { *m = AdminScatterRequest{} } func (m *AdminScatterRequest) String() string { return proto.CompactTextString(m) } func (*AdminScatterRequest) ProtoMessage() {} func (*AdminScatterRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{79} + return fileDescriptor_api_9819b93932ee2bad, []int{79} } func (m *AdminScatterRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4275,7 +4279,7 @@ func (m *AdminScatterResponse) Reset() { *m = AdminScatterResponse{} } func (m *AdminScatterResponse) String() string { return proto.CompactTextString(m) } func (*AdminScatterResponse) ProtoMessage() {} func (*AdminScatterResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{80} + return fileDescriptor_api_9819b93932ee2bad, []int{80} } func (m *AdminScatterResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4308,7 +4312,7 @@ func (m *AdminScatterResponse_Range) Reset() { *m = AdminScatterResponse func (m *AdminScatterResponse_Range) String() string { return proto.CompactTextString(m) } func (*AdminScatterResponse_Range) ProtoMessage() {} func (*AdminScatterResponse_Range) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{80, 0} + return fileDescriptor_api_9819b93932ee2bad, []int{80, 0} } func (m *AdminScatterResponse_Range) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4353,7 +4357,7 @@ func (m *AdminVerifyProtectedTimestampRequest) Reset() { *m = AdminVerif func (m *AdminVerifyProtectedTimestampRequest) String() string { return proto.CompactTextString(m) } func (*AdminVerifyProtectedTimestampRequest) ProtoMessage() {} func (*AdminVerifyProtectedTimestampRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{81} + return fileDescriptor_api_9819b93932ee2bad, []int{81} } func (m *AdminVerifyProtectedTimestampRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4391,7 +4395,7 @@ func (m *AdminVerifyProtectedTimestampResponse) Reset() { *m = AdminVeri func (m *AdminVerifyProtectedTimestampResponse) String() string { return proto.CompactTextString(m) } func (*AdminVerifyProtectedTimestampResponse) ProtoMessage() {} func (*AdminVerifyProtectedTimestampResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{82} + return fileDescriptor_api_9819b93932ee2bad, []int{82} } func (m *AdminVerifyProtectedTimestampResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4444,7 +4448,7 @@ func (m *AddSSTableRequest) Reset() { *m = AddSSTableRequest{} } func (m *AddSSTableRequest) String() string { return proto.CompactTextString(m) } func (*AddSSTableRequest) ProtoMessage() {} func (*AddSSTableRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{83} + return fileDescriptor_api_9819b93932ee2bad, []int{83} } func (m *AddSSTableRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4478,7 +4482,7 @@ func (m *AddSSTableResponse) Reset() { *m = AddSSTableResponse{} } func (m *AddSSTableResponse) String() string { return proto.CompactTextString(m) } func (*AddSSTableResponse) ProtoMessage() {} func (*AddSSTableResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{84} + return fileDescriptor_api_9819b93932ee2bad, []int{84} } func (m *AddSSTableResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4522,7 +4526,7 @@ func (m *RefreshRequest) Reset() { *m = RefreshRequest{} } func (m *RefreshRequest) String() string { return proto.CompactTextString(m) } func (*RefreshRequest) ProtoMessage() {} func (*RefreshRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{85} + return fileDescriptor_api_9819b93932ee2bad, []int{85} } func (m *RefreshRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4556,7 +4560,7 @@ func (m *RefreshResponse) Reset() { *m = RefreshResponse{} } func (m *RefreshResponse) String() string { return proto.CompactTextString(m) } func (*RefreshResponse) ProtoMessage() {} func (*RefreshResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{86} + return fileDescriptor_api_9819b93932ee2bad, []int{86} } func (m *RefreshResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4595,7 +4599,7 @@ func (m *RefreshRangeRequest) Reset() { *m = RefreshRangeRequest{} } func (m *RefreshRangeRequest) String() string { return proto.CompactTextString(m) } func (*RefreshRangeRequest) ProtoMessage() {} func (*RefreshRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{87} + return fileDescriptor_api_9819b93932ee2bad, []int{87} } func (m *RefreshRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4629,7 +4633,7 @@ func (m *RefreshRangeResponse) Reset() { *m = RefreshRangeResponse{} } func (m *RefreshRangeResponse) String() string { return proto.CompactTextString(m) } func (*RefreshRangeResponse) ProtoMessage() {} func (*RefreshRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{88} + return fileDescriptor_api_9819b93932ee2bad, []int{88} } func (m *RefreshRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4678,7 +4682,7 @@ func (m *SubsumeRequest) Reset() { *m = SubsumeRequest{} } func (m *SubsumeRequest) String() string { return proto.CompactTextString(m) } func (*SubsumeRequest) ProtoMessage() {} func (*SubsumeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{89} + return fileDescriptor_api_9819b93932ee2bad, []int{89} } func (m *SubsumeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4720,14 +4724,14 @@ type SubsumeResponse struct { // before it stopped responding to requests altogether (in anticipation of // being subsumed). It is suitable for use as the timestamp cache's low water // mark for the keys previously owned by the subsumed range. - FreezeStart hlc.Timestamp `protobuf:"bytes,5,opt,name=freeze_start,json=freezeStart,proto3" json:"freeze_start"` + FreezeStart github_com_cockroachdb_cockroach_pkg_util_hlc.ClockTimestamp `protobuf:"bytes,5,opt,name=freeze_start,json=freezeStart,proto3,casttype=github.com/cockroachdb/cockroach/pkg/util/hlc.ClockTimestamp" json:"freeze_start"` } func (m *SubsumeResponse) Reset() { *m = SubsumeResponse{} } func (m *SubsumeResponse) String() string { return proto.CompactTextString(m) } func (*SubsumeResponse) ProtoMessage() {} func (*SubsumeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{90} + return fileDescriptor_api_9819b93932ee2bad, []int{90} } func (m *SubsumeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4762,7 +4766,7 @@ func (m *RangeStatsRequest) Reset() { *m = RangeStatsRequest{} } func (m *RangeStatsRequest) String() string { return proto.CompactTextString(m) } func (*RangeStatsRequest) ProtoMessage() {} func (*RangeStatsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{91} + return fileDescriptor_api_9819b93932ee2bad, []int{91} } func (m *RangeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4804,7 +4808,7 @@ func (m *RangeStatsResponse) Reset() { *m = RangeStatsResponse{} } func (m *RangeStatsResponse) String() string { return proto.CompactTextString(m) } func (*RangeStatsResponse) ProtoMessage() {} func (*RangeStatsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{92} + return fileDescriptor_api_9819b93932ee2bad, []int{92} } func (m *RangeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4887,7 +4891,7 @@ func (m *RequestUnion) Reset() { *m = RequestUnion{} } func (m *RequestUnion) String() string { return proto.CompactTextString(m) } func (*RequestUnion) ProtoMessage() {} func (*RequestUnion) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{93} + return fileDescriptor_api_9819b93932ee2bad, []int{93} } func (m *RequestUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -6337,7 +6341,7 @@ func (m *ResponseUnion) Reset() { *m = ResponseUnion{} } func (m *ResponseUnion) String() string { return proto.CompactTextString(m) } func (*ResponseUnion) ProtoMessage() {} func (*ResponseUnion) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{94} + return fileDescriptor_api_9819b93932ee2bad, []int{94} } func (m *ResponseUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7872,7 +7876,7 @@ func (m *Header) Reset() { *m = Header{} } func (m *Header) String() string { return proto.CompactTextString(m) } func (*Header) ProtoMessage() {} func (*Header) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{95} + return fileDescriptor_api_9819b93932ee2bad, []int{95} } func (m *Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7910,7 +7914,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_api_a1ecaa7abf76de44, []int{96} + return fileDescriptor_api_9819b93932ee2bad, []int{96} } func (m *ClientRangeInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7946,7 +7950,7 @@ type BatchRequest struct { func (m *BatchRequest) Reset() { *m = BatchRequest{} } func (*BatchRequest) ProtoMessage() {} func (*BatchRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{97} + return fileDescriptor_api_9819b93932ee2bad, []int{97} } func (m *BatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7983,7 +7987,7 @@ type BatchResponse struct { func (m *BatchResponse) Reset() { *m = BatchResponse{} } func (*BatchResponse) ProtoMessage() {} func (*BatchResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{98} + return fileDescriptor_api_9819b93932ee2bad, []int{98} } func (m *BatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8036,7 +8040,7 @@ type BatchResponse_Header struct { Txn *Transaction `protobuf:"bytes,3,opt,name=txn,proto3" json:"txn,omitempty"` // now is the highest current time from any node contacted during the request. // It can be used by the receiver to update its local HLC. - Now hlc.Timestamp `protobuf:"bytes,5,opt,name=now,proto3" json:"now"` + Now github_com_cockroachdb_cockroach_pkg_util_hlc.ClockTimestamp `protobuf:"bytes,5,opt,name=now,proto3,casttype=github.com/cockroachdb/cockroach/pkg/util/hlc.ClockTimestamp" json:"now"` // collected_spans stores trace spans recorded during the execution of this // request. CollectedSpans []tracingpb.RecordedSpan `protobuf:"bytes,6,rep,name=collected_spans,json=collectedSpans,proto3" json:"collected_spans"` @@ -8062,7 +8066,7 @@ func (m *BatchResponse_Header) Reset() { *m = BatchResponse_Header{} } func (m *BatchResponse_Header) String() string { return proto.CompactTextString(m) } func (*BatchResponse_Header) ProtoMessage() {} func (*BatchResponse_Header) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{98, 0} + return fileDescriptor_api_9819b93932ee2bad, []int{98, 0} } func (m *BatchResponse_Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8100,7 +8104,7 @@ func (m *RangeLookupRequest) Reset() { *m = RangeLookupRequest{} } func (m *RangeLookupRequest) String() string { return proto.CompactTextString(m) } func (*RangeLookupRequest) ProtoMessage() {} func (*RangeLookupRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{99} + return fileDescriptor_api_9819b93932ee2bad, []int{99} } func (m *RangeLookupRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8138,7 +8142,7 @@ func (m *RangeLookupResponse) Reset() { *m = RangeLookupResponse{} } func (m *RangeLookupResponse) String() string { return proto.CompactTextString(m) } func (*RangeLookupResponse) ProtoMessage() {} func (*RangeLookupResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{100} + return fileDescriptor_api_9819b93932ee2bad, []int{100} } func (m *RangeLookupResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8177,7 +8181,7 @@ func (m *RangeFeedRequest) Reset() { *m = RangeFeedRequest{} } func (m *RangeFeedRequest) String() string { return proto.CompactTextString(m) } func (*RangeFeedRequest) ProtoMessage() {} func (*RangeFeedRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{101} + return fileDescriptor_api_9819b93932ee2bad, []int{101} } func (m *RangeFeedRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8218,7 +8222,7 @@ func (m *RangeFeedValue) Reset() { *m = RangeFeedValue{} } func (m *RangeFeedValue) String() string { return proto.CompactTextString(m) } func (*RangeFeedValue) ProtoMessage() {} func (*RangeFeedValue) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{102} + return fileDescriptor_api_9819b93932ee2bad, []int{102} } func (m *RangeFeedValue) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8259,7 +8263,7 @@ func (m *RangeFeedCheckpoint) Reset() { *m = RangeFeedCheckpoint{} } func (m *RangeFeedCheckpoint) String() string { return proto.CompactTextString(m) } func (*RangeFeedCheckpoint) ProtoMessage() {} func (*RangeFeedCheckpoint) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{103} + return fileDescriptor_api_9819b93932ee2bad, []int{103} } func (m *RangeFeedCheckpoint) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8296,7 +8300,7 @@ func (m *RangeFeedError) Reset() { *m = RangeFeedError{} } func (m *RangeFeedError) String() string { return proto.CompactTextString(m) } func (*RangeFeedError) ProtoMessage() {} func (*RangeFeedError) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{104} + return fileDescriptor_api_9819b93932ee2bad, []int{104} } func (m *RangeFeedError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8333,7 +8337,7 @@ func (m *RangeFeedEvent) Reset() { *m = RangeFeedEvent{} } func (m *RangeFeedEvent) String() string { return proto.CompactTextString(m) } func (*RangeFeedEvent) ProtoMessage() {} func (*RangeFeedEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{105} + return fileDescriptor_api_9819b93932ee2bad, []int{105} } func (m *RangeFeedEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8377,7 +8381,7 @@ func (m *ResetQuorumRequest) Reset() { *m = ResetQuorumRequest{} } func (m *ResetQuorumRequest) String() string { return proto.CompactTextString(m) } func (*ResetQuorumRequest) ProtoMessage() {} func (*ResetQuorumRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{106} + return fileDescriptor_api_9819b93932ee2bad, []int{106} } func (m *ResetQuorumRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8409,7 +8413,7 @@ func (m *ResetQuorumResponse) Reset() { *m = ResetQuorumResponse{} } func (m *ResetQuorumResponse) String() string { return proto.CompactTextString(m) } func (*ResetQuorumResponse) ProtoMessage() {} func (*ResetQuorumResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{107} + return fileDescriptor_api_9819b93932ee2bad, []int{107} } func (m *ResetQuorumResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8450,7 +8454,7 @@ func (m *GossipSubscriptionRequest) Reset() { *m = GossipSubscriptionReq func (m *GossipSubscriptionRequest) String() string { return proto.CompactTextString(m) } func (*GossipSubscriptionRequest) ProtoMessage() {} func (*GossipSubscriptionRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{108} + return fileDescriptor_api_9819b93932ee2bad, []int{108} } func (m *GossipSubscriptionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8490,7 +8494,7 @@ func (m *GossipSubscriptionEvent) Reset() { *m = GossipSubscriptionEvent func (m *GossipSubscriptionEvent) String() string { return proto.CompactTextString(m) } func (*GossipSubscriptionEvent) ProtoMessage() {} func (*GossipSubscriptionEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{109} + return fileDescriptor_api_9819b93932ee2bad, []int{109} } func (m *GossipSubscriptionEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8526,7 +8530,7 @@ func (m *JoinNodeRequest) Reset() { *m = JoinNodeRequest{} } func (m *JoinNodeRequest) String() string { return proto.CompactTextString(m) } func (*JoinNodeRequest) ProtoMessage() {} func (*JoinNodeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{110} + return fileDescriptor_api_9819b93932ee2bad, []int{110} } func (m *JoinNodeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8565,7 +8569,7 @@ func (m *JoinNodeResponse) Reset() { *m = JoinNodeResponse{} } func (m *JoinNodeResponse) String() string { return proto.CompactTextString(m) } func (*JoinNodeResponse) ProtoMessage() {} func (*JoinNodeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{111} + return fileDescriptor_api_9819b93932ee2bad, []int{111} } func (m *JoinNodeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8608,7 +8612,7 @@ func (m *ContentionEvent) Reset() { *m = ContentionEvent{} } func (m *ContentionEvent) String() string { return proto.CompactTextString(m) } func (*ContentionEvent) ProtoMessage() {} func (*ContentionEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_api_a1ecaa7abf76de44, []int{112} + return fileDescriptor_api_9819b93932ee2bad, []int{112} } func (m *ContentionEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -27058,7 +27062,7 @@ func (m *RequestLeaseRequest) Unmarshal(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.MinProposedTS == nil { - m.MinProposedTS = &hlc.Timestamp{} + m.MinProposedTS = &github_com_cockroachdb_cockroach_pkg_util_hlc.ClockTimestamp{} } if err := m.MinProposedTS.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err @@ -38545,516 +38549,518 @@ var ( ErrIntOverflowApi = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_a1ecaa7abf76de44) } - -var fileDescriptor_api_a1ecaa7abf76de44 = []byte{ - // 8127 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x7d, 0x5d, 0x6c, 0x23, 0x59, - 0x76, 0x9e, 0x8a, 0xa4, 0x24, 0xf2, 0x50, 0xfc, 0xd1, 0x95, 0xba, 0x9b, 0xad, 0x99, 0x69, 0xa9, - 0xab, 0xff, 0x7b, 0x67, 0xa4, 0xe9, 0xee, 0x1d, 0xcf, 0x78, 0x7a, 0x3c, 0x6b, 0x91, 0x62, 0x37, - 0x29, 0xb5, 0xd4, 0xea, 0x22, 0xd5, 0xed, 0x19, 0xaf, 0x53, 0x5b, 0xaa, 0xba, 0xa2, 0x6a, 0x45, - 0x56, 0xb1, 0xab, 0x8a, 0xfa, 0x19, 0x20, 0x40, 0x1c, 0x1b, 0x89, 0x81, 0x00, 0x0b, 0x3f, 0x38, - 0xc8, 0x1a, 0x4e, 0xe2, 0x75, 0x1c, 0xc7, 0x01, 0x82, 0x20, 0x01, 0x12, 0xc4, 0x41, 0x90, 0xd8, - 0x2f, 0x06, 0xb2, 0x08, 0x0c, 0x64, 0xfd, 0x14, 0x23, 0x40, 0x14, 0x5b, 0x9b, 0xb7, 0x04, 0x46, - 0x90, 0x97, 0x00, 0xf3, 0x10, 0x04, 0xf7, 0xa7, 0xfe, 0xc8, 0x22, 0x45, 0xf5, 0xd6, 0xc0, 0x03, - 0xec, 0x8b, 0xc4, 0x3a, 0x75, 0xcf, 0xa9, 0x7b, 0xcf, 0xbd, 0xf7, 0xdc, 0xf3, 0xdd, 0x3a, 0xf7, - 0x14, 0xcc, 0x5a, 0xa6, 0xa2, 0xee, 0x77, 0x77, 0x57, 0x94, 0xae, 0xbe, 0xdc, 0xb5, 0x4c, 0xc7, - 0x44, 0xb3, 0xaa, 0xa9, 0x1e, 0x50, 0xf2, 0x32, 0xbf, 0xb9, 0x70, 0xff, 0xe0, 0x70, 0xe5, 0xe0, - 0xd0, 0xc6, 0xd6, 0x21, 0xb6, 0x56, 0x54, 0xd3, 0x50, 0x7b, 0x96, 0x85, 0x0d, 0xf5, 0x64, 0xa5, - 0x6d, 0xaa, 0x07, 0xf4, 0x8f, 0x6e, 0xb4, 0x18, 0xfb, 0x02, 0x72, 0x25, 0x6a, 0x8a, 0xa3, 0x70, - 0xda, 0xbc, 0x4b, 0xc3, 0x96, 0x65, 0x5a, 0x36, 0xa7, 0x5e, 0x76, 0xa9, 0x1d, 0xec, 0x28, 0x81, - 0xd2, 0x6f, 0xd9, 0x8e, 0x69, 0x29, 0x2d, 0xbc, 0x82, 0x8d, 0x96, 0x6e, 0x60, 0x52, 0xe0, 0x50, - 0x55, 0xf9, 0xcd, 0xb7, 0x23, 0x6f, 0x3e, 0xe2, 0x77, 0x4b, 0x3d, 0x47, 0x6f, 0xaf, 0xec, 0xb7, - 0xd5, 0x15, 0x47, 0xef, 0x60, 0xdb, 0x51, 0x3a, 0x5d, 0x7e, 0xe7, 0x3e, 0xbd, 0xe3, 0x58, 0x8a, - 0xaa, 0x1b, 0x2d, 0xf7, 0x7f, 0x77, 0x77, 0xc5, 0xc2, 0xaa, 0x69, 0x69, 0x58, 0x93, 0xed, 0xae, - 0x62, 0xb8, 0xd5, 0x6d, 0x99, 0x2d, 0x93, 0xfe, 0x5c, 0x21, 0xbf, 0x38, 0xf5, 0x5a, 0xcb, 0x34, - 0x5b, 0x6d, 0xbc, 0x42, 0xaf, 0x76, 0x7b, 0x7b, 0x2b, 0x5a, 0xcf, 0x52, 0x1c, 0xdd, 0xe4, 0x5c, - 0xe2, 0xbf, 0x11, 0x20, 0x27, 0xe1, 0xd7, 0x3d, 0x6c, 0x3b, 0x35, 0xac, 0x68, 0xd8, 0x42, 0x57, - 0x21, 0x79, 0x80, 0x4f, 0x4a, 0xc9, 0x25, 0xe1, 0xee, 0x4c, 0x79, 0xfa, 0xcb, 0xd3, 0xc5, 0xe4, - 0x06, 0x3e, 0x91, 0x08, 0x0d, 0x2d, 0xc1, 0x34, 0x36, 0x34, 0x99, 0xdc, 0x4e, 0x85, 0x6f, 0x4f, - 0x61, 0x43, 0xdb, 0xc0, 0x27, 0xe8, 0xdb, 0x90, 0xb6, 0x89, 0x34, 0x43, 0xc5, 0xa5, 0xc9, 0x25, - 0xe1, 0xee, 0x64, 0xf9, 0xe7, 0xbf, 0x3c, 0x5d, 0xfc, 0xa4, 0xa5, 0x3b, 0xfb, 0xbd, 0xdd, 0x65, - 0xd5, 0xec, 0xac, 0x78, 0xfd, 0xa4, 0xed, 0xfa, 0xbf, 0x57, 0xba, 0x07, 0xad, 0x95, 0x7e, 0x1d, - 0x2d, 0x37, 0x8f, 0x8d, 0x06, 0x7e, 0x2d, 0x79, 0x12, 0xd7, 0x53, 0x69, 0xa1, 0x98, 0x58, 0x4f, - 0xa5, 0x13, 0xc5, 0xa4, 0xf8, 0xbb, 0x49, 0xc8, 0x4b, 0xd8, 0xee, 0x9a, 0x86, 0x8d, 0x79, 0xcd, - 0xdf, 0x87, 0xa4, 0x73, 0x6c, 0xd0, 0x9a, 0x67, 0x1f, 0x5e, 0x5b, 0x1e, 0x18, 0x11, 0xcb, 0x4d, - 0x4b, 0x31, 0x6c, 0x45, 0x25, 0xcd, 0x97, 0x48, 0x51, 0xf4, 0x11, 0x64, 0x2d, 0x6c, 0xf7, 0x3a, - 0x98, 0x2a, 0x92, 0x36, 0x2a, 0xfb, 0xf0, 0x4a, 0x04, 0x67, 0xa3, 0xab, 0x18, 0x12, 0xb0, 0xb2, - 0xe4, 0x37, 0xba, 0x0a, 0x69, 0xa3, 0xd7, 0x21, 0xaa, 0xb0, 0x69, 0x43, 0x93, 0xd2, 0xb4, 0xd1, - 0xeb, 0x6c, 0xe0, 0x13, 0x1b, 0xfd, 0x02, 0x5c, 0xd6, 0x70, 0xd7, 0xc2, 0xaa, 0xe2, 0x60, 0x4d, - 0xb6, 0x14, 0xa3, 0x85, 0x65, 0xdd, 0xd8, 0x33, 0xed, 0xd2, 0xd4, 0x52, 0xf2, 0x6e, 0xf6, 0xe1, - 0xdb, 0x11, 0xf2, 0x25, 0x52, 0xaa, 0x6e, 0xec, 0x99, 0xe5, 0xd4, 0x0f, 0x4f, 0x17, 0x27, 0xa4, - 0x79, 0x5f, 0x82, 0x77, 0xcb, 0x46, 0x0d, 0xc8, 0xf1, 0xea, 0x5a, 0x58, 0xb1, 0x4d, 0xa3, 0x34, - 0xbd, 0x24, 0xdc, 0xcd, 0x3f, 0x5c, 0x8e, 0x12, 0x18, 0x52, 0x0d, 0xb9, 0xec, 0x75, 0xb0, 0x44, - 0xb9, 0xa4, 0x19, 0x2b, 0x70, 0x85, 0xde, 0x82, 0x0c, 0x69, 0xc9, 0xee, 0x89, 0x83, 0xed, 0x52, - 0x9a, 0x36, 0x85, 0x34, 0xad, 0x4c, 0xae, 0xc5, 0x4f, 0x61, 0x26, 0xc8, 0x8a, 0x10, 0xe4, 0xa5, - 0x6a, 0x63, 0x67, 0xb3, 0x2a, 0xef, 0x6c, 0x6d, 0x6c, 0x3d, 0x7f, 0xb5, 0x55, 0x9c, 0x40, 0xf3, - 0x50, 0xe4, 0xb4, 0x8d, 0xea, 0x67, 0xf2, 0xb3, 0xfa, 0x66, 0xbd, 0x59, 0x14, 0x16, 0x52, 0xbf, - 0xf6, 0xbb, 0xd7, 0x26, 0xc4, 0x6d, 0x80, 0xa7, 0xd8, 0xe1, 0x03, 0x0c, 0x95, 0x61, 0x6a, 0x9f, - 0xd6, 0xa7, 0x24, 0x50, 0x4d, 0x2f, 0x45, 0x56, 0x3c, 0x30, 0x18, 0xcb, 0x69, 0xa2, 0x8d, 0x1f, - 0x9d, 0x2e, 0x0a, 0x12, 0xe7, 0x14, 0xff, 0x48, 0x80, 0x2c, 0x15, 0xc9, 0xda, 0x87, 0x2a, 0x7d, - 0x32, 0xaf, 0x9f, 0xab, 0x8c, 0x41, 0xa1, 0x68, 0x19, 0x26, 0x0f, 0x95, 0x76, 0x0f, 0x97, 0x12, - 0x54, 0x46, 0x29, 0x42, 0xc6, 0x4b, 0x72, 0x5f, 0x62, 0xc5, 0xd0, 0x63, 0x98, 0xd1, 0x0d, 0x07, - 0x1b, 0x8e, 0xcc, 0xd8, 0x92, 0xe7, 0xb0, 0x65, 0x59, 0x69, 0x7a, 0x21, 0xfe, 0x6b, 0x01, 0x60, - 0xbb, 0x17, 0xa7, 0x52, 0xd0, 0x37, 0xc7, 0xac, 0x3f, 0x1f, 0x5d, 0xbc, 0x15, 0x97, 0x61, 0x4a, - 0x37, 0xda, 0xba, 0xc1, 0xea, 0x9f, 0x96, 0xf8, 0x15, 0x9a, 0x87, 0xc9, 0xdd, 0xb6, 0x6e, 0x68, - 0x74, 0x3e, 0xa4, 0x25, 0x76, 0x21, 0x4a, 0x90, 0xa5, 0xb5, 0x8e, 0x51, 0xef, 0xe2, 0x69, 0x02, - 0x2e, 0x55, 0x4c, 0x43, 0xd3, 0xc9, 0x94, 0x54, 0xda, 0x5f, 0x0b, 0xad, 0xac, 0x43, 0x60, 0xf2, - 0xc9, 0xf8, 0xb8, 0x3b, 0x66, 0x1f, 0x23, 0x9f, 0xab, 0x7a, 0xdc, 0xa5, 0xb4, 0x68, 0x4d, 0xa2, - 0x6f, 0xc2, 0x15, 0xa5, 0xdd, 0x36, 0x8f, 0x64, 0x7d, 0x4f, 0xd6, 0x4c, 0x6c, 0xcb, 0x86, 0xe9, - 0xc8, 0xf8, 0x58, 0xb7, 0x1d, 0x6a, 0x4a, 0xd2, 0xd2, 0x1c, 0xbd, 0x5d, 0xdf, 0x5b, 0x33, 0xb1, - 0xbd, 0x65, 0x3a, 0x55, 0x72, 0x8b, 0xcc, 0x53, 0x52, 0x19, 0x36, 0x4f, 0xa7, 0x88, 0xf9, 0x95, - 0xd2, 0xf8, 0xb8, 0x4b, 0xe7, 0x69, 0xa0, 0x2b, 0xa7, 0x83, 0x5d, 0x29, 0xfe, 0x12, 0x5c, 0xee, - 0xd7, 0x6f, 0x9c, 0xfd, 0xf7, 0x27, 0x02, 0xe4, 0xeb, 0x86, 0xee, 0x7c, 0x2d, 0x3a, 0xce, 0x53, - 0x76, 0x32, 0xa8, 0xec, 0xfb, 0x50, 0xdc, 0x53, 0xf4, 0xf6, 0x73, 0xa3, 0x69, 0x76, 0x76, 0x6d, - 0xc7, 0x34, 0xb0, 0xcd, 0x7b, 0x63, 0x80, 0x2e, 0xbe, 0x84, 0x82, 0xd7, 0x9a, 0x38, 0xd5, 0xe4, - 0x40, 0xb1, 0x6e, 0xa8, 0x16, 0xee, 0x60, 0x23, 0x56, 0x3d, 0xbd, 0x0d, 0x19, 0xdd, 0x95, 0x4b, - 0x75, 0x95, 0x94, 0x7c, 0x82, 0xd8, 0x83, 0xd9, 0xc0, 0x53, 0xe3, 0x34, 0x97, 0x64, 0xc9, 0xc0, - 0x47, 0xb2, 0xdf, 0x47, 0x64, 0xc9, 0xc0, 0x47, 0xcc, 0xbc, 0x35, 0x20, 0xb7, 0x86, 0xdb, 0xd8, - 0xc1, 0x71, 0x5a, 0xfd, 0x1d, 0xc8, 0xbb, 0x42, 0xe3, 0xec, 0x98, 0xbf, 0x27, 0x00, 0xe2, 0x72, - 0xc9, 0x2a, 0x1b, 0x67, 0xdf, 0x2c, 0x12, 0xd7, 0xc2, 0xe9, 0x59, 0x06, 0xf3, 0x11, 0xd8, 0x98, - 0x04, 0x46, 0xa2, 0x6e, 0x82, 0x3f, 0x65, 0x53, 0xc1, 0x29, 0xcb, 0xdd, 0x9b, 0x23, 0x98, 0x0b, - 0x55, 0x2c, 0xde, 0xee, 0x4b, 0xd1, 0x3a, 0x25, 0x96, 0x92, 0x41, 0x1f, 0x8e, 0x12, 0xc5, 0xef, - 0x0b, 0x30, 0x5b, 0x69, 0x63, 0xc5, 0x8a, 0x5d, 0x23, 0xdf, 0x82, 0xb4, 0x86, 0x15, 0x8d, 0x36, - 0x99, 0x4d, 0xec, 0x77, 0x02, 0x52, 0x88, 0xa7, 0xbb, 0xbc, 0xdf, 0x56, 0x97, 0x9b, 0xae, 0x0f, - 0xcc, 0x67, 0xb7, 0xc7, 0x24, 0x7e, 0x06, 0x28, 0x58, 0xb3, 0x38, 0x07, 0xc2, 0x3f, 0x14, 0x00, - 0x49, 0xf8, 0x10, 0x5b, 0x4e, 0xec, 0xcd, 0x5e, 0x83, 0xac, 0xa3, 0x58, 0x2d, 0xec, 0xc8, 0xc4, - 0xbb, 0xbf, 0x48, 0xcb, 0x81, 0xf1, 0x11, 0xb2, 0xf8, 0x39, 0xcc, 0x85, 0xea, 0x17, 0x67, 0xe3, - 0xff, 0xb7, 0x00, 0xd9, 0x86, 0xaa, 0x18, 0x71, 0xb6, 0xfa, 0x53, 0xc8, 0xda, 0xaa, 0x62, 0xc8, - 0x7b, 0xa6, 0xd5, 0x51, 0x1c, 0x3a, 0xc4, 0xf3, 0xa1, 0x56, 0x7b, 0x9e, 0xb5, 0xaa, 0x18, 0x4f, - 0x68, 0x21, 0x09, 0x6c, 0xef, 0x37, 0x7a, 0x01, 0xd9, 0x03, 0x7c, 0x22, 0x73, 0x94, 0x46, 0xd7, - 0xc5, 0xfc, 0xc3, 0xf7, 0x03, 0xfc, 0x07, 0x87, 0xcb, 0x2e, 0xb8, 0x5b, 0x0e, 0x80, 0xbb, 0x65, - 0xc2, 0xb1, 0xdc, 0x70, 0x2c, 0x6c, 0xb4, 0x9c, 0x7d, 0x09, 0x0e, 0xf0, 0xc9, 0x33, 0x26, 0x83, - 0x4d, 0xac, 0xf5, 0x54, 0x3a, 0x59, 0x4c, 0x89, 0xff, 0x57, 0x80, 0x19, 0xd6, 0xe4, 0x38, 0x27, - 0xd6, 0x07, 0x90, 0xb2, 0xcc, 0x23, 0x36, 0xb1, 0xb2, 0x0f, 0xdf, 0x8a, 0x10, 0xb1, 0x81, 0x4f, - 0x82, 0x2b, 0x17, 0x2d, 0x8e, 0xca, 0xc0, 0xfd, 0x43, 0x99, 0x72, 0x27, 0xc7, 0xe5, 0x06, 0xc6, - 0x25, 0x11, 0x19, 0x77, 0xa0, 0xb0, 0xab, 0x38, 0xea, 0xbe, 0x6c, 0xf1, 0x4a, 0x92, 0x55, 0x2e, - 0x79, 0x77, 0x46, 0xca, 0x53, 0xb2, 0x5b, 0x75, 0x9b, 0xb4, 0x9c, 0x8d, 0x74, 0x1b, 0xff, 0x94, - 0xf5, 0xf9, 0xff, 0x13, 0xf8, 0x1c, 0x72, 0x5b, 0xfe, 0xd3, 0xd6, 0xf5, 0xbf, 0x99, 0x80, 0x2b, - 0x95, 0x7d, 0xac, 0x1e, 0x54, 0x4c, 0xc3, 0xd6, 0x6d, 0x87, 0xe8, 0x2e, 0xce, 0xfe, 0x7f, 0x0b, - 0x32, 0x47, 0xba, 0xb3, 0x2f, 0x6b, 0xfa, 0xde, 0x1e, 0xb5, 0x73, 0x69, 0x29, 0x4d, 0x08, 0x6b, - 0xfa, 0xde, 0x1e, 0x7a, 0x04, 0xa9, 0x8e, 0xa9, 0x31, 0x37, 0x3a, 0xff, 0x70, 0x31, 0x42, 0x3c, - 0xad, 0x9a, 0xdd, 0xeb, 0x6c, 0x9a, 0x1a, 0x96, 0x68, 0x61, 0x74, 0x0d, 0x40, 0x25, 0xd4, 0xae, - 0xa9, 0x1b, 0x0e, 0x5f, 0x27, 0x03, 0x14, 0x54, 0x83, 0x8c, 0x83, 0xad, 0x8e, 0x6e, 0x28, 0x0e, - 0x2e, 0x4d, 0x52, 0xe5, 0xdd, 0x8c, 0xac, 0x78, 0xb7, 0xad, 0xab, 0xca, 0x1a, 0xb6, 0x55, 0x4b, - 0xef, 0x3a, 0xa6, 0xc5, 0xb5, 0xe8, 0x33, 0x8b, 0xdf, 0x4b, 0x41, 0x69, 0x50, 0x37, 0x71, 0x8e, - 0x90, 0x6d, 0x98, 0x22, 0xb8, 0xbb, 0xed, 0xf0, 0x31, 0xf2, 0x70, 0x98, 0x0a, 0x22, 0x6a, 0x40, - 0xf1, 0x7b, 0xdb, 0xe1, 0xd5, 0xe6, 0x72, 0x16, 0xfe, 0xbd, 0x00, 0x53, 0xec, 0x06, 0x7a, 0x00, - 0x69, 0xbe, 0xd1, 0xa0, 0xd1, 0x3a, 0x26, 0xcb, 0x97, 0xcf, 0x4e, 0x17, 0xa7, 0xd9, 0xde, 0xc1, - 0xda, 0x97, 0xfe, 0x4f, 0x69, 0x9a, 0x96, 0xab, 0x6b, 0xa4, 0xb7, 0x6c, 0x47, 0xb1, 0x1c, 0xba, - 0x9d, 0x93, 0x60, 0x78, 0x82, 0x12, 0x36, 0xf0, 0x09, 0x5a, 0x87, 0x29, 0xdb, 0x51, 0x9c, 0x9e, - 0xcd, 0xfb, 0xeb, 0x42, 0x95, 0x6d, 0x50, 0x4e, 0x89, 0x4b, 0x20, 0x8e, 0x8e, 0x86, 0x1d, 0x45, - 0x6f, 0xd3, 0x0e, 0xcc, 0x48, 0xfc, 0x4a, 0xfc, 0x2d, 0x01, 0xa6, 0x58, 0x51, 0x74, 0x05, 0xe6, - 0xa4, 0xd5, 0xad, 0xa7, 0x55, 0xb9, 0xbe, 0xb5, 0x56, 0x6d, 0x56, 0xa5, 0xcd, 0xfa, 0xd6, 0x6a, - 0xb3, 0x5a, 0x9c, 0x40, 0x97, 0x01, 0xb9, 0x37, 0x2a, 0xcf, 0xb7, 0x1a, 0xf5, 0x46, 0xb3, 0xba, - 0xd5, 0x2c, 0x0a, 0x74, 0xcf, 0x81, 0xd2, 0x03, 0xd4, 0x04, 0xba, 0x09, 0x4b, 0xfd, 0x54, 0xb9, - 0xd1, 0x5c, 0x6d, 0x36, 0xe4, 0x6a, 0xa3, 0x59, 0xdf, 0x5c, 0x6d, 0x56, 0xd7, 0x8a, 0xc9, 0x11, - 0xa5, 0xc8, 0x43, 0x24, 0xa9, 0x5a, 0x69, 0x16, 0x53, 0xa2, 0x03, 0x97, 0x24, 0xac, 0x9a, 0x9d, - 0x6e, 0xcf, 0xc1, 0xa4, 0x96, 0x76, 0x9c, 0x33, 0xe5, 0x0a, 0x4c, 0x6b, 0xd6, 0x89, 0x6c, 0xf5, - 0x0c, 0x3e, 0x4f, 0xa6, 0x34, 0xeb, 0x44, 0xea, 0x19, 0xe2, 0xbf, 0x14, 0xe0, 0x72, 0xff, 0x63, - 0xe3, 0x1c, 0x84, 0x2f, 0x20, 0xab, 0x68, 0x1a, 0xd6, 0x64, 0x0d, 0xb7, 0x1d, 0x85, 0x3b, 0x23, - 0xf7, 0x03, 0x92, 0xf8, 0x26, 0xdc, 0xb2, 0xb7, 0x09, 0xb7, 0xf9, 0xb2, 0x52, 0xa1, 0x15, 0x59, - 0x23, 0x1c, 0xae, 0xf9, 0xa1, 0x42, 0x28, 0x45, 0xfc, 0x5f, 0x29, 0xc8, 0x55, 0x0d, 0xad, 0x79, - 0x1c, 0xeb, 0x5a, 0x72, 0x19, 0xa6, 0x54, 0xb3, 0xd3, 0xd1, 0x1d, 0x57, 0x41, 0xec, 0x0a, 0xfd, - 0x6c, 0xc0, 0x89, 0x4c, 0x8e, 0xe1, 0x4a, 0xf9, 0xee, 0x23, 0xfa, 0x0e, 0x5c, 0x21, 0x56, 0xd3, - 0x32, 0x94, 0xb6, 0xcc, 0xa4, 0xc9, 0x8e, 0xa5, 0xb7, 0x5a, 0xd8, 0xe2, 0x1b, 0x7f, 0x77, 0x23, - 0xea, 0x59, 0xe7, 0x1c, 0x15, 0xca, 0xd0, 0x64, 0xe5, 0xa5, 0x4b, 0x7a, 0x14, 0x19, 0x7d, 0x02, - 0x40, 0x96, 0x22, 0xba, 0x99, 0x68, 0x73, 0x7b, 0x34, 0x6c, 0x37, 0xd1, 0x35, 0x41, 0x84, 0x81, - 0x5c, 0xdb, 0x68, 0x85, 0x20, 0x86, 0xd7, 0x3d, 0xdd, 0xc2, 0xf2, 0x83, 0xae, 0x4a, 0x21, 0x7e, - 0xba, 0x9c, 0x3f, 0x3b, 0x5d, 0x04, 0x89, 0x91, 0x1f, 0x6c, 0x57, 0x08, 0x82, 0x60, 0xbf, 0xbb, - 0x2a, 0x7a, 0x05, 0xf7, 0x02, 0x3b, 0x15, 0x64, 0xe5, 0xe5, 0xcd, 0x52, 0x1c, 0x79, 0x5f, 0x6f, - 0xed, 0x63, 0x4b, 0xf6, 0x36, 0x94, 0xe9, 0xce, 0x5e, 0x5a, 0xba, 0xe9, 0x33, 0x54, 0x14, 0x83, - 0xd5, 0x7e, 0xd5, 0xa9, 0xd1, 0xc2, 0x9e, 0xce, 0x88, 0xf2, 0xbb, 0xa6, 0x6e, 0x9b, 0x46, 0x29, - 0xc3, 0x94, 0xcf, 0xae, 0xd0, 0x3d, 0x28, 0x3a, 0xc7, 0x86, 0xbc, 0x8f, 0x15, 0xcb, 0xd9, 0xc5, - 0x8a, 0x43, 0x56, 0x69, 0xa0, 0x25, 0x0a, 0xce, 0xb1, 0x51, 0x0b, 0x90, 0xd1, 0x0b, 0x28, 0xea, - 0x86, 0xbc, 0xd7, 0xd6, 0x5b, 0xfb, 0x8e, 0x7c, 0x64, 0xe9, 0x0e, 0xb6, 0x4b, 0xb3, 0x54, 0x21, - 0x51, 0xe3, 0xb6, 0xc1, 0x77, 0x78, 0xb5, 0x57, 0xa4, 0x24, 0x57, 0x4d, 0x5e, 0x37, 0x9e, 0x50, - 0x7e, 0x4a, 0xb4, 0xd7, 0x53, 0xe9, 0xe9, 0x62, 0x5a, 0xfc, 0x6f, 0x02, 0xe4, 0xdd, 0xe1, 0x16, - 0xe7, 0xcc, 0xb8, 0x0b, 0x45, 0xd3, 0xc0, 0x72, 0x77, 0x5f, 0xb1, 0x31, 0xd7, 0x23, 0x5f, 0x70, - 0xf2, 0xa6, 0x81, 0xb7, 0x09, 0x99, 0xa9, 0x0b, 0x6d, 0xc3, 0xac, 0xed, 0x28, 0x2d, 0xdd, 0x68, - 0x05, 0xd4, 0x3b, 0x39, 0xbe, 0x5b, 0x5f, 0xe4, 0xdc, 0x1e, 0x3d, 0xe4, 0xa5, 0xfc, 0xa9, 0x00, - 0xb3, 0xab, 0x5a, 0x47, 0x37, 0x1a, 0xdd, 0xb6, 0x1e, 0xeb, 0x6e, 0xc1, 0x4d, 0xc8, 0xd8, 0x44, - 0xa6, 0x6f, 0xf0, 0x7d, 0xec, 0x97, 0xa6, 0x77, 0x88, 0xe5, 0x7f, 0x06, 0x05, 0x7c, 0xdc, 0xd5, - 0xd9, 0x4b, 0x02, 0x06, 0x59, 0x52, 0xe3, 0xb7, 0x2d, 0xef, 0xf3, 0x92, 0x5b, 0xbc, 0x4d, 0x9f, - 0x01, 0x0a, 0x36, 0x29, 0x4e, 0xec, 0xf2, 0x19, 0xcc, 0x51, 0xd1, 0x3b, 0x86, 0x1d, 0xb3, 0xbe, - 0xc4, 0x5f, 0x84, 0xf9, 0xb0, 0xe8, 0x38, 0xeb, 0xfd, 0x8a, 0xf7, 0xf2, 0x26, 0xb6, 0x62, 0x85, - 0x9b, 0x9e, 0xae, 0xb9, 0xe0, 0x38, 0xeb, 0xfc, 0xab, 0x02, 0x5c, 0xa5, 0xb2, 0xe9, 0x7b, 0x94, - 0x3d, 0x6c, 0x3d, 0xc3, 0x8a, 0x1d, 0x2b, 0x56, 0xbe, 0x01, 0x53, 0x0c, 0xf3, 0xd2, 0xf1, 0x39, - 0x59, 0xce, 0x12, 0xcf, 0xa5, 0xe1, 0x98, 0x16, 0xf1, 0x5c, 0xf8, 0x2d, 0x51, 0x81, 0x85, 0xa8, - 0x5a, 0xc4, 0xbc, 0x1d, 0x30, 0xcb, 0x9d, 0x46, 0x32, 0x94, 0x2b, 0xfb, 0xc4, 0x67, 0x42, 0x55, - 0xc8, 0xaa, 0xf4, 0x97, 0xec, 0x9c, 0x74, 0x31, 0x95, 0x9f, 0x1f, 0xe5, 0x6f, 0x32, 0xb6, 0xe6, - 0x49, 0x17, 0x13, 0xa7, 0xd5, 0xfd, 0x4d, 0x14, 0x15, 0x68, 0xe4, 0x48, 0x8f, 0x95, 0xce, 0x23, - 0x5a, 0xd6, 0x75, 0xfd, 0xb8, 0x0e, 0xfe, 0x55, 0x92, 0x2b, 0x81, 0x3d, 0x83, 0x17, 0x8f, 0xd5, - 0x47, 0xf9, 0x3c, 0xf4, 0x1a, 0x2b, 0xd8, 0xf0, 0xc4, 0x05, 0x1a, 0x1e, 0xd8, 0x4b, 0xf7, 0xa9, - 0xe8, 0x33, 0x08, 0xec, 0x96, 0xcb, 0xac, 0x4d, 0x2e, 0xfa, 0xb9, 0x88, 0x3a, 0x66, 0x7d, 0x29, - 0x8c, 0x6e, 0xa3, 0x0a, 0xa4, 0xf1, 0x71, 0x57, 0xd6, 0xb0, 0xad, 0x72, 0xc3, 0x25, 0x0e, 0x7b, - 0xdf, 0x36, 0x80, 0x07, 0xa6, 0xf1, 0x71, 0x97, 0x10, 0xd1, 0x0e, 0x59, 0xbd, 0x5c, 0x57, 0x81, - 0x56, 0xdb, 0x3e, 0x1f, 0x5e, 0xf8, 0x23, 0x85, 0x8b, 0x2b, 0x78, 0x5e, 0x02, 0x13, 0x21, 0xfe, - 0x40, 0x80, 0xb7, 0x22, 0x7b, 0x2d, 0xce, 0x85, 0xec, 0x13, 0x48, 0xd1, 0xc6, 0x27, 0x2e, 0xd8, - 0x78, 0xca, 0x25, 0xfe, 0x9e, 0x3b, 0xc7, 0x25, 0xdc, 0x36, 0x89, 0x62, 0xbf, 0x82, 0xfd, 0xb0, - 0x69, 0xb7, 0xc3, 0x13, 0x17, 0xee, 0x70, 0x97, 0xd5, 0x33, 0x02, 0x7d, 0xd5, 0x8c, 0xd3, 0x08, - 0xfc, 0xba, 0x00, 0x73, 0x9e, 0x4f, 0x13, 0xb3, 0x7b, 0xfb, 0x01, 0x24, 0x0d, 0xf3, 0xe8, 0x22, - 0x9b, 0x81, 0xa4, 0x3c, 0x59, 0x92, 0xc2, 0x35, 0x8a, 0xb3, 0xbd, 0xff, 0x21, 0x01, 0x99, 0xa7, - 0x95, 0x38, 0x5b, 0xf9, 0x09, 0xdf, 0x68, 0x66, 0x13, 0x3b, 0x6a, 0x28, 0x7a, 0xcf, 0x5b, 0x7e, - 0x5a, 0xd9, 0xc0, 0x27, 0xee, 0x50, 0x24, 0x5c, 0x68, 0x15, 0x32, 0xce, 0xbe, 0x85, 0xed, 0x7d, - 0xb3, 0xad, 0x5d, 0xc4, 0x07, 0xf1, 0xb9, 0x16, 0x30, 0x4c, 0x52, 0xb9, 0x6e, 0x50, 0x83, 0x10, - 0x11, 0xd4, 0x40, 0x1e, 0xe3, 0xb9, 0x71, 0x89, 0x8b, 0x3c, 0x26, 0xe0, 0xbf, 0x4d, 0x16, 0xa7, - 0xc4, 0x17, 0x00, 0xa4, 0x39, 0x71, 0x76, 0xc9, 0xdf, 0x4a, 0x42, 0x7e, 0xbb, 0x67, 0xef, 0xc7, - 0x3c, 0xfa, 0x2a, 0x00, 0xdd, 0x9e, 0x4d, 0xf1, 0xc1, 0xb1, 0xc1, 0xdb, 0x7c, 0x4e, 0xbc, 0x84, - 0xdb, 0x68, 0xc6, 0xd7, 0x3c, 0x36, 0x50, 0x8d, 0x0b, 0xc1, 0xb2, 0x1f, 0x74, 0x71, 0x63, 0x14, - 0x92, 0x6c, 0x1e, 0x1b, 0x9b, 0xd8, 0x83, 0x90, 0x4c, 0x12, 0x26, 0x92, 0x3e, 0x81, 0x69, 0x72, - 0x21, 0x3b, 0xe6, 0x45, 0xba, 0x79, 0x8a, 0xf0, 0x34, 0x4d, 0xf4, 0x18, 0x32, 0x8c, 0x9b, 0x2c, - 0x4d, 0x53, 0x74, 0x69, 0x8a, 0x6a, 0x0b, 0x57, 0x23, 0x5d, 0x94, 0xd2, 0x94, 0x95, 0x2c, 0x44, - 0xf3, 0x30, 0xb9, 0x67, 0x5a, 0xaa, 0xfb, 0xda, 0x94, 0x5d, 0xb0, 0xfe, 0x5c, 0x4f, 0xa5, 0xd3, - 0xc5, 0xcc, 0x7a, 0x2a, 0x9d, 0x29, 0x82, 0xf8, 0x5b, 0x02, 0x14, 0xbc, 0x8e, 0x88, 0xd3, 0x5a, - 0x57, 0x42, 0x5a, 0xbc, 0x78, 0x57, 0x10, 0x05, 0x8a, 0xff, 0x91, 0xba, 0x2b, 0xaa, 0x79, 0x48, - 0x7b, 0x26, 0xce, 0x91, 0xf2, 0x98, 0x85, 0xd4, 0x24, 0x2e, 0xda, 0xbb, 0x34, 0xba, 0xe6, 0x01, - 0xcc, 0xeb, 0x1d, 0x62, 0xc7, 0x75, 0xa7, 0x7d, 0xc2, 0x31, 0x95, 0x83, 0xdd, 0xf7, 0xb3, 0x73, - 0xfe, 0xbd, 0x8a, 0x7b, 0x4b, 0xfc, 0xc7, 0x74, 0x77, 0xda, 0x6f, 0x49, 0x9c, 0xaa, 0xae, 0x43, - 0xce, 0x62, 0xa2, 0x89, 0xcf, 0x71, 0x41, 0x6d, 0xcf, 0x78, 0xac, 0x44, 0xe1, 0xbf, 0x93, 0x80, - 0xc2, 0x8b, 0x1e, 0xb6, 0x4e, 0xbe, 0x4e, 0xea, 0xbe, 0x0d, 0x85, 0x23, 0x45, 0x77, 0xe4, 0x3d, - 0xd3, 0x92, 0x7b, 0x5d, 0x4d, 0x71, 0xdc, 0xb8, 0x8e, 0x1c, 0x21, 0x3f, 0x31, 0xad, 0x1d, 0x4a, - 0x44, 0x18, 0xd0, 0x81, 0x61, 0x1e, 0x19, 0x32, 0x21, 0x53, 0x14, 0x7b, 0x6c, 0xf0, 0x2d, 0xe3, - 0xf2, 0x87, 0xff, 0xf5, 0x74, 0xf1, 0xd1, 0x58, 0xd1, 0x5a, 0x34, 0x32, 0xad, 0xd7, 0xd3, 0xb5, - 0xe5, 0x9d, 0x9d, 0xfa, 0x9a, 0x54, 0xa4, 0x22, 0x5f, 0x31, 0x89, 0xcd, 0x63, 0xc3, 0x16, 0xff, - 0x49, 0x02, 0x8a, 0xbe, 0x8e, 0xe2, 0xec, 0xc8, 0x2a, 0x64, 0x5f, 0xf7, 0xb0, 0xa5, 0xbf, 0x41, - 0x37, 0x02, 0x67, 0x24, 0x66, 0xe7, 0x73, 0x98, 0x09, 0x69, 0x20, 0xf9, 0x93, 0x69, 0x20, 0x7b, - 0xe4, 0x37, 0x1e, 0xdd, 0x87, 0x59, 0xe7, 0xd8, 0x90, 0x59, 0x9c, 0x1e, 0x8b, 0xed, 0x70, 0xc3, - 0x0e, 0x0a, 0x0e, 0xd1, 0x07, 0xa1, 0xd3, 0xb8, 0x0e, 0x5b, 0xfc, 0x23, 0x01, 0x10, 0x55, 0x54, - 0x9d, 0xed, 0xe9, 0x7f, 0x5d, 0xc6, 0xd3, 0x5d, 0x28, 0xd2, 0xc8, 0x47, 0x59, 0xdf, 0x93, 0x3b, - 0xba, 0x6d, 0xeb, 0x46, 0x8b, 0x0f, 0xa8, 0x3c, 0xa5, 0xd7, 0xf7, 0x36, 0x19, 0x55, 0xfc, 0xeb, - 0x30, 0x17, 0x6a, 0x40, 0x9c, 0x9d, 0x7d, 0x1d, 0x66, 0xf6, 0xcc, 0x9e, 0xa1, 0xc9, 0xec, 0x8d, - 0x07, 0xdf, 0x0e, 0xcc, 0x52, 0x1a, 0x7b, 0x9e, 0xf8, 0x3f, 0x13, 0x30, 0x2f, 0x61, 0xdb, 0x6c, - 0x1f, 0xe2, 0xf8, 0x55, 0x58, 0x03, 0xfe, 0xae, 0x45, 0x7e, 0x23, 0x4d, 0x66, 0x18, 0x33, 0x5b, - 0xe6, 0xc2, 0x7b, 0xea, 0x37, 0x47, 0x8f, 0xd8, 0xc1, 0x5d, 0x74, 0xbe, 0x27, 0x97, 0x0a, 0xed, - 0xc9, 0x99, 0x50, 0xd0, 0x5b, 0x86, 0x49, 0x6c, 0x9a, 0x8d, 0x5f, 0x1b, 0xbd, 0x8e, 0x8b, 0x54, - 0x96, 0x47, 0x55, 0xb2, 0xce, 0x58, 0x1a, 0xf8, 0xf5, 0x56, 0xaf, 0x43, 0x7d, 0xe7, 0xf2, 0x65, - 0x52, 0xdf, 0xb3, 0xd3, 0xc5, 0x7c, 0xe8, 0x9e, 0x2d, 0xe5, 0x75, 0xef, 0x9a, 0x48, 0x17, 0xbf, - 0x0d, 0x97, 0xfa, 0x94, 0x1d, 0xa7, 0xc7, 0xf3, 0xef, 0x92, 0x70, 0x35, 0x2c, 0x3e, 0x6e, 0xfc, - 0xf1, 0x75, 0xef, 0xd0, 0x1a, 0xe4, 0x3a, 0xba, 0xf1, 0x66, 0x5b, 0x8b, 0x33, 0x1d, 0xdd, 0xf0, - 0xb7, 0x71, 0x23, 0x86, 0xc6, 0xd4, 0x57, 0x3a, 0x34, 0x14, 0x58, 0x88, 0xea, 0xbb, 0x38, 0xc7, - 0xc7, 0xaf, 0x09, 0x30, 0x13, 0xf7, 0x9e, 0xd9, 0x9b, 0xc5, 0x9b, 0x89, 0x4d, 0xc8, 0x7d, 0x05, - 0x9b, 0x6c, 0xbf, 0x23, 0x00, 0x6a, 0x5a, 0x3d, 0x83, 0x80, 0xda, 0x67, 0x66, 0x2b, 0xce, 0x66, - 0xce, 0xc3, 0xa4, 0x6e, 0x68, 0xf8, 0x98, 0x36, 0x33, 0x25, 0xb1, 0x8b, 0xd0, 0xab, 0xc3, 0xe4, - 0x58, 0xaf, 0x0e, 0xc5, 0xcf, 0x61, 0x2e, 0x54, 0xc5, 0x38, 0xdb, 0xff, 0x4f, 0x13, 0x30, 0xc7, - 0x1b, 0x12, 0xfb, 0xf6, 0xe2, 0x37, 0x61, 0xb2, 0x4d, 0x64, 0x8e, 0xe8, 0x67, 0xfa, 0x4c, 0xb7, - 0x9f, 0x69, 0x61, 0xf4, 0x73, 0x00, 0x5d, 0x0b, 0x1f, 0xca, 0x8c, 0x35, 0x39, 0x16, 0x6b, 0x86, - 0x70, 0x50, 0x02, 0xfa, 0x05, 0x28, 0x90, 0xf9, 0xdc, 0xb5, 0xcc, 0xae, 0x69, 0x13, 0x97, 0xc5, - 0x1e, 0x0f, 0xe5, 0xcc, 0x9e, 0x9d, 0x2e, 0xe6, 0x36, 0x75, 0x63, 0x9b, 0x33, 0x36, 0x1b, 0x12, - 0x31, 0x0c, 0xde, 0xa5, 0x2d, 0xfe, 0x67, 0x01, 0xe6, 0xbf, 0xb2, 0xad, 0xd8, 0xbf, 0x0a, 0x5d, - 0x89, 0x2f, 0xa1, 0x48, 0x7f, 0xd4, 0x8d, 0x3d, 0x33, 0xce, 0x4d, 0xf1, 0xef, 0x09, 0x30, 0x1b, - 0x10, 0x1c, 0xa7, 0x7f, 0xf2, 0x46, 0x7a, 0x12, 0x7f, 0x91, 0x78, 0x2c, 0xc1, 0x41, 0x1e, 0xe7, - 0x14, 0xfa, 0x83, 0x04, 0x5c, 0xae, 0xb0, 0x57, 0xc8, 0x6e, 0x4c, 0x45, 0x9c, 0x23, 0xa3, 0x04, - 0xd3, 0x87, 0xd8, 0xb2, 0x75, 0x93, 0xad, 0x9e, 0x39, 0xc9, 0xbd, 0x44, 0x0b, 0x90, 0xb6, 0x0d, - 0xa5, 0x6b, 0xef, 0x9b, 0xee, 0xbb, 0x33, 0xef, 0xda, 0x8b, 0xff, 0x98, 0x7c, 0xf3, 0xf8, 0x8f, - 0xa9, 0xd1, 0xf1, 0x1f, 0xd3, 0x3f, 0x41, 0xfc, 0x07, 0x7f, 0x51, 0xf5, 0x9f, 0x04, 0xb8, 0x32, - 0xa0, 0xb9, 0x38, 0x47, 0xcb, 0x77, 0x21, 0xab, 0x72, 0xc1, 0xc4, 0xde, 0xb2, 0xb7, 0x70, 0x75, - 0x52, 0xec, 0x0d, 0x61, 0xc7, 0xd9, 0xe9, 0x22, 0xb8, 0x55, 0xad, 0xaf, 0x71, 0xe5, 0x90, 0xdf, - 0x9a, 0xf8, 0x2b, 0x39, 0x28, 0x54, 0x8f, 0xd9, 0x0e, 0x74, 0x83, 0xad, 0xf2, 0xe8, 0x09, 0xa4, - 0xbb, 0x96, 0x79, 0xa8, 0xbb, 0xcd, 0xc8, 0x87, 0x5e, 0xfe, 0xbb, 0xcd, 0xe8, 0xe3, 0xda, 0xe6, - 0x1c, 0x92, 0xc7, 0x8b, 0x9a, 0x90, 0x79, 0x66, 0xaa, 0x4a, 0xfb, 0x89, 0xde, 0x76, 0x47, 0xfe, - 0xfb, 0xe7, 0x0b, 0x5a, 0xf6, 0x78, 0xb6, 0x15, 0x67, 0xdf, 0xed, 0x04, 0x8f, 0x88, 0xea, 0x90, - 0xae, 0x39, 0x4e, 0x97, 0xdc, 0xe4, 0xb6, 0xe3, 0xce, 0x18, 0x42, 0x09, 0x8b, 0x1b, 0x2b, 0xea, - 0xb2, 0xa3, 0x26, 0xcc, 0x3e, 0xa5, 0x27, 0x9f, 0x2a, 0x6d, 0xb3, 0xa7, 0x55, 0x4c, 0x63, 0x4f, - 0x6f, 0x71, 0xbb, 0x7b, 0x7b, 0x0c, 0x99, 0x4f, 0x2b, 0x0d, 0x69, 0x50, 0x00, 0x5a, 0x85, 0x74, - 0xe3, 0x11, 0x17, 0xc6, 0xdc, 0xb2, 0x5b, 0x63, 0x08, 0x6b, 0x3c, 0x92, 0x3c, 0x36, 0xb4, 0x0e, - 0xd9, 0xd5, 0x2f, 0x7a, 0x16, 0xe6, 0x52, 0xa6, 0x86, 0x46, 0x1e, 0xf4, 0x4b, 0xa1, 0x5c, 0x52, - 0x90, 0x19, 0x35, 0x20, 0xff, 0xca, 0xb4, 0x0e, 0xda, 0xa6, 0xe2, 0xb6, 0x70, 0x9a, 0x8a, 0xfb, - 0xc6, 0x18, 0xe2, 0x5c, 0x46, 0xa9, 0x4f, 0x04, 0xfa, 0x36, 0x14, 0x48, 0x67, 0x34, 0x95, 0xdd, - 0xb6, 0x5b, 0xc9, 0x34, 0x95, 0xfa, 0xee, 0x18, 0x52, 0x3d, 0x4e, 0xf7, 0x15, 0x48, 0x9f, 0xa8, - 0x05, 0x09, 0x72, 0xa1, 0x41, 0x80, 0x10, 0xa4, 0xba, 0xa4, 0xbf, 0x05, 0x1a, 0x1b, 0x44, 0x7f, - 0xa3, 0xf7, 0x60, 0xda, 0x30, 0x35, 0xec, 0xce, 0x90, 0x5c, 0x79, 0xfe, 0xec, 0x74, 0x71, 0x6a, - 0xcb, 0xd4, 0x98, 0x43, 0xc2, 0x7f, 0x49, 0x53, 0xa4, 0x50, 0x5d, 0x5b, 0x58, 0x82, 0x14, 0xe9, - 0x77, 0x62, 0x98, 0x76, 0x15, 0x1b, 0xef, 0x58, 0x3a, 0x97, 0xe6, 0x5e, 0x2e, 0xfc, 0x8b, 0x04, - 0x24, 0x1a, 0x8f, 0x88, 0xcb, 0xbd, 0xdb, 0x53, 0x0f, 0xb0, 0xc3, 0xef, 0xf3, 0x2b, 0xea, 0x8a, - 0x5b, 0x78, 0x4f, 0x67, 0x9e, 0x51, 0x46, 0xe2, 0x57, 0xe8, 0x1d, 0x00, 0x45, 0x55, 0xb1, 0x6d, - 0xcb, 0xee, 0x89, 0xb8, 0x8c, 0x94, 0x61, 0x94, 0x0d, 0x7c, 0x42, 0xd8, 0x6c, 0xac, 0x5a, 0xd8, - 0x71, 0x03, 0x9b, 0xd8, 0x15, 0x61, 0x73, 0x70, 0xa7, 0x2b, 0x3b, 0xe6, 0x01, 0x36, 0xe8, 0x38, - 0xc9, 0x10, 0x53, 0xd3, 0xe9, 0x36, 0x09, 0x81, 0x58, 0x49, 0x6c, 0x68, 0xbe, 0x49, 0xcb, 0x48, - 0xde, 0x35, 0x11, 0x69, 0xe1, 0x96, 0xce, 0x8f, 0x76, 0x65, 0x24, 0x7e, 0x45, 0xb4, 0xa4, 0xf4, - 0x9c, 0x7d, 0xda, 0x13, 0x19, 0x89, 0xfe, 0x46, 0xb7, 0xa1, 0xc0, 0x62, 0x21, 0x65, 0x6c, 0xa8, - 0x32, 0x35, 0xae, 0x19, 0x7a, 0x3b, 0xc7, 0xc8, 0x55, 0x43, 0x25, 0xa6, 0x14, 0x3d, 0x02, 0x4e, - 0x90, 0x0f, 0x3a, 0x36, 0xd1, 0x29, 0x90, 0x52, 0xe5, 0xc2, 0xd9, 0xe9, 0x62, 0xb6, 0x41, 0x6f, - 0x6c, 0x6c, 0x36, 0xea, 0x6b, 0x52, 0x96, 0x95, 0xda, 0xe8, 0xd8, 0x75, 0x6d, 0xe1, 0x37, 0x04, - 0x48, 0x3e, 0xad, 0x34, 0x2e, 0xac, 0x32, 0xb7, 0xa2, 0xc9, 0x40, 0x45, 0xef, 0x40, 0x61, 0x57, - 0x6f, 0xb7, 0x75, 0xa3, 0x45, 0xbc, 0xa0, 0xef, 0x62, 0xd5, 0x55, 0x58, 0x9e, 0x93, 0xb7, 0x19, - 0x15, 0x2d, 0x41, 0x56, 0xb5, 0xb0, 0x86, 0x0d, 0x47, 0x57, 0xda, 0x36, 0xd7, 0x5c, 0x90, 0xb4, - 0xf0, 0xcb, 0x02, 0x4c, 0xd2, 0x19, 0x80, 0xde, 0x86, 0x8c, 0x6a, 0x1a, 0x8e, 0xa2, 0x1b, 0xdc, - 0x94, 0x65, 0x24, 0x9f, 0x30, 0xb4, 0x7a, 0xd7, 0x61, 0x46, 0x51, 0x55, 0xb3, 0x67, 0x38, 0xb2, - 0xa1, 0x74, 0x30, 0xaf, 0x66, 0x96, 0xd3, 0xb6, 0x94, 0x0e, 0x46, 0x8b, 0xe0, 0x5e, 0x7a, 0x07, - 0x1d, 0x33, 0x12, 0x70, 0xd2, 0x06, 0x3e, 0x59, 0xf8, 0x63, 0x01, 0xd2, 0xee, 0x9c, 0x21, 0xd5, - 0x68, 0x61, 0x03, 0x5b, 0x8a, 0x63, 0x7a, 0xd5, 0xf0, 0x08, 0xfd, 0x4b, 0x65, 0xc6, 0x5f, 0x2a, - 0xe7, 0x61, 0xd2, 0x21, 0xd3, 0x82, 0xd7, 0x80, 0x5d, 0xd0, 0xed, 0xe8, 0xb6, 0xd2, 0x62, 0xbb, - 0x71, 0x19, 0x89, 0x5d, 0x90, 0xc6, 0xf0, 0x90, 0x5a, 0xa6, 0x11, 0x7e, 0x45, 0x6a, 0xca, 0x02, - 0x3f, 0x77, 0x71, 0x4b, 0x37, 0xe8, 0x58, 0x4a, 0x4a, 0x40, 0x49, 0x65, 0x42, 0x41, 0x6f, 0x41, - 0x86, 0x15, 0xc0, 0x86, 0x46, 0x07, 0x54, 0x52, 0x4a, 0x53, 0x42, 0xd5, 0xd0, 0x16, 0x30, 0x64, - 0xbc, 0xc9, 0x49, 0xba, 0xad, 0x67, 0x7b, 0x8a, 0xa4, 0xbf, 0xd1, 0xfb, 0x30, 0xff, 0xba, 0xa7, - 0xb4, 0xf5, 0x3d, 0xba, 0xd1, 0x46, 0x8a, 0x31, 0x9d, 0xb1, 0x96, 0x20, 0xef, 0x1e, 0x95, 0x40, - 0x55, 0xe7, 0xce, 0xe5, 0xa4, 0x3f, 0x97, 0xc5, 0xdf, 0x17, 0x60, 0x96, 0x45, 0xf5, 0xb0, 0x60, - 0xd4, 0xf8, 0xfc, 0x90, 0x8f, 0x21, 0xa3, 0x29, 0x8e, 0xc2, 0x8e, 0x6e, 0x26, 0x46, 0x1e, 0xdd, - 0xf4, 0x8e, 0x12, 0x28, 0x8e, 0x42, 0x8f, 0x6f, 0x22, 0x48, 0x91, 0xdf, 0xec, 0x94, 0xab, 0x44, - 0x7f, 0x8b, 0x9f, 0x01, 0x0a, 0x56, 0x34, 0x4e, 0x8f, 0xec, 0x1e, 0x5c, 0x22, 0xba, 0xae, 0x1a, - 0xaa, 0x75, 0xd2, 0x75, 0x74, 0xd3, 0x78, 0x4e, 0xff, 0xda, 0xa8, 0x18, 0x78, 0x2f, 0x45, 0x5f, - 0x47, 0x89, 0x7f, 0x38, 0x05, 0xb9, 0xea, 0x71, 0xd7, 0xb4, 0x62, 0xdd, 0xc5, 0x2a, 0xc3, 0x34, - 0x07, 0xfa, 0x23, 0xde, 0x0b, 0xf7, 0x19, 0x73, 0xf7, 0x95, 0x2b, 0x67, 0x44, 0x65, 0x00, 0x16, - 0x30, 0x4a, 0x83, 0x82, 0x92, 0x17, 0x78, 0x53, 0x46, 0xd9, 0x08, 0x15, 0x6d, 0x41, 0xb6, 0x73, - 0xa8, 0xaa, 0xf2, 0x9e, 0xde, 0x76, 0x78, 0xdc, 0x5d, 0x74, 0x88, 0xf8, 0xe6, 0xcb, 0x4a, 0xe5, - 0x09, 0x2d, 0xc4, 0x42, 0xe0, 0xfc, 0x6b, 0x09, 0x88, 0x04, 0xf6, 0x1b, 0xbd, 0x0b, 0xfc, 0x48, - 0x8d, 0x6c, 0xbb, 0xa7, 0xe7, 0xca, 0xb9, 0xb3, 0xd3, 0xc5, 0x8c, 0x44, 0xa9, 0x8d, 0x46, 0x53, - 0xca, 0xb0, 0x02, 0x0d, 0xdb, 0x41, 0x37, 0x20, 0x67, 0x76, 0x74, 0x47, 0x76, 0x9d, 0x24, 0xee, - 0x51, 0xce, 0x10, 0xa2, 0xeb, 0x44, 0xa1, 0x26, 0xdc, 0xc1, 0x06, 0x1d, 0xed, 0xa4, 0x9d, 0xf2, - 0x2e, 0xdb, 0x7c, 0x74, 0xd8, 0x8c, 0x96, 0xcd, 0xae, 0xa3, 0x77, 0xf4, 0x2f, 0xe8, 0x9b, 0x69, - 0xfe, 0xd2, 0xe8, 0x06, 0x2b, 0x4e, 0xda, 0x57, 0xa6, 0xbb, 0x92, 0xbc, 0xec, 0xf3, 0x40, 0x51, - 0xf4, 0xb7, 0x05, 0xb8, 0xcc, 0x15, 0x29, 0xef, 0xd2, 0x18, 0x77, 0xa5, 0xad, 0x3b, 0x27, 0xf2, - 0xc1, 0x61, 0x29, 0x4d, 0xfd, 0xd6, 0x9f, 0x8d, 0xec, 0x90, 0xc0, 0x38, 0x58, 0x76, 0xbb, 0xe5, - 0xe4, 0x19, 0x67, 0xde, 0x38, 0xac, 0x1a, 0x8e, 0x75, 0x52, 0xbe, 0x72, 0x76, 0xba, 0x38, 0x37, - 0x78, 0xf7, 0xa5, 0x34, 0x67, 0x0f, 0xb2, 0xa0, 0x1a, 0x00, 0xf6, 0xc6, 0x21, 0x5d, 0x31, 0xa2, - 0xfd, 0x8f, 0xc8, 0x01, 0x2b, 0x05, 0x78, 0xd1, 0x5d, 0x28, 0xf2, 0x93, 0x2d, 0x7b, 0x7a, 0x1b, - 0xcb, 0xb6, 0xfe, 0x05, 0xa6, 0x6b, 0x4b, 0x52, 0xca, 0x33, 0x3a, 0x11, 0xd1, 0xd0, 0xbf, 0xc0, - 0x0b, 0xdf, 0x85, 0xd2, 0xb0, 0xda, 0x07, 0xa7, 0x40, 0x86, 0xbd, 0x91, 0xfd, 0x28, 0xbc, 0x1d, - 0x33, 0xc6, 0x50, 0xe5, 0x5b, 0x32, 0x1f, 0x27, 0x3e, 0x12, 0xc4, 0x7f, 0x96, 0x80, 0x5c, 0xb9, - 0xd7, 0x3e, 0x78, 0xde, 0x6d, 0xf4, 0x3a, 0x1d, 0xc5, 0x3a, 0x21, 0x66, 0x90, 0x19, 0x0a, 0x52, - 0x41, 0x81, 0x99, 0x41, 0x6a, 0x09, 0xf4, 0x2f, 0x30, 0x59, 0x9c, 0x82, 0xa7, 0xb5, 0x59, 0x0c, - 0x3f, 0x6d, 0x43, 0xe0, 0x08, 0xb6, 0x79, 0x64, 0xa3, 0x8f, 0xa0, 0x14, 0x28, 0x48, 0xf7, 0x4e, - 0x64, 0x6c, 0x38, 0x96, 0x8e, 0xd9, 0xfe, 0x5f, 0x52, 0x0a, 0xc4, 0xcb, 0xd4, 0xc9, 0xed, 0x2a, - 0xbb, 0x8b, 0x9a, 0x30, 0x43, 0x0a, 0x9e, 0xc8, 0x74, 0x09, 0x71, 0xf7, 0x67, 0x1f, 0x44, 0x34, - 0x2b, 0x54, 0xef, 0x65, 0xaa, 0x9f, 0x0a, 0xe5, 0xa1, 0x3f, 0xa5, 0x2c, 0xf6, 0x29, 0x0b, 0x9f, - 0x42, 0xb1, 0xbf, 0x40, 0x50, 0x97, 0x29, 0xa6, 0xcb, 0xf9, 0xa0, 0x2e, 0x93, 0x01, 0x3d, 0xad, - 0xa7, 0xd2, 0xa9, 0xe2, 0xa4, 0xf8, 0x17, 0x49, 0xc8, 0xbb, 0xc3, 0x2c, 0x4e, 0xa0, 0x53, 0x86, - 0x49, 0x32, 0x28, 0xdc, 0x18, 0x8f, 0xdb, 0x23, 0x46, 0x37, 0x8f, 0x1a, 0x27, 0x83, 0xc5, 0x05, - 0xc9, 0x94, 0x35, 0x0e, 0x83, 0xb3, 0xf0, 0xcb, 0x09, 0x48, 0x51, 0x6c, 0xf1, 0x00, 0x52, 0x74, - 0xa1, 0x10, 0xc6, 0x59, 0x28, 0x68, 0x51, 0x6f, 0x39, 0x4b, 0x04, 0x5c, 0x53, 0xe2, 0xf3, 0xed, - 0x2b, 0x1f, 0x3c, 0x78, 0x48, 0x8d, 0xcd, 0x8c, 0xc4, 0xaf, 0x50, 0x99, 0x86, 0x1d, 0x99, 0x96, - 0x83, 0x35, 0xee, 0xd3, 0x2f, 0x9d, 0xd7, 0xbf, 0xee, 0xa2, 0xe4, 0xf2, 0xa1, 0xab, 0x90, 0x24, - 0x56, 0x6c, 0x9a, 0x05, 0x29, 0x9c, 0x9d, 0x2e, 0x26, 0x89, 0xfd, 0x22, 0x34, 0xb4, 0x02, 0xd9, - 0xb0, 0xc9, 0x20, 0x1e, 0x1c, 0x35, 0x8c, 0x81, 0xe9, 0x0e, 0x6d, 0x6f, 0x6a, 0x31, 0x3c, 0xcb, - 0xfb, 0xf8, 0x2f, 0x53, 0x90, 0xab, 0x77, 0xe2, 0x5e, 0x52, 0x56, 0xc3, 0x3d, 0x1c, 0x05, 0x84, - 0x42, 0x0f, 0x8d, 0xe8, 0xe0, 0xd0, 0x0a, 0x9e, 0xbc, 0xd8, 0x0a, 0x5e, 0x27, 0x9e, 0x32, 0x4f, - 0xbf, 0x90, 0x1c, 0x82, 0x79, 0xc2, 0xcf, 0xa7, 0x7e, 0x8a, 0x44, 0x78, 0xfc, 0x73, 0x14, 0x34, - 0xd0, 0xe4, 0x53, 0xea, 0x90, 0xb3, 0x51, 0x36, 0x35, 0xfe, 0x28, 0x9b, 0xc6, 0x86, 0x46, 0x17, - 0xb5, 0xb0, 0x45, 0x9d, 0x7e, 0x73, 0x8b, 0xba, 0xe0, 0xf0, 0xc1, 0xfa, 0x31, 0x24, 0x35, 0xdd, - 0xed, 0x9c, 0xf1, 0x97, 0x6a, 0xc2, 0x74, 0xce, 0xa8, 0x4d, 0x05, 0x47, 0x2d, 0x1b, 0x25, 0x0b, - 0x75, 0x00, 0x5f, 0x37, 0x68, 0x09, 0xa6, 0xcc, 0xb6, 0xe6, 0x1e, 0x24, 0xc9, 0x95, 0x33, 0x67, - 0xa7, 0x8b, 0x93, 0xcf, 0xdb, 0x5a, 0x7d, 0x4d, 0x9a, 0x34, 0xdb, 0x5a, 0x5d, 0xa3, 0xb9, 0x2f, - 0xf0, 0x91, 0xec, 0x45, 0x99, 0xcd, 0x48, 0xd3, 0x06, 0x3e, 0x5a, 0xc3, 0xb6, 0xca, 0x07, 0xdc, - 0x6f, 0x0b, 0x90, 0x77, 0x75, 0x1f, 0xaf, 0x51, 0x49, 0xeb, 0x1d, 0x3e, 0xc9, 0x92, 0x17, 0x9b, - 0x64, 0x2e, 0x1f, 0x3f, 0x5e, 0xfb, 0xab, 0x02, 0x8f, 0x1b, 0x6e, 0xa8, 0x8a, 0x43, 0x9c, 0x8a, - 0x18, 0x27, 0xc6, 0x3d, 0x28, 0x5a, 0x8a, 0xa1, 0x99, 0x1d, 0xfd, 0x0b, 0xcc, 0x36, 0x42, 0x6d, - 0xfe, 0xd6, 0xb2, 0xe0, 0xd1, 0xe9, 0xae, 0x9f, 0x2d, 0xfe, 0x71, 0x82, 0xc7, 0x18, 0x7b, 0xd5, - 0x88, 0x53, 0x5d, 0xdf, 0x81, 0xd9, 0xfe, 0x44, 0x24, 0xee, 0x6c, 0x7d, 0x2f, 0x42, 0x5e, 0x54, - 0x45, 0x58, 0xac, 0xa0, 0x1b, 0xb8, 0xde, 0x97, 0x94, 0xc4, 0x46, 0x15, 0xc8, 0x06, 0xf3, 0x9b, - 0x24, 0xc7, 0xce, 0x6f, 0x02, 0x96, 0x97, 0xd5, 0x64, 0xe1, 0xe7, 0x61, 0x92, 0xde, 0x7e, 0x03, - 0x13, 0xcd, 0x7b, 0xf3, 0xcf, 0x13, 0x70, 0x93, 0xd6, 0xfe, 0x25, 0xb6, 0xf4, 0xbd, 0x93, 0x6d, - 0xcb, 0x74, 0xb0, 0xea, 0x60, 0xcd, 0x3f, 0x05, 0x12, 0xab, 0xdd, 0xcb, 0x74, 0xdd, 0x07, 0x5c, - 0x28, 0x5e, 0xcc, 0xe3, 0x42, 0x1b, 0x50, 0xe0, 0x91, 0x01, 0x4a, 0x5b, 0x3f, 0xc4, 0xb2, 0xe2, - 0x5c, 0x64, 0x75, 0xcb, 0x31, 0xde, 0x55, 0xc2, 0xba, 0xea, 0x20, 0x0d, 0x32, 0x5c, 0x98, 0xae, - 0xf1, 0xb4, 0x3c, 0x4f, 0x7f, 0xb2, 0x0d, 0xc5, 0x34, 0x0b, 0x4f, 0xa8, 0xaf, 0x49, 0x69, 0x26, - 0xb9, 0xae, 0x89, 0xff, 0x45, 0x80, 0x5b, 0xe7, 0xa8, 0x38, 0xce, 0xa1, 0xbb, 0x00, 0xe9, 0x43, - 0xf2, 0x20, 0x9d, 0xeb, 0x38, 0x2d, 0x79, 0xd7, 0x68, 0x13, 0x72, 0x7b, 0x8a, 0xde, 0xf6, 0x87, - 0xf4, 0xf0, 0xf0, 0xc2, 0xe8, 0x48, 0xd7, 0x19, 0xc6, 0xce, 0xc6, 0xb0, 0xf8, 0x9b, 0x09, 0x98, - 0x5d, 0xd5, 0xb4, 0x46, 0x83, 0xdb, 0xc0, 0xf8, 0x46, 0x8a, 0x0b, 0x32, 0x13, 0x3e, 0xc8, 0x44, - 0xef, 0x01, 0xd2, 0x74, 0x9b, 0xa5, 0xff, 0xb0, 0xf7, 0x15, 0xcd, 0x3c, 0xf2, 0xc3, 0x2a, 0x66, - 0xdd, 0x3b, 0x0d, 0xf7, 0x06, 0x6a, 0x00, 0x45, 0x3b, 0xb2, 0xed, 0x28, 0xde, 0x7b, 0xa3, 0x5b, - 0x63, 0x1d, 0xd7, 0x62, 0x30, 0xc8, 0xbb, 0x94, 0x32, 0x44, 0x0e, 0xfd, 0x49, 0xfc, 0x76, 0x9d, - 0x34, 0xdd, 0x91, 0x15, 0xdb, 0x3d, 0x9b, 0xc3, 0x12, 0x8f, 0xe4, 0x19, 0x7d, 0xd5, 0x66, 0x47, - 0x6e, 0xd8, 0x61, 0x02, 0x5f, 0x35, 0x71, 0x42, 0xe2, 0x7f, 0x24, 0x40, 0x5e, 0xc2, 0x7b, 0x16, - 0xb6, 0x63, 0xdd, 0x14, 0x78, 0x02, 0x33, 0x16, 0x93, 0x2a, 0xef, 0x59, 0x66, 0xe7, 0x22, 0xf3, - 0x2a, 0xcb, 0x19, 0x9f, 0x58, 0x66, 0x87, 0x1b, 0x96, 0x97, 0x50, 0xf0, 0xea, 0x18, 0x67, 0xe3, - 0x7f, 0x9f, 0x1e, 0x45, 0x66, 0x82, 0xe3, 0x8e, 0x6f, 0x88, 0x57, 0x03, 0xf4, 0x45, 0x55, 0xb0, - 0xa2, 0x71, 0xaa, 0xe1, 0x7f, 0x08, 0x90, 0x6f, 0xf4, 0x76, 0x59, 0x7e, 0xa9, 0xf8, 0x34, 0x50, - 0x85, 0x4c, 0x1b, 0xef, 0x39, 0xf2, 0x1b, 0x85, 0xc1, 0xa7, 0x09, 0x2b, 0x3d, 0x04, 0xf0, 0x14, - 0xc0, 0xa2, 0xc7, 0xd7, 0xa8, 0x9c, 0xe4, 0x05, 0xe5, 0x64, 0x28, 0x2f, 0x21, 0x8b, 0xff, 0x3c, - 0x01, 0x05, 0xaf, 0x99, 0x71, 0x5a, 0xc9, 0x57, 0x21, 0xeb, 0x90, 0xbc, 0x88, 0x75, 0x98, 0xe5, - 0x21, 0x1d, 0xd1, 0x16, 0x62, 0x19, 0xe6, 0xa8, 0xe3, 0x22, 0x2b, 0xdd, 0x6e, 0x5b, 0x77, 0xe1, - 0x2e, 0xb5, 0x3f, 0x29, 0x69, 0x96, 0xde, 0x5a, 0x65, 0x77, 0x28, 0xd0, 0x25, 0x63, 0x6e, 0xcf, - 0xc2, 0xf8, 0x0b, 0x2c, 0x53, 0xe4, 0x75, 0x91, 0x90, 0x95, 0x2c, 0x63, 0x6c, 0x10, 0x3e, 0x3e, - 0xe6, 0x5e, 0xc1, 0x2c, 0xd5, 0x69, 0xdc, 0xc7, 0x6e, 0xc5, 0x7f, 0x90, 0x00, 0x14, 0x94, 0xfc, - 0xd5, 0xf5, 0x45, 0x22, 0xbe, 0xbe, 0x78, 0x17, 0x10, 0x0b, 0x5a, 0xb4, 0xe5, 0x2e, 0xb6, 0x64, - 0x1b, 0xab, 0x26, 0xcf, 0x71, 0x24, 0x48, 0x45, 0x7e, 0x67, 0x1b, 0x5b, 0x0d, 0x4a, 0x47, 0x8f, - 0x01, 0x7c, 0x8f, 0x8c, 0x2f, 0x18, 0x23, 0x1d, 0x32, 0x29, 0xe3, 0xb9, 0x62, 0xe2, 0xf7, 0x16, - 0x60, 0x86, 0xeb, 0x70, 0xc7, 0xd0, 0x4d, 0x03, 0x3d, 0x80, 0x64, 0x8b, 0x6f, 0xfd, 0x67, 0x23, - 0xb7, 0xe9, 0xfc, 0xe4, 0x6e, 0xb5, 0x09, 0x89, 0x94, 0x25, 0x2c, 0xdd, 0x9e, 0x13, 0xe1, 0x18, - 0xf9, 0x81, 0xd8, 0x41, 0x96, 0x6e, 0xcf, 0x41, 0x0d, 0x28, 0xa8, 0x7e, 0x92, 0x2a, 0x99, 0xb0, - 0x27, 0x87, 0x82, 0xa8, 0xc8, 0x74, 0x61, 0xb5, 0x09, 0x29, 0xaf, 0x86, 0x6e, 0xa0, 0x4a, 0x30, - 0x37, 0x52, 0x6a, 0x20, 0xca, 0xcb, 0x3f, 0xdf, 0x1b, 0xce, 0xcb, 0x54, 0x9b, 0x08, 0xa4, 0x50, - 0x42, 0x1f, 0xc3, 0x94, 0x46, 0xb3, 0xf0, 0xf0, 0x11, 0x1d, 0x35, 0xe8, 0x42, 0xc9, 0x8e, 0x6a, - 0x13, 0x12, 0xe7, 0x40, 0xeb, 0x30, 0xc3, 0x7e, 0x31, 0x37, 0x85, 0x23, 0xcb, 0x5b, 0xc3, 0x25, - 0x04, 0x16, 0x82, 0xda, 0x84, 0x94, 0xd5, 0x7c, 0x2a, 0xfa, 0x26, 0xa4, 0x6c, 0x55, 0x71, 0xb1, - 0xe5, 0xb5, 0x21, 0x29, 0x35, 0x7c, 0x66, 0x5a, 0x1a, 0x3d, 0x66, 0xe9, 0x1a, 0x9d, 0x63, 0x77, - 0x9b, 0x2f, 0xaa, 0xfa, 0xa1, 0xa3, 0xdb, 0xa4, 0xfa, 0x98, 0x12, 0xd0, 0x53, 0xc8, 0x2a, 0xc4, - 0xdf, 0x93, 0xe9, 0xe1, 0x47, 0xba, 0xaf, 0x17, 0xfd, 0x72, 0x7d, 0xe0, 0xb0, 0x6a, 0x8d, 0x9e, - 0x0f, 0x77, 0x89, 0xbe, 0xa0, 0x0e, 0xb6, 0x5a, 0xb8, 0x94, 0x1d, 0x2d, 0x28, 0x18, 0xdb, 0xe5, - 0x09, 0xa2, 0x44, 0xe2, 0xf7, 0x79, 0x27, 0x8f, 0x69, 0xa3, 0x66, 0x86, 0xbe, 0xce, 0x8d, 0x38, - 0xb6, 0x53, 0x9b, 0x90, 0x66, 0xf6, 0x03, 0x64, 0xb4, 0x0c, 0x89, 0x96, 0x5a, 0xca, 0x0d, 0x9d, - 0x21, 0xde, 0xd1, 0x94, 0xda, 0x84, 0x94, 0x68, 0xa9, 0xe8, 0x53, 0x48, 0xb3, 0x73, 0x06, 0xc7, - 0x46, 0x29, 0x3f, 0xd4, 0x4e, 0x84, 0x4f, 0x6b, 0xd4, 0x26, 0x24, 0x7a, 0xb4, 0x81, 0x3c, 0x6f, - 0x1b, 0xf2, 0x16, 0x0b, 0x8e, 0x73, 0x43, 0x59, 0x8b, 0x43, 0x5f, 0x71, 0x47, 0x45, 0xb3, 0xd6, - 0xa8, 0xe7, 0x1f, 0xa0, 0xa3, 0xef, 0xc0, 0x7c, 0x58, 0x22, 0x1f, 0x69, 0xb3, 0x43, 0x5f, 0xd7, - 0x0e, 0x8d, 0xac, 0xac, 0x4d, 0x48, 0xc8, 0x1a, 0xb8, 0x89, 0x3e, 0x84, 0x49, 0xd6, 0x6b, 0x88, - 0x8a, 0x8c, 0x8a, 0xda, 0xe8, 0xeb, 0x30, 0x56, 0x9e, 0x0c, 0x7e, 0x87, 0x47, 0x88, 0xc9, 0x6d, - 0xb3, 0x55, 0x9a, 0x1b, 0x3a, 0xf8, 0x07, 0x63, 0xdd, 0xc8, 0xe0, 0x77, 0x7c, 0x2a, 0xe9, 0x77, - 0x8b, 0xdd, 0xe1, 0x61, 0x45, 0xf3, 0x43, 0xfb, 0x3d, 0x22, 0x70, 0xac, 0x46, 0x63, 0xf7, 0x7d, - 0x32, 0xa9, 0x9a, 0xc5, 0xb2, 0xc0, 0xc8, 0x74, 0x4e, 0x5d, 0x1a, 0x5a, 0xb5, 0xc1, 0x34, 0x39, - 0x35, 0xea, 0x23, 0x79, 0x54, 0xf4, 0x12, 0x8a, 0x3c, 0x57, 0x83, 0xff, 0x4e, 0xe1, 0x32, 0x95, - 0x77, 0x2f, 0xd2, 0x74, 0x45, 0xc5, 0xe4, 0xd4, 0x26, 0xa4, 0x82, 0x1a, 0xbe, 0x83, 0x3e, 0x83, - 0x59, 0x2a, 0x4f, 0x56, 0xfd, 0xf4, 0x1a, 0xa5, 0xd2, 0x40, 0xb2, 0x86, 0xe1, 0x99, 0x38, 0x5c, - 0xc9, 0x45, 0xb5, 0xef, 0x16, 0x19, 0xc6, 0xba, 0xa1, 0x3b, 0xd4, 0xca, 0x2e, 0x0c, 0x1d, 0xc6, - 0xe1, 0xa4, 0x7e, 0x64, 0x18, 0xeb, 0x8c, 0x42, 0x86, 0xb1, 0xc3, 0x63, 0xce, 0x78, 0x77, 0xbc, - 0x3d, 0x74, 0x18, 0x47, 0x05, 0xa7, 0x91, 0x61, 0xec, 0x04, 0xe9, 0x64, 0x18, 0x33, 0x03, 0xd1, - 0x27, 0xf7, 0x9d, 0xa1, 0xc3, 0x78, 0xe8, 0x21, 0x64, 0x32, 0x8c, 0x95, 0x81, 0x9b, 0x68, 0x0d, - 0x80, 0xb9, 0x33, 0x74, 0x51, 0xbc, 0x36, 0x74, 0x31, 0xe8, 0x8f, 0x3d, 0x23, 0x8b, 0x41, 0xdb, - 0xa5, 0x11, 0x43, 0x46, 0xc1, 0x92, 0x4c, 0x5f, 0xa1, 0x96, 0x16, 0x87, 0x1a, 0xb2, 0x81, 0xd7, - 0x9d, 0xc4, 0x90, 0x1d, 0x79, 0x44, 0xb2, 0xaa, 0xb0, 0x3d, 0xdf, 0xd2, 0xd2, 0x70, 0xb3, 0x1c, - 0x7c, 0xf5, 0x43, 0xcd, 0x32, 0x25, 0xa0, 0x55, 0xc8, 0x90, 0x35, 0xff, 0x84, 0x9a, 0xa1, 0xeb, - 0x43, 0x7d, 0xd2, 0xbe, 0xc3, 0x29, 0xb5, 0x09, 0x29, 0xfd, 0x9a, 0x93, 0xc8, 0xe3, 0xd9, 0x6e, - 0x58, 0x49, 0x1c, 0xfa, 0xf8, 0xd0, 0xce, 0x29, 0x79, 0x3c, 0xe3, 0x40, 0x2a, 0x5c, 0x62, 0x7d, - 0xc5, 0xcf, 0x03, 0x5b, 0xfc, 0x08, 0x6b, 0xe9, 0x06, 0x15, 0x35, 0x74, 0x5b, 0x29, 0xf2, 0x98, - 0x72, 0x6d, 0x42, 0x9a, 0x53, 0x06, 0xef, 0x92, 0x09, 0xcf, 0x97, 0x1e, 0xb6, 0x19, 0x55, 0xba, - 0x39, 0x74, 0xc2, 0x47, 0xec, 0xe1, 0x91, 0x09, 0xaf, 0x04, 0xc8, 0x6c, 0x01, 0xd2, 0x64, 0xdb, - 0x66, 0x2f, 0xdc, 0x6f, 0x8d, 0x58, 0x80, 0xfa, 0x76, 0x01, 0xd8, 0x02, 0xa4, 0x35, 0x18, 0x27, - 0x11, 0xa4, 0xb6, 0xb1, 0x62, 0x71, 0x33, 0x7b, 0x7b, 0xa8, 0xa0, 0x81, 0xfc, 0x79, 0x44, 0x90, - 0xea, 0x11, 0x89, 0xc3, 0x63, 0xb9, 0x49, 0x5e, 0xb8, 0xc3, 0x78, 0x67, 0xa8, 0xc3, 0x13, 0x99, - 0x85, 0x86, 0x38, 0x3c, 0x56, 0xe8, 0x06, 0xfa, 0x39, 0x98, 0xe6, 0xf0, 0xad, 0x74, 0x77, 0x84, - 0x1b, 0x1b, 0x44, 0xdc, 0x64, 0x5e, 0x73, 0x1e, 0x66, 0x65, 0x19, 0x6c, 0x64, 0xcd, 0xbb, 0x37, - 0xc2, 0xca, 0x0e, 0x20, 0x57, 0x66, 0x65, 0x7d, 0x32, 0xb1, 0xb2, 0x6c, 0x9c, 0xf2, 0xb5, 0xee, - 0xfe, 0x50, 0x2b, 0x3b, 0x78, 0xf4, 0x85, 0x58, 0xd9, 0xd7, 0x3e, 0x95, 0xb4, 0xcc, 0x66, 0xf0, - 0xa9, 0xf4, 0x8d, 0xa1, 0x2d, 0x0b, 0xe3, 0x48, 0xd2, 0x32, 0xce, 0x43, 0xba, 0x8d, 0xb9, 0xc4, - 0x4c, 0xd3, 0xef, 0x0e, 0x3f, 0x74, 0xdf, 0x0f, 0x3a, 0x6a, 0xee, 0x46, 0x25, 0xd3, 0xb0, 0x67, - 0xa8, 0x2c, 0x7e, 0xe8, 0x98, 0x6b, 0xea, 0xbd, 0xd1, 0x86, 0x2a, 0xea, 0x24, 0xb5, 0x67, 0xa8, - 0x42, 0x37, 0x69, 0x55, 0xd9, 0x39, 0x33, 0x3a, 0xbf, 0x97, 0x47, 0xe4, 0x07, 0xe8, 0x3b, 0xed, - 0x47, 0xab, 0xea, 0x11, 0xfd, 0x29, 0xd4, 0x63, 0xc9, 0x2b, 0x4a, 0x2b, 0xa3, 0xa7, 0x50, 0x38, - 0x7d, 0x86, 0x37, 0x85, 0x38, 0xd9, 0x5b, 0x33, 0x5d, 0x0f, 0xe3, 0xfd, 0xd1, 0x6b, 0x66, 0xbf, - 0x6b, 0xc1, 0xd6, 0x4c, 0xee, 0x53, 0xfc, 0x4d, 0x01, 0x96, 0x58, 0xdd, 0xe8, 0x8e, 0xde, 0x89, - 0xec, 0xed, 0x8b, 0x06, 0xce, 0x3c, 0x3c, 0xa0, 0x0f, 0xf8, 0x70, 0x58, 0x75, 0xcf, 0xd9, 0xe7, - 0xad, 0x4d, 0x48, 0xef, 0x28, 0xa3, 0xca, 0x95, 0xa7, 0xf9, 0x6b, 0x51, 0xef, 0x90, 0x67, 0xa1, - 0x58, 0x5c, 0x4f, 0xa5, 0xaf, 0x14, 0x4b, 0xeb, 0xa9, 0xf4, 0xd5, 0xe2, 0xc2, 0x7a, 0x2a, 0xfd, - 0x56, 0xf1, 0x6d, 0xf1, 0x2f, 0xaf, 0x42, 0xce, 0x45, 0x7e, 0x0c, 0x11, 0x3d, 0x0c, 0x22, 0xa2, - 0x6b, 0xc3, 0x10, 0x11, 0xc7, 0x8a, 0x1c, 0x12, 0x3d, 0x0c, 0x42, 0xa2, 0x6b, 0xc3, 0x20, 0x91, - 0xcf, 0x43, 0x30, 0x51, 0x73, 0x18, 0x26, 0xba, 0x37, 0x06, 0x26, 0xf2, 0x44, 0xf5, 0x83, 0xa2, - 0xb5, 0x41, 0x50, 0x74, 0x73, 0x34, 0x28, 0xf2, 0x44, 0x05, 0x50, 0xd1, 0xe3, 0x3e, 0x54, 0x74, - 0x7d, 0x04, 0x2a, 0xf2, 0xf8, 0x5d, 0x58, 0xb4, 0x11, 0x09, 0x8b, 0x6e, 0x9f, 0x07, 0x8b, 0x3c, - 0x39, 0x21, 0x5c, 0xf4, 0x41, 0x08, 0x17, 0x2d, 0x0e, 0xc5, 0x45, 0x1e, 0x37, 0x03, 0x46, 0x9f, - 0xf4, 0x03, 0xa3, 0xeb, 0x23, 0x80, 0x91, 0xdf, 0x02, 0x8e, 0x8c, 0x6a, 0x51, 0xc8, 0xe8, 0xd6, - 0x39, 0xc8, 0xc8, 0x93, 0x12, 0x84, 0x46, 0xb5, 0x28, 0x68, 0x74, 0xeb, 0x1c, 0x68, 0xd4, 0x27, - 0x89, 0x61, 0xa3, 0xad, 0x68, 0x6c, 0x74, 0xe7, 0x5c, 0x6c, 0xe4, 0x49, 0x0b, 0x83, 0xa3, 0x95, - 0x00, 0x38, 0x7a, 0x67, 0x08, 0x38, 0xf2, 0x58, 0x09, 0x3a, 0xfa, 0xd6, 0x00, 0x3a, 0x12, 0x47, - 0xa1, 0x23, 0x8f, 0xd7, 0x83, 0x47, 0x2f, 0x86, 0xc0, 0xa3, 0xbb, 0xe7, 0xc3, 0x23, 0x4f, 0x58, - 0x1f, 0x3e, 0x52, 0x46, 0xe2, 0xa3, 0xf7, 0xc6, 0xc4, 0x47, 0x9e, 0xf4, 0x28, 0x80, 0xf4, 0x51, - 0x18, 0x20, 0x2d, 0x0d, 0x07, 0x48, 0x9e, 0x18, 0x8e, 0x90, 0x36, 0x22, 0x11, 0xd2, 0xed, 0xf3, - 0x10, 0x92, 0x3f, 0x0f, 0x82, 0x10, 0x69, 0x2b, 0x1a, 0x22, 0xdd, 0x39, 0x17, 0x22, 0xf9, 0xdd, - 0x1f, 0xc2, 0x48, 0x1b, 0x91, 0x18, 0xe9, 0xf6, 0x79, 0x18, 0xc9, 0xaf, 0x5c, 0x10, 0x24, 0xbd, - 0x1a, 0x0a, 0x92, 0xee, 0x8f, 0x03, 0x92, 0x3c, 0xa1, 0x03, 0x28, 0xe9, 0xf3, 0xe1, 0x28, 0xe9, - 0x1b, 0x17, 0xc8, 0x57, 0x18, 0x09, 0x93, 0xbe, 0x35, 0x00, 0x93, 0xc4, 0x51, 0x30, 0xc9, 0x1f, - 0xcf, 0x2e, 0x4e, 0x52, 0x46, 0xa2, 0x9a, 0xf7, 0xc6, 0x44, 0x35, 0xfe, 0xe0, 0x8b, 0x80, 0x35, - 0xd5, 0x08, 0x58, 0x73, 0x73, 0x34, 0xac, 0xf1, 0xcd, 0xb9, 0x8f, 0x6b, 0x6a, 0x51, 0xb8, 0xe6, - 0xd6, 0x39, 0xb8, 0xc6, 0xb7, 0x42, 0x01, 0x60, 0xf3, 0xb8, 0x0f, 0xd8, 0x5c, 0x3f, 0x37, 0xea, - 0x27, 0x80, 0x6c, 0xca, 0x83, 0xc8, 0xe6, 0xc6, 0x48, 0x64, 0xe3, 0x49, 0xf0, 0xa1, 0xcd, 0xe3, - 0x3e, 0x68, 0x73, 0x7d, 0x04, 0xb4, 0xf1, 0x2b, 0xc0, 0xb1, 0x8d, 0x36, 0x1a, 0xdb, 0x2c, 0x8f, - 0x8b, 0x6d, 0x3c, 0xc1, 0x91, 0xe0, 0x66, 0x2b, 0x1a, 0xdc, 0xdc, 0x19, 0xf3, 0x85, 0xfc, 0x00, - 0xba, 0xa9, 0x45, 0xa1, 0x9b, 0x5b, 0xe7, 0xa0, 0x9b, 0xe0, 0x1a, 0xe2, 0xc1, 0x9b, 0x5a, 0x14, - 0xbc, 0xb9, 0x75, 0x0e, 0xbc, 0xf1, 0x25, 0x05, 0xf0, 0x4d, 0x73, 0x18, 0xbe, 0xb9, 0x37, 0x06, - 0xbe, 0xf1, 0x9d, 0x97, 0x3e, 0x80, 0xf3, 0x69, 0x3f, 0xc0, 0x11, 0x47, 0x01, 0x1c, 0x7f, 0x46, - 0xba, 0x08, 0x67, 0x2b, 0x1a, 0xe1, 0xdc, 0x39, 0x17, 0xe1, 0x04, 0x8d, 0x64, 0x00, 0xe2, 0x6c, - 0x44, 0x42, 0x9c, 0xdb, 0xe7, 0x41, 0x1c, 0xdf, 0x48, 0x06, 0x31, 0xce, 0xa7, 0xfd, 0x18, 0x47, - 0x1c, 0x85, 0x71, 0xfc, 0xc6, 0xb9, 0x20, 0xa7, 0x16, 0x05, 0x72, 0x6e, 0x9d, 0x03, 0x72, 0xfc, - 0xce, 0x0b, 0xa0, 0x1c, 0x65, 0x24, 0xca, 0x79, 0x6f, 0x4c, 0x94, 0xd3, 0x67, 0xb8, 0xc2, 0x30, - 0xa7, 0x16, 0x05, 0x73, 0x6e, 0x9d, 0x03, 0x73, 0x02, 0x95, 0xf5, 0x71, 0xce, 0x56, 0x34, 0xce, - 0xb9, 0x73, 0x2e, 0xce, 0xe9, 0x9b, 0x4d, 0x2e, 0xd0, 0xd9, 0x88, 0x04, 0x3a, 0xb7, 0xcf, 0x03, - 0x3a, 0x7d, 0x0b, 0x1f, 0x77, 0x0e, 0x7e, 0x65, 0x7c, 0xa4, 0xf3, 0xd1, 0xc5, 0x91, 0x8e, 0xf7, - 0xcc, 0x58, 0xa0, 0xce, 0x7a, 0x2a, 0xfd, 0x76, 0xf1, 0x1d, 0xf1, 0xef, 0x4c, 0xc3, 0x54, 0xcd, - 0x8b, 0x73, 0xf1, 0x6b, 0x29, 0xbc, 0x49, 0x5e, 0x24, 0xb4, 0x46, 0x66, 0x2c, 0xb5, 0x7b, 0xe7, - 0xa7, 0xba, 0x1b, 0x4c, 0xc6, 0xc6, 0x59, 0xdf, 0xe0, 0x80, 0x32, 0xfa, 0x00, 0x72, 0x3d, 0x1b, - 0x5b, 0x72, 0xd7, 0xd2, 0x4d, 0x4b, 0x77, 0xd8, 0x29, 0x0e, 0xa1, 0x5c, 0xfc, 0xf2, 0x74, 0x71, - 0x66, 0xc7, 0xc6, 0xd6, 0x36, 0xa7, 0x4b, 0x33, 0xbd, 0xc0, 0x95, 0xfb, 0x01, 0xa9, 0xc9, 0xf1, - 0x3f, 0x20, 0xf5, 0x02, 0x8a, 0x16, 0x56, 0xb4, 0x90, 0x07, 0xc2, 0x72, 0x10, 0x45, 0x8f, 0x19, - 0x7a, 0xca, 0xca, 0x2d, 0x49, 0x73, 0x11, 0x15, 0xac, 0x30, 0x11, 0x3d, 0x80, 0x4b, 0x1d, 0xe5, - 0x98, 0xc6, 0x44, 0xca, 0xae, 0x53, 0x47, 0xe3, 0x1c, 0xd9, 0xb7, 0x99, 0x50, 0x47, 0x39, 0xa6, - 0x5f, 0xa3, 0x62, 0xb7, 0xe8, 0xa7, 0x24, 0x6e, 0x41, 0x5e, 0xd3, 0x6d, 0x47, 0x37, 0x54, 0x87, - 0xe7, 0x9e, 0x65, 0x79, 0x5b, 0x73, 0x2e, 0x95, 0x25, 0x98, 0xbd, 0x0f, 0xb3, 0x3c, 0x58, 0x3e, - 0xf0, 0x8a, 0x90, 0xe7, 0x6f, 0x65, 0x37, 0xbc, 0xb7, 0x82, 0xa8, 0x02, 0x85, 0x96, 0xe2, 0xe0, - 0x23, 0xe5, 0x44, 0x76, 0x8f, 0x62, 0x65, 0x69, 0x4a, 0xc6, 0xb7, 0xce, 0x4e, 0x17, 0x73, 0x4f, - 0xd9, 0xad, 0x81, 0x13, 0x59, 0xb9, 0x56, 0xe0, 0x86, 0x86, 0xee, 0x40, 0x41, 0xb1, 0x4f, 0x0c, - 0x95, 0xaa, 0x07, 0x1b, 0x76, 0xcf, 0xa6, 0x90, 0x22, 0x2d, 0xe5, 0x29, 0xb9, 0xe2, 0x52, 0xd1, - 0x75, 0x98, 0xe1, 0x91, 0xe4, 0xec, 0xf3, 0x36, 0x05, 0xda, 0x54, 0xfe, 0xdd, 0x04, 0xf6, 0x85, - 0x9b, 0xc7, 0xb0, 0xc0, 0x73, 0xcb, 0x1f, 0x29, 0x96, 0x26, 0x53, 0xad, 0xfb, 0xe3, 0xb3, 0x48, - 0xc5, 0x5e, 0x61, 0xb9, 0xe4, 0x49, 0x01, 0xa2, 0x6a, 0x3f, 0x13, 0xc2, 0x16, 0xcc, 0xaa, 0x6d, - 0xdd, 0x43, 0x00, 0xac, 0xe5, 0xb3, 0x43, 0xed, 0x6c, 0x85, 0x96, 0xf5, 0x5f, 0x91, 0x16, 0xd4, - 0x30, 0x01, 0x35, 0x80, 0x66, 0x7b, 0x91, 0xbb, 0x66, 0x5b, 0x57, 0x4f, 0xa8, 0xf3, 0x1f, 0xce, - 0x91, 0x3d, 0x32, 0x53, 0xfd, 0x2b, 0x45, 0x77, 0xb6, 0x29, 0xa7, 0x04, 0x47, 0xde, 0x6f, 0x96, - 0xdf, 0x76, 0x3d, 0x95, 0x9e, 0x29, 0xe6, 0xd6, 0x53, 0xe9, 0x7c, 0xb1, 0x20, 0xfe, 0x5d, 0x01, - 0x0a, 0x7d, 0x75, 0x41, 0x35, 0xb8, 0xa4, 0x79, 0x53, 0x45, 0xe6, 0x47, 0x8d, 0x74, 0xd3, 0xe0, - 0x69, 0xbf, 0xe7, 0xbe, 0x3c, 0x5d, 0x2c, 0xd0, 0xd2, 0x4f, 0xbd, 0x5b, 0xd2, 0xbc, 0xcf, 0xe1, - 0x53, 0xd1, 0x47, 0x90, 0x67, 0xee, 0xa3, 0xf7, 0xc5, 0x36, 0x1a, 0x23, 0x5e, 0x9e, 0xfd, 0xf2, - 0x74, 0x31, 0x47, 0x7d, 0x46, 0x37, 0x2d, 0xaf, 0x94, 0x6b, 0x07, 0x2f, 0xc5, 0xdf, 0x10, 0x60, - 0x26, 0x74, 0x9c, 0xe7, 0x71, 0xdf, 0x1b, 0xf4, 0xab, 0xd1, 0xb8, 0x73, 0x58, 0x40, 0x5d, 0x9a, - 0x8f, 0x73, 0x37, 0x3a, 0x71, 0x71, 0x38, 0x6e, 0xa1, 0xbb, 0x30, 0x6e, 0xa8, 0x86, 0xcb, 0xf6, - 0x71, 0xea, 0xfb, 0x3f, 0x58, 0x9c, 0x10, 0xff, 0x20, 0x05, 0xb9, 0xf0, 0xe1, 0x9d, 0x7a, 0x5f, - 0xbd, 0xa2, 0xd6, 0x85, 0x10, 0xc7, 0xf2, 0x88, 0xb4, 0x85, 0x19, 0x3f, 0xbf, 0x3e, 0xab, 0xe6, - 0xd2, 0x88, 0x38, 0x81, 0x60, 0x3d, 0x7d, 0xc6, 0x85, 0xef, 0x25, 0x3d, 0xfb, 0xba, 0x0c, 0x93, - 0x34, 0x8d, 0x0e, 0xaf, 0x5a, 0xa9, 0x7f, 0xf4, 0x10, 0x5f, 0x99, 0xdc, 0x97, 0x58, 0x31, 0x62, - 0x8f, 0x9b, 0x6f, 0x94, 0xa7, 0xce, 0x9f, 0x06, 0x17, 0xff, 0x40, 0x1e, 0xcf, 0x53, 0x38, 0x79, - 0xb1, 0x3c, 0x85, 0xe8, 0x97, 0xa0, 0xa0, 0x9a, 0xed, 0x36, 0x5b, 0xeb, 0x98, 0x45, 0x1a, 0xcc, - 0x3c, 0x42, 0x45, 0xf0, 0x6f, 0x1a, 0x2e, 0x7b, 0xdf, 0x36, 0x5c, 0x96, 0xf8, 0xb7, 0x0d, 0x03, - 0xb1, 0x9e, 0x79, 0x4f, 0x18, 0x33, 0x64, 0x7d, 0x61, 0xa7, 0xd3, 0x6f, 0x12, 0x76, 0xca, 0x42, - 0x95, 0xf9, 0xc8, 0xf9, 0x53, 0x81, 0x07, 0x86, 0x3c, 0x33, 0xcd, 0x83, 0x9e, 0x17, 0x2e, 0xba, - 0x10, 0xcc, 0x1a, 0x98, 0xfe, 0xf2, 0x74, 0x31, 0x25, 0x79, 0x69, 0x03, 0xa3, 0x2c, 0x7f, 0xe2, - 0x27, 0xb3, 0xfc, 0xd7, 0x61, 0xa6, 0x6b, 0xe1, 0x3d, 0xec, 0xa8, 0xfb, 0xb2, 0xd1, 0xeb, 0xf0, - 0x53, 0x25, 0x59, 0x97, 0xb6, 0xd5, 0xeb, 0xa0, 0x7b, 0x50, 0xf4, 0x8a, 0x70, 0x8c, 0xed, 0xa6, - 0x95, 0x72, 0xe9, 0x1c, 0x91, 0x8b, 0xff, 0x47, 0x80, 0xb9, 0x50, 0x9b, 0xf8, 0x9c, 0x58, 0x87, - 0xac, 0x6f, 0x0e, 0xec, 0x92, 0x70, 0xc1, 0xe0, 0xc9, 0x20, 0x33, 0x92, 0xe1, 0xb2, 0xfb, 0x58, - 0x9a, 0x53, 0xde, 0x17, 0x9b, 0xb8, 0xa0, 0xd8, 0x4b, 0xbe, 0x9c, 0xb5, 0xc0, 0x03, 0xbc, 0x49, - 0x92, 0x1c, 0x6b, 0x92, 0x88, 0xbf, 0x2d, 0x40, 0x91, 0x3e, 0xe0, 0x09, 0xc6, 0x5a, 0x2c, 0xd6, - 0xc9, 0x0d, 0x4a, 0x4e, 0x8c, 0x7f, 0x6e, 0x24, 0xf4, 0x1d, 0x8c, 0x64, 0xf8, 0x3b, 0x18, 0xe2, - 0x0f, 0x04, 0xc8, 0x7b, 0x35, 0x64, 0x5f, 0x89, 0x1b, 0x91, 0x9c, 0xf2, 0xcd, 0xbe, 0x84, 0xe6, - 0x66, 0xe1, 0x18, 0xeb, 0xc3, 0x75, 0xc1, 0x2c, 0x1c, 0xec, 0x0b, 0x5e, 0x7f, 0xdf, 0x1d, 0x39, - 0xa4, 0x8a, 0x15, 0x3f, 0xdb, 0xc2, 0x1b, 0x1c, 0xa1, 0x91, 0xe8, 0x07, 0x36, 0xcd, 0xf6, 0x21, - 0x4b, 0x7c, 0x32, 0x96, 0xd9, 0x42, 0x3c, 0x1c, 0x0a, 0xf8, 0x6e, 0x9c, 0xd6, 0x6c, 0xd0, 0x4f, - 0x6f, 0xb2, 0xdf, 0xb6, 0xf8, 0x24, 0xa0, 0x40, 0xda, 0xf9, 0x44, 0x4b, 0x63, 0x99, 0x52, 0x57, - 0x4b, 0x6c, 0xac, 0xfc, 0x49, 0xb0, 0x27, 0xaa, 0x87, 0x04, 0x85, 0x3d, 0x82, 0xe4, 0xa1, 0xd2, - 0x1e, 0x15, 0x06, 0x16, 0xea, 0x39, 0x89, 0x94, 0x46, 0x4f, 0x42, 0x49, 0x2a, 0x12, 0xc3, 0x11, - 0xc3, 0xa0, 0x4a, 0x43, 0xc9, 0x2c, 0x3e, 0x0c, 0x8f, 0xf5, 0x91, 0x8f, 0x0f, 0x0e, 0xfa, 0x8f, - 0x53, 0x3f, 0xfc, 0xc1, 0xa2, 0x20, 0x7e, 0x02, 0x48, 0xc2, 0x36, 0x76, 0x5e, 0xf4, 0x4c, 0xcb, - 0x4f, 0xf8, 0x71, 0xbb, 0xef, 0xc3, 0x20, 0x93, 0xe5, 0xec, 0x59, 0x54, 0x4a, 0x9f, 0x4b, 0x30, - 0x17, 0xe2, 0x66, 0xc6, 0x42, 0xfc, 0x10, 0xae, 0x3e, 0x35, 0x6d, 0x5b, 0xef, 0x12, 0xe8, 0x49, - 0x67, 0x25, 0x59, 0x1a, 0x3c, 0xf3, 0x98, 0xee, 0xd2, 0x4d, 0x08, 0x83, 0x99, 0x91, 0x8c, 0xe4, - 0x5d, 0x8b, 0x7f, 0x28, 0xc0, 0x95, 0x41, 0x4e, 0xa6, 0xe5, 0xa8, 0x13, 0x7f, 0xd3, 0xaa, 0xe9, - 0xe7, 0x77, 0x3b, 0x7f, 0xb4, 0xba, 0xc5, 0x89, 0x8b, 0xc9, 0x9f, 0x29, 0x77, 0x14, 0x6a, 0x3e, - 0xf8, 0xe9, 0xe3, 0x3c, 0x27, 0x6f, 0x32, 0xaa, 0x6f, 0x49, 0x52, 0xe3, 0x59, 0x92, 0x26, 0x14, - 0xd6, 0x4d, 0xdd, 0x20, 0x9e, 0xac, 0xdb, 0xde, 0x55, 0xc8, 0xef, 0xea, 0x86, 0x62, 0x9d, 0xc8, - 0xee, 0xa1, 0x6e, 0x36, 0x50, 0x16, 0xa2, 0x2a, 0xcb, 0x4a, 0x48, 0x39, 0xc6, 0xc1, 0x2f, 0xc5, - 0x1f, 0x09, 0x50, 0xf4, 0xc5, 0x72, 0x8b, 0xfc, 0x2e, 0x80, 0xda, 0xee, 0xd9, 0x0e, 0xb6, 0xdc, - 0x5e, 0x9a, 0x61, 0xd1, 0xdb, 0x15, 0x46, 0xad, 0xaf, 0x49, 0x19, 0x5e, 0xa0, 0xae, 0xa1, 0x1b, - 0xe1, 0xe4, 0x08, 0x93, 0x65, 0x38, 0x1b, 0x48, 0x89, 0x40, 0xba, 0xdd, 0x76, 0x4c, 0xcb, 0xc3, - 0x4c, 0xbc, 0xdb, 0xdd, 0x54, 0xea, 0xf4, 0x4c, 0x2f, 0x29, 0xb7, 0x0a, 0x79, 0xb2, 0xdc, 0x1f, - 0x62, 0xaf, 0x49, 0xa9, 0xf3, 0x9b, 0xc4, 0x38, 0xdc, 0x26, 0xfd, 0x1e, 0x71, 0x52, 0x59, 0x6f, - 0x78, 0x3d, 0x3c, 0xc2, 0xa2, 0xfd, 0x4c, 0x30, 0x25, 0xe1, 0x78, 0x49, 0x1b, 0xa9, 0x27, 0xf2, - 0x2d, 0x48, 0xbb, 0x9f, 0x2e, 0xe6, 0x13, 0xe4, 0xea, 0x32, 0xfb, 0xb6, 0xf1, 0xb2, 0xfb, 0x6d, - 0xe3, 0xe5, 0x35, 0x5e, 0x80, 0x99, 0xf1, 0xef, 0xff, 0xf7, 0x45, 0x41, 0xf2, 0x98, 0xee, 0x37, - 0xc8, 0x08, 0x1f, 0x58, 0x85, 0x51, 0x1e, 0x20, 0xf0, 0x0d, 0x19, 0xfe, 0x35, 0xdb, 0xd5, 0x35, - 0x79, 0x67, 0xab, 0xf2, 0x7c, 0x73, 0xb3, 0xde, 0x6c, 0x56, 0xd7, 0x8a, 0x02, 0x2a, 0xc2, 0x4c, - 0xe8, 0x0b, 0x34, 0x09, 0xf6, 0x7d, 0xdb, 0xfb, 0x3f, 0x03, 0xe0, 0x7f, 0xcc, 0x8a, 0xc8, 0xda, - 0xa8, 0x7e, 0x26, 0xbf, 0x5c, 0x7d, 0xb6, 0x53, 0x6d, 0x14, 0x27, 0x10, 0x82, 0x7c, 0x79, 0xb5, - 0x59, 0xa9, 0xc9, 0x52, 0xb5, 0xb1, 0xfd, 0x7c, 0xab, 0x51, 0x75, 0xbf, 0x8b, 0x7b, 0x7f, 0x0d, - 0x66, 0x82, 0xe9, 0x6e, 0xd0, 0x1c, 0x14, 0x2a, 0xb5, 0x6a, 0x65, 0x43, 0x7e, 0x59, 0x5f, 0x95, - 0x5f, 0xec, 0x54, 0x77, 0xaa, 0xc5, 0x09, 0x5a, 0x35, 0x4a, 0x7c, 0xb2, 0xf3, 0xec, 0x59, 0x51, - 0x40, 0x05, 0xc8, 0xb2, 0x6b, 0xfa, 0xb5, 0x9a, 0x62, 0xe2, 0xfe, 0x26, 0x64, 0x03, 0x69, 0x6d, - 0xc9, 0xe3, 0xb6, 0x77, 0x1a, 0x35, 0xb9, 0x59, 0xdf, 0xac, 0x36, 0x9a, 0xab, 0x9b, 0xdb, 0x4c, - 0x06, 0xa5, 0xad, 0x96, 0x9f, 0x4b, 0xcd, 0xa2, 0xe0, 0x5d, 0x37, 0x9f, 0xef, 0x54, 0x6a, 0x6e, - 0x33, 0xc4, 0x54, 0x3a, 0x59, 0x4c, 0xde, 0xff, 0x1b, 0x02, 0x5c, 0x19, 0x92, 0xfa, 0x05, 0x65, - 0x61, 0x7a, 0xc7, 0xa0, 0x39, 0x3e, 0x8b, 0x13, 0x28, 0x17, 0xc8, 0xfe, 0x52, 0x14, 0x50, 0x9a, - 0xe5, 0xdf, 0x28, 0x26, 0xd0, 0x14, 0x24, 0x1a, 0x8f, 0x8a, 0x49, 0x52, 0xd3, 0x40, 0xf2, 0x94, - 0x62, 0x0a, 0x65, 0x78, 0xda, 0x86, 0xe2, 0x24, 0x9a, 0xf1, 0xb3, 0x27, 0x14, 0xa7, 0x88, 0x28, - 0x2f, 0x0b, 0x41, 0x71, 0xfa, 0xfe, 0x75, 0x08, 0x9c, 0xf4, 0x46, 0x00, 0x53, 0xcf, 0x14, 0x07, - 0xdb, 0x4e, 0x71, 0x02, 0x4d, 0x43, 0x72, 0xb5, 0xdd, 0x2e, 0x0a, 0x0f, 0xff, 0x6d, 0x0a, 0xd2, - 0xee, 0x57, 0x59, 0xd0, 0x33, 0x98, 0x64, 0x5b, 0xcf, 0x8b, 0xc3, 0x3d, 0x7b, 0x3a, 0x79, 0x17, - 0x96, 0xce, 0x73, 0xfd, 0xc5, 0x09, 0xf4, 0xd7, 0x20, 0x1b, 0xf0, 0x98, 0xd0, 0xd0, 0xed, 0xb3, - 0x90, 0x97, 0xb8, 0x70, 0xfb, 0xbc, 0x62, 0x9e, 0xfc, 0x57, 0x90, 0xf1, 0x2c, 0x38, 0xba, 0x31, - 0xca, 0xbe, 0xbb, 0xb2, 0x47, 0x2f, 0x02, 0x64, 0xae, 0x89, 0x13, 0xef, 0x0b, 0xc8, 0x02, 0x34, - 0x68, 0x6c, 0x51, 0x54, 0x44, 0xc2, 0x50, 0x6b, 0xbe, 0x70, 0x7f, 0xac, 0xd2, 0xfe, 0x33, 0x89, - 0xb2, 0xfc, 0x15, 0x23, 0x5a, 0x59, 0x03, 0xeb, 0x51, 0xb4, 0xb2, 0x22, 0x16, 0x9e, 0x09, 0xf4, - 0x02, 0x52, 0xc4, 0x52, 0xa2, 0x28, 0x1f, 0xb2, 0xcf, 0x32, 0x2f, 0xdc, 0x18, 0x59, 0xc6, 0x15, - 0x59, 0xbe, 0xf7, 0xc3, 0xbf, 0xb8, 0x36, 0xf1, 0xc3, 0xb3, 0x6b, 0xc2, 0x8f, 0xce, 0xae, 0x09, - 0x7f, 0x76, 0x76, 0x4d, 0xf8, 0xf3, 0xb3, 0x6b, 0xc2, 0xaf, 0xff, 0xf8, 0xda, 0xc4, 0x8f, 0x7e, - 0x7c, 0x6d, 0xe2, 0xcf, 0x7e, 0x7c, 0x6d, 0xe2, 0xf3, 0x69, 0xce, 0xbd, 0x3b, 0x45, 0x8d, 0xca, - 0xa3, 0xff, 0x1f, 0x00, 0x00, 0xff, 0xff, 0x9c, 0xe1, 0xc2, 0x1d, 0x6b, 0x7e, 0x00, 0x00, +func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_9819b93932ee2bad) } + +var fileDescriptor_api_9819b93932ee2bad = []byte{ + // 8155 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x7d, 0x5f, 0x6c, 0x23, 0x49, + 0x7a, 0x9f, 0x9a, 0xa4, 0x24, 0xf2, 0xa3, 0xf8, 0x47, 0x25, 0xcd, 0x0c, 0x47, 0xbb, 0x3b, 0xd2, + 0xf4, 0xfc, 0x9f, 0xbb, 0x95, 0x76, 0x66, 0x6e, 0xbd, 0xeb, 0x9d, 0xf5, 0x9e, 0x45, 0x8a, 0x33, + 0xa4, 0x34, 0xd2, 0x68, 0x9a, 0xd4, 0x4c, 0x76, 0x7d, 0x4e, 0x5f, 0xab, 0xbb, 0x44, 0xf5, 0x89, + 0xec, 0xe6, 0x74, 0x37, 0xf5, 0x67, 0x81, 0x00, 0x71, 0x6c, 0x38, 0x06, 0x02, 0x1c, 0xfc, 0xe0, + 0x20, 0x67, 0x38, 0x89, 0xcf, 0x71, 0x1c, 0x3f, 0xe4, 0x21, 0x01, 0x12, 0x24, 0x41, 0x90, 0xd8, + 0x2f, 0x06, 0x72, 0x08, 0x8c, 0xe4, 0xfc, 0x14, 0x23, 0x40, 0x14, 0x5b, 0x97, 0x97, 0x20, 0xc1, + 0x21, 0xc8, 0x8b, 0x81, 0x7d, 0x08, 0x82, 0xfa, 0xd3, 0xff, 0xc8, 0x26, 0x45, 0xcd, 0xf6, 0x26, + 0x0b, 0xf8, 0x45, 0x62, 0x7f, 0x55, 0xdf, 0xd7, 0x55, 0x5f, 0x55, 0x7d, 0xf5, 0xfd, 0xaa, 0xbe, + 0xaa, 0x86, 0x59, 0xcb, 0x54, 0xd4, 0xfd, 0xee, 0xee, 0x8a, 0xd2, 0xd5, 0x97, 0xbb, 0x96, 0xe9, + 0x98, 0x68, 0x56, 0x35, 0xd5, 0x03, 0x4a, 0x5e, 0xe6, 0x89, 0x0b, 0xf7, 0x0f, 0x0e, 0x57, 0x0e, + 0x0e, 0x6d, 0x6c, 0x1d, 0x62, 0x6b, 0x45, 0x35, 0x0d, 0xb5, 0x67, 0x59, 0xd8, 0x50, 0x4f, 0x56, + 0xda, 0xa6, 0x7a, 0x40, 0xff, 0xe8, 0x46, 0x8b, 0xb1, 0x2f, 0x20, 0x57, 0xa2, 0xa6, 0x38, 0x0a, + 0xa7, 0xcd, 0xbb, 0x34, 0x6c, 0x59, 0xa6, 0x65, 0x73, 0xea, 0x65, 0x97, 0xda, 0xc1, 0x8e, 0x12, + 0xc8, 0xfd, 0x96, 0xed, 0x98, 0x96, 0xd2, 0xc2, 0x2b, 0xd8, 0x68, 0xe9, 0x06, 0x26, 0x19, 0x0e, + 0x55, 0x95, 0x27, 0xbe, 0x1d, 0x99, 0xf8, 0x88, 0xa7, 0x96, 0x7a, 0x8e, 0xde, 0x5e, 0xd9, 0x6f, + 0xab, 0x2b, 0x8e, 0xde, 0xc1, 0xb6, 0xa3, 0x74, 0xba, 0x3c, 0xe5, 0x3e, 0x4d, 0x71, 0x2c, 0x45, + 0xd5, 0x8d, 0x96, 0xfb, 0xbf, 0xbb, 0xbb, 0x62, 0x61, 0xd5, 0xb4, 0x34, 0xac, 0xc9, 0x76, 0x57, + 0x31, 0xdc, 0xe2, 0xb6, 0xcc, 0x96, 0x49, 0x7f, 0xae, 0x90, 0x5f, 0x9c, 0x7a, 0xad, 0x65, 0x9a, + 0xad, 0x36, 0x5e, 0xa1, 0x4f, 0xbb, 0xbd, 0xbd, 0x15, 0xad, 0x67, 0x29, 0x8e, 0x6e, 0x72, 0x2e, + 0xf1, 0x5f, 0x08, 0x90, 0x93, 0xf0, 0xeb, 0x1e, 0xb6, 0x9d, 0x1a, 0x56, 0x34, 0x6c, 0xa1, 0xab, + 0x90, 0x3c, 0xc0, 0x27, 0xa5, 0xe4, 0x92, 0x70, 0x77, 0xa6, 0x3c, 0xfd, 0xc5, 0xe9, 0x62, 0x72, + 0x03, 0x9f, 0x48, 0x84, 0x86, 0x96, 0x60, 0x1a, 0x1b, 0x9a, 0x4c, 0x92, 0x53, 0xe1, 0xe4, 0x29, + 0x6c, 0x68, 0x1b, 0xf8, 0x04, 0x7d, 0x07, 0xd2, 0x36, 0x91, 0x66, 0xa8, 0xb8, 0x34, 0xb9, 0x24, + 0xdc, 0x9d, 0x2c, 0xff, 0xfc, 0x17, 0xa7, 0x8b, 0x1f, 0xb7, 0x74, 0x67, 0xbf, 0xb7, 0xbb, 0xac, + 0x9a, 0x9d, 0x15, 0xaf, 0x9d, 0xb4, 0x5d, 0xff, 0xf7, 0x4a, 0xf7, 0xa0, 0xb5, 0xd2, 0xaf, 0xa3, + 0xe5, 0xe6, 0xb1, 0xd1, 0xc0, 0xaf, 0x25, 0x4f, 0xe2, 0x7a, 0x2a, 0x2d, 0x14, 0x13, 0xeb, 0xa9, + 0x74, 0xa2, 0x98, 0x14, 0x7f, 0x37, 0x09, 0x79, 0x09, 0xdb, 0x5d, 0xd3, 0xb0, 0x31, 0x2f, 0xf9, + 0x7b, 0x90, 0x74, 0x8e, 0x0d, 0x5a, 0xf2, 0xec, 0xc3, 0x6b, 0xcb, 0x03, 0x3d, 0x62, 0xb9, 0x69, + 0x29, 0x86, 0xad, 0xa8, 0xa4, 0xfa, 0x12, 0xc9, 0x8a, 0x3e, 0x84, 0xac, 0x85, 0xed, 0x5e, 0x07, + 0x53, 0x45, 0xd2, 0x4a, 0x65, 0x1f, 0x5e, 0x89, 0xe0, 0x6c, 0x74, 0x15, 0x43, 0x02, 0x96, 0x97, + 0xfc, 0x46, 0x57, 0x21, 0x6d, 0xf4, 0x3a, 0x44, 0x15, 0x36, 0xad, 0x68, 0x52, 0x9a, 0x36, 0x7a, + 0x9d, 0x0d, 0x7c, 0x62, 0xa3, 0xbf, 0x02, 0x97, 0x35, 0xdc, 0xb5, 0xb0, 0xaa, 0x38, 0x58, 0x93, + 0x2d, 0xc5, 0x68, 0x61, 0x59, 0x37, 0xf6, 0x4c, 0xbb, 0x34, 0xb5, 0x94, 0xbc, 0x9b, 0x7d, 0xf8, + 0x76, 0x84, 0x7c, 0x89, 0xe4, 0xaa, 0x1b, 0x7b, 0x66, 0x39, 0xf5, 0xa3, 0xd3, 0xc5, 0x09, 0x69, + 0xde, 0x97, 0xe0, 0x25, 0xd9, 0xa8, 0x01, 0x39, 0x5e, 0x5c, 0x0b, 0x2b, 0xb6, 0x69, 0x94, 0xa6, + 0x97, 0x84, 0xbb, 0xf9, 0x87, 0xcb, 0x51, 0x02, 0x43, 0xaa, 0x21, 0x8f, 0xbd, 0x0e, 0x96, 0x28, + 0x97, 0x34, 0x63, 0x05, 0x9e, 0xd0, 0x5b, 0x90, 0x21, 0x35, 0xd9, 0x3d, 0x71, 0xb0, 0x5d, 0x4a, + 0xd3, 0xaa, 0x90, 0xaa, 0x95, 0xc9, 0xb3, 0xf8, 0x09, 0xcc, 0x04, 0x59, 0x11, 0x82, 0xbc, 0x54, + 0x6d, 0xec, 0x6c, 0x56, 0xe5, 0x9d, 0xad, 0x8d, 0xad, 0xe7, 0xaf, 0xb6, 0x8a, 0x13, 0x68, 0x1e, + 0x8a, 0x9c, 0xb6, 0x51, 0xfd, 0x54, 0x7e, 0x56, 0xdf, 0xac, 0x37, 0x8b, 0xc2, 0x42, 0xea, 0xd7, + 0x7e, 0xf7, 0xda, 0x84, 0xb8, 0x0d, 0xf0, 0x14, 0x3b, 0xbc, 0x83, 0xa1, 0x32, 0x4c, 0xed, 0xd3, + 0xf2, 0x94, 0x04, 0xaa, 0xe9, 0xa5, 0xc8, 0x82, 0x07, 0x3a, 0x63, 0x39, 0x4d, 0xb4, 0xf1, 0xe3, + 0xd3, 0x45, 0x41, 0xe2, 0x9c, 0xe2, 0x1f, 0x0a, 0x90, 0xa5, 0x22, 0x59, 0xfd, 0x50, 0xa5, 0x4f, + 0xe6, 0xf5, 0x73, 0x95, 0x31, 0x28, 0x14, 0x2d, 0xc3, 0xe4, 0xa1, 0xd2, 0xee, 0xe1, 0x52, 0x82, + 0xca, 0x28, 0x45, 0xc8, 0x78, 0x49, 0xd2, 0x25, 0x96, 0x0d, 0x3d, 0x86, 0x19, 0xdd, 0x70, 0xb0, + 0xe1, 0xc8, 0x8c, 0x2d, 0x79, 0x0e, 0x5b, 0x96, 0xe5, 0xa6, 0x0f, 0xe2, 0x3f, 0x17, 0x00, 0xb6, + 0x7b, 0x71, 0x2a, 0x05, 0x7d, 0x6b, 0xcc, 0xf2, 0xf3, 0xde, 0xc5, 0x6b, 0x71, 0x19, 0xa6, 0x74, + 0xa3, 0xad, 0x1b, 0xac, 0xfc, 0x69, 0x89, 0x3f, 0xa1, 0x79, 0x98, 0xdc, 0x6d, 0xeb, 0x86, 0x46, + 0xc7, 0x43, 0x5a, 0x62, 0x0f, 0xa2, 0x04, 0x59, 0x5a, 0xea, 0x18, 0xf5, 0x2e, 0x9e, 0x26, 0xe0, + 0x52, 0xc5, 0x34, 0x34, 0x9d, 0x0c, 0x49, 0xa5, 0xfd, 0xb5, 0xd0, 0xca, 0x3a, 0x04, 0x06, 0x9f, + 0x8c, 0x8f, 0xbb, 0x63, 0xb6, 0x31, 0xf2, 0xb9, 0xaa, 0xc7, 0x5d, 0x4a, 0x8b, 0xd6, 0x24, 0xfa, + 0x16, 0x5c, 0x51, 0xda, 0x6d, 0xf3, 0x48, 0xd6, 0xf7, 0x64, 0xcd, 0xc4, 0xb6, 0x6c, 0x98, 0x8e, + 0x8c, 0x8f, 0x75, 0xdb, 0xa1, 0xa6, 0x24, 0x2d, 0xcd, 0xd1, 0xe4, 0xfa, 0xde, 0x9a, 0x89, 0xed, + 0x2d, 0xd3, 0xa9, 0x92, 0x24, 0x32, 0x4e, 0x49, 0x61, 0xd8, 0x38, 0x9d, 0x22, 0xe6, 0x57, 0x4a, + 0xe3, 0xe3, 0x2e, 0x1d, 0xa7, 0x81, 0xa6, 0x9c, 0x0e, 0x36, 0xa5, 0xf8, 0x8b, 0x70, 0xb9, 0x5f, + 0xbf, 0x71, 0xb6, 0xdf, 0x1f, 0x0b, 0x90, 0xaf, 0x1b, 0xba, 0xf3, 0xb5, 0x68, 0x38, 0x4f, 0xd9, + 0xc9, 0xa0, 0xb2, 0xef, 0x43, 0x71, 0x4f, 0xd1, 0xdb, 0xcf, 0x8d, 0xa6, 0xd9, 0xd9, 0xb5, 0x1d, + 0xd3, 0xc0, 0x36, 0x6f, 0x8d, 0x01, 0xba, 0xf8, 0x12, 0x0a, 0x5e, 0x6d, 0xe2, 0x54, 0x93, 0x03, + 0xc5, 0xba, 0xa1, 0x5a, 0xb8, 0x83, 0x8d, 0x58, 0xf5, 0xf4, 0x36, 0x64, 0x74, 0x57, 0x2e, 0xd5, + 0x55, 0x52, 0xf2, 0x09, 0x62, 0x0f, 0x66, 0x03, 0x6f, 0x8d, 0xd3, 0x5c, 0x92, 0x29, 0x03, 0x1f, + 0xc9, 0x7e, 0x1b, 0x91, 0x29, 0x03, 0x1f, 0x31, 0xf3, 0xd6, 0x80, 0xdc, 0x1a, 0x6e, 0x63, 0x07, + 0xc7, 0x69, 0xf5, 0x77, 0x20, 0xef, 0x0a, 0x8d, 0xb3, 0x61, 0xfe, 0x8e, 0x00, 0x88, 0xcb, 0x25, + 0xb3, 0x6c, 0x9c, 0x6d, 0xb3, 0x48, 0x5c, 0x0b, 0xa7, 0x67, 0x19, 0xcc, 0x47, 0x60, 0x7d, 0x12, + 0x18, 0x89, 0xba, 0x09, 0xfe, 0x90, 0x4d, 0x05, 0x87, 0x2c, 0x77, 0x6f, 0x8e, 0x60, 0x2e, 0x54, + 0xb0, 0x78, 0x9b, 0x2f, 0x45, 0xcb, 0x94, 0x58, 0x4a, 0x06, 0x7d, 0x38, 0x4a, 0x14, 0x7f, 0x20, + 0xc0, 0x6c, 0xa5, 0x8d, 0x15, 0x2b, 0x76, 0x8d, 0x7c, 0x1b, 0xd2, 0x1a, 0x56, 0x34, 0x5a, 0x65, + 0x36, 0xb0, 0xdf, 0x09, 0x48, 0x21, 0x9e, 0xee, 0xf2, 0x7e, 0x5b, 0x5d, 0x6e, 0xba, 0x3e, 0x30, + 0x1f, 0xdd, 0x1e, 0x93, 0xf8, 0x29, 0xa0, 0x60, 0xc9, 0xe2, 0xec, 0x08, 0x7f, 0x5f, 0x00, 0x24, + 0xe1, 0x43, 0x6c, 0x39, 0xb1, 0x57, 0x7b, 0x0d, 0xb2, 0x8e, 0x62, 0xb5, 0xb0, 0x23, 0x13, 0xef, + 0xfe, 0x22, 0x35, 0x07, 0xc6, 0x47, 0xc8, 0xe2, 0x67, 0x30, 0x17, 0x2a, 0x5f, 0x9c, 0x95, 0xff, + 0x5f, 0x02, 0x64, 0x1b, 0xaa, 0x62, 0xc4, 0x59, 0xeb, 0x4f, 0x20, 0x6b, 0xab, 0x8a, 0x21, 0xef, + 0x99, 0x56, 0x47, 0x71, 0x68, 0x17, 0xcf, 0x87, 0x6a, 0xed, 0x79, 0xd6, 0xaa, 0x62, 0x3c, 0xa1, + 0x99, 0x24, 0xb0, 0xbd, 0xdf, 0xe8, 0x05, 0x64, 0x0f, 0xf0, 0x89, 0xcc, 0x51, 0x1a, 0x9d, 0x17, + 0xf3, 0x0f, 0xdf, 0x0b, 0xf0, 0x1f, 0x1c, 0x2e, 0xbb, 0xe0, 0x6e, 0x39, 0x00, 0xee, 0x96, 0x09, + 0xc7, 0x72, 0xc3, 0xb1, 0xb0, 0xd1, 0x72, 0xf6, 0x25, 0x38, 0xc0, 0x27, 0xcf, 0x98, 0x0c, 0x36, + 0xb0, 0xd6, 0x53, 0xe9, 0x64, 0x31, 0x25, 0xfe, 0x85, 0x00, 0x33, 0xac, 0xca, 0x71, 0x0e, 0xac, + 0xf7, 0x21, 0x65, 0x99, 0x47, 0x6c, 0x60, 0x65, 0x1f, 0xbe, 0x15, 0x21, 0x62, 0x03, 0x9f, 0x04, + 0x67, 0x2e, 0x9a, 0x1d, 0x95, 0x81, 0xfb, 0x87, 0x32, 0xe5, 0x4e, 0x8e, 0xcb, 0x0d, 0x8c, 0x4b, + 0x22, 0x32, 0xee, 0x40, 0x61, 0x57, 0x71, 0xd4, 0x7d, 0xd9, 0xe2, 0x85, 0x24, 0xb3, 0x5c, 0xf2, + 0xee, 0x8c, 0x94, 0xa7, 0x64, 0xb7, 0xe8, 0x36, 0xa9, 0x39, 0xeb, 0xe9, 0x36, 0xfe, 0x4b, 0xd6, + 0xe6, 0xff, 0x47, 0xe0, 0x63, 0xc8, 0xad, 0xf9, 0x5f, 0xb6, 0xa6, 0xff, 0xcd, 0x04, 0x5c, 0xa9, + 0xec, 0x63, 0xf5, 0xa0, 0x62, 0x1a, 0xb6, 0x6e, 0x3b, 0x44, 0x77, 0x71, 0xb6, 0xff, 0x5b, 0x90, + 0x39, 0xd2, 0x9d, 0x7d, 0x59, 0xd3, 0xf7, 0xf6, 0xa8, 0x9d, 0x4b, 0x4b, 0x69, 0x42, 0x58, 0xd3, + 0xf7, 0xf6, 0xd0, 0x23, 0x48, 0x75, 0x4c, 0x8d, 0xb9, 0xd1, 0xf9, 0x87, 0x8b, 0x11, 0xe2, 0x69, + 0xd1, 0xec, 0x5e, 0x67, 0xd3, 0xd4, 0xb0, 0x44, 0x33, 0xa3, 0x6b, 0x00, 0x2a, 0xa1, 0x76, 0x4d, + 0xdd, 0x70, 0xf8, 0x3c, 0x19, 0xa0, 0xa0, 0x1a, 0x64, 0x1c, 0x6c, 0x75, 0x74, 0x43, 0x71, 0x70, + 0x69, 0x92, 0x2a, 0xef, 0x66, 0x64, 0xc1, 0xbb, 0x6d, 0x5d, 0x55, 0xd6, 0xb0, 0xad, 0x5a, 0x7a, + 0xd7, 0x31, 0x2d, 0xae, 0x45, 0x9f, 0x59, 0xfc, 0x7e, 0x0a, 0x4a, 0x83, 0xba, 0x89, 0xb3, 0x87, + 0x6c, 0xc3, 0x14, 0xc1, 0xdd, 0x6d, 0x87, 0xf7, 0x91, 0x87, 0xc3, 0x54, 0x10, 0x51, 0x02, 0x8a, + 0xdf, 0xdb, 0x0e, 0x2f, 0x36, 0x97, 0xb3, 0xf0, 0x6f, 0x04, 0x98, 0x62, 0x09, 0xe8, 0x01, 0xa4, + 0xf9, 0x42, 0x83, 0x46, 0xcb, 0x98, 0x2c, 0x5f, 0x3e, 0x3b, 0x5d, 0x9c, 0x66, 0x6b, 0x07, 0x6b, + 0x5f, 0xf8, 0x3f, 0xa5, 0x69, 0x9a, 0xaf, 0xae, 0x91, 0xd6, 0xb2, 0x1d, 0xc5, 0x72, 0xe8, 0x72, + 0x4e, 0x82, 0xe1, 0x09, 0x4a, 0xd8, 0xc0, 0x27, 0x68, 0x1d, 0xa6, 0x6c, 0x47, 0x71, 0x7a, 0x36, + 0x6f, 0xaf, 0x0b, 0x15, 0xb6, 0x41, 0x39, 0x25, 0x2e, 0x81, 0x38, 0x3a, 0x1a, 0x76, 0x14, 0xbd, + 0x4d, 0x1b, 0x30, 0x23, 0xf1, 0x27, 0xf1, 0xb7, 0x04, 0x98, 0x62, 0x59, 0xd1, 0x15, 0x98, 0x93, + 0x56, 0xb7, 0x9e, 0x56, 0xe5, 0xfa, 0xd6, 0x5a, 0xb5, 0x59, 0x95, 0x36, 0xeb, 0x5b, 0xab, 0xcd, + 0x6a, 0x71, 0x02, 0x5d, 0x06, 0xe4, 0x26, 0x54, 0x9e, 0x6f, 0x35, 0xea, 0x8d, 0x66, 0x75, 0xab, + 0x59, 0x14, 0xe8, 0x9a, 0x03, 0xa5, 0x07, 0xa8, 0x09, 0x74, 0x13, 0x96, 0xfa, 0xa9, 0x72, 0xa3, + 0xb9, 0xda, 0x6c, 0xc8, 0xd5, 0x46, 0xb3, 0xbe, 0xb9, 0xda, 0xac, 0xae, 0x15, 0x93, 0x23, 0x72, + 0x91, 0x97, 0x48, 0x52, 0xb5, 0xd2, 0x2c, 0xa6, 0x44, 0x07, 0x2e, 0x49, 0x58, 0x35, 0x3b, 0xdd, + 0x9e, 0x83, 0x49, 0x29, 0xed, 0x38, 0x47, 0xca, 0x15, 0x98, 0xd6, 0xac, 0x13, 0xd9, 0xea, 0x19, + 0x7c, 0x9c, 0x4c, 0x69, 0xd6, 0x89, 0xd4, 0x33, 0xc4, 0x7f, 0x2a, 0xc0, 0xe5, 0xfe, 0xd7, 0xc6, + 0xd9, 0x09, 0x5f, 0x40, 0x56, 0xd1, 0x34, 0xac, 0xc9, 0x1a, 0x6e, 0x3b, 0x0a, 0x77, 0x46, 0xee, + 0x07, 0x24, 0xf1, 0x45, 0xb8, 0x65, 0x6f, 0x11, 0x6e, 0xf3, 0x65, 0xa5, 0x42, 0x0b, 0xb2, 0x46, + 0x38, 0x5c, 0xf3, 0x43, 0x85, 0x50, 0x8a, 0xf8, 0x3f, 0x53, 0x90, 0xab, 0x1a, 0x5a, 0xf3, 0x38, + 0xd6, 0xb9, 0xe4, 0x32, 0x4c, 0xa9, 0x66, 0xa7, 0xa3, 0x3b, 0xae, 0x82, 0xd8, 0x13, 0xfa, 0xd9, + 0x80, 0x13, 0x99, 0x1c, 0xc3, 0x95, 0xf2, 0xdd, 0x47, 0xf4, 0x5d, 0xb8, 0x42, 0xac, 0xa6, 0x65, + 0x28, 0x6d, 0x99, 0x49, 0x93, 0x1d, 0x4b, 0x6f, 0xb5, 0xb0, 0xc5, 0x17, 0xfe, 0xee, 0x46, 0x94, + 0xb3, 0xce, 0x39, 0x2a, 0x94, 0xa1, 0xc9, 0xf2, 0x4b, 0x97, 0xf4, 0x28, 0x32, 0xfa, 0x18, 0x80, + 0x4c, 0x45, 0x74, 0x31, 0xd1, 0xe6, 0xf6, 0x68, 0xd8, 0x6a, 0xa2, 0x6b, 0x82, 0x08, 0x03, 0x79, + 0xb6, 0xd1, 0x0a, 0x41, 0x0c, 0xaf, 0x7b, 0xba, 0x85, 0xe5, 0x07, 0x5d, 0x95, 0x42, 0xfc, 0x74, + 0x39, 0x7f, 0x76, 0xba, 0x08, 0x12, 0x23, 0x3f, 0xd8, 0xae, 0x10, 0x04, 0xc1, 0x7e, 0x77, 0x55, + 0xf4, 0x0a, 0xee, 0x05, 0x56, 0x2a, 0xc8, 0xcc, 0xcb, 0xab, 0xa5, 0x38, 0xf2, 0xbe, 0xde, 0xda, + 0xc7, 0x96, 0xec, 0x2d, 0x28, 0xd3, 0x95, 0xbd, 0xb4, 0x74, 0xd3, 0x67, 0xa8, 0x28, 0x06, 0x2b, + 0xfd, 0xaa, 0x53, 0xa3, 0x99, 0x3d, 0x9d, 0x11, 0xe5, 0x77, 0x4d, 0xdd, 0x36, 0x8d, 0x52, 0x86, + 0x29, 0x9f, 0x3d, 0xa1, 0x7b, 0x50, 0x74, 0x8e, 0x0d, 0x79, 0x1f, 0x2b, 0x96, 0xb3, 0x8b, 0x15, + 0x87, 0xcc, 0xd2, 0x40, 0x73, 0x14, 0x9c, 0x63, 0xa3, 0x16, 0x20, 0xa3, 0x17, 0x50, 0xd4, 0x0d, + 0x79, 0xaf, 0xad, 0xb7, 0xf6, 0x1d, 0xf9, 0xc8, 0xd2, 0x1d, 0x6c, 0x97, 0x66, 0xa9, 0x42, 0xa2, + 0xfa, 0x6d, 0x83, 0xaf, 0xf0, 0x6a, 0xaf, 0x48, 0x4e, 0xae, 0x9a, 0xbc, 0x6e, 0x3c, 0xa1, 0xfc, + 0x94, 0x68, 0xaf, 0xa7, 0xd2, 0xd3, 0xc5, 0xb4, 0xf8, 0x5f, 0x04, 0xc8, 0xbb, 0xdd, 0x2d, 0xce, + 0x91, 0x71, 0x17, 0x8a, 0xa6, 0x81, 0xe5, 0xee, 0xbe, 0x62, 0x63, 0xae, 0x47, 0x3e, 0xe1, 0xe4, + 0x4d, 0x03, 0x6f, 0x13, 0x32, 0x53, 0x17, 0xda, 0x86, 0x59, 0xdb, 0x51, 0x5a, 0xba, 0xd1, 0x0a, + 0xa8, 0x77, 0x72, 0x7c, 0xb7, 0xbe, 0xc8, 0xb9, 0x3d, 0x7a, 0xc8, 0x4b, 0xf9, 0x13, 0x01, 0x66, + 0x57, 0xb5, 0x8e, 0x6e, 0x34, 0xba, 0x6d, 0x3d, 0xd6, 0xd5, 0x82, 0x9b, 0x90, 0xb1, 0x89, 0x4c, + 0xdf, 0xe0, 0xfb, 0xd8, 0x2f, 0x4d, 0x53, 0x88, 0xe5, 0x7f, 0x06, 0x05, 0x7c, 0xdc, 0xd5, 0xd9, + 0x26, 0x01, 0x83, 0x2c, 0xa9, 0xf1, 0xeb, 0x96, 0xf7, 0x79, 0x49, 0x12, 0xaf, 0xd3, 0xa7, 0x80, + 0x82, 0x55, 0x8a, 0x13, 0xbb, 0x7c, 0x0a, 0x73, 0x54, 0xf4, 0x8e, 0x61, 0xc7, 0xac, 0x2f, 0xf1, + 0x17, 0x60, 0x3e, 0x2c, 0x3a, 0xce, 0x72, 0xbf, 0xe2, 0xad, 0xbc, 0x89, 0xad, 0x58, 0xe1, 0xa6, + 0xa7, 0x6b, 0x2e, 0x38, 0xce, 0x32, 0xff, 0x8a, 0x00, 0x57, 0xa9, 0x6c, 0xba, 0x8f, 0xb2, 0x87, + 0xad, 0x67, 0x58, 0xb1, 0x63, 0xc5, 0xca, 0x37, 0x60, 0x8a, 0x61, 0x5e, 0xda, 0x3f, 0x27, 0xcb, + 0x59, 0xe2, 0xb9, 0x34, 0x1c, 0xd3, 0x22, 0x9e, 0x0b, 0x4f, 0x12, 0x15, 0x58, 0x88, 0x2a, 0x45, + 0xcc, 0xcb, 0x01, 0xb3, 0xdc, 0x69, 0x24, 0x5d, 0xb9, 0xb2, 0x4f, 0x7c, 0x26, 0x54, 0x85, 0xac, + 0x4a, 0x7f, 0xc9, 0xce, 0x49, 0x17, 0x53, 0xf9, 0xf9, 0x51, 0xfe, 0x26, 0x63, 0x6b, 0x9e, 0x74, + 0x31, 0x71, 0x5a, 0xdd, 0xdf, 0x44, 0x51, 0x81, 0x4a, 0x8e, 0xf4, 0x58, 0xe9, 0x38, 0xa2, 0x79, + 0x5d, 0xd7, 0x8f, 0xeb, 0xe0, 0x9f, 0x25, 0xb9, 0x12, 0xd8, 0x3b, 0x78, 0xf6, 0x58, 0x7d, 0x94, + 0xcf, 0x42, 0xdb, 0x58, 0xc1, 0x8a, 0x27, 0x2e, 0x50, 0xf1, 0xc0, 0x5a, 0xba, 0x4f, 0x45, 0x9f, + 0x42, 0x60, 0xb5, 0x5c, 0x66, 0x75, 0x72, 0xd1, 0xcf, 0x45, 0xd4, 0x31, 0xeb, 0x4b, 0x61, 0x74, + 0x1b, 0x55, 0x20, 0x8d, 0x8f, 0xbb, 0xb2, 0x86, 0x6d, 0x95, 0x1b, 0x2e, 0x71, 0xd8, 0x7e, 0xdb, + 0x00, 0x1e, 0x98, 0xc6, 0xc7, 0x5d, 0x42, 0x44, 0x3b, 0x64, 0xf6, 0x72, 0x5d, 0x05, 0x5a, 0x6c, + 0xfb, 0x7c, 0x78, 0xe1, 0xf7, 0x14, 0x2e, 0xae, 0xe0, 0x79, 0x09, 0x4c, 0x84, 0xf8, 0x43, 0x01, + 0xde, 0x8a, 0x6c, 0xb5, 0x38, 0x27, 0xb2, 0x8f, 0x21, 0x45, 0x2b, 0x9f, 0xb8, 0x60, 0xe5, 0x29, + 0x97, 0xf8, 0x7b, 0xee, 0x18, 0x97, 0x70, 0xdb, 0x24, 0x8a, 0xfd, 0x0a, 0xd6, 0xc3, 0xa6, 0xdd, + 0x06, 0x4f, 0x5c, 0xb8, 0xc1, 0x5d, 0x56, 0xcf, 0x08, 0xf4, 0x15, 0x33, 0x4e, 0x23, 0xf0, 0xeb, + 0x02, 0xcc, 0x79, 0x3e, 0x4d, 0xcc, 0xee, 0xed, 0xfb, 0x90, 0x34, 0xcc, 0xa3, 0x8b, 0x2c, 0x06, + 0x92, 0xfc, 0x64, 0x4a, 0x0a, 0x97, 0x28, 0xce, 0xfa, 0xfe, 0xdb, 0x04, 0x64, 0x9e, 0x56, 0xe2, + 0xac, 0xe5, 0xc7, 0x7c, 0xa1, 0x99, 0x0d, 0xec, 0xa8, 0xae, 0xe8, 0xbd, 0x6f, 0xf9, 0x69, 0x65, + 0x03, 0x9f, 0xb8, 0x5d, 0x91, 0x70, 0xa1, 0x55, 0xc8, 0x38, 0xfb, 0x16, 0xb6, 0xf7, 0xcd, 0xb6, + 0x76, 0x11, 0x1f, 0xc4, 0xe7, 0x5a, 0xc0, 0x30, 0x49, 0xe5, 0xba, 0x41, 0x0d, 0x42, 0x44, 0x50, + 0x03, 0x79, 0x8d, 0xe7, 0xc6, 0x25, 0x2e, 0xf2, 0x9a, 0x80, 0xff, 0x36, 0x59, 0x9c, 0x12, 0x5f, + 0x00, 0x90, 0xea, 0xc4, 0xd9, 0x24, 0xbf, 0x9a, 0x84, 0xfc, 0x76, 0xcf, 0xde, 0x8f, 0xb9, 0xf7, + 0x55, 0x00, 0xba, 0x3d, 0x9b, 0xe2, 0x83, 0x63, 0x83, 0xd7, 0xf9, 0x9c, 0x78, 0x09, 0xb7, 0xd2, + 0x8c, 0xaf, 0x79, 0x6c, 0xa0, 0x1a, 0x17, 0x82, 0x65, 0x3f, 0xe8, 0xe2, 0xc6, 0x28, 0x24, 0xd9, + 0x3c, 0x36, 0x36, 0xb1, 0x07, 0x21, 0x99, 0x24, 0x4c, 0x24, 0x7d, 0x0c, 0xd3, 0xe4, 0x41, 0x76, + 0xcc, 0x8b, 0x34, 0xf3, 0x14, 0xe1, 0x69, 0x9a, 0xe8, 0x31, 0x64, 0x18, 0x37, 0x99, 0x9a, 0xa6, + 0xe8, 0xd4, 0x14, 0x55, 0x17, 0xae, 0x46, 0x3a, 0x29, 0xa5, 0x29, 0x2b, 0x99, 0x88, 0xe6, 0x61, + 0x72, 0xcf, 0xb4, 0x54, 0x77, 0xdb, 0x94, 0x3d, 0xb0, 0xf6, 0x5c, 0x4f, 0xa5, 0xd3, 0xc5, 0xcc, + 0x7a, 0x2a, 0x9d, 0x29, 0x82, 0xf8, 0x5b, 0x02, 0x14, 0xbc, 0x86, 0x88, 0xd3, 0x5a, 0x57, 0x42, + 0x5a, 0xbc, 0x78, 0x53, 0x10, 0x05, 0x8a, 0xff, 0x8e, 0xba, 0x2b, 0xaa, 0x79, 0x48, 0x5b, 0x26, + 0xce, 0x9e, 0xf2, 0x98, 0x85, 0xd4, 0x24, 0x2e, 0xda, 0xba, 0x34, 0xba, 0xe6, 0x01, 0xcc, 0xeb, + 0x1d, 0x62, 0xc7, 0x75, 0xa7, 0x7d, 0xc2, 0x31, 0x95, 0x83, 0xdd, 0xfd, 0xd9, 0x39, 0x3f, 0xad, + 0xe2, 0x26, 0x89, 0xff, 0x90, 0xae, 0x4e, 0xfb, 0x35, 0x89, 0x53, 0xd5, 0x75, 0xc8, 0x59, 0x4c, + 0x34, 0xf1, 0x39, 0x2e, 0xa8, 0xed, 0x19, 0x8f, 0x95, 0x28, 0xfc, 0x77, 0x12, 0x50, 0x78, 0xd1, + 0xc3, 0xd6, 0xc9, 0xd7, 0x49, 0xdd, 0xb7, 0xa1, 0x70, 0xa4, 0xe8, 0x8e, 0xbc, 0x67, 0x5a, 0x72, + 0xaf, 0xab, 0x29, 0x8e, 0x1b, 0xd7, 0x91, 0x23, 0xe4, 0x27, 0xa6, 0xb5, 0x43, 0x89, 0x08, 0x03, + 0x3a, 0x30, 0xcc, 0x23, 0x43, 0x26, 0x64, 0x8a, 0x62, 0x8f, 0x0d, 0xbe, 0x64, 0x5c, 0xfe, 0xe0, + 0x3f, 0x9f, 0x2e, 0x3e, 0x1a, 0x2b, 0x5a, 0x8b, 0x46, 0xa6, 0xf5, 0x7a, 0xba, 0xb6, 0xbc, 0xb3, + 0x53, 0x5f, 0x93, 0x8a, 0x54, 0xe4, 0x2b, 0x26, 0xb1, 0x79, 0x6c, 0xd8, 0xe2, 0x3f, 0x4a, 0x40, + 0xd1, 0xd7, 0x51, 0x9c, 0x0d, 0x59, 0x85, 0xec, 0xeb, 0x1e, 0xb6, 0xf4, 0x37, 0x68, 0x46, 0xe0, + 0x8c, 0xc4, 0xec, 0x7c, 0x06, 0x33, 0x21, 0x0d, 0x24, 0xbf, 0x9c, 0x06, 0xb2, 0x47, 0x7e, 0xe5, + 0xd1, 0x7d, 0x98, 0x75, 0x8e, 0x0d, 0x99, 0xc5, 0xe9, 0xb1, 0xd8, 0x0e, 0x37, 0xec, 0xa0, 0xe0, + 0x10, 0x7d, 0x10, 0x3a, 0x8d, 0xeb, 0xb0, 0xc5, 0x3f, 0x14, 0x00, 0x51, 0x45, 0xd5, 0xd9, 0x9a, + 0xfe, 0xd7, 0xa5, 0x3f, 0xdd, 0x85, 0x22, 0x8d, 0x7c, 0x94, 0xf5, 0x3d, 0xb9, 0xa3, 0xdb, 0xb6, + 0x6e, 0xb4, 0x78, 0x87, 0xca, 0x53, 0x7a, 0x7d, 0x6f, 0x93, 0x51, 0xc5, 0xbf, 0x06, 0x73, 0xa1, + 0x0a, 0xc4, 0xd9, 0xd8, 0xd7, 0x61, 0x66, 0xcf, 0xec, 0x19, 0x9a, 0xcc, 0x76, 0x3c, 0xf8, 0x72, + 0x60, 0x96, 0xd2, 0xd8, 0xfb, 0xc4, 0xff, 0x91, 0x80, 0x79, 0x09, 0xdb, 0x66, 0xfb, 0x10, 0xc7, + 0xaf, 0xc2, 0x1a, 0xf0, 0xbd, 0x16, 0xf9, 0x8d, 0x34, 0x99, 0x61, 0xcc, 0x6c, 0x9a, 0x0b, 0xaf, + 0xa9, 0xdf, 0x1c, 0xdd, 0x63, 0x07, 0x57, 0xd1, 0xf9, 0x9a, 0x5c, 0x2a, 0xb4, 0x26, 0x67, 0x42, + 0x41, 0x6f, 0x19, 0x26, 0xb1, 0x69, 0x36, 0x7e, 0x6d, 0xf4, 0x3a, 0x2e, 0x52, 0x59, 0x1e, 0x55, + 0xc8, 0x3a, 0x63, 0x69, 0xe0, 0xd7, 0x5b, 0xbd, 0x0e, 0xf5, 0x9d, 0xcb, 0x97, 0x49, 0x79, 0xcf, + 0x4e, 0x17, 0xf3, 0xa1, 0x34, 0x5b, 0xca, 0xeb, 0xde, 0x33, 0x91, 0x2e, 0x7e, 0x07, 0x2e, 0xf5, + 0x29, 0x3b, 0x4e, 0x8f, 0xe7, 0x5f, 0x27, 0xe1, 0x6a, 0x58, 0x7c, 0xdc, 0xf8, 0xe3, 0xeb, 0xde, + 0xa0, 0x35, 0xc8, 0x75, 0x74, 0xe3, 0xcd, 0x96, 0x16, 0x67, 0x3a, 0xba, 0xe1, 0x2f, 0xe3, 0x46, + 0x74, 0x8d, 0xa9, 0xaf, 0xb4, 0x6b, 0x28, 0xb0, 0x10, 0xd5, 0x76, 0x71, 0xf6, 0x8f, 0x5f, 0x13, + 0x60, 0x26, 0xee, 0x35, 0xb3, 0x37, 0x8b, 0x37, 0x13, 0x9b, 0x90, 0xfb, 0x0a, 0x16, 0xd9, 0x7e, + 0x47, 0x00, 0xd4, 0xb4, 0x7a, 0x06, 0x01, 0xb5, 0xcf, 0xcc, 0x56, 0x9c, 0xd5, 0x9c, 0x87, 0x49, + 0xdd, 0xd0, 0xf0, 0x31, 0xad, 0x66, 0x4a, 0x62, 0x0f, 0xa1, 0xad, 0xc3, 0xe4, 0x58, 0x5b, 0x87, + 0xe2, 0x67, 0x30, 0x17, 0x2a, 0x62, 0x9c, 0xf5, 0xff, 0xef, 0x09, 0x98, 0xe3, 0x15, 0x89, 0x7d, + 0x79, 0xf1, 0x5b, 0x30, 0xd9, 0x26, 0x32, 0x47, 0xb4, 0x33, 0x7d, 0xa7, 0xdb, 0xce, 0x34, 0x33, + 0xfa, 0x39, 0x80, 0xae, 0x85, 0x0f, 0x65, 0xc6, 0x9a, 0x1c, 0x8b, 0x35, 0x43, 0x38, 0x28, 0x01, + 0xfd, 0x40, 0x80, 0x02, 0x19, 0xd0, 0x5d, 0xcb, 0xec, 0x9a, 0x36, 0xf1, 0x59, 0xec, 0xf1, 0x60, + 0xce, 0x8b, 0xb3, 0xd3, 0xc5, 0xdc, 0xa6, 0x6e, 0x6c, 0x73, 0xc6, 0x66, 0x63, 0xec, 0x50, 0x7a, + 0xf7, 0x40, 0xc1, 0x72, 0xa5, 0x6d, 0xaa, 0x07, 0xfe, 0x66, 0x18, 0xb1, 0x2c, 0x9e, 0x38, 0x5b, + 0xfc, 0x8f, 0x02, 0xcc, 0x7f, 0x65, 0x6b, 0xb9, 0xff, 0x3f, 0x94, 0x2d, 0xbe, 0x84, 0x22, 0xfd, + 0x51, 0x37, 0xf6, 0xcc, 0x38, 0x57, 0xd5, 0xbf, 0x2f, 0xc0, 0x6c, 0x40, 0x70, 0x9c, 0x0e, 0xce, + 0x1b, 0xe9, 0x49, 0xfc, 0x05, 0xe2, 0xf2, 0x04, 0x47, 0x49, 0x9c, 0x63, 0xf0, 0x5f, 0x26, 0xe0, + 0x72, 0x85, 0xed, 0x41, 0xbb, 0x41, 0x19, 0x71, 0xf6, 0x8c, 0x12, 0x4c, 0x1f, 0x62, 0xcb, 0xd6, + 0x4d, 0x36, 0xfd, 0xe6, 0x24, 0xf7, 0x11, 0x2d, 0x40, 0xda, 0x36, 0x94, 0xae, 0xbd, 0x6f, 0xba, + 0x9b, 0x6f, 0xde, 0xb3, 0x17, 0x40, 0x32, 0xf9, 0xe6, 0x01, 0x24, 0x53, 0xa3, 0x03, 0x48, 0xa6, + 0xbf, 0x44, 0x00, 0x09, 0xdf, 0xe9, 0xfa, 0xf7, 0x02, 0x5c, 0x19, 0xd0, 0x5c, 0x9c, 0xbd, 0xe5, + 0x7b, 0x90, 0x55, 0xb9, 0x60, 0x62, 0xb0, 0xd9, 0x36, 0x5e, 0x9d, 0x64, 0x7b, 0x43, 0xdc, 0x72, + 0x76, 0xba, 0x08, 0x6e, 0x51, 0xeb, 0x6b, 0x5c, 0x39, 0xe4, 0xb7, 0x26, 0xfe, 0x72, 0x0e, 0x0a, + 0xd5, 0x63, 0xb6, 0x84, 0xdd, 0x60, 0x6e, 0x02, 0x7a, 0x02, 0xe9, 0xae, 0x65, 0x1e, 0xea, 0x6e, + 0x35, 0xf2, 0xa1, 0xe8, 0x01, 0xb7, 0x1a, 0x7d, 0x5c, 0xdb, 0x9c, 0x43, 0xf2, 0x78, 0x51, 0x13, + 0x32, 0xcf, 0x4c, 0x55, 0x69, 0x3f, 0xd1, 0xdb, 0x6e, 0xcf, 0x7f, 0xef, 0x7c, 0x41, 0xcb, 0x1e, + 0xcf, 0xb6, 0xe2, 0xec, 0xbb, 0x8d, 0xe0, 0x11, 0x51, 0x1d, 0xd2, 0x35, 0xc7, 0xe9, 0x92, 0x44, + 0x6e, 0x3b, 0xee, 0x8c, 0x21, 0x94, 0xb0, 0xb8, 0xc1, 0xa6, 0x2e, 0x3b, 0x6a, 0xc2, 0xec, 0x53, + 0x7a, 0x74, 0xaa, 0xd2, 0x36, 0x7b, 0x5a, 0xc5, 0x34, 0xf6, 0xf4, 0x16, 0xb7, 0xdb, 0xb7, 0xc7, + 0x90, 0xf9, 0xb4, 0xd2, 0x90, 0x06, 0x05, 0xa0, 0x55, 0x48, 0x37, 0x1e, 0x71, 0x61, 0xcc, 0xaf, + 0xbb, 0x35, 0x86, 0xb0, 0xc6, 0x23, 0xc9, 0x63, 0x43, 0xeb, 0x90, 0x5d, 0xfd, 0xbc, 0x67, 0x61, + 0x2e, 0x65, 0x6a, 0x68, 0xe8, 0x42, 0xbf, 0x14, 0xca, 0x25, 0x05, 0x99, 0x51, 0x03, 0xf2, 0xaf, + 0x4c, 0xeb, 0xa0, 0x6d, 0x2a, 0x6e, 0x0d, 0xa7, 0xa9, 0xb8, 0x6f, 0x8c, 0x21, 0xce, 0x65, 0x94, + 0xfa, 0x44, 0xa0, 0xef, 0x40, 0x81, 0x34, 0x46, 0x53, 0xd9, 0x6d, 0xbb, 0x85, 0x4c, 0x53, 0xa9, + 0xdf, 0x1c, 0x43, 0xaa, 0xc7, 0xe9, 0xee, 0xa1, 0xf4, 0x89, 0x5a, 0x90, 0x20, 0x17, 0xea, 0x04, + 0x08, 0x41, 0xaa, 0x4b, 0xda, 0x5b, 0xa0, 0xc1, 0x45, 0xf4, 0x37, 0x7a, 0x17, 0xa6, 0x0d, 0x53, + 0xc3, 0xee, 0x08, 0xc9, 0x95, 0xe7, 0xcf, 0x4e, 0x17, 0xa7, 0xb6, 0x4c, 0x8d, 0x79, 0x34, 0xfc, + 0x97, 0x34, 0x45, 0x32, 0xd5, 0xb5, 0x85, 0x25, 0x48, 0x91, 0x76, 0x27, 0x86, 0x69, 0x57, 0xb1, + 0xf1, 0x8e, 0xa5, 0x73, 0x69, 0xee, 0xe3, 0xc2, 0x3f, 0x49, 0x40, 0xa2, 0xf1, 0x88, 0xf8, 0xec, + 0xbb, 0x3d, 0xf5, 0x00, 0x3b, 0x3c, 0x9d, 0x3f, 0x51, 0x5f, 0xde, 0xc2, 0x7b, 0x3a, 0x73, 0xad, + 0x32, 0x12, 0x7f, 0x42, 0xef, 0x00, 0x28, 0xaa, 0x8a, 0x6d, 0x5b, 0x76, 0x8f, 0xd4, 0x65, 0xa4, + 0x0c, 0xa3, 0x6c, 0xe0, 0x13, 0xc2, 0x66, 0x63, 0xd5, 0xc2, 0x8e, 0x1b, 0x19, 0xc5, 0x9e, 0x08, + 0x9b, 0x83, 0x3b, 0x5d, 0xd9, 0x31, 0x0f, 0xb0, 0x41, 0xfb, 0x49, 0x86, 0x98, 0x9a, 0x4e, 0xb7, + 0x49, 0x08, 0xc4, 0x4a, 0x62, 0x43, 0xf3, 0x4d, 0x5a, 0x46, 0xf2, 0x9e, 0x89, 0x48, 0x0b, 0xb7, + 0x74, 0x7e, 0x36, 0x2c, 0x23, 0xf1, 0x27, 0xa2, 0x25, 0xa5, 0xe7, 0xec, 0xd3, 0x96, 0xc8, 0x48, + 0xf4, 0x37, 0xba, 0x0d, 0x05, 0x16, 0x4c, 0x29, 0x63, 0x43, 0x95, 0xa9, 0x71, 0xcd, 0xd0, 0xe4, + 0x1c, 0x23, 0x57, 0x0d, 0x95, 0x98, 0x52, 0xf4, 0x08, 0x38, 0x41, 0x3e, 0xe8, 0xd8, 0x44, 0xa7, + 0x40, 0x72, 0x95, 0x0b, 0x67, 0xa7, 0x8b, 0xd9, 0x06, 0x4d, 0xd8, 0xd8, 0x6c, 0xd4, 0xd7, 0xa4, + 0x2c, 0xcb, 0xb5, 0xd1, 0xb1, 0xeb, 0xda, 0xc2, 0x6f, 0x08, 0x90, 0x7c, 0x5a, 0x69, 0x5c, 0x58, + 0x65, 0x6e, 0x41, 0x93, 0x81, 0x82, 0xde, 0x81, 0xc2, 0xae, 0xde, 0x6e, 0xeb, 0x46, 0x8b, 0x78, + 0x51, 0xdf, 0xc3, 0xaa, 0xab, 0xb0, 0x3c, 0x27, 0x6f, 0x33, 0x2a, 0x5a, 0x82, 0xac, 0x6a, 0x61, + 0x0d, 0x1b, 0x8e, 0xae, 0xb4, 0x6d, 0xae, 0xb9, 0x20, 0x69, 0xe1, 0x97, 0x04, 0x98, 0xa4, 0x23, + 0x00, 0xbd, 0x0d, 0x19, 0xd5, 0x34, 0x1c, 0x45, 0x37, 0xb8, 0x29, 0xcb, 0x48, 0x3e, 0x61, 0x68, + 0xf1, 0xae, 0xc3, 0x8c, 0xa2, 0xaa, 0x66, 0xcf, 0x70, 0x64, 0x43, 0xe9, 0x60, 0x5e, 0xcc, 0x2c, + 0xa7, 0x6d, 0x29, 0x1d, 0x8c, 0x16, 0xc1, 0x7d, 0xf4, 0x4e, 0x4a, 0x66, 0x24, 0xe0, 0xa4, 0x0d, + 0x7c, 0xb2, 0xf0, 0x47, 0x02, 0xa4, 0xdd, 0x31, 0x43, 0x8a, 0xd1, 0xc2, 0x06, 0xb6, 0x14, 0xc7, + 0xf4, 0x8a, 0xe1, 0x11, 0xfa, 0xa7, 0xca, 0x8c, 0x3f, 0x55, 0xce, 0xc3, 0xa4, 0x43, 0x86, 0x05, + 0x2f, 0x01, 0x7b, 0xa0, 0xeb, 0xd9, 0x6d, 0xa5, 0xc5, 0x96, 0xf3, 0x32, 0x12, 0x7b, 0x20, 0x95, + 0xe1, 0x31, 0xb9, 0x4c, 0x23, 0xfc, 0x89, 0x94, 0x94, 0x45, 0x8e, 0xee, 0xe2, 0x96, 0x6e, 0xd0, + 0xbe, 0x94, 0x94, 0x80, 0x92, 0xca, 0x84, 0x82, 0xde, 0x82, 0x0c, 0xcb, 0x80, 0x0d, 0x8d, 0x76, + 0xa8, 0xa4, 0x94, 0xa6, 0x84, 0xaa, 0xa1, 0x2d, 0x60, 0xc8, 0x78, 0x83, 0x93, 0x34, 0x5b, 0xcf, + 0xf6, 0x14, 0x49, 0x7f, 0xa3, 0xf7, 0x60, 0xfe, 0x75, 0x4f, 0x69, 0xeb, 0x7b, 0x74, 0xa5, 0x8e, + 0x64, 0x63, 0x3a, 0x63, 0x35, 0x41, 0x5e, 0x1a, 0x95, 0x40, 0x55, 0xe7, 0x8e, 0xe5, 0xa4, 0x3f, + 0x96, 0xc5, 0xdf, 0x17, 0x60, 0x96, 0x85, 0x05, 0xb1, 0x68, 0xd6, 0xf8, 0xfc, 0x90, 0x8f, 0x20, + 0xa3, 0x29, 0x8e, 0xc2, 0xce, 0x7e, 0x26, 0x46, 0x9e, 0xfd, 0xf4, 0xce, 0x22, 0x28, 0x8e, 0x42, + 0xcf, 0x7f, 0x22, 0x48, 0x91, 0xdf, 0xec, 0x98, 0xac, 0x44, 0x7f, 0x8b, 0x9f, 0x02, 0x0a, 0x16, + 0x34, 0x4e, 0x8f, 0xec, 0x1e, 0x5c, 0x22, 0xba, 0xae, 0x1a, 0xaa, 0x75, 0xd2, 0x75, 0x74, 0xd3, + 0x78, 0x4e, 0xff, 0xda, 0xa8, 0x18, 0xd8, 0xd8, 0xa2, 0xfb, 0x59, 0xe2, 0x1f, 0x4c, 0x41, 0xae, + 0x7a, 0xdc, 0x35, 0xad, 0x58, 0x97, 0xc1, 0xca, 0x30, 0xcd, 0x57, 0x0a, 0x46, 0x6c, 0x2c, 0xf7, + 0x19, 0x73, 0x77, 0xcf, 0x96, 0x33, 0xa2, 0x32, 0x00, 0x8b, 0x38, 0xa5, 0x51, 0x45, 0xc9, 0x0b, + 0x6c, 0xb5, 0x51, 0x36, 0x42, 0x45, 0x5b, 0x90, 0xed, 0x1c, 0xaa, 0xaa, 0xbc, 0xa7, 0xb7, 0x1d, + 0x1e, 0xb8, 0x17, 0x1d, 0x63, 0xbe, 0xf9, 0xb2, 0x52, 0x79, 0x42, 0x33, 0xb1, 0x18, 0x3a, 0xff, + 0x59, 0x02, 0x22, 0x81, 0xfd, 0x46, 0xdf, 0x04, 0x7e, 0x26, 0x47, 0xb6, 0xdd, 0xe3, 0x77, 0xe5, + 0xdc, 0xd9, 0xe9, 0x62, 0x46, 0xa2, 0xd4, 0x46, 0xa3, 0x29, 0x65, 0x58, 0x86, 0x86, 0xed, 0xa0, + 0x1b, 0x90, 0x33, 0x3b, 0xba, 0x23, 0xbb, 0x4e, 0x12, 0xf7, 0x28, 0x67, 0x08, 0xd1, 0x75, 0xa2, + 0x50, 0x13, 0xee, 0x60, 0x83, 0xf6, 0x76, 0x52, 0x4f, 0x79, 0x97, 0xad, 0x5e, 0x3a, 0x6c, 0x44, + 0xcb, 0x66, 0xd7, 0xd1, 0x3b, 0xfa, 0xe7, 0x74, 0x6b, 0x9b, 0xef, 0x3a, 0xdd, 0x60, 0xd9, 0x49, + 0xfd, 0xca, 0x74, 0x59, 0x93, 0xe7, 0x7d, 0x1e, 0xc8, 0x8a, 0xfe, 0xa6, 0x00, 0x97, 0xb9, 0x22, + 0xe5, 0x5d, 0x1a, 0x24, 0xaf, 0xb4, 0x75, 0xe7, 0x44, 0x3e, 0x38, 0x2c, 0xa5, 0xa9, 0xdf, 0xfa, + 0xb3, 0x91, 0x0d, 0x12, 0xe8, 0x07, 0xcb, 0x6e, 0xb3, 0x9c, 0x3c, 0xe3, 0xcc, 0x1b, 0x87, 0x55, + 0xc3, 0xb1, 0x4e, 0xca, 0x57, 0xce, 0x4e, 0x17, 0xe7, 0x06, 0x53, 0x5f, 0x4a, 0x73, 0xf6, 0x20, + 0x0b, 0xaa, 0x01, 0x60, 0xaf, 0x1f, 0xd2, 0x19, 0x23, 0xda, 0xff, 0x88, 0xec, 0xb0, 0x52, 0x80, + 0x17, 0xdd, 0x85, 0x22, 0x3f, 0x1a, 0xb3, 0xa7, 0xb7, 0xb1, 0x6c, 0xeb, 0x9f, 0x63, 0x3a, 0xb7, + 0x24, 0xa5, 0x3c, 0xa3, 0x13, 0x11, 0x0d, 0xfd, 0x73, 0xbc, 0xf0, 0x3d, 0x28, 0x0d, 0x2b, 0x7d, + 0x70, 0x08, 0x64, 0xd8, 0x96, 0xee, 0x87, 0xe1, 0xf5, 0x9c, 0x31, 0xba, 0x2a, 0x5f, 0xd3, 0xf9, + 0x28, 0xf1, 0xa1, 0x20, 0xfe, 0xe3, 0x04, 0xe4, 0xca, 0xbd, 0xf6, 0xc1, 0xf3, 0x6e, 0xa3, 0xd7, + 0xe9, 0x28, 0xd6, 0x09, 0x31, 0x83, 0xcc, 0x50, 0x90, 0x02, 0x0a, 0xcc, 0x0c, 0x52, 0x4b, 0xa0, + 0x7f, 0x8e, 0xc9, 0xe4, 0x14, 0x3c, 0xee, 0xcd, 0x0e, 0x01, 0xd0, 0x3a, 0x04, 0xce, 0x70, 0x9b, + 0x47, 0x36, 0xfa, 0x10, 0x4a, 0x81, 0x8c, 0x74, 0xf1, 0x45, 0xc6, 0x86, 0x63, 0xe9, 0x98, 0x2d, + 0x20, 0x26, 0xa5, 0x40, 0xc0, 0x4d, 0x9d, 0x24, 0x57, 0x59, 0x2a, 0x6a, 0xc2, 0x0c, 0xc9, 0x78, + 0x22, 0xd3, 0x29, 0xc4, 0x5d, 0xe0, 0x7d, 0x10, 0x51, 0xad, 0x50, 0xb9, 0x97, 0xa9, 0x7e, 0x2a, + 0x94, 0x87, 0xfe, 0x94, 0xb2, 0xd8, 0xa7, 0x2c, 0x7c, 0x02, 0xc5, 0xfe, 0x0c, 0x41, 0x5d, 0xa6, + 0x98, 0x2e, 0xe7, 0x83, 0xba, 0x4c, 0x06, 0xf4, 0xb4, 0x9e, 0x4a, 0xa7, 0x8a, 0x93, 0xe2, 0x9f, + 0x27, 0x21, 0xef, 0x76, 0xb3, 0x38, 0x81, 0x4e, 0x19, 0x26, 0x49, 0xa7, 0x70, 0x83, 0x44, 0x6e, + 0x8f, 0xe8, 0xdd, 0x3c, 0xec, 0x9c, 0x74, 0x16, 0x17, 0x24, 0x53, 0xd6, 0x38, 0x0c, 0xce, 0xc2, + 0x2f, 0x25, 0x20, 0x45, 0xb1, 0xc5, 0x03, 0x48, 0xd1, 0x89, 0x42, 0x18, 0x67, 0xa2, 0xa0, 0x59, + 0xbd, 0xe9, 0x2c, 0x11, 0x70, 0x4d, 0x89, 0xcf, 0xb7, 0xaf, 0xbc, 0xff, 0xe0, 0x21, 0x35, 0x36, + 0x33, 0x12, 0x7f, 0x42, 0x65, 0x1a, 0xb7, 0x64, 0x5a, 0x0e, 0xd6, 0xb8, 0x4f, 0xbf, 0x74, 0x5e, + 0xfb, 0xba, 0x93, 0x92, 0xcb, 0x87, 0xae, 0x42, 0x92, 0x58, 0xb1, 0x69, 0x16, 0xe5, 0x70, 0x76, + 0xba, 0x98, 0x24, 0xf6, 0x8b, 0xd0, 0xd0, 0x0a, 0x64, 0xc3, 0x26, 0x83, 0x78, 0x70, 0xd4, 0x30, + 0x06, 0x86, 0x3b, 0xb4, 0xbd, 0xa1, 0xc5, 0xf0, 0x2c, 0x6f, 0xe3, 0x9f, 0xa6, 0x20, 0x57, 0xef, + 0xc4, 0x3d, 0xa5, 0xac, 0x86, 0x5b, 0x38, 0x0a, 0x08, 0x85, 0x5e, 0x1a, 0xd1, 0xc0, 0xa1, 0x19, + 0x3c, 0x79, 0xb1, 0x19, 0xbc, 0x4e, 0x3c, 0x65, 0x7e, 0x7f, 0x43, 0x72, 0x08, 0xe6, 0x09, 0xbf, + 0x9f, 0xfa, 0x29, 0x12, 0xe1, 0xf1, 0x0f, 0x62, 0xd0, 0x48, 0x95, 0x4f, 0xa8, 0x43, 0xce, 0x7a, + 0xd9, 0xd4, 0xf8, 0xbd, 0x6c, 0x1a, 0x1b, 0x1a, 0x9d, 0xd4, 0xc2, 0x16, 0x75, 0xfa, 0xcd, 0x2d, + 0xea, 0x82, 0xc3, 0x3b, 0xeb, 0x47, 0x90, 0xd4, 0x74, 0xb7, 0x71, 0xc6, 0x9f, 0xaa, 0x09, 0xd3, + 0x39, 0xbd, 0x36, 0x15, 0xec, 0xb5, 0xac, 0x97, 0x2c, 0xd4, 0x01, 0x7c, 0xdd, 0xa0, 0x25, 0x98, + 0x32, 0xdb, 0x9a, 0x7b, 0x12, 0x25, 0x57, 0xce, 0x9c, 0x9d, 0x2e, 0x4e, 0x3e, 0x6f, 0x6b, 0xf5, + 0x35, 0x69, 0xd2, 0x6c, 0x6b, 0x75, 0x8d, 0x5e, 0x9e, 0x81, 0x8f, 0x64, 0x2f, 0x4c, 0x6d, 0x46, + 0x9a, 0x36, 0xf0, 0xd1, 0x1a, 0xb6, 0x55, 0xde, 0xe1, 0x7e, 0x5b, 0x80, 0xbc, 0xab, 0xfb, 0x78, + 0x8d, 0x4a, 0x5a, 0xef, 0xf0, 0x41, 0x96, 0xbc, 0xd8, 0x20, 0x73, 0xf9, 0xf8, 0xf9, 0xdc, 0x5f, + 0x11, 0x78, 0xe0, 0x71, 0x43, 0x55, 0x1c, 0xe2, 0x54, 0xc4, 0x38, 0x30, 0xee, 0x41, 0xd1, 0x52, + 0x0c, 0xcd, 0xec, 0xe8, 0x9f, 0x63, 0xb6, 0x10, 0x6a, 0xf3, 0x6d, 0xcf, 0x82, 0x47, 0xa7, 0xab, + 0x7e, 0xb6, 0xf8, 0x47, 0x09, 0x1e, 0xa4, 0xec, 0x15, 0x23, 0x4e, 0x75, 0x7d, 0x17, 0x66, 0xfb, + 0x6f, 0x32, 0x71, 0x47, 0xeb, 0xbb, 0x11, 0xf2, 0xa2, 0x0a, 0xc2, 0x82, 0x0d, 0xdd, 0xc8, 0xf7, + 0xbe, 0x5b, 0x4d, 0x6c, 0x54, 0x81, 0x6c, 0xf0, 0x82, 0x94, 0xe4, 0xd8, 0x17, 0xa4, 0x80, 0xe5, + 0x5d, 0x8b, 0xb2, 0xf0, 0xf3, 0x30, 0x49, 0x93, 0xdf, 0xc0, 0x44, 0xf3, 0xd6, 0xfc, 0xb3, 0x04, + 0xdc, 0xa4, 0xa5, 0x7f, 0x89, 0x2d, 0x7d, 0xef, 0x64, 0xdb, 0x32, 0x1d, 0xac, 0x3a, 0x58, 0xf3, + 0x57, 0xce, 0x63, 0xb5, 0x7b, 0x99, 0xae, 0xfb, 0x82, 0x0b, 0x05, 0x9c, 0x79, 0x5c, 0x68, 0x03, + 0x0a, 0x3c, 0xb4, 0x40, 0x69, 0xeb, 0x87, 0x58, 0x56, 0x9c, 0x8b, 0xcc, 0x6e, 0x39, 0xc6, 0xbb, + 0x4a, 0x58, 0x57, 0x1d, 0xa4, 0x41, 0x86, 0x0b, 0xd3, 0x35, 0x7e, 0xaf, 0xcf, 0xd3, 0x2f, 0xb7, + 0xa0, 0x98, 0x66, 0xf1, 0x0d, 0xf5, 0x35, 0x29, 0xcd, 0x24, 0xd7, 0x35, 0xf1, 0x3f, 0x09, 0x70, + 0xeb, 0x1c, 0x15, 0xc7, 0xd9, 0x75, 0x17, 0x20, 0x7d, 0x48, 0x5e, 0xa4, 0x73, 0x1d, 0xa7, 0x25, + 0xef, 0x19, 0x6d, 0x42, 0x6e, 0x4f, 0xd1, 0xdb, 0x7e, 0x97, 0x1e, 0x1e, 0x9f, 0x18, 0x1d, 0x2a, + 0x3b, 0xc3, 0xd8, 0x59, 0x1f, 0x16, 0x7f, 0x33, 0x01, 0xb3, 0xab, 0x9a, 0xd6, 0x68, 0x70, 0x1b, + 0x18, 0x5f, 0x4f, 0x71, 0x41, 0x66, 0xc2, 0x07, 0x99, 0xe8, 0x5d, 0x40, 0x9a, 0x6e, 0xb3, 0xfb, + 0x43, 0xec, 0x7d, 0x45, 0x33, 0x8f, 0xfc, 0xb8, 0x8c, 0x59, 0x37, 0xa5, 0xe1, 0x26, 0xa0, 0x06, + 0x50, 0xb4, 0x23, 0xdb, 0x8e, 0xe2, 0xed, 0x3b, 0xdd, 0x1a, 0xeb, 0xbc, 0x17, 0x83, 0x41, 0xde, + 0xa3, 0x94, 0x21, 0x72, 0xe8, 0x4f, 0xe2, 0xb7, 0xeb, 0xa4, 0xea, 0x8e, 0xac, 0xd8, 0xee, 0xe1, + 0x1e, 0x76, 0x73, 0x49, 0x9e, 0xd1, 0x57, 0x6d, 0x76, 0x66, 0x87, 0x9d, 0x46, 0xf0, 0x55, 0x13, + 0x27, 0x24, 0xfe, 0x07, 0x02, 0xe4, 0x25, 0xbc, 0x67, 0x61, 0x3b, 0xd6, 0x45, 0x81, 0x27, 0x30, + 0x63, 0x31, 0xa9, 0xf2, 0x9e, 0x65, 0x76, 0x2e, 0x32, 0xae, 0xb2, 0x9c, 0xf1, 0x89, 0x65, 0x76, + 0xb8, 0x61, 0x79, 0x09, 0x05, 0xaf, 0x8c, 0x71, 0x56, 0xfe, 0xf7, 0xe9, 0x59, 0x66, 0x26, 0x38, + 0xee, 0x00, 0x89, 0x78, 0x35, 0x40, 0x37, 0xaa, 0x82, 0x05, 0x8d, 0x53, 0x0d, 0xff, 0x4d, 0x80, + 0x7c, 0xa3, 0xb7, 0xcb, 0x2e, 0xa8, 0x8a, 0x4f, 0x03, 0x55, 0xc8, 0xb4, 0xf1, 0x9e, 0x23, 0xbf, + 0x51, 0x1c, 0x7d, 0x9a, 0xb0, 0xd2, 0x53, 0x04, 0x4f, 0x01, 0x2c, 0x7a, 0xfe, 0x8d, 0xca, 0x49, + 0x5e, 0x50, 0x4e, 0x86, 0xf2, 0x12, 0xb2, 0xf8, 0xd3, 0x04, 0x14, 0xbc, 0x6a, 0xc6, 0x69, 0x25, + 0x5f, 0x85, 0xac, 0x43, 0xf2, 0x22, 0xd6, 0x61, 0x96, 0xc7, 0x84, 0x44, 0x5b, 0x88, 0x65, 0x98, + 0xa3, 0x8e, 0x8b, 0xac, 0x74, 0xbb, 0x6d, 0xdd, 0x85, 0xbb, 0xd4, 0xfe, 0xa4, 0xa4, 0x59, 0x9a, + 0xb4, 0xca, 0x52, 0x28, 0xd0, 0x45, 0xbf, 0x2a, 0xc0, 0xcc, 0x9e, 0x85, 0xf1, 0xe7, 0x58, 0xa6, + 0xd0, 0x6b, 0xbc, 0xa0, 0x97, 0x35, 0x52, 0x86, 0x2f, 0xbd, 0x29, 0x9e, 0x65, 0x2f, 0x6e, 0x90, + 0xf7, 0xf2, 0x4e, 0xfb, 0x0a, 0x66, 0x69, 0xa3, 0xc4, 0x7d, 0xf0, 0x57, 0xfc, 0x7b, 0x09, 0x40, + 0x41, 0xc9, 0x5f, 0x5d, 0x63, 0x26, 0xe2, 0x6b, 0xcc, 0x6f, 0x02, 0x62, 0x61, 0x93, 0xb6, 0xdc, + 0xc5, 0x96, 0x6c, 0x63, 0xd5, 0xe4, 0xb7, 0x2c, 0x09, 0x52, 0x91, 0xa7, 0x6c, 0x63, 0xab, 0x41, + 0xe9, 0xe8, 0x31, 0x80, 0xef, 0xd2, 0xf1, 0x19, 0x67, 0xa4, 0x47, 0x27, 0x65, 0x3c, 0x5f, 0x4e, + 0xfc, 0xfe, 0x02, 0xcc, 0x70, 0x1d, 0xee, 0x18, 0xba, 0x69, 0xa0, 0x07, 0x90, 0x6c, 0xf1, 0xbd, + 0x83, 0x6c, 0xe4, 0x3a, 0x9f, 0x7f, 0xbd, 0x5c, 0x6d, 0x42, 0x22, 0x79, 0x09, 0x4b, 0xb7, 0xe7, + 0x44, 0x78, 0x56, 0x7e, 0x28, 0x78, 0x90, 0xa5, 0xdb, 0x73, 0x50, 0x03, 0x0a, 0xaa, 0x7f, 0x4d, + 0x96, 0x4c, 0xd8, 0x93, 0x43, 0x51, 0x58, 0xe4, 0x85, 0x65, 0xb5, 0x09, 0x29, 0xaf, 0x86, 0x12, + 0x50, 0x25, 0x78, 0x3b, 0x53, 0x6a, 0x20, 0xce, 0xcc, 0x3f, 0x61, 0x1c, 0xbe, 0x19, 0xaa, 0x36, + 0x11, 0xb8, 0xc4, 0x09, 0x7d, 0x04, 0x53, 0x1a, 0xbd, 0x07, 0x88, 0x8f, 0x88, 0xa8, 0x4e, 0x17, + 0xba, 0x6e, 0xa9, 0x36, 0x21, 0x71, 0x0e, 0xb4, 0x0e, 0x33, 0xec, 0x17, 0xf3, 0x73, 0x38, 0x34, + 0xbd, 0x35, 0x5c, 0x42, 0x60, 0x26, 0xa9, 0x4d, 0x48, 0x59, 0xcd, 0xa7, 0xa2, 0x6f, 0x41, 0xca, + 0x56, 0x15, 0x17, 0x9c, 0x5e, 0x1b, 0x72, 0xa9, 0x87, 0xcf, 0x4c, 0x73, 0xa3, 0xc7, 0xec, 0xc2, + 0x48, 0xe7, 0xd8, 0x5d, 0x27, 0x8c, 0x2a, 0x7e, 0xe8, 0xf0, 0x38, 0x29, 0x3e, 0xa6, 0x04, 0xf4, + 0x14, 0xb2, 0x0a, 0x71, 0x18, 0x65, 0x7a, 0xfc, 0x92, 0x2e, 0x0c, 0x46, 0xef, 0xce, 0x0f, 0x1c, + 0x97, 0xad, 0xd1, 0x13, 0xea, 0x2e, 0xd1, 0x17, 0xd4, 0xc1, 0x56, 0x0b, 0x97, 0xb2, 0xa3, 0x05, + 0x05, 0xa3, 0xcb, 0x3c, 0x41, 0x94, 0x48, 0x1c, 0x47, 0xef, 0xec, 0x33, 0xad, 0xd4, 0xcc, 0xd0, + 0xfd, 0xe0, 0x88, 0x83, 0x43, 0xb5, 0x09, 0x69, 0x66, 0x3f, 0x40, 0x46, 0xcb, 0x90, 0x68, 0xa9, + 0xa5, 0xdc, 0xd0, 0x11, 0xe2, 0x1d, 0x8e, 0xa9, 0x4d, 0x48, 0x89, 0x96, 0x8a, 0x3e, 0x81, 0x34, + 0x3b, 0xe9, 0x70, 0x6c, 0x94, 0xf2, 0x43, 0xed, 0x44, 0xf8, 0xbc, 0x48, 0x6d, 0x42, 0xa2, 0x87, + 0x2b, 0xc8, 0xfb, 0xb6, 0x21, 0x6f, 0xb1, 0xf0, 0x3c, 0x37, 0x98, 0xb6, 0x38, 0x74, 0x8f, 0x3c, + 0x2a, 0x9e, 0xb6, 0x46, 0xa1, 0x43, 0x80, 0x8e, 0xbe, 0x0b, 0xf3, 0x61, 0x89, 0xbc, 0xa7, 0xcd, + 0x0e, 0xdd, 0xef, 0x1d, 0x1a, 0xdb, 0x59, 0x9b, 0x90, 0x90, 0x35, 0x90, 0x88, 0x3e, 0x80, 0x49, + 0xd6, 0x6a, 0x88, 0x8a, 0x8c, 0x0a, 0xfb, 0xe8, 0x6b, 0x30, 0x96, 0x9f, 0x74, 0x7e, 0x87, 0xc7, + 0xa8, 0xc9, 0x6d, 0xb3, 0x55, 0x9a, 0x1b, 0xda, 0xf9, 0x07, 0xa3, 0xed, 0x48, 0xe7, 0x77, 0x7c, + 0x2a, 0x69, 0x77, 0x8b, 0xa5, 0xf0, 0xb8, 0xa4, 0xf9, 0xa1, 0xed, 0x1e, 0x11, 0xba, 0x56, 0xa3, + 0xa7, 0x07, 0x7c, 0x32, 0x29, 0x9a, 0xc5, 0xee, 0xa1, 0x91, 0xe9, 0x98, 0xba, 0x34, 0xb4, 0x68, + 0x83, 0x17, 0xf5, 0xd4, 0xa8, 0x93, 0xe5, 0x51, 0xd1, 0x4b, 0x28, 0xf2, 0xdb, 0x22, 0xfc, 0x4d, + 0x89, 0xcb, 0x54, 0xde, 0xbd, 0x48, 0xd3, 0x15, 0x15, 0xd4, 0x53, 0x9b, 0x90, 0x0a, 0x6a, 0x38, + 0x05, 0x7d, 0x0a, 0xb3, 0x54, 0x9e, 0xac, 0xfa, 0x17, 0x7c, 0x94, 0x4a, 0x03, 0xd7, 0x45, 0x0c, + 0xbf, 0x0b, 0xc4, 0x95, 0x5c, 0x54, 0xfb, 0x92, 0x48, 0x37, 0xd6, 0x0d, 0xdd, 0xa1, 0x56, 0x76, + 0x61, 0x68, 0x37, 0x0e, 0x5f, 0x2b, 0x48, 0xba, 0xb1, 0xce, 0x28, 0xa4, 0x1b, 0x3b, 0x3c, 0x68, + 0x8d, 0x37, 0xc7, 0xdb, 0x43, 0xbb, 0x71, 0x54, 0x74, 0x1b, 0xe9, 0xc6, 0x4e, 0x90, 0x4e, 0xba, + 0x31, 0x33, 0x10, 0x7d, 0x72, 0xdf, 0x19, 0xda, 0x8d, 0x87, 0x1e, 0x83, 0x26, 0xdd, 0x58, 0x19, + 0x48, 0x44, 0x6b, 0x00, 0xcc, 0x1f, 0xa2, 0x93, 0xe2, 0xb5, 0xa1, 0x93, 0x41, 0x7f, 0xf0, 0x1a, + 0x99, 0x0c, 0xda, 0x2e, 0x8d, 0x18, 0x32, 0x8a, 0xb6, 0x64, 0xba, 0x07, 0x5b, 0x5a, 0x1c, 0x6a, + 0xc8, 0x06, 0xf6, 0x4b, 0x89, 0x21, 0x3b, 0xf2, 0x88, 0x64, 0x56, 0x61, 0x8b, 0xc6, 0xa5, 0xa5, + 0xe1, 0x66, 0x39, 0xb8, 0x77, 0x44, 0xcd, 0x32, 0x25, 0xa0, 0x55, 0xc8, 0x90, 0x39, 0xff, 0x84, + 0x9a, 0xa1, 0xeb, 0x43, 0x9d, 0xda, 0xbe, 0xe3, 0x31, 0xb5, 0x09, 0x29, 0xfd, 0x9a, 0x93, 0xc8, + 0xeb, 0xd9, 0x72, 0x5a, 0x49, 0x1c, 0xfa, 0xfa, 0xd0, 0xd2, 0x2b, 0x79, 0x3d, 0xe3, 0x40, 0x2a, + 0x5c, 0x62, 0x6d, 0xc5, 0x4f, 0x24, 0x5b, 0xfc, 0x10, 0x6d, 0xe9, 0x06, 0x15, 0x35, 0x74, 0x5d, + 0x2a, 0xf2, 0xa0, 0x74, 0x6d, 0x42, 0x9a, 0x53, 0x06, 0x53, 0xc9, 0x80, 0xe7, 0x53, 0x0f, 0x5b, + 0xcd, 0x2a, 0xdd, 0x1c, 0x3a, 0xe0, 0x23, 0x16, 0x01, 0xc9, 0x80, 0x57, 0x02, 0x64, 0x36, 0x01, + 0x69, 0xb2, 0x6d, 0xb3, 0x1d, 0xfb, 0x5b, 0x23, 0x26, 0xa0, 0xbe, 0x65, 0x04, 0x36, 0x01, 0x69, + 0x0d, 0xc6, 0x49, 0x04, 0xa9, 0x6d, 0xac, 0x58, 0xdc, 0xcc, 0xde, 0x1e, 0x2a, 0x68, 0xe0, 0x06, + 0x3f, 0x22, 0x48, 0xf5, 0x88, 0xc4, 0xe1, 0xb1, 0xdc, 0x6b, 0x66, 0xb8, 0xc3, 0x78, 0x67, 0xa8, + 0xc3, 0x13, 0x79, 0x0f, 0x0e, 0x71, 0x78, 0xac, 0x50, 0x02, 0xfa, 0x39, 0x98, 0xe6, 0xf8, 0xaf, + 0x74, 0x77, 0x84, 0x1b, 0x1b, 0x84, 0xec, 0x64, 0x5c, 0x73, 0x1e, 0x66, 0x65, 0x19, 0xee, 0x64, + 0xd5, 0xbb, 0x37, 0xc2, 0xca, 0x0e, 0x40, 0x5f, 0x66, 0x65, 0x7d, 0x32, 0xb1, 0xb2, 0xac, 0x9f, + 0xf2, 0xb9, 0xee, 0xfe, 0x50, 0x2b, 0x3b, 0x78, 0xf8, 0x86, 0x58, 0xd9, 0xd7, 0x3e, 0x95, 0xd4, + 0xcc, 0x66, 0xf8, 0xab, 0xf4, 0x8d, 0xa1, 0x35, 0x0b, 0x03, 0x51, 0x52, 0x33, 0xce, 0x43, 0x9a, + 0x8d, 0xb9, 0xc4, 0x4c, 0xd3, 0xdf, 0x1c, 0x7e, 0xec, 0xbf, 0x1f, 0x74, 0xd4, 0xdc, 0x95, 0x4e, + 0xa6, 0x61, 0xcf, 0x50, 0x59, 0xfc, 0xd8, 0x33, 0xd7, 0xd4, 0xbb, 0xa3, 0x0d, 0x55, 0xd4, 0x59, + 0x6e, 0xcf, 0x50, 0x85, 0x12, 0x69, 0x51, 0xd9, 0x49, 0x37, 0x3a, 0xbe, 0x97, 0x47, 0xdc, 0x50, + 0xd0, 0x77, 0xde, 0x90, 0x16, 0xd5, 0x23, 0xfa, 0x43, 0xa8, 0xc7, 0xae, 0xcf, 0x28, 0xad, 0x8c, + 0x1e, 0x42, 0xe1, 0x0b, 0x3c, 0xbc, 0x21, 0xc4, 0xc9, 0xde, 0x9c, 0xe9, 0x7a, 0x18, 0xef, 0x8d, + 0x9e, 0x33, 0xfb, 0x5d, 0x0b, 0x36, 0x67, 0x72, 0x9f, 0xe2, 0x6f, 0x08, 0xb0, 0xc4, 0xca, 0x46, + 0x97, 0x04, 0x4f, 0x64, 0x6f, 0x61, 0x35, 0x70, 0xea, 0xe2, 0x01, 0x7d, 0xc1, 0x07, 0xc3, 0x8a, + 0x7b, 0xce, 0x42, 0x71, 0x6d, 0x42, 0x7a, 0x47, 0x19, 0x95, 0xaf, 0x3c, 0xcd, 0xf7, 0x55, 0xbd, + 0x63, 0xa6, 0x85, 0x62, 0x71, 0x3d, 0x95, 0xbe, 0x52, 0x2c, 0xad, 0xa7, 0xd2, 0x57, 0x8b, 0x0b, + 0xeb, 0xa9, 0xf4, 0x5b, 0xc5, 0xb7, 0xc5, 0x9f, 0x5e, 0x85, 0x9c, 0x8b, 0xfc, 0x18, 0x22, 0x7a, + 0x18, 0x44, 0x44, 0xd7, 0x86, 0x21, 0x22, 0x8e, 0x15, 0x39, 0x24, 0x7a, 0x18, 0x84, 0x44, 0xd7, + 0x86, 0x41, 0x22, 0x9f, 0x87, 0x60, 0xa2, 0xe6, 0x30, 0x4c, 0x74, 0x6f, 0x0c, 0x4c, 0xe4, 0x89, + 0xea, 0x07, 0x45, 0x6b, 0x83, 0xa0, 0xe8, 0xe6, 0x68, 0x50, 0xe4, 0x89, 0x0a, 0xa0, 0xa2, 0xc7, + 0x7d, 0xa8, 0xe8, 0xfa, 0x08, 0x54, 0xe4, 0xf1, 0xbb, 0xb0, 0x68, 0x23, 0x12, 0x16, 0xdd, 0x3e, + 0x0f, 0x16, 0x79, 0x72, 0x42, 0xb8, 0xe8, 0xfd, 0x10, 0x2e, 0x5a, 0x1c, 0x8a, 0x8b, 0x3c, 0x6e, + 0x06, 0x8c, 0x3e, 0xee, 0x07, 0x46, 0xd7, 0x47, 0x00, 0x23, 0xbf, 0x06, 0x1c, 0x19, 0xd5, 0xa2, + 0x90, 0xd1, 0xad, 0x73, 0x90, 0x91, 0x27, 0x25, 0x08, 0x8d, 0x6a, 0x51, 0xd0, 0xe8, 0xd6, 0x39, + 0xd0, 0xa8, 0x4f, 0x12, 0xc3, 0x46, 0x5b, 0xd1, 0xd8, 0xe8, 0xce, 0xb9, 0xd8, 0xc8, 0x93, 0x16, + 0x06, 0x47, 0x2b, 0x01, 0x70, 0xf4, 0xce, 0x10, 0x70, 0xe4, 0xb1, 0x12, 0x74, 0xf4, 0xed, 0x01, + 0x74, 0x24, 0x8e, 0x42, 0x47, 0x1e, 0xaf, 0x07, 0x8f, 0x5e, 0x0c, 0x81, 0x47, 0x77, 0xcf, 0x87, + 0x47, 0x9e, 0xb0, 0x3e, 0x7c, 0xa4, 0x8c, 0xc4, 0x47, 0xef, 0x8e, 0x89, 0x8f, 0x3c, 0xe9, 0x51, + 0x00, 0xe9, 0xc3, 0x30, 0x40, 0x5a, 0x1a, 0x0e, 0x90, 0x3c, 0x31, 0x1c, 0x21, 0x6d, 0x44, 0x22, + 0xa4, 0xdb, 0xe7, 0x21, 0x24, 0x7f, 0x1c, 0x04, 0x21, 0xd2, 0x56, 0x34, 0x44, 0xba, 0x73, 0x2e, + 0x44, 0xf2, 0x9b, 0x3f, 0x84, 0x91, 0x36, 0x22, 0x31, 0xd2, 0xed, 0xf3, 0x30, 0x92, 0x5f, 0xb8, + 0x20, 0x48, 0x7a, 0x35, 0x14, 0x24, 0xdd, 0x1f, 0x07, 0x24, 0x79, 0x42, 0x07, 0x50, 0xd2, 0x67, + 0xc3, 0x51, 0xd2, 0x37, 0x2e, 0x70, 0x63, 0x62, 0x24, 0x4c, 0xfa, 0xf6, 0x00, 0x4c, 0x12, 0x47, + 0xc1, 0x24, 0xbf, 0x3f, 0xbb, 0x38, 0x49, 0x19, 0x89, 0x6a, 0xde, 0x1d, 0x13, 0xd5, 0xf8, 0x9d, + 0x2f, 0x02, 0xd6, 0x54, 0x23, 0x60, 0xcd, 0xcd, 0xd1, 0xb0, 0xc6, 0x37, 0xe7, 0x3e, 0xae, 0xa9, + 0x45, 0xe1, 0x9a, 0x5b, 0xe7, 0xe0, 0x1a, 0xdf, 0x0a, 0x05, 0x80, 0xcd, 0xe3, 0x3e, 0x60, 0x73, + 0xfd, 0xdc, 0xb0, 0xa1, 0x00, 0xb2, 0x29, 0x0f, 0x22, 0x9b, 0x1b, 0x23, 0x91, 0x8d, 0x27, 0xc1, + 0x87, 0x36, 0x8f, 0xfb, 0xa0, 0xcd, 0xf5, 0x11, 0xd0, 0xc6, 0x2f, 0x00, 0xc7, 0x36, 0xda, 0x68, + 0x6c, 0xb3, 0x3c, 0x2e, 0xb6, 0xf1, 0x04, 0x47, 0x82, 0x9b, 0xad, 0x68, 0x70, 0x73, 0x67, 0xcc, + 0x1d, 0xfd, 0x01, 0x74, 0x53, 0x8b, 0x42, 0x37, 0xb7, 0xce, 0x41, 0x37, 0xc1, 0x39, 0xc4, 0x83, + 0x37, 0xb5, 0x28, 0x78, 0x73, 0xeb, 0x1c, 0x78, 0xe3, 0x4b, 0x0a, 0xe0, 0x9b, 0xe6, 0x30, 0x7c, + 0x73, 0x6f, 0x0c, 0x7c, 0xe3, 0x3b, 0x2f, 0x7d, 0x00, 0xe7, 0x93, 0x7e, 0x80, 0x23, 0x8e, 0x02, + 0x38, 0xfe, 0x88, 0x74, 0x11, 0xce, 0x56, 0x34, 0xc2, 0xb9, 0x73, 0x2e, 0xc2, 0x09, 0x1a, 0xc9, + 0x00, 0xc4, 0xd9, 0x88, 0x84, 0x38, 0xb7, 0xcf, 0x83, 0x38, 0xbe, 0x91, 0x0c, 0x62, 0x9c, 0x4f, + 0xfa, 0x31, 0x8e, 0x38, 0x0a, 0xe3, 0xf8, 0x95, 0x73, 0x41, 0x4e, 0x2d, 0x0a, 0xe4, 0xdc, 0x3a, + 0x07, 0xe4, 0xf8, 0x8d, 0x17, 0x40, 0x39, 0xca, 0x48, 0x94, 0xf3, 0xee, 0x98, 0x28, 0xa7, 0xcf, + 0x70, 0x85, 0x61, 0x4e, 0x2d, 0x0a, 0xe6, 0xdc, 0x3a, 0x07, 0xe6, 0x04, 0x0a, 0xeb, 0xe3, 0x9c, + 0xad, 0x68, 0x9c, 0x73, 0xe7, 0x5c, 0x9c, 0xd3, 0x37, 0x9a, 0x5c, 0xa0, 0xb3, 0x11, 0x09, 0x74, + 0x6e, 0x9f, 0x07, 0x74, 0xfa, 0x26, 0x3e, 0xee, 0x1c, 0xfc, 0xf2, 0xf8, 0x48, 0xe7, 0xc3, 0x8b, + 0x23, 0x1d, 0xef, 0x9d, 0xb1, 0x40, 0x9d, 0xf5, 0x54, 0xfa, 0xed, 0xe2, 0x3b, 0xe2, 0xdf, 0x9a, + 0x86, 0xa9, 0x9a, 0x17, 0x28, 0xe3, 0x97, 0x52, 0x78, 0x93, 0x9b, 0x99, 0xd0, 0x1a, 0x19, 0xb1, + 0xd4, 0xee, 0x9d, 0x7f, 0xd9, 0xde, 0xe0, 0x75, 0x70, 0x9c, 0xf5, 0x0d, 0x8e, 0x48, 0xa3, 0xf7, + 0x21, 0xd7, 0xb3, 0xb1, 0x25, 0x77, 0x2d, 0xdd, 0xb4, 0x74, 0x87, 0x1d, 0x03, 0x11, 0xca, 0xc5, + 0x2f, 0x4e, 0x17, 0x67, 0x76, 0x6c, 0x6c, 0x6d, 0x73, 0xba, 0x34, 0xd3, 0x0b, 0x3c, 0xb9, 0x9f, + 0xb0, 0x9a, 0x1c, 0xff, 0x13, 0x56, 0x2f, 0xa0, 0x68, 0x61, 0x45, 0x0b, 0x79, 0x20, 0xec, 0x16, + 0xa4, 0xe8, 0x3e, 0x43, 0x8f, 0x69, 0xb9, 0x39, 0xe9, 0x6d, 0x48, 0x05, 0x2b, 0x4c, 0x44, 0x0f, + 0xe0, 0x52, 0x47, 0x39, 0xa6, 0x41, 0x95, 0xb2, 0xeb, 0xd4, 0xd1, 0x40, 0x49, 0xf6, 0x75, 0x28, + 0xd4, 0x51, 0x8e, 0xe9, 0xf7, 0xb0, 0x58, 0x12, 0xfd, 0x98, 0xc5, 0x2d, 0xc8, 0x6b, 0xba, 0xed, + 0xe8, 0x86, 0xea, 0xf0, 0xdb, 0x6f, 0xd9, 0xcd, 0xb1, 0x39, 0x97, 0xca, 0xae, 0xb8, 0xbd, 0x0f, + 0xb3, 0x3c, 0xda, 0x3e, 0xb0, 0x45, 0xc8, 0x6f, 0x90, 0x65, 0x09, 0xde, 0xae, 0x20, 0xaa, 0x40, + 0xa1, 0xa5, 0x38, 0xf8, 0x48, 0x39, 0x91, 0xdd, 0xb3, 0x5c, 0x59, 0x7a, 0x29, 0xe4, 0x5b, 0x67, + 0xa7, 0x8b, 0xb9, 0xa7, 0x2c, 0x69, 0xe0, 0x48, 0x57, 0xae, 0x15, 0x48, 0xd0, 0xd0, 0x1d, 0x28, + 0x28, 0xf6, 0x89, 0xa1, 0x52, 0xf5, 0x60, 0xc3, 0xee, 0xd9, 0x14, 0x52, 0xa4, 0xa5, 0x3c, 0x25, + 0x57, 0x5c, 0x2a, 0xba, 0x0e, 0x33, 0x3c, 0x14, 0x9d, 0x7d, 0x60, 0xa7, 0x40, 0xab, 0xca, 0xbf, + 0xdc, 0xc0, 0xbe, 0xb1, 0xf3, 0x18, 0x16, 0xf8, 0xed, 0xf6, 0x47, 0x8a, 0xa5, 0xc9, 0x54, 0xeb, + 0x7e, 0xff, 0x2c, 0x52, 0xb1, 0x57, 0xd8, 0x6d, 0xf6, 0x24, 0x03, 0x51, 0xb5, 0x7f, 0x17, 0xc3, + 0x16, 0xcc, 0xaa, 0x6d, 0xdd, 0x43, 0x00, 0xac, 0xe6, 0xb3, 0x43, 0xed, 0x6c, 0x85, 0xe6, 0xf5, + 0xb7, 0x48, 0x0b, 0x6a, 0x98, 0x80, 0x1a, 0x40, 0xef, 0x9b, 0x91, 0xbb, 0x66, 0x5b, 0x57, 0x4f, + 0xa8, 0xf3, 0x1f, 0xbe, 0xa5, 0x7b, 0xe4, 0x5d, 0xf9, 0xaf, 0x14, 0xdd, 0xd9, 0xa6, 0x9c, 0x12, + 0x1c, 0x79, 0xbf, 0xd9, 0x0d, 0xbb, 0xeb, 0xa9, 0xf4, 0x4c, 0x31, 0xb7, 0x9e, 0x4a, 0xe7, 0x8b, + 0x05, 0xf1, 0x6f, 0x0b, 0x50, 0xe8, 0x2b, 0x0b, 0xaa, 0xc1, 0x25, 0xcd, 0x1b, 0x2a, 0x32, 0x3f, + 0xab, 0xa4, 0x9b, 0x06, 0xbf, 0x78, 0x7c, 0xee, 0x8b, 0xd3, 0xc5, 0x02, 0xcd, 0xfd, 0xd4, 0x4b, + 0x92, 0xe6, 0x7d, 0x0e, 0x9f, 0x8a, 0x3e, 0x84, 0x3c, 0x73, 0x1f, 0xbd, 0x6f, 0xc6, 0xd1, 0x20, + 0xf3, 0xf2, 0xec, 0x17, 0xa7, 0x8b, 0x39, 0xea, 0x33, 0xba, 0x17, 0x03, 0x4b, 0xb9, 0x76, 0xf0, + 0x51, 0xfc, 0x0d, 0x01, 0x66, 0x42, 0xe7, 0x81, 0x1e, 0xf7, 0xed, 0xa0, 0x5f, 0x8d, 0xc6, 0x9d, + 0xc3, 0x22, 0xf2, 0xd2, 0xbc, 0x9f, 0xbb, 0xe1, 0x8d, 0x8b, 0xc3, 0x71, 0x0b, 0x5d, 0x85, 0x71, + 0x63, 0x3d, 0x5c, 0xb6, 0x8f, 0x52, 0x3f, 0xf8, 0xe1, 0xe2, 0x84, 0xf8, 0x17, 0x29, 0xc8, 0x85, + 0x4f, 0xff, 0xd4, 0xfb, 0xca, 0x15, 0x35, 0x2f, 0x84, 0x38, 0x96, 0x47, 0x5c, 0x9c, 0x98, 0xf1, + 0x6f, 0xf8, 0x67, 0xc5, 0x5c, 0x1a, 0x11, 0x27, 0x10, 0x2c, 0xa7, 0xcf, 0xb8, 0xf0, 0x1f, 0x92, + 0x9e, 0x7d, 0x5d, 0x86, 0x49, 0x7a, 0x91, 0x0f, 0x2f, 0x5a, 0xa9, 0xbf, 0xf7, 0x10, 0x5f, 0x99, + 0xa4, 0x4b, 0x2c, 0x1b, 0xb1, 0xc7, 0xcd, 0x37, 0xba, 0x29, 0xcf, 0x1f, 0x06, 0x17, 0xff, 0x44, + 0x5f, 0x8f, 0xdd, 0x94, 0xf8, 0xff, 0x30, 0x1e, 0x84, 0xbc, 0x0f, 0xfd, 0x22, 0x14, 0x54, 0xb3, + 0xdd, 0x66, 0x73, 0x25, 0xb3, 0x68, 0x83, 0x77, 0xa7, 0xd0, 0x22, 0xf0, 0xaf, 0x32, 0x2e, 0x7b, + 0x5f, 0x67, 0x5c, 0x96, 0xf8, 0xd7, 0x19, 0x03, 0xc1, 0xa6, 0x79, 0x4f, 0x18, 0x33, 0x84, 0x7d, + 0x71, 0xaf, 0xd3, 0x6f, 0x12, 0xf7, 0xca, 0x62, 0xa5, 0x79, 0xcf, 0xfb, 0x13, 0x81, 0x07, 0x96, + 0x3c, 0x33, 0xcd, 0x83, 0x9e, 0x17, 0xaf, 0xba, 0x10, 0xbc, 0xf7, 0x30, 0xfd, 0xc5, 0xe9, 0x62, + 0x4a, 0xf2, 0x2e, 0x3e, 0x8c, 0x9a, 0x39, 0x12, 0x5f, 0x6e, 0xe6, 0xb8, 0x0e, 0x33, 0x5d, 0x0b, + 0xef, 0x61, 0x47, 0xdd, 0x97, 0x8d, 0x5e, 0x87, 0x1f, 0x6b, 0xc9, 0xba, 0xb4, 0xad, 0x5e, 0x07, + 0xdd, 0x83, 0xa2, 0x97, 0x85, 0x63, 0x74, 0xf7, 0x62, 0x2c, 0x97, 0xce, 0x11, 0xbd, 0xf8, 0xbf, + 0x05, 0x98, 0x0b, 0xd5, 0x89, 0x8f, 0xa9, 0x75, 0xc8, 0xfa, 0xe6, 0xc4, 0x2e, 0x09, 0x17, 0x8c, + 0xde, 0x0c, 0x32, 0x23, 0x19, 0x2e, 0xbb, 0xaf, 0xa5, 0xb7, 0xe2, 0xfb, 0x62, 0x13, 0x17, 0x14, + 0x7b, 0xc9, 0x97, 0xb3, 0x16, 0x78, 0x81, 0x37, 0xc8, 0x92, 0x63, 0x0d, 0x32, 0xf1, 0xb7, 0x05, + 0x28, 0xd2, 0x17, 0x3c, 0xc1, 0x58, 0x8b, 0xc5, 0xba, 0xb9, 0x51, 0xd1, 0x89, 0xf1, 0x0f, 0xae, + 0x84, 0xbe, 0xe4, 0x91, 0x0c, 0x7f, 0xc9, 0x43, 0xfc, 0xa1, 0x00, 0x79, 0xaf, 0x84, 0xec, 0x3b, + 0x77, 0x23, 0xae, 0xd7, 0x7c, 0xb3, 0x6f, 0xb9, 0xb9, 0xd7, 0x80, 0x8c, 0xf5, 0xe9, 0xbd, 0xe0, + 0x35, 0x20, 0xec, 0x1b, 0x64, 0x7f, 0xd7, 0xed, 0x39, 0xa4, 0x88, 0x15, 0xff, 0xba, 0x87, 0x37, + 0x38, 0xc3, 0x23, 0xd1, 0x4f, 0x84, 0x9a, 0xed, 0x43, 0x76, 0x73, 0xcb, 0x58, 0x66, 0x0f, 0xf1, + 0x70, 0x2a, 0xe0, 0xab, 0x79, 0x5a, 0xb3, 0x41, 0x3f, 0x1e, 0xca, 0x7e, 0xdb, 0xe2, 0x93, 0x80, + 0x02, 0x69, 0xe3, 0x13, 0x2d, 0x8d, 0x65, 0x8a, 0x5d, 0x2d, 0xb1, 0xbe, 0xf2, 0xc7, 0xc1, 0x96, + 0xa8, 0x1e, 0x12, 0x14, 0xf7, 0x08, 0x92, 0x87, 0x4a, 0x7b, 0x54, 0x18, 0x59, 0xa8, 0xe5, 0x24, + 0x92, 0x1b, 0x3d, 0x09, 0xdd, 0x92, 0x91, 0x18, 0x8e, 0x38, 0x06, 0x55, 0x1a, 0xba, 0x4d, 0xe3, + 0x83, 0x70, 0x5f, 0x1f, 0xf9, 0xfa, 0x60, 0xa7, 0xff, 0x28, 0xf5, 0xa3, 0x1f, 0x2e, 0x0a, 0xe2, + 0xc7, 0x80, 0x24, 0x6c, 0x63, 0xe7, 0x45, 0xcf, 0xb4, 0xfc, 0x1b, 0x47, 0x6e, 0xf7, 0x7d, 0xda, + 0x64, 0xb2, 0x9c, 0x3d, 0x8b, 0xba, 0x94, 0xe8, 0x12, 0xcc, 0x85, 0xb8, 0x99, 0xb1, 0x10, 0x3f, + 0x80, 0xab, 0x4f, 0x4d, 0xdb, 0xd6, 0xbb, 0x04, 0xba, 0xd2, 0x51, 0x49, 0xa6, 0x16, 0xcf, 0x3c, + 0xa6, 0xbb, 0x74, 0x11, 0xc3, 0x60, 0x66, 0x24, 0x23, 0x79, 0xcf, 0xe2, 0x1f, 0x08, 0x70, 0x65, + 0x90, 0x93, 0x69, 0x39, 0xea, 0xc8, 0xe1, 0xb4, 0x6a, 0xfa, 0x37, 0xd4, 0x9d, 0xdf, 0x5b, 0xdd, + 0xec, 0xc4, 0x45, 0xe5, 0xef, 0x94, 0x3b, 0x0a, 0x35, 0x1f, 0xfc, 0xf8, 0x73, 0x9e, 0x93, 0x37, + 0x19, 0xd5, 0xb7, 0x24, 0xa9, 0xf1, 0x2c, 0x49, 0x13, 0x0a, 0xeb, 0xa6, 0x6e, 0x10, 0x4f, 0xd8, + 0xad, 0xef, 0x2a, 0xe4, 0x77, 0x75, 0x43, 0xb1, 0x4e, 0x64, 0xf7, 0x54, 0x39, 0xeb, 0x28, 0x0b, + 0x51, 0x85, 0x65, 0x39, 0xa4, 0x1c, 0xe3, 0xe0, 0x8f, 0xe2, 0x8f, 0x05, 0x28, 0xfa, 0x62, 0xb9, + 0x45, 0xfe, 0x26, 0x80, 0xda, 0xee, 0xd9, 0x0e, 0xb6, 0xdc, 0x56, 0x9a, 0x61, 0xe1, 0xe3, 0x15, + 0x46, 0xad, 0xaf, 0x49, 0x19, 0x9e, 0xa1, 0xae, 0xa1, 0x1b, 0xe1, 0xdb, 0x19, 0x26, 0xcb, 0x70, + 0x36, 0x70, 0x27, 0x03, 0x69, 0x76, 0xdb, 0x31, 0x2d, 0x0f, 0x73, 0xf1, 0x66, 0x77, 0x2f, 0x83, + 0xa7, 0x87, 0x8a, 0x49, 0xbe, 0x55, 0xc8, 0x13, 0x77, 0xe1, 0x10, 0x7b, 0x55, 0x4a, 0x9d, 0x5f, + 0x25, 0xc6, 0xe1, 0x56, 0xe9, 0xf7, 0x88, 0x93, 0xcb, 0x5a, 0xc3, 0x6b, 0xe1, 0x11, 0x16, 0xed, + 0x67, 0x82, 0x97, 0x2a, 0x8e, 0x77, 0xed, 0x24, 0xf5, 0x64, 0xbe, 0x0d, 0x69, 0xf7, 0xe3, 0xcb, + 0x7c, 0x80, 0x5c, 0x5d, 0x66, 0x5f, 0x67, 0x5e, 0x76, 0xbf, 0xce, 0xbc, 0xbc, 0xc6, 0x33, 0x30, + 0x33, 0xfe, 0x83, 0xff, 0xba, 0x28, 0x48, 0x1e, 0xd3, 0xfd, 0x06, 0xe9, 0xe1, 0x03, 0xb3, 0x30, + 0xca, 0x03, 0x04, 0xbe, 0x82, 0xc3, 0xbf, 0xc7, 0xbb, 0xba, 0x26, 0xef, 0x6c, 0x55, 0x9e, 0x6f, + 0x6e, 0xd6, 0x9b, 0xcd, 0xea, 0x5a, 0x51, 0x40, 0x45, 0x98, 0x09, 0x7d, 0x43, 0x27, 0xc1, 0xbe, + 0xd0, 0x7b, 0xff, 0x67, 0x00, 0xfc, 0xcf, 0x71, 0x11, 0x59, 0x1b, 0xd5, 0x4f, 0xe5, 0x97, 0xab, + 0xcf, 0x76, 0xaa, 0x8d, 0xe2, 0x04, 0x42, 0x90, 0x2f, 0xaf, 0x36, 0x2b, 0x35, 0x59, 0xaa, 0x36, + 0xb6, 0x9f, 0x6f, 0x35, 0xaa, 0xee, 0x97, 0x7d, 0xef, 0xaf, 0xc1, 0x4c, 0xf0, 0xbe, 0x1d, 0x34, + 0x07, 0x85, 0x4a, 0xad, 0x5a, 0xd9, 0x90, 0x5f, 0xd6, 0x57, 0xe5, 0x17, 0x3b, 0xd5, 0x9d, 0x6a, + 0x71, 0x82, 0x16, 0x8d, 0x12, 0x9f, 0xec, 0x3c, 0x7b, 0x56, 0x14, 0x50, 0x01, 0xb2, 0xec, 0x99, + 0x7e, 0x6f, 0xa7, 0x98, 0xb8, 0xbf, 0x09, 0xd9, 0xc0, 0xc5, 0xbc, 0xe4, 0x75, 0xdb, 0x3b, 0x8d, + 0x9a, 0xdc, 0xac, 0x6f, 0x56, 0x1b, 0xcd, 0xd5, 0xcd, 0x6d, 0x26, 0x83, 0xd2, 0x56, 0xcb, 0xcf, + 0xa5, 0x66, 0x51, 0xf0, 0x9e, 0x9b, 0xcf, 0x77, 0x2a, 0x35, 0xb7, 0x1a, 0x62, 0x2a, 0x9d, 0x2c, + 0x26, 0xef, 0xff, 0x75, 0x01, 0xae, 0x0c, 0xb9, 0x7b, 0x06, 0x65, 0x61, 0x7a, 0xc7, 0xa0, 0xb7, + 0x94, 0x16, 0x27, 0x50, 0x2e, 0x70, 0xfd, 0x4c, 0x51, 0x40, 0x69, 0x76, 0x01, 0x48, 0x31, 0x81, + 0xa6, 0x20, 0xd1, 0x78, 0x54, 0x4c, 0x92, 0x92, 0x06, 0x6e, 0x6f, 0x29, 0xa6, 0x50, 0x86, 0xdf, + 0x1b, 0x51, 0x9c, 0x44, 0x33, 0xfe, 0xf5, 0x0d, 0xc5, 0x29, 0x22, 0xca, 0xbb, 0x06, 0xa1, 0x38, + 0x7d, 0xff, 0x3a, 0x04, 0x8e, 0x9a, 0x23, 0x80, 0xa9, 0x67, 0x8a, 0x83, 0x6d, 0xa7, 0x38, 0x81, + 0xa6, 0x21, 0xb9, 0xda, 0x6e, 0x17, 0x85, 0x87, 0xff, 0x2a, 0x05, 0x69, 0xf7, 0xbb, 0x32, 0xe8, + 0x19, 0x4c, 0xb2, 0xa5, 0xeb, 0xc5, 0xe1, 0xc8, 0x80, 0x0e, 0xde, 0x85, 0xa5, 0xf3, 0xa0, 0x83, + 0x38, 0x81, 0xfe, 0x2a, 0x64, 0x03, 0x1e, 0x13, 0x1a, 0xba, 0xfc, 0x16, 0xf2, 0x12, 0x17, 0x6e, + 0x9f, 0x97, 0xcd, 0x93, 0xff, 0x0a, 0x32, 0x9e, 0x05, 0x47, 0x37, 0x46, 0xd9, 0x77, 0x57, 0xf6, + 0xe8, 0x49, 0x80, 0x8c, 0x35, 0x71, 0xe2, 0x3d, 0x01, 0x59, 0x80, 0x06, 0x8d, 0x2d, 0x8a, 0x8a, + 0x68, 0x18, 0x6a, 0xcd, 0x17, 0xee, 0x8f, 0x95, 0xdb, 0x7f, 0x27, 0x51, 0x96, 0x3f, 0x63, 0x44, + 0x2b, 0x6b, 0x60, 0x3e, 0x8a, 0x56, 0x56, 0xc4, 0xc4, 0x33, 0x81, 0x5e, 0x40, 0x8a, 0x58, 0x4a, + 0x14, 0xe5, 0x43, 0xf6, 0x59, 0xe6, 0x85, 0x1b, 0x23, 0xf3, 0xb8, 0x22, 0xcb, 0xf7, 0x7e, 0xf4, + 0xe7, 0xd7, 0x26, 0x7e, 0x74, 0x76, 0x4d, 0xf8, 0xf1, 0xd9, 0x35, 0xe1, 0x4f, 0xcf, 0xae, 0x09, + 0x7f, 0x76, 0x76, 0x4d, 0xf8, 0xf5, 0x9f, 0x5c, 0x9b, 0xf8, 0xf1, 0x4f, 0xae, 0x4d, 0xfc, 0xe9, + 0x4f, 0xae, 0x4d, 0x7c, 0x36, 0xcd, 0xb9, 0x77, 0xa7, 0xa8, 0x51, 0x79, 0xf4, 0x7f, 0x03, 0x00, + 0x00, 0xff, 0xff, 0xa1, 0x0e, 0xed, 0x04, 0x2d, 0x7f, 0x00, 0x00, } diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index f09a518503d2..95afcd8f3348 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -836,6 +836,9 @@ message AdminRelocateRangeResponse { // gossip protocol. message HeartbeatTxnRequest { RequestHeader header = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true]; + // NOTE: this could use a ClockTimestamp type, but doing so results in a + // large diff that doesn't seem worth it, given that we never feed this + // timestamp back into a clock. util.hlc.Timestamp now = 2 [(gogoproto.nullable) = false]; } @@ -1141,7 +1144,8 @@ message RequestLeaseRequest { // The MinLeaseProposedTS of the proposing replica to make sure that leases // issued after a node restart receive a new sequence number (instead of // counting as a lease extension). See #23204. - util.hlc.Timestamp min_proposed_ts = 4 [(gogoproto.customname) = "MinProposedTS"]; + util.hlc.Timestamp min_proposed_ts = 4 [(gogoproto.customname) = "MinProposedTS", + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/util/hlc.ClockTimestamp"]; } // A TransferLeaseRequest represents the arguments to the TransferLease() @@ -1645,7 +1649,8 @@ message SubsumeResponse { // before it stopped responding to requests altogether (in anticipation of // being subsumed). It is suitable for use as the timestamp cache's low water // mark for the keys previously owned by the subsumed range. - util.hlc.Timestamp freeze_start = 5 [(gogoproto.nullable) = false]; + util.hlc.Timestamp freeze_start = 5 [(gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/util/hlc.ClockTimestamp"]; } // RangeStatsRequest is the argument to the RangeStats() method. It requests the @@ -2001,7 +2006,8 @@ message BatchResponse { Transaction txn = 3; // now is the highest current time from any node contacted during the request. // It can be used by the receiver to update its local HLC. - util.hlc.Timestamp now = 5 [(gogoproto.nullable) = false]; + util.hlc.Timestamp now = 5 [(gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/util/hlc.ClockTimestamp"]; // collected_spans stores trace spans recorded during the execution of this // request. repeated util.tracing.tracingpb.RecordedSpan collected_spans = 6 [(gogoproto.nullable) = false]; diff --git a/pkg/roachpb/data.go b/pkg/roachpb/data.go index 605e68560118..32a71e4f2230 100644 --- a/pkg/roachpb/data.go +++ b/pkg/roachpb/data.go @@ -906,7 +906,11 @@ func MakeTransaction( // occurred, i.e. the maximum of ReadTimestamp and LastHeartbeat. func (t Transaction) LastActive() hlc.Timestamp { ts := t.LastHeartbeat - ts.Forward(t.ReadTimestamp) + // Only forward by the ReadTimestamp if it is a clock timestamp. + // TODO(nvanbenschoten): replace this with look at the Synthetic bool. + if readTS, ok := t.ReadTimestamp.TryToClockTimestamp(); ok { + ts.Forward(readTS.ToTimestamp()) + } return ts } @@ -1244,7 +1248,7 @@ func (t *Transaction) ResetObservedTimestamps() { // UpdateObservedTimestamp stores a timestamp off a node's clock for future // operations in the transaction. When multiple calls are made for a single // nodeID, the lowest timestamp prevails. -func (t *Transaction) UpdateObservedTimestamp(nodeID NodeID, maxTS hlc.Timestamp) { +func (t *Transaction) UpdateObservedTimestamp(nodeID NodeID, maxTS hlc.ClockTimestamp) { // Fast path optimization for either no observed timestamps or // exactly one, for the same nodeID as we're updating. if l := len(t.ObservedTimestamps); l == 0 { @@ -1264,7 +1268,7 @@ func (t *Transaction) UpdateObservedTimestamp(nodeID NodeID, maxTS hlc.Timestamp // given node's clock during the transaction. The returned boolean is false if // no observation about the requested node was found. Otherwise, MaxTimestamp // can be lowered to the returned timestamp when reading from nodeID. -func (t *Transaction) GetObservedTimestamp(nodeID NodeID) (hlc.Timestamp, bool) { +func (t *Transaction) GetObservedTimestamp(nodeID NodeID) (hlc.ClockTimestamp, bool) { s := observedTimestampSlice(t.ObservedTimestamps) return s.get(nodeID) } @@ -1382,14 +1386,14 @@ func PrepareTransactionForRetry( // Start the new transaction at the current time from the local clock. // The local hlc should have been advanced to at least the error's // timestamp already. - now := clock.Now() + now := clock.NowAsClockTimestamp() txn = MakeTransaction( txn.Name, nil, // baseKey // We have errTxnPri, but this wants a UserPriority. So we're going to // overwrite the priority below. NormalUserPriority, - now, + now.ToTimestamp(), clock.MaxOffset().Nanoseconds(), ) // Use the priority communicated back by the server. @@ -1471,7 +1475,7 @@ func readWithinUncertaintyIntervalRetryTimestamp( // If the reader encountered a newer write within the uncertainty // interval, we advance the txn's timestamp just past the last observed // timestamp from the node. - ts, ok := txn.GetObservedTimestamp(origin) + clockTS, ok := txn.GetObservedTimestamp(origin) if !ok { log.Fatalf(ctx, "missing observed timestamp for node %d found on uncertainty restart. "+ @@ -1479,6 +1483,7 @@ func readWithinUncertaintyIntervalRetryTimestamp( origin, err, txn, txn.ObservedTimestamps) } // Also forward by the existing timestamp. + ts := clockTS.ToTimestamp() ts.Forward(err.ExistingTimestamp.Next()) return ts } @@ -2316,18 +2321,18 @@ func (s observedTimestampSlice) index(nodeID NodeID) int { // get the observed timestamp for the specified node, returning false if no // timestamp exists. -func (s observedTimestampSlice) get(nodeID NodeID) (hlc.Timestamp, bool) { +func (s observedTimestampSlice) get(nodeID NodeID) (hlc.ClockTimestamp, bool) { i := s.index(nodeID) if i < len(s) && s[i].NodeID == nodeID { return s[i].Timestamp, true } - return hlc.Timestamp{}, false + return hlc.ClockTimestamp{}, false } // update the timestamp for the specified node, or add a new entry in the // correct (sorted) location. The receiver is not mutated. func (s observedTimestampSlice) update( - nodeID NodeID, timestamp hlc.Timestamp, + nodeID NodeID, timestamp hlc.ClockTimestamp, ) observedTimestampSlice { i := s.index(nodeID) if i < len(s) && s[i].NodeID == nodeID { diff --git a/pkg/roachpb/data.pb.go b/pkg/roachpb/data.pb.go index 113726cb254e..5fb6152c2031 100644 --- a/pkg/roachpb/data.pb.go +++ b/pkg/roachpb/data.pb.go @@ -11,6 +11,7 @@ import enginepb "github.com/cockroachdb/cockroach/pkg/storage/enginepb" import hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" import github_com_cockroachdb_cockroach_pkg_util_uuid "github.com/cockroachdb/cockroach/pkg/util/uuid" +import github_com_cockroachdb_cockroach_pkg_util_hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" import github_com_cockroachdb_cockroach_pkg_storage_enginepb "github.com/cockroachdb/cockroach/pkg/storage/enginepb" import bytes "bytes" @@ -97,7 +98,7 @@ func (x ValueType) String() string { return proto.EnumName(ValueType_name, int32(x)) } func (ValueType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_data_11933f4ce39aae12, []int{0} + return fileDescriptor_data_7c13a0dbc42fb34d, []int{0} } // ReplicaChangeType is a parameter of ChangeReplicasTrigger. @@ -127,7 +128,7 @@ func (x ReplicaChangeType) String() string { return proto.EnumName(ReplicaChangeType_name, int32(x)) } func (ReplicaChangeType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_data_11933f4ce39aae12, []int{1} + return fileDescriptor_data_7c13a0dbc42fb34d, []int{1} } // TransactionStatus specifies possible states for a transaction. @@ -179,7 +180,7 @@ func (x TransactionStatus) String() string { return proto.EnumName(TransactionStatus_name, int32(x)) } func (TransactionStatus) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_data_11933f4ce39aae12, []int{2} + return fileDescriptor_data_7c13a0dbc42fb34d, []int{2} } // Span is a key range with an inclusive start Key and an exclusive end Key. @@ -196,7 +197,7 @@ type Span struct { func (m *Span) Reset() { *m = Span{} } func (*Span) ProtoMessage() {} func (*Span) Descriptor() ([]byte, []int) { - return fileDescriptor_data_11933f4ce39aae12, []int{0} + return fileDescriptor_data_7c13a0dbc42fb34d, []int{0} } func (m *Span) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -248,7 +249,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_11933f4ce39aae12, []int{1} + return fileDescriptor_data_7c13a0dbc42fb34d, []int{1} } func (m *Value) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -284,7 +285,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_11933f4ce39aae12, []int{2} + return fileDescriptor_data_7c13a0dbc42fb34d, []int{2} } func (m *KeyValue) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -322,7 +323,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_11933f4ce39aae12, []int{3} + return fileDescriptor_data_7c13a0dbc42fb34d, []int{3} } func (m *StoreIdent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -362,7 +363,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_11933f4ce39aae12, []int{4} + return fileDescriptor_data_7c13a0dbc42fb34d, []int{4} } func (m *SplitTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -403,14 +404,14 @@ type MergeTrigger struct { // before it stopped responding to requests altogether (in anticipation of // being subsumed). It is suitable for use as the timestamp cache's low water // mark for the keys previously owned by the subsumed range. - FreezeStart hlc.Timestamp `protobuf:"bytes,5,opt,name=freeze_start,json=freezeStart,proto3" json:"freeze_start"` + FreezeStart github_com_cockroachdb_cockroach_pkg_util_hlc.ClockTimestamp `protobuf:"bytes,5,opt,name=freeze_start,json=freezeStart,proto3,casttype=github.com/cockroachdb/cockroach/pkg/util/hlc.ClockTimestamp" json:"freeze_start"` } 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_11933f4ce39aae12, []int{5} + return fileDescriptor_data_7c13a0dbc42fb34d, []int{5} } func (m *MergeTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -480,7 +481,7 @@ type ChangeReplicasTrigger struct { func (m *ChangeReplicasTrigger) Reset() { *m = ChangeReplicasTrigger{} } func (*ChangeReplicasTrigger) ProtoMessage() {} func (*ChangeReplicasTrigger) Descriptor() ([]byte, []int) { - return fileDescriptor_data_11933f4ce39aae12, []int{6} + return fileDescriptor_data_7c13a0dbc42fb34d, []int{6} } func (m *ChangeReplicasTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -522,7 +523,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_11933f4ce39aae12, []int{7} + return fileDescriptor_data_7c13a0dbc42fb34d, []int{7} } func (m *ModifiedSpanTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -568,7 +569,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_11933f4ce39aae12, []int{8} + return fileDescriptor_data_7c13a0dbc42fb34d, []int{8} } func (m *StickyBitTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -607,7 +608,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_11933f4ce39aae12, []int{9} + return fileDescriptor_data_7c13a0dbc42fb34d, []int{9} } func (m *InternalCommitTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -668,15 +669,15 @@ func (m *InternalCommitTrigger) GetStickyBitTrigger() *StickyBitTrigger { } type ObservedTimestamp struct { - NodeID NodeID `protobuf:"varint,1,opt,name=node_id,json=nodeId,proto3,casttype=NodeID" json:"node_id,omitempty"` - Timestamp hlc.Timestamp `protobuf:"bytes,2,opt,name=timestamp,proto3" json:"timestamp"` + NodeID NodeID `protobuf:"varint,1,opt,name=node_id,json=nodeId,proto3,casttype=NodeID" json:"node_id,omitempty"` + Timestamp github_com_cockroachdb_cockroach_pkg_util_hlc.ClockTimestamp `protobuf:"bytes,2,opt,name=timestamp,proto3,casttype=github.com/cockroachdb/cockroach/pkg/util/hlc.ClockTimestamp" json:"timestamp"` } 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_11933f4ce39aae12, []int{10} + return fileDescriptor_data_7c13a0dbc42fb34d, []int{10} } func (m *ObservedTimestamp) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -720,6 +721,10 @@ type Transaction struct { // The last time that the transaction's record was sent a heartbeat by its // coordinator to indicate client activity. Concurrent transactions will // avoid aborting a transaction if it observes recent-enough activity. + // + // NOTE: this could use a ClockTimestamp type, but doing so results in a + // large diff that doesn't seem worth it, given that we never feed this + // timestamp back into a clock. LastHeartbeat hlc.Timestamp `protobuf:"bytes,5,opt,name=last_heartbeat,json=lastHeartbeat,proto3" json:"last_heartbeat"` // This flag is set if the transaction's timestamp was "leaked" beyond the // transaction (e.g. via cluster_logical_timestamp()). If true, this prevents @@ -829,7 +834,7 @@ type Transaction struct { func (m *Transaction) Reset() { *m = Transaction{} } func (*Transaction) ProtoMessage() {} func (*Transaction) Descriptor() ([]byte, []int) { - return fileDescriptor_data_11933f4ce39aae12, []int{11} + return fileDescriptor_data_7c13a0dbc42fb34d, []int{11} } func (m *Transaction) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -880,7 +885,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_11933f4ce39aae12, []int{12} + return fileDescriptor_data_7c13a0dbc42fb34d, []int{12} } func (m *TransactionRecord) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -920,7 +925,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_11933f4ce39aae12, []int{13} + return fileDescriptor_data_7c13a0dbc42fb34d, []int{13} } func (m *Intent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -957,7 +962,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_11933f4ce39aae12, []int{13, 0} + return fileDescriptor_data_7c13a0dbc42fb34d, []int{13, 0} } func (m *Intent_SingleKeySpan) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -994,7 +999,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_11933f4ce39aae12, []int{14} + return fileDescriptor_data_7c13a0dbc42fb34d, []int{14} } func (m *LockAcquisition) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1035,7 +1040,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_11933f4ce39aae12, []int{15} + return fileDescriptor_data_7c13a0dbc42fb34d, []int{15} } func (m *LockUpdate) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1072,7 +1077,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_11933f4ce39aae12, []int{16} + return fileDescriptor_data_7c13a0dbc42fb34d, []int{16} } func (m *SequencedWrite) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1118,7 +1123,7 @@ type Lease struct { // to help with the rollout (such that a lease applied by some nodes before // the rollout and some nodes after the rollout is serialized the same). // TODO(andrei): Make this non-nullable after the rollout. - ProposedTS *hlc.Timestamp `protobuf:"bytes,5,opt,name=proposed_ts,json=proposedTs,proto3" json:"proposed_ts,omitempty"` + ProposedTS *github_com_cockroachdb_cockroach_pkg_util_hlc.ClockTimestamp `protobuf:"bytes,5,opt,name=proposed_ts,json=proposedTs,proto3,casttype=github.com/cockroachdb/cockroach/pkg/util/hlc.ClockTimestamp" json:"proposed_ts,omitempty"` // The epoch of the lease holder's node liveness entry. If this value is // non-zero, the expiration field is ignored. Epoch int64 `protobuf:"varint,6,opt,name=epoch,proto3" json:"epoch,omitempty"` @@ -1137,7 +1142,7 @@ type Lease struct { func (m *Lease) Reset() { *m = Lease{} } func (*Lease) ProtoMessage() {} func (*Lease) Descriptor() ([]byte, []int) { - return fileDescriptor_data_11933f4ce39aae12, []int{17} + return fileDescriptor_data_7c13a0dbc42fb34d, []int{17} } func (m *Lease) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1182,7 +1187,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_11933f4ce39aae12, []int{18} + return fileDescriptor_data_7c13a0dbc42fb34d, []int{18} } func (m *AbortSpanEntry) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1239,7 +1244,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_11933f4ce39aae12, []int{19} + return fileDescriptor_data_7c13a0dbc42fb34d, []int{19} } func (m *LeafTxnInputState) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1292,7 +1297,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_11933f4ce39aae12, []int{20} + return fileDescriptor_data_7c13a0dbc42fb34d, []int{20} } func (m *LeafTxnFinalState) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1327,7 +1332,7 @@ type RangeInfo struct { func (m *RangeInfo) Reset() { *m = RangeInfo{} } func (*RangeInfo) ProtoMessage() {} func (*RangeInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_data_11933f4ce39aae12, []int{21} + return fileDescriptor_data_7c13a0dbc42fb34d, []int{21} } func (m *RangeInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2613,7 +2618,7 @@ func NewPopulatedObservedTimestamp(r randyData, easy bool) *ObservedTimestamp { if r.Intn(2) == 0 { this.NodeID *= -1 } - v3 := hlc.NewPopulatedTimestamp(r, easy) + v3 := github_com_cockroachdb_cockroach_pkg_util_hlc.NewPopulatedClockTimestamp(r, easy) this.Timestamp = *v3 if !easy && r.Intn(10) != 0 { } @@ -2736,7 +2741,7 @@ func NewPopulatedLease(r randyData, easy bool) *Lease { this.DeprecatedStartStasis = hlc.NewPopulatedTimestamp(r, easy) } if r.Intn(10) != 0 { - this.ProposedTS = hlc.NewPopulatedTimestamp(r, easy) + this.ProposedTS = github_com_cockroachdb_cockroach_pkg_util_hlc.NewPopulatedClockTimestamp(r, easy) } this.Epoch = int64(r.Int63()) if r.Intn(2) == 0 { @@ -6175,7 +6180,7 @@ func (m *Lease) Unmarshal(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.ProposedTS == nil { - m.ProposedTS = &hlc.Timestamp{} + m.ProposedTS = &github_com_cockroachdb_cockroach_pkg_util_hlc.ClockTimestamp{} } if err := m.ProposedTS.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err @@ -6905,158 +6910,160 @@ var ( ErrIntOverflowData = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/data.proto", fileDescriptor_data_11933f4ce39aae12) } - -var fileDescriptor_data_11933f4ce39aae12 = []byte{ - // 2397 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xdc, 0x59, 0xdd, 0x6f, 0x1b, 0x59, - 0x15, 0xcf, 0xd8, 0x63, 0x7b, 0x7c, 0xfc, 0x91, 0xc9, 0x6d, 0xd2, 0x7a, 0xb3, 0x10, 0x17, 0x2f, - 0xb0, 0xa5, 0x62, 0x1d, 0x91, 0x5d, 0x56, 0x10, 0x8a, 0x84, 0x1d, 0xbb, 0xad, 0xdd, 0xd8, 0xee, - 0x8e, 0x9d, 0x2e, 0xed, 0x22, 0x0d, 0xe3, 0x99, 0x1b, 0x67, 0x88, 0x3d, 0xe3, 0xcc, 0x1d, 0xa7, - 0x31, 0xef, 0x48, 0xab, 0xe5, 0x01, 0x1e, 0x79, 0xac, 0xc4, 0x1b, 0x3c, 0xc0, 0x23, 0x42, 0xe2, - 0xbd, 0x8f, 0x7d, 0xdb, 0x85, 0x95, 0x2c, 0x48, 0x85, 0xb4, 0x7f, 0x43, 0x25, 0x24, 0x74, 0xef, - 0x9d, 0xaf, 0xa4, 0x6e, 0x48, 0x36, 0x8b, 0x58, 0xf1, 0xe2, 0xcc, 0x9c, 0x7b, 0xce, 0xef, 0xdc, - 0x7b, 0xce, 0xb9, 0xe7, 0x9c, 0x39, 0x01, 0xe4, 0xd8, 0x9a, 0xbe, 0x37, 0xee, 0xaf, 0x1b, 0x9a, - 0xab, 0x95, 0xc7, 0x8e, 0xed, 0xda, 0x68, 0x49, 0xb7, 0xf5, 0x7d, 0x46, 0x2f, 0x7b, 0xab, 0xab, - 0x37, 0xf7, 0x0f, 0xd7, 0xf7, 0x0f, 0x09, 0x76, 0x0e, 0xb1, 0xb3, 0xae, 0xdb, 0x96, 0x3e, 0x71, - 0x1c, 0x6c, 0xe9, 0xd3, 0xf5, 0xa1, 0xad, 0xef, 0xb3, 0x1f, 0xd3, 0x1a, 0x70, 0xf1, 0xd5, 0xab, - 0x3e, 0xe4, 0x08, 0xbb, 0x5a, 0x08, 0xbb, 0xfa, 0x3a, 0x71, 0x6d, 0x47, 0x1b, 0xe0, 0x75, 0x6c, - 0x0d, 0x4c, 0x0b, 0x53, 0x86, 0x43, 0x5d, 0xf7, 0x16, 0xbf, 0x32, 0x77, 0xf1, 0x6d, 0x6f, 0xb5, - 0x30, 0x71, 0xcd, 0xe1, 0xfa, 0xde, 0x50, 0x5f, 0x77, 0xcd, 0x11, 0x26, 0xae, 0x36, 0x1a, 0x7b, - 0x2b, 0xcb, 0x03, 0x7b, 0x60, 0xb3, 0xc7, 0x75, 0xfa, 0xc4, 0xa9, 0xa5, 0x87, 0x20, 0x76, 0xc7, - 0x9a, 0x85, 0x5e, 0x83, 0xf8, 0x3e, 0x9e, 0x16, 0xe2, 0xd7, 0x85, 0x1b, 0xd9, 0x6a, 0xea, 0xc5, - 0xac, 0x18, 0xbf, 0x87, 0xa7, 0x0a, 0xa5, 0xa1, 0xeb, 0x90, 0xc2, 0x96, 0xa1, 0xd2, 0x65, 0xf1, - 0xe4, 0x72, 0x12, 0x5b, 0xc6, 0x3d, 0x3c, 0xdd, 0x94, 0x7e, 0xf3, 0xa4, 0xb8, 0xf0, 0xa7, 0x27, - 0x45, 0xa1, 0x29, 0x4a, 0x82, 0x1c, 0x6b, 0x8a, 0x52, 0x4c, 0x8e, 0x97, 0x06, 0x90, 0x78, 0xa0, - 0x0d, 0x27, 0x18, 0xbd, 0x0e, 0x69, 0x47, 0x7b, 0xac, 0xf6, 0xa7, 0x2e, 0x26, 0x05, 0x81, 0x42, - 0x28, 0x92, 0xa3, 0x3d, 0xae, 0xd2, 0x77, 0x54, 0x81, 0x74, 0xb0, 0xd3, 0x42, 0xec, 0xba, 0x70, - 0x23, 0xb3, 0xf1, 0xd5, 0x72, 0x68, 0x56, 0x7a, 0x9c, 0xf2, 0xde, 0x50, 0x2f, 0xf7, 0x7c, 0xa6, - 0xaa, 0xf8, 0x74, 0x56, 0x5c, 0x50, 0x42, 0xa9, 0xd2, 0x07, 0x20, 0xdd, 0xc3, 0x53, 0xae, 0xcb, - 0x3b, 0x87, 0x30, 0xe7, 0x1c, 0xef, 0x40, 0xe2, 0x90, 0xf2, 0x78, 0x5a, 0x0a, 0xe5, 0x97, 0x9c, - 0x57, 0x66, 0x18, 0x9e, 0x02, 0xce, 0x5c, 0xfa, 0x58, 0x00, 0xe8, 0xba, 0xb6, 0x83, 0x1b, 0x06, - 0xb6, 0x5c, 0x34, 0x00, 0xd0, 0x87, 0x13, 0xe2, 0x62, 0x47, 0x35, 0x0d, 0x4f, 0xcd, 0x5d, 0xca, - 0xff, 0xb7, 0x59, 0xf1, 0xed, 0x81, 0xe9, 0xee, 0x4d, 0xfa, 0x65, 0xdd, 0x1e, 0xad, 0x07, 0xd8, - 0x46, 0x3f, 0x7c, 0x5e, 0x1f, 0xef, 0x0f, 0xd6, 0x99, 0x83, 0x26, 0x13, 0xd3, 0x28, 0xef, 0xec, - 0x34, 0x6a, 0xc7, 0xb3, 0x62, 0x7a, 0x8b, 0x03, 0x36, 0x6a, 0x4a, 0xda, 0xc3, 0x6e, 0x18, 0xe8, - 0x2d, 0x48, 0x59, 0xb6, 0x81, 0xa9, 0x16, 0xba, 0xdf, 0x44, 0x75, 0xf9, 0x78, 0x56, 0x4c, 0xb6, - 0x6d, 0x03, 0x37, 0x6a, 0x2f, 0x82, 0x27, 0x25, 0x49, 0x99, 0x1a, 0x06, 0xfa, 0x0e, 0x48, 0x34, - 0x2e, 0x18, 0x7f, 0x9c, 0xf1, 0x5f, 0x3d, 0x9e, 0x15, 0x53, 0x7c, 0xe7, 0x54, 0xc0, 0x7f, 0x54, - 0x52, 0x84, 0x9f, 0xa6, 0xf4, 0x3b, 0x01, 0xb2, 0xdd, 0xf1, 0xd0, 0x74, 0x7b, 0x8e, 0x39, 0x18, - 0x60, 0x07, 0xd5, 0x21, 0x3d, 0xc4, 0xbb, 0xae, 0x6a, 0x60, 0xa2, 0xb3, 0xa3, 0x65, 0x36, 0x4a, - 0x73, 0x8c, 0xa4, 0x68, 0xd6, 0x00, 0xd7, 0x30, 0xd1, 0x1d, 0x73, 0xec, 0xda, 0x8e, 0x67, 0x2e, - 0x89, 0x8a, 0x52, 0x2a, 0xba, 0x03, 0xe0, 0x98, 0x83, 0x3d, 0x0f, 0x27, 0x76, 0x41, 0x9c, 0x34, - 0x93, 0xa5, 0xe4, 0x4d, 0xf1, 0x33, 0x1e, 0x52, 0x71, 0x59, 0x2c, 0x7d, 0x1a, 0x83, 0x6c, 0x0b, - 0x3b, 0x03, 0xfc, 0x25, 0xdd, 0x2c, 0x1a, 0x80, 0xcc, 0x81, 0xe8, 0x6d, 0x54, 0x89, 0xab, 0xb9, - 0x84, 0x5d, 0x97, 0xcc, 0xc6, 0x37, 0x22, 0x70, 0xde, 0xdd, 0x2d, 0xfb, 0x77, 0xb7, 0xdc, 0x7a, - 0xb0, 0xb5, 0xd5, 0xa5, 0xcc, 0xd5, 0xab, 0x14, 0xf1, 0x78, 0x56, 0xcc, 0x2b, 0x14, 0x26, 0xa0, - 0x2b, 0x79, 0x06, 0xdb, 0x3a, 0xd4, 0x75, 0xf6, 0x8e, 0x6e, 0x43, 0x76, 0xd7, 0xc1, 0xf8, 0xe7, - 0x98, 0x2a, 0x71, 0xdc, 0x42, 0xe2, 0xfc, 0x77, 0x26, 0xc3, 0x05, 0xbb, 0x54, 0xee, 0x84, 0x75, - 0x7f, 0x9f, 0x80, 0x95, 0xad, 0x3d, 0x7a, 0x44, 0x05, 0x8f, 0x87, 0xa6, 0xae, 0x11, 0xdf, 0xcc, - 0x8f, 0xe0, 0xaa, 0x81, 0xc7, 0x0e, 0xd6, 0x35, 0x17, 0x1b, 0xaa, 0xce, 0x78, 0x54, 0x77, 0x3a, - 0xc6, 0xcc, 0xe6, 0xf9, 0x8d, 0xaf, 0xcf, 0xb3, 0x15, 0xc7, 0xe0, 0x80, 0xbd, 0xe9, 0x18, 0x2b, - 0xcb, 0x21, 0x46, 0x48, 0x45, 0x0f, 0x01, 0x45, 0xb0, 0x1d, 0x2e, 0xe5, 0xf9, 0xe0, 0x0c, 0xdc, - 0x97, 0xbc, 0xb0, 0x14, 0xa2, 0x78, 0x2c, 0xe8, 0x67, 0xf0, 0x7a, 0x04, 0x7a, 0x32, 0x36, 0xa2, - 0x2a, 0x48, 0x21, 0x7e, 0x3d, 0x7e, 0x41, 0x1d, 0xaf, 0x85, 0x70, 0x3b, 0x1c, 0xcd, 0xb7, 0x14, - 0xc2, 0xb0, 0x1a, 0xd1, 0x65, 0xe1, 0x23, 0xd7, 0x57, 0x44, 0x2f, 0xa3, 0xc8, 0x2e, 0xe3, 0x8d, - 0xe3, 0x59, 0xf1, 0x5a, 0x2d, 0xe0, 0x6a, 0xe3, 0x23, 0xd7, 0x93, 0x67, 0x97, 0x33, 0x1d, 0xbc, - 0x28, 0xd7, 0x8c, 0xb9, 0x5c, 0x06, 0x7a, 0x17, 0x44, 0x16, 0xa3, 0x89, 0xf3, 0xc6, 0xa8, 0xc2, - 0xf8, 0x51, 0x1f, 0xae, 0x99, 0x96, 0x8b, 0x1d, 0x4b, 0x1b, 0xaa, 0x9a, 0x61, 0x44, 0xcd, 0x90, - 0xbc, 0xb0, 0x19, 0x56, 0x7c, 0xa8, 0x0a, 0x45, 0x0a, 0x4c, 0xb0, 0x0b, 0xaf, 0x05, 0x3a, 0x1c, - 0x3c, 0xb2, 0x0f, 0xa3, 0x5a, 0x52, 0x17, 0xd6, 0x12, 0x6c, 0x58, 0xe1, 0x58, 0xbe, 0x9e, 0x4d, - 0x91, 0x16, 0x9a, 0xd2, 0x47, 0x02, 0x5c, 0x69, 0xd9, 0x86, 0xb9, 0x6b, 0x62, 0x83, 0x16, 0x2f, - 0x3f, 0x56, 0xbf, 0x0d, 0x88, 0x4c, 0x89, 0x8b, 0x47, 0xaa, 0x6e, 0x5b, 0xbb, 0xe6, 0x40, 0x25, - 0x63, 0xcd, 0x62, 0x71, 0x2a, 0x29, 0x32, 0x5f, 0xd9, 0x62, 0x0b, 0xac, 0xe2, 0xd5, 0x01, 0xb1, - 0x04, 0x3b, 0x34, 0x0f, 0xb1, 0x85, 0x09, 0xe1, 0xdc, 0x3c, 0xfa, 0xae, 0xcd, 0xd9, 0x2c, 0x15, - 0x52, 0x64, 0x2a, 0xb2, 0xed, 0x49, 0x50, 0x4a, 0xe9, 0x01, 0xc8, 0x5d, 0xd7, 0xd4, 0xf7, 0xa7, - 0xd5, 0x30, 0x91, 0x56, 0x01, 0x08, 0xa3, 0xa9, 0x7d, 0xd3, 0xf5, 0x92, 0xd3, 0xf9, 0x8a, 0x1a, - 0xf1, 0xa1, 0x4a, 0x7f, 0x8e, 0xc3, 0x4a, 0xc3, 0x33, 0xc3, 0x96, 0x3d, 0x1a, 0x85, 0xe8, 0x35, - 0xc8, 0x11, 0x9a, 0xb6, 0x55, 0x97, 0x13, 0x3c, 0x05, 0xc5, 0xb9, 0x7b, 0x0e, 0xd3, 0xbb, 0x92, - 0x25, 0xd1, 0x64, 0x5f, 0x83, 0xdc, 0x88, 0xe6, 0xd3, 0x00, 0x25, 0xf6, 0x4a, 0x94, 0x68, 0xde, - 0x55, 0xb2, 0xa3, 0x68, 0x16, 0xfe, 0x29, 0x5c, 0xf3, 0x72, 0x82, 0xef, 0xee, 0x00, 0x2f, 0xce, - 0xf0, 0x6e, 0xcc, 0xc1, 0x9b, 0x9b, 0x69, 0x94, 0x15, 0xfd, 0x15, 0x09, 0x68, 0x65, 0xe4, 0xf9, - 0x9a, 0x79, 0x28, 0xc0, 0xe7, 0xc9, 0xf5, 0x9b, 0xf3, 0xf6, 0xfb, 0x72, 0x6c, 0x28, 0x57, 0x46, - 0x73, 0x02, 0xe6, 0x3d, 0x40, 0xa1, 0x9f, 0x02, 0x60, 0x7e, 0xc1, 0xde, 0x98, 0x67, 0xce, 0x53, - 0x8e, 0x56, 0x64, 0x72, 0x8a, 0xb2, 0x29, 0x7e, 0xf8, 0xa4, 0x28, 0x94, 0x7e, 0x29, 0xc0, 0x52, - 0xa7, 0xcf, 0x9a, 0x40, 0x23, 0xf0, 0x71, 0xb4, 0xa4, 0x0b, 0xe7, 0x28, 0xe9, 0x97, 0xef, 0x8c, - 0x36, 0x45, 0xda, 0x94, 0x95, 0xfe, 0x99, 0x84, 0x4c, 0xcf, 0xd1, 0x2c, 0xa2, 0xe9, 0xae, 0x69, - 0x5b, 0xa8, 0x02, 0x22, 0x6d, 0x38, 0xbd, 0xb8, 0x79, 0xe3, 0xac, 0xf2, 0xd4, 0x3b, 0xb2, 0x5a, - 0xd8, 0xd5, 0xaa, 0x12, 0x45, 0x7e, 0x36, 0x2b, 0x0a, 0x0a, 0x13, 0x45, 0x08, 0x44, 0x4b, 0x1b, - 0xf1, 0x56, 0x2a, 0xad, 0xb0, 0x67, 0x74, 0x0b, 0x92, 0xb4, 0xec, 0x4d, 0x78, 0xdd, 0x9b, 0x5f, - 0x1a, 0x22, 0xdb, 0xe8, 0x32, 0x5e, 0xc5, 0x93, 0x41, 0x4d, 0xc8, 0x0f, 0x35, 0xe2, 0xaa, 0x7b, - 0x58, 0x73, 0xdc, 0x3e, 0xd6, 0x2e, 0x54, 0xd8, 0x72, 0x54, 0xf4, 0xae, 0x2f, 0x89, 0xee, 0x42, - 0x6e, 0xa4, 0x1d, 0xa9, 0xa1, 0xf5, 0x52, 0xe7, 0x87, 0xca, 0x8e, 0xb4, 0xa3, 0xd0, 0x65, 0x1f, - 0xc0, 0x15, 0xdb, 0xf3, 0x63, 0x08, 0x47, 0x0a, 0xd2, 0x2b, 0x53, 0xda, 0x4b, 0x5e, 0xf7, 0x60, - 0x91, 0x7d, 0x7a, 0x81, 0xa0, 0x5b, 0x00, 0xf4, 0x7b, 0x80, 0x85, 0x35, 0x29, 0x64, 0x18, 0xe6, - 0xab, 0x32, 0x8f, 0xef, 0x5b, 0x2a, 0x40, 0xdf, 0x09, 0x2a, 0x41, 0xee, 0xb1, 0x63, 0xba, 0x58, - 0x75, 0x6d, 0x5b, 0xb5, 0x87, 0x46, 0x21, 0xcb, 0x12, 0x5d, 0x86, 0x11, 0x7b, 0xb6, 0xdd, 0x19, - 0x1a, 0xd4, 0xa8, 0x0e, 0xd6, 0x22, 0x5b, 0x2f, 0x2c, 0x5e, 0xc0, 0xa8, 0x54, 0x34, 0x34, 0xc5, - 0x3b, 0x70, 0x55, 0x67, 0x79, 0x28, 0x44, 0x53, 0x77, 0xcd, 0x23, 0x6c, 0x14, 0x64, 0xa6, 0x78, - 0x99, 0xaf, 0x06, 0x02, 0xb7, 0xe9, 0x1a, 0x7a, 0x0f, 0x64, 0xd3, 0x52, 0x77, 0x87, 0xac, 0x35, - 0x62, 0x5b, 0x23, 0x85, 0x25, 0x76, 0xd2, 0xaf, 0xcd, 0x3b, 0x29, 0x3e, 0x98, 0x60, 0x4b, 0xc7, - 0xc6, 0xfb, 0x94, 0xd3, 0xdb, 0x47, 0xde, 0xb4, 0x6e, 0x33, 0x79, 0x46, 0x24, 0xc8, 0x86, 0x45, - 0x73, 0x60, 0xd9, 0x0e, 0x4d, 0x08, 0xf8, 0xc0, 0x9a, 0x8c, 0x48, 0x01, 0x31, 0xc4, 0xf2, 0x59, - 0x91, 0xdc, 0xe0, 0x22, 0x5d, 0x7c, 0xd0, 0x9e, 0x8c, 0x58, 0xa5, 0x0c, 0x3b, 0xae, 0x13, 0x6b, - 0x44, 0xc9, 0x9b, 0xc1, 0x3b, 0x45, 0x3f, 0xf1, 0x79, 0x13, 0x97, 0xc5, 0xa6, 0x28, 0x25, 0xe5, - 0x54, 0x53, 0x94, 0xd2, 0x32, 0x34, 0x45, 0x29, 0x27, 0xe7, 0x9b, 0xa2, 0x94, 0x97, 0x17, 0x4b, - 0x7f, 0x14, 0x61, 0x29, 0x12, 0xe0, 0x0a, 0xd6, 0x6d, 0xc7, 0xf8, 0x22, 0x6e, 0xdb, 0x97, 0xe7, - 0x66, 0x5d, 0x2e, 0x64, 0xff, 0x1f, 0x82, 0x41, 0xf4, 0x02, 0x21, 0x26, 0xc7, 0x4f, 0x85, 0x43, - 0x4a, 0x96, 0x9a, 0xa2, 0x24, 0xc9, 0xe9, 0x20, 0x34, 0x40, 0xce, 0x34, 0x45, 0x29, 0x2b, 0xe7, - 0xa2, 0x61, 0xd2, 0x14, 0xa5, 0x45, 0x59, 0x6e, 0x8a, 0x92, 0x2c, 0x2f, 0x95, 0x66, 0x02, 0x24, - 0x69, 0x95, 0xb7, 0x5c, 0xf4, 0x10, 0x16, 0x89, 0x69, 0x0d, 0x86, 0x98, 0x7e, 0x69, 0x87, 0xad, - 0x4b, 0x66, 0xe3, 0xcd, 0x39, 0xb6, 0xe1, 0x32, 0xe5, 0x2e, 0x13, 0xb8, 0x87, 0xa7, 0xcc, 0xde, - 0x61, 0xd8, 0xe4, 0x48, 0x74, 0x01, 0xfd, 0x00, 0xe2, 0xee, 0x91, 0xdf, 0xdb, 0x9c, 0x2b, 0x02, - 0xb9, 0xb1, 0xa9, 0xd4, 0xea, 0x2d, 0xc8, 0x9d, 0x50, 0x73, 0xc6, 0xa8, 0x20, 0xfa, 0xf9, 0xdf, - 0x14, 0x25, 0x51, 0x4e, 0x94, 0xfe, 0x2a, 0xc0, 0xe2, 0xb6, 0xad, 0xef, 0x57, 0xf4, 0x83, 0x89, - 0x49, 0x4c, 0x56, 0x7f, 0xbe, 0x0b, 0x62, 0xe4, 0x78, 0xaf, 0x0c, 0x9f, 0xc8, 0x2d, 0x20, 0x97, - 0x3d, 0x05, 0x52, 0x00, 0x8c, 0x89, 0xa3, 0xf5, 0xcd, 0xa1, 0xe9, 0xf2, 0xbd, 0xe7, 0x37, 0x36, - 0x22, 0x18, 0xfb, 0x87, 0x65, 0x7f, 0x6a, 0x53, 0x8e, 0x4c, 0x6d, 0xca, 0x34, 0x78, 0xcb, 0xb5, - 0x40, 0x52, 0x89, 0xa0, 0x94, 0xfe, 0x10, 0x03, 0xa0, 0x67, 0xe3, 0x1f, 0x04, 0xff, 0x93, 0x63, - 0x85, 0x99, 0x21, 0xfe, 0x39, 0x32, 0xc3, 0x9c, 0xcb, 0x23, 0xfe, 0x37, 0x2f, 0x4f, 0xe9, 0x57, - 0x02, 0xe4, 0x4f, 0x5e, 0xeb, 0xb3, 0x06, 0x36, 0x3f, 0x01, 0x89, 0x78, 0xcc, 0xde, 0x0c, 0xe4, - 0x47, 0x2f, 0x66, 0xc5, 0x5b, 0xe7, 0x9a, 0xb2, 0x9c, 0x1e, 0x92, 0x51, 0xe3, 0x75, 0xf1, 0x81, - 0x12, 0x20, 0x7a, 0xbd, 0xd1, 0x5f, 0xe2, 0x90, 0xd8, 0xc6, 0x1a, 0xc1, 0xe8, 0xfb, 0x90, 0xe0, - 0x1f, 0xd4, 0x17, 0xe8, 0xd7, 0xb9, 0x04, 0xfa, 0x21, 0x00, 0x3e, 0x1a, 0x9b, 0x8e, 0x46, 0x6d, - 0x7c, 0xae, 0x56, 0x4d, 0x89, 0x08, 0xa0, 0x1a, 0xa4, 0xfc, 0x8f, 0xdf, 0xf8, 0x85, 0x3f, 0x7e, - 0x7d, 0x51, 0xb4, 0x03, 0x91, 0x4f, 0x47, 0x3e, 0x1b, 0xa0, 0xbf, 0xc4, 0xf4, 0xe7, 0x10, 0xff, - 0x61, 0x47, 0x2b, 0xa1, 0x34, 0x1b, 0x10, 0x74, 0x99, 0x2c, 0x6a, 0x43, 0x66, 0xec, 0xd8, 0x63, - 0x9b, 0xd0, 0x0e, 0x88, 0x9c, 0xaf, 0x74, 0xe4, 0x8f, 0x67, 0x45, 0xb8, 0xef, 0x49, 0xf5, 0xba, - 0x0a, 0xf8, 0x08, 0x3d, 0x82, 0x96, 0x21, 0x81, 0xc7, 0xb6, 0xbe, 0x57, 0x48, 0x5e, 0x17, 0x6e, - 0xc4, 0x15, 0xfe, 0x82, 0xde, 0x8a, 0xb8, 0x9a, 0x36, 0x6b, 0xf1, 0xea, 0xd2, 0x8b, 0x59, 0x31, - 0xc7, 0x3c, 0xe3, 0x07, 0x4c, 0xc4, 0x77, 0x41, 0x45, 0x2e, 0x7d, 0x2a, 0x40, 0xbe, 0xd2, 0xb7, - 0x1d, 0x97, 0xde, 0xad, 0xba, 0xe5, 0x3a, 0xd3, 0xb3, 0x22, 0xea, 0xf2, 0x2d, 0x35, 0xd2, 0x40, - 0x1a, 0x3b, 0xa6, 0xed, 0xf8, 0x69, 0x24, 0x51, 0xad, 0xbf, 0x98, 0x15, 0x2b, 0x9f, 0x3b, 0x28, - 0xef, 0x7b, 0x60, 0x4a, 0x00, 0xbb, 0x29, 0xd1, 0x93, 0x7d, 0x46, 0x4f, 0xf7, 0xaf, 0x18, 0x2c, - 0x6d, 0x63, 0x6d, 0xb7, 0x77, 0x64, 0x35, 0xac, 0xf1, 0x84, 0xfa, 0xc4, 0xc5, 0xe8, 0x5d, 0x9e, - 0x31, 0x78, 0x9c, 0xae, 0x9d, 0x7d, 0xe3, 0xa3, 0xc9, 0xe2, 0x4d, 0x58, 0x74, 0xf0, 0xae, 0x83, - 0xc9, 0x9e, 0x6a, 0x5a, 0x87, 0xda, 0xd0, 0x34, 0x98, 0xad, 0x25, 0x25, 0xef, 0x91, 0x1b, 0x9c, - 0x3a, 0xb7, 0x4e, 0x4b, 0x97, 0xab, 0xd3, 0x1b, 0xb0, 0x42, 0x5c, 0x3c, 0x1e, 0x9b, 0xd6, 0x40, - 0x1d, 0xd1, 0x8f, 0x20, 0x6c, 0x69, 0xfd, 0x21, 0x36, 0x0a, 0x69, 0xb6, 0x83, 0x2b, 0xfe, 0x62, - 0xcb, 0x36, 0x70, 0x9d, 0x2f, 0xa1, 0x3e, 0x64, 0x59, 0xf7, 0x4a, 0xf0, 0x81, 0x6a, 0x4d, 0x46, - 0x05, 0xf8, 0x82, 0x72, 0x00, 0x50, 0x54, 0x9e, 0xa2, 0x4e, 0x15, 0x72, 0x51, 0x4e, 0x34, 0x45, - 0x29, 0x21, 0x27, 0x79, 0x51, 0x2f, 0x7d, 0x14, 0xda, 0xff, 0xb6, 0x69, 0x69, 0xc3, 0xcb, 0xd9, - 0xff, 0x7b, 0x50, 0x88, 0xce, 0xd2, 0xec, 0xd1, 0x48, 0xb3, 0xe8, 0xdf, 0x89, 0xe5, 0xf2, 0x50, - 0x52, 0x22, 0xb3, 0xb6, 0x2d, 0xbe, 0xbc, 0x45, 0x57, 0x51, 0x15, 0x72, 0xbe, 0xe7, 0x78, 0xe7, - 0x25, 0x9e, 0xa7, 0xf3, 0xca, 0x7a, 0x32, 0xbc, 0xf9, 0x3a, 0xaf, 0xf7, 0x03, 0x93, 0x04, 0x66, - 0xe0, 0x5d, 0x4d, 0xe9, 0x17, 0x02, 0xa4, 0x59, 0xba, 0x6f, 0x58, 0xbb, 0x36, 0xba, 0xe5, 0x8d, - 0xa3, 0x2e, 0x3a, 0x7a, 0xe5, 0x43, 0xa9, 0x77, 0x20, 0x31, 0xa4, 0x77, 0xfb, 0x8c, 0x59, 0x3c, - 0xbb, 0xfb, 0x7e, 0x9e, 0x65, 0xcc, 0x7c, 0xfc, 0x73, 0xf3, 0x63, 0x01, 0xd2, 0x6c, 0x50, 0xcf, - 0x86, 0x88, 0x19, 0x48, 0xed, 0xb4, 0xef, 0xb5, 0x3b, 0xef, 0xb7, 0xe5, 0x05, 0x94, 0x82, 0x78, - 0xa3, 0xdd, 0x93, 0x05, 0x94, 0x86, 0xc4, 0xed, 0xed, 0x4e, 0xa5, 0x27, 0xc7, 0xe8, 0x63, 0xf5, - 0x61, 0xaf, 0xde, 0x95, 0xe3, 0xe8, 0x0a, 0x2c, 0xd6, 0xea, 0xdb, 0x8d, 0x56, 0xa3, 0x57, 0xaf, - 0xa9, 0x9c, 0x28, 0x21, 0x09, 0xc4, 0x5e, 0xa3, 0x55, 0x97, 0x45, 0x0a, 0x55, 0xab, 0x6f, 0x35, - 0x5a, 0x95, 0x6d, 0x39, 0x81, 0x56, 0x60, 0x29, 0xe4, 0xf5, 0xc9, 0x69, 0x94, 0x05, 0xa9, 0xb6, - 0xa3, 0x54, 0x7a, 0x8d, 0x4e, 0x5b, 0x4e, 0x22, 0x80, 0x24, 0x95, 0xed, 0x3d, 0x92, 0xb3, 0x54, - 0xf7, 0x9d, 0x7a, 0x47, 0xce, 0x31, 0x85, 0x9d, 0x1f, 0x6f, 0xd4, 0xe4, 0x3c, 0x7d, 0xec, 0xed, - 0xdc, 0xdf, 0xae, 0xcb, 0x40, 0x05, 0xab, 0x8d, 0x5e, 0x45, 0x51, 0x2a, 0x0f, 0xe5, 0x0c, 0xca, - 0x03, 0x50, 0xc1, 0x6e, 0x5d, 0x69, 0xd4, 0xbb, 0xb2, 0x51, 0xa2, 0xdd, 0x63, 0xea, 0xa6, 0x0e, - 0x4b, 0x2f, 0xcd, 0x4e, 0x51, 0x0e, 0xd2, 0x95, 0x5a, 0x4d, 0x7d, 0xd0, 0xe9, 0xd5, 0x15, 0x79, - 0x01, 0xc9, 0x90, 0x55, 0xea, 0xad, 0xce, 0x83, 0xba, 0x47, 0x11, 0xd0, 0x12, 0xe4, 0x28, 0x43, - 0xbb, 0xd3, 0xf6, 0x48, 0x31, 0xb4, 0x0c, 0xb2, 0xc7, 0x14, 0x52, 0xe3, 0xab, 0xe2, 0x87, 0xbf, - 0x5d, 0x5b, 0xb8, 0xd9, 0x39, 0xf1, 0x91, 0xc2, 0x3b, 0x02, 0x7a, 0xf4, 0xfb, 0xf5, 0x76, 0xad, - 0xd1, 0xbe, 0x23, 0x2f, 0xd0, 0x97, 0x6e, 0xaf, 0x72, 0x87, 0xbe, 0xc4, 0xa9, 0xfa, 0xad, 0x4e, - 0xab, 0xd5, 0xe8, 0xf5, 0xea, 0x35, 0x59, 0xa0, 0x6b, 0x95, 0x6a, 0x47, 0xa1, 0x2f, 0x31, 0x0e, - 0x58, 0xfd, 0xd6, 0xd3, 0x7f, 0xac, 0x2d, 0x3c, 0x3d, 0x5e, 0x13, 0x9e, 0x1d, 0xaf, 0x09, 0x9f, - 0x1c, 0xaf, 0x09, 0x7f, 0x3f, 0x5e, 0x13, 0x7e, 0xfd, 0x7c, 0x6d, 0xe1, 0xd9, 0xf3, 0xb5, 0x85, - 0x4f, 0x9e, 0xaf, 0x2d, 0x3c, 0x4a, 0x79, 0x3e, 0xed, 0x27, 0xd9, 0x3f, 0x9d, 0xde, 0xfe, 0x77, - 0x00, 0x00, 0x00, 0xff, 0xff, 0x41, 0xcf, 0x09, 0xa3, 0x4c, 0x1b, 0x00, 0x00, +func init() { proto.RegisterFile("roachpb/data.proto", fileDescriptor_data_7c13a0dbc42fb34d) } + +var fileDescriptor_data_7c13a0dbc42fb34d = []byte{ + // 2429 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xdc, 0x59, 0xdf, 0x6f, 0x1b, 0x59, + 0xf5, 0xcf, 0x64, 0xc6, 0xf6, 0xf8, 0xf8, 0x47, 0x26, 0xb7, 0x49, 0xeb, 0xcd, 0x7e, 0xbf, 0x71, + 0xf1, 0x02, 0x5b, 0x2a, 0xd6, 0x11, 0x69, 0x59, 0x41, 0x28, 0x12, 0xfe, 0xd5, 0xd6, 0x6e, 0x6c, + 0x77, 0xc7, 0x4e, 0x97, 0x76, 0x91, 0x86, 0xf1, 0xcc, 0x8d, 0x33, 0xc4, 0x9e, 0x71, 0x66, 0xc6, + 0x69, 0xcc, 0x1b, 0x0f, 0xa0, 0xd5, 0x3e, 0x00, 0x8f, 0x3c, 0xf0, 0x50, 0x89, 0x37, 0x78, 0x80, + 0x47, 0xc4, 0x5f, 0xd0, 0x07, 0x1e, 0xfa, 0xb6, 0x0b, 0x48, 0x16, 0xa4, 0x42, 0xda, 0xbf, 0xa1, + 0x12, 0x08, 0xdd, 0x7b, 0xe7, 0x57, 0x52, 0x37, 0x38, 0x9b, 0x22, 0x56, 0xbc, 0x24, 0x33, 0xe7, + 0x9e, 0xf3, 0x39, 0xf7, 0x9e, 0x5f, 0xf7, 0xcc, 0x31, 0x20, 0xdb, 0x52, 0xb5, 0xbd, 0x51, 0x6f, + 0x43, 0x57, 0x5d, 0xb5, 0x38, 0xb2, 0x2d, 0xd7, 0x42, 0xcb, 0x9a, 0xa5, 0xed, 0x53, 0x7a, 0xd1, + 0x5b, 0x5d, 0xbb, 0xbe, 0x7f, 0xb8, 0xb1, 0x7f, 0xe8, 0x60, 0xfb, 0x10, 0xdb, 0x1b, 0x9a, 0x65, + 0x6a, 0x63, 0xdb, 0xc6, 0xa6, 0x36, 0xd9, 0x18, 0x58, 0xda, 0x3e, 0xfd, 0x63, 0x98, 0x7d, 0x26, + 0xbe, 0x76, 0xd9, 0x87, 0x1c, 0x62, 0x57, 0x0d, 0x61, 0xd7, 0xde, 0x74, 0x5c, 0xcb, 0x56, 0xfb, + 0x78, 0x03, 0x9b, 0x7d, 0xc3, 0xc4, 0x84, 0xe1, 0x50, 0xd3, 0xbc, 0xc5, 0xff, 0x9b, 0xb9, 0x78, + 0xc3, 0x5b, 0xcd, 0x8d, 0x5d, 0x63, 0xb0, 0xb1, 0x37, 0xd0, 0x36, 0x5c, 0x63, 0x88, 0x1d, 0x57, + 0x1d, 0x8e, 0xbc, 0x95, 0x95, 0xbe, 0xd5, 0xb7, 0xe8, 0xe3, 0x06, 0x79, 0x62, 0xd4, 0xc2, 0x43, + 0x10, 0x3a, 0x23, 0xd5, 0x44, 0x6f, 0x00, 0xbf, 0x8f, 0x27, 0x39, 0xfe, 0x2a, 0x77, 0x2d, 0x5d, + 0x4e, 0xbc, 0x98, 0xe6, 0xf9, 0x7b, 0x78, 0x22, 0x13, 0x1a, 0xba, 0x0a, 0x09, 0x6c, 0xea, 0x0a, + 0x59, 0x16, 0x4e, 0x2e, 0xc7, 0xb1, 0xa9, 0xdf, 0xc3, 0x93, 0x2d, 0xf1, 0x17, 0x4f, 0xf2, 0x0b, + 0xbf, 0x7f, 0x92, 0xe7, 0x1a, 0x82, 0xc8, 0x49, 0x8b, 0x0d, 0x41, 0x5c, 0x94, 0xf8, 0x42, 0x1f, + 0x62, 0x0f, 0xd4, 0xc1, 0x18, 0xa3, 0x37, 0x21, 0x69, 0xab, 0x8f, 0x95, 0xde, 0xc4, 0xc5, 0x4e, + 0x8e, 0x23, 0x10, 0xb2, 0x68, 0xab, 0x8f, 0xcb, 0xe4, 0x1d, 0x95, 0x20, 0x19, 0xec, 0x34, 0xb7, + 0x78, 0x95, 0xbb, 0x96, 0xda, 0xfc, 0xff, 0x62, 0x68, 0x56, 0x72, 0x9c, 0xe2, 0xde, 0x40, 0x2b, + 0x76, 0x7d, 0xa6, 0xb2, 0xf0, 0x74, 0x9a, 0x5f, 0x90, 0x43, 0xa9, 0xc2, 0x07, 0x20, 0xde, 0xc3, + 0x13, 0xa6, 0xcb, 0x3b, 0x07, 0x37, 0xe3, 0x1c, 0x37, 0x21, 0x76, 0x48, 0x78, 0x3c, 0x2d, 0xb9, + 0xe2, 0x4b, 0xce, 0x2b, 0x52, 0x0c, 0x4f, 0x01, 0x63, 0x2e, 0x7c, 0xcc, 0x01, 0x74, 0x5c, 0xcb, + 0xc6, 0x75, 0x1d, 0x9b, 0x2e, 0xea, 0x03, 0x68, 0x83, 0xb1, 0xe3, 0x62, 0x5b, 0x31, 0x74, 0x4f, + 0xcd, 0x5d, 0xc2, 0xff, 0xe7, 0x69, 0xfe, 0x46, 0xdf, 0x70, 0xf7, 0xc6, 0xbd, 0xa2, 0x66, 0x0d, + 0x37, 0x02, 0x6c, 0xbd, 0x17, 0x3e, 0x6f, 0x8c, 0xf6, 0xfb, 0x1b, 0xd4, 0x41, 0xe3, 0xb1, 0xa1, + 0x17, 0x77, 0x76, 0xea, 0xd5, 0xe3, 0x69, 0x3e, 0x59, 0x61, 0x80, 0xf5, 0xaa, 0x9c, 0xf4, 0xb0, + 0xeb, 0x3a, 0x7a, 0x07, 0x12, 0xa6, 0xa5, 0x63, 0xa2, 0x85, 0xec, 0x37, 0x56, 0x5e, 0x39, 0x9e, + 0xe6, 0xe3, 0x2d, 0x4b, 0xc7, 0xf5, 0xea, 0x8b, 0xe0, 0x49, 0x8e, 0x13, 0xa6, 0xba, 0x8e, 0xbe, + 0x06, 0x22, 0x89, 0x0b, 0xca, 0xcf, 0x53, 0xfe, 0xcb, 0xc7, 0xd3, 0x7c, 0x82, 0xed, 0x9c, 0x08, + 0xf8, 0x8f, 0x72, 0xc2, 0x61, 0xa7, 0x29, 0xfc, 0x9a, 0x83, 0x74, 0x67, 0x34, 0x30, 0xdc, 0xae, + 0x6d, 0xf4, 0xfb, 0xd8, 0x46, 0x35, 0x48, 0x0e, 0xf0, 0xae, 0xab, 0xe8, 0xd8, 0xd1, 0xe8, 0xd1, + 0x52, 0x9b, 0x85, 0x19, 0x46, 0x92, 0x55, 0xb3, 0x8f, 0xab, 0xd8, 0xd1, 0x6c, 0x63, 0xe4, 0x5a, + 0xb6, 0x67, 0x2e, 0x91, 0x88, 0x12, 0x2a, 0xba, 0x03, 0x60, 0x1b, 0xfd, 0x3d, 0x0f, 0x67, 0xf1, + 0x9c, 0x38, 0x49, 0x2a, 0x4b, 0xc8, 0x5b, 0xc2, 0xa7, 0x2c, 0xa4, 0x78, 0x49, 0x28, 0xfc, 0x92, + 0x87, 0x74, 0x13, 0xdb, 0x7d, 0xfc, 0x39, 0xdd, 0x2c, 0xea, 0x83, 0xc4, 0x80, 0x48, 0x36, 0x2a, + 0x8e, 0xab, 0xba, 0x0e, 0x4d, 0x97, 0xd4, 0xe6, 0x97, 0x22, 0x70, 0x5e, 0xee, 0x16, 0xfd, 0xdc, + 0x2d, 0x36, 0x1f, 0x54, 0x2a, 0x1d, 0xc2, 0x5c, 0xbe, 0x4c, 0x10, 0x8f, 0xa7, 0xf9, 0xac, 0x4c, + 0x60, 0x02, 0xba, 0x9c, 0xa5, 0xb0, 0xcd, 0x43, 0x4d, 0xa3, 0xef, 0xe8, 0x27, 0x1c, 0xa4, 0x77, + 0x6d, 0x8c, 0x7f, 0x88, 0x89, 0x16, 0xdb, 0xcd, 0xc5, 0xe6, 0x49, 0x9a, 0x2a, 0x41, 0x7f, 0x31, + 0xcd, 0xdf, 0x9a, 0x3f, 0x46, 0x09, 0x40, 0x85, 0x14, 0xac, 0x00, 0x45, 0x4e, 0x31, 0xc5, 0x1d, + 0xa2, 0xf7, 0x84, 0x7b, 0x7e, 0x13, 0x83, 0xd5, 0xca, 0x1e, 0xb1, 0x91, 0x8c, 0x47, 0x03, 0x43, + 0x53, 0x1d, 0xdf, 0x4f, 0x8f, 0xe0, 0xb2, 0x8e, 0x47, 0x36, 0xd6, 0x54, 0x17, 0xeb, 0x8a, 0x46, + 0x79, 0x14, 0x77, 0x32, 0xc2, 0xd4, 0x69, 0xd9, 0xcd, 0x2f, 0xce, 0x32, 0x36, 0xc3, 0x60, 0x80, + 0xdd, 0xc9, 0x08, 0xcb, 0x2b, 0x21, 0x46, 0x48, 0x45, 0x0f, 0x01, 0x45, 0xb0, 0x6d, 0x26, 0xe5, + 0x39, 0xf1, 0x0c, 0xdc, 0x97, 0xdc, 0xb8, 0x1c, 0xa2, 0x78, 0x2c, 0xe8, 0x07, 0xf0, 0x66, 0x04, + 0x7a, 0x3c, 0xd2, 0xa3, 0x2a, 0x9c, 0x1c, 0x7f, 0x95, 0x3f, 0xa7, 0x8e, 0x37, 0x42, 0xb8, 0x1d, + 0x86, 0xe6, 0x5b, 0x0a, 0x61, 0x58, 0x8b, 0xe8, 0x32, 0xf1, 0x91, 0xeb, 0x2b, 0x22, 0xd9, 0x2c, + 0xd0, 0x6c, 0xbe, 0x76, 0x3c, 0xcd, 0x5f, 0xa9, 0x06, 0x5c, 0x2d, 0x7c, 0xe4, 0x7a, 0xf2, 0x34, + 0xbb, 0x93, 0xc1, 0x8b, 0x7c, 0x45, 0x9f, 0xc9, 0xa5, 0xa3, 0x77, 0x41, 0xa0, 0x41, 0x1e, 0x9b, + 0x37, 0xc8, 0x65, 0xca, 0x8f, 0x7a, 0x70, 0xc5, 0x30, 0x5d, 0x6c, 0x9b, 0xea, 0x40, 0x51, 0x75, + 0x3d, 0x6a, 0x86, 0xf8, 0xb9, 0xcd, 0xb0, 0xea, 0x43, 0x95, 0x08, 0x52, 0x60, 0x82, 0x5d, 0x78, + 0x23, 0xd0, 0x61, 0xe3, 0xa1, 0x75, 0x18, 0xd5, 0x92, 0x38, 0xb7, 0x96, 0x60, 0xc3, 0x32, 0xc3, + 0xf2, 0xf5, 0x6c, 0x09, 0xe4, 0xa6, 0x2a, 0x7c, 0xc4, 0xc1, 0xa5, 0xa6, 0xa5, 0x1b, 0xbb, 0x06, + 0xd6, 0xc9, 0xed, 0xe7, 0xc7, 0xea, 0x57, 0x01, 0x39, 0x13, 0xc7, 0xc5, 0x43, 0x45, 0xb3, 0xcc, + 0x5d, 0xa3, 0xaf, 0x38, 0x23, 0xd5, 0xa4, 0x71, 0x2a, 0xca, 0x12, 0x5b, 0xa9, 0xd0, 0x05, 0x7a, + 0x65, 0xd6, 0x00, 0xd1, 0x0a, 0x3d, 0x30, 0x0e, 0xb1, 0x89, 0x1d, 0x87, 0x71, 0xb3, 0xe8, 0xbb, + 0x32, 0x63, 0xb3, 0x44, 0x48, 0x96, 0x88, 0xc8, 0xb6, 0x27, 0x41, 0x28, 0x85, 0x07, 0x20, 0x75, + 0x5c, 0x43, 0xdb, 0x9f, 0x94, 0xc3, 0x4a, 0x5c, 0x06, 0x70, 0x28, 0x4d, 0xe9, 0x19, 0xae, 0x57, + 0xdd, 0xe6, 0xbb, 0x15, 0x1d, 0x1f, 0xaa, 0xf0, 0x07, 0x1e, 0x56, 0xeb, 0x9e, 0x19, 0x2a, 0xd6, + 0x70, 0x18, 0xa2, 0x57, 0x21, 0xe3, 0x90, 0xba, 0xaf, 0xb8, 0x8c, 0xe0, 0x29, 0xc8, 0xcf, 0xdc, + 0x73, 0x78, 0x3f, 0xc8, 0x69, 0x27, 0x7a, 0x5b, 0x54, 0x21, 0x33, 0x24, 0x05, 0x39, 0x40, 0x59, + 0x7c, 0x25, 0x4a, 0xb4, 0x70, 0xcb, 0xe9, 0x61, 0xb4, 0x8c, 0x7f, 0x1f, 0xae, 0x78, 0x35, 0xc1, + 0x77, 0x77, 0x80, 0xc7, 0x53, 0xbc, 0x6b, 0x33, 0xf0, 0x66, 0x56, 0x1a, 0x79, 0x55, 0x7b, 0x45, + 0x01, 0x5a, 0x1d, 0x7a, 0xbe, 0xa6, 0x1e, 0x0a, 0xf0, 0x59, 0x75, 0xfe, 0xf2, 0xac, 0xfd, 0xbe, + 0x1c, 0x1b, 0xf2, 0xa5, 0xe1, 0x8c, 0x80, 0x79, 0x0f, 0x50, 0xe8, 0xa7, 0x00, 0x98, 0x25, 0xd8, + 0x5b, 0xb3, 0xcc, 0x79, 0xca, 0xd1, 0xb2, 0xe4, 0x9c, 0xa2, 0x6c, 0x09, 0x1f, 0x3e, 0xc9, 0x73, + 0x85, 0x3f, 0x72, 0xb0, 0xdc, 0xee, 0xd1, 0x2e, 0x52, 0x0f, 0x7c, 0x1c, 0xed, 0x09, 0xb8, 0x39, + 0x7a, 0x82, 0x1f, 0x71, 0xe7, 0xee, 0xad, 0x5e, 0xcf, 0x35, 0x11, 0x6a, 0xdd, 0x12, 0x48, 0x5b, + 0x58, 0xf8, 0x7b, 0x1c, 0x52, 0x5d, 0x5b, 0x35, 0x1d, 0x55, 0x73, 0x0d, 0xcb, 0x44, 0x25, 0x10, + 0x48, 0xcb, 0xeb, 0x05, 0xde, 0x5b, 0x67, 0x5d, 0x90, 0xdd, 0x23, 0xb3, 0x89, 0x5d, 0xb5, 0x2c, + 0x92, 0x9d, 0x3d, 0x9b, 0xe6, 0x39, 0x99, 0x8a, 0x22, 0x04, 0x82, 0xa9, 0x0e, 0x59, 0x33, 0x97, + 0x94, 0xe9, 0x33, 0xba, 0x05, 0x71, 0x72, 0xf1, 0x8e, 0xd9, 0xcd, 0x3b, 0xfb, 0x6e, 0x89, 0x6c, + 0xa3, 0x43, 0x79, 0x65, 0x4f, 0x06, 0x35, 0x20, 0x3b, 0x50, 0x1d, 0x57, 0xd9, 0xc3, 0xaa, 0xed, + 0xf6, 0xb0, 0x3a, 0xe7, 0xcd, 0xca, 0x12, 0x2f, 0x43, 0x44, 0xef, 0xfa, 0x92, 0xe8, 0x2e, 0x64, + 0x86, 0xea, 0x91, 0x12, 0x5a, 0x3f, 0x31, 0x3f, 0x54, 0x7a, 0xa8, 0x1e, 0x85, 0x3e, 0xff, 0x00, + 0x2e, 0x59, 0x5e, 0x20, 0x84, 0x70, 0x4e, 0x4e, 0x7c, 0x65, 0x4d, 0x7c, 0x29, 0x6c, 0x3c, 0x58, + 0x64, 0x9d, 0x5e, 0x70, 0xd0, 0x2d, 0x00, 0xe2, 0x39, 0x9a, 0x17, 0x4e, 0x2e, 0x45, 0x31, 0x5f, + 0x55, 0xba, 0xfc, 0x0a, 0x43, 0x04, 0xc8, 0xbb, 0x83, 0x0a, 0x90, 0x79, 0x6c, 0x1b, 0x2e, 0x56, + 0x5c, 0xcb, 0x52, 0xac, 0x81, 0x9e, 0x4b, 0xd3, 0x4a, 0x99, 0xa2, 0xc4, 0xae, 0x65, 0xb5, 0x07, + 0x3a, 0x31, 0xaa, 0x8d, 0xd5, 0xc8, 0xd6, 0x73, 0x4b, 0xe7, 0x30, 0x2a, 0x11, 0x0d, 0x4d, 0x71, + 0x13, 0x2e, 0x6b, 0xb4, 0x90, 0x85, 0x68, 0xca, 0xae, 0x71, 0x84, 0xf5, 0x9c, 0x44, 0x15, 0xaf, + 0xb0, 0xd5, 0x40, 0xe0, 0x36, 0x59, 0x43, 0xef, 0x81, 0x64, 0x98, 0xca, 0xee, 0x80, 0x36, 0x67, + 0x74, 0x6b, 0x4e, 0x6e, 0x99, 0x9e, 0xf4, 0x0b, 0xb3, 0x4e, 0x8a, 0x0f, 0xc6, 0xd8, 0xd4, 0xb0, + 0xfe, 0x3e, 0xe1, 0xf4, 0xf6, 0x91, 0x35, 0xcc, 0xdb, 0x54, 0x9e, 0x12, 0x1d, 0x64, 0xc1, 0x92, + 0xd1, 0x37, 0x2d, 0x9b, 0x54, 0x14, 0x7c, 0x60, 0x8e, 0x87, 0x4e, 0x0e, 0x51, 0xc4, 0xe2, 0x59, + 0x91, 0x5c, 0x67, 0x22, 0x1d, 0x7c, 0xd0, 0x1a, 0x0f, 0xe9, 0x55, 0x1b, 0xf6, 0x7c, 0x27, 0xd6, + 0x1c, 0x39, 0x6b, 0x04, 0xef, 0x04, 0xfd, 0xc4, 0x07, 0x16, 0x2f, 0x09, 0x0d, 0x41, 0x8c, 0x4b, + 0x89, 0x86, 0x20, 0x26, 0x25, 0x68, 0x08, 0x62, 0x46, 0xca, 0x36, 0x04, 0x31, 0x2b, 0x2d, 0x15, + 0x7e, 0x27, 0xc0, 0x72, 0x24, 0xc0, 0x65, 0xac, 0x59, 0xb6, 0xfe, 0x3a, 0xb2, 0xed, 0xf3, 0x93, + 0x59, 0x17, 0x0b, 0xd9, 0xff, 0x85, 0x60, 0x10, 0xbc, 0x40, 0x58, 0x94, 0xf8, 0x53, 0xe1, 0x90, + 0x90, 0xc4, 0x86, 0x20, 0x8a, 0x52, 0x32, 0x08, 0x0d, 0x90, 0x52, 0x0d, 0x41, 0x4c, 0x4b, 0x99, + 0x68, 0x98, 0x34, 0x04, 0x71, 0x49, 0x92, 0x1a, 0x82, 0x28, 0x49, 0xcb, 0x85, 0x29, 0x07, 0x71, + 0xd2, 0x26, 0x98, 0x2e, 0x7a, 0x08, 0x4b, 0x8e, 0x61, 0xf6, 0x07, 0x98, 0x7c, 0xeb, 0x87, 0xbd, + 0x4f, 0x6a, 0xf3, 0xed, 0x19, 0xb6, 0x61, 0x32, 0xc5, 0x0e, 0x15, 0xb8, 0x87, 0x27, 0xd4, 0xde, + 0x61, 0xd8, 0x64, 0x9c, 0xe8, 0x02, 0xfa, 0x16, 0xf0, 0xee, 0x91, 0xdf, 0x1c, 0xcd, 0x15, 0x81, + 0xcc, 0xd8, 0x44, 0x6a, 0xed, 0x16, 0x64, 0x4e, 0xa8, 0x39, 0x63, 0x58, 0x11, 0x1d, 0x40, 0x34, + 0x04, 0x51, 0x90, 0x62, 0x85, 0x3f, 0x71, 0xb0, 0xb4, 0x6d, 0x69, 0xfb, 0x25, 0xed, 0x60, 0x6c, + 0x38, 0x06, 0xbd, 0x7f, 0xbe, 0x0e, 0x42, 0xe4, 0x78, 0xaf, 0x0c, 0x9f, 0x48, 0x16, 0x38, 0x17, + 0x3d, 0x05, 0x92, 0x01, 0xf4, 0xb1, 0xad, 0xf6, 0x8c, 0x81, 0xe1, 0xb2, 0xbd, 0x67, 0x37, 0x37, + 0x23, 0x18, 0xfb, 0x87, 0x45, 0x7f, 0x6e, 0x54, 0x8c, 0xcc, 0x8d, 0x8a, 0x24, 0x78, 0x8b, 0xd5, + 0x40, 0x52, 0x8e, 0xa0, 0x14, 0x7e, 0xbb, 0x08, 0x40, 0xce, 0xc6, 0xbe, 0x28, 0xfe, 0x2b, 0xc7, + 0x0a, 0x2b, 0x03, 0xff, 0x19, 0x2a, 0xc3, 0x8c, 0xe4, 0x11, 0xfe, 0x93, 0xc9, 0x53, 0xf8, 0x19, + 0x07, 0xd9, 0x93, 0x69, 0x7d, 0xd6, 0xc8, 0xe8, 0x7b, 0x20, 0x3a, 0x1e, 0xb3, 0x37, 0x85, 0xf9, + 0xce, 0xdc, 0xcd, 0xd1, 0xe9, 0x31, 0x1d, 0x31, 0x5e, 0x07, 0x1f, 0xc8, 0x01, 0xa2, 0xd7, 0x1b, + 0xfd, 0x93, 0x87, 0xd8, 0x36, 0x56, 0x1d, 0x8c, 0xbe, 0x09, 0x31, 0xf6, 0x45, 0x7f, 0x8e, 0x86, + 0x9f, 0x49, 0xa0, 0x6f, 0x03, 0xe0, 0xa3, 0x91, 0x61, 0xab, 0xc4, 0xc6, 0x73, 0xb5, 0x7a, 0x72, + 0x44, 0x00, 0x55, 0x21, 0xe1, 0x7f, 0x3d, 0xf3, 0xe7, 0xfe, 0x7a, 0xf6, 0x45, 0xd1, 0x0e, 0x44, + 0xbe, 0x3d, 0xd9, 0x70, 0x82, 0xfc, 0x75, 0x0c, 0x7f, 0x12, 0xf2, 0x6f, 0x76, 0xb4, 0x1a, 0x4a, + 0xd3, 0x09, 0x43, 0x87, 0xca, 0xa2, 0x9f, 0x72, 0x90, 0x1a, 0xd9, 0xd6, 0xc8, 0x72, 0x48, 0x0b, + 0xe4, 0xcc, 0x77, 0x77, 0xb4, 0x8e, 0xa7, 0x79, 0xb8, 0xef, 0x49, 0x75, 0x3b, 0x17, 0x6e, 0x69, + 0xc1, 0xdf, 0x41, 0xd7, 0x41, 0x2b, 0x10, 0xc3, 0x23, 0x4b, 0xdb, 0xcb, 0xc5, 0xaf, 0x72, 0xd7, + 0x78, 0x99, 0xbd, 0xa0, 0x77, 0x22, 0xb1, 0x42, 0xba, 0x3d, 0xbe, 0xbc, 0xfc, 0x62, 0x9a, 0xcf, + 0x50, 0xd7, 0xfa, 0x11, 0x17, 0x71, 0x7e, 0x70, 0xa5, 0x17, 0xfe, 0xc2, 0x41, 0xb6, 0xd4, 0xb3, + 0x6c, 0x97, 0x24, 0x67, 0xcd, 0x74, 0xed, 0xc9, 0x59, 0x21, 0x79, 0xf1, 0x79, 0x29, 0x52, 0x41, + 0x1c, 0xd9, 0x86, 0x65, 0xfb, 0x75, 0x28, 0x56, 0xae, 0xbd, 0x98, 0xe6, 0x4b, 0x9f, 0x39, 0xaa, + 0xef, 0x7b, 0x60, 0x72, 0x00, 0xbb, 0x25, 0x92, 0x93, 0x7d, 0x4a, 0x4e, 0xf7, 0x8f, 0x45, 0x58, + 0xde, 0xc6, 0xea, 0x6e, 0xf7, 0xc8, 0xac, 0x9b, 0xa3, 0x31, 0x71, 0xaa, 0x8b, 0xd1, 0xbb, 0xac, + 0xe4, 0xb0, 0x40, 0x5f, 0x3f, 0xbb, 0x64, 0x44, 0xab, 0xcd, 0xdb, 0xb0, 0x64, 0xe3, 0x5d, 0x1b, + 0x3b, 0x7b, 0x8a, 0x61, 0x1e, 0xaa, 0x03, 0x43, 0xa7, 0xb6, 0x16, 0xe5, 0xac, 0x47, 0xae, 0x33, + 0xea, 0xcc, 0x8b, 0x5e, 0xbc, 0xd8, 0x45, 0xbf, 0x09, 0xab, 0x8e, 0x8b, 0x47, 0x23, 0xc3, 0xec, + 0x2b, 0x43, 0xf2, 0x19, 0x86, 0x4d, 0xb5, 0x37, 0xc0, 0x7a, 0x2e, 0x49, 0x77, 0x70, 0xc9, 0x5f, + 0x6c, 0x5a, 0x3a, 0xae, 0xb1, 0x25, 0xd4, 0x83, 0x34, 0x6d, 0x7f, 0x1d, 0x7c, 0xa0, 0x98, 0xe3, + 0x61, 0x0e, 0x5e, 0x53, 0x11, 0x01, 0x82, 0xca, 0x6a, 0xdc, 0xa9, 0x4e, 0x40, 0x90, 0x62, 0x0d, + 0x41, 0x8c, 0x49, 0x71, 0xd6, 0x15, 0x14, 0x3e, 0x0a, 0xed, 0x7f, 0xdb, 0x30, 0xd5, 0xc1, 0xc5, + 0xec, 0xff, 0x0d, 0xc8, 0x45, 0xa7, 0x79, 0xd6, 0x70, 0xa8, 0x9a, 0xe4, 0xff, 0xd8, 0x74, 0x59, + 0x28, 0xc9, 0x91, 0x69, 0x5f, 0x85, 0x2d, 0x57, 0xc8, 0x2a, 0x2a, 0x43, 0xc6, 0xf7, 0x1c, 0x6b, + 0xdd, 0x84, 0x79, 0x5a, 0xb7, 0xb4, 0x27, 0xc3, 0xba, 0xb7, 0x79, 0xbd, 0x1f, 0x98, 0x24, 0x30, + 0x03, 0x6b, 0x8b, 0x0a, 0x3f, 0xe6, 0x20, 0x49, 0xef, 0x8b, 0xba, 0xb9, 0x6b, 0xa1, 0x5b, 0xde, + 0x40, 0xec, 0xbc, 0xd3, 0x63, 0x36, 0x16, 0xbb, 0x09, 0xb1, 0x01, 0xc9, 0xed, 0x33, 0x7e, 0x4e, + 0xa0, 0xb9, 0xef, 0x17, 0x6a, 0xca, 0xcc, 0x06, 0x50, 0xd7, 0x3f, 0xe6, 0x20, 0x49, 0x7f, 0x6b, + 0xa0, 0x63, 0xcc, 0x14, 0x24, 0x76, 0x5a, 0xf7, 0x5a, 0xed, 0xf7, 0x5b, 0xd2, 0x02, 0x4a, 0x00, + 0x5f, 0x6f, 0x75, 0x25, 0x0e, 0x25, 0x21, 0x76, 0x7b, 0xbb, 0x5d, 0xea, 0x4a, 0x8b, 0xe4, 0xb1, + 0xfc, 0xb0, 0x5b, 0xeb, 0x48, 0x3c, 0xba, 0x04, 0x4b, 0xd5, 0xda, 0x76, 0xbd, 0x59, 0xef, 0xd6, + 0xaa, 0x0a, 0x23, 0x8a, 0x48, 0x04, 0xa1, 0x5b, 0x6f, 0xd6, 0x24, 0x81, 0x40, 0x55, 0x6b, 0x95, + 0x7a, 0xb3, 0xb4, 0x2d, 0xc5, 0xd0, 0x2a, 0x2c, 0x87, 0xbc, 0x3e, 0x39, 0x89, 0xd2, 0x20, 0x56, + 0x77, 0xe4, 0x52, 0xb7, 0xde, 0x6e, 0x49, 0x71, 0x04, 0x10, 0x27, 0xb2, 0xdd, 0x47, 0x52, 0x9a, + 0xe8, 0xbe, 0x53, 0x6b, 0x4b, 0x19, 0xaa, 0xb0, 0xfd, 0xdd, 0xcd, 0xaa, 0x94, 0x25, 0x8f, 0xdd, + 0x9d, 0xfb, 0xdb, 0x35, 0x09, 0x88, 0x60, 0xb9, 0xde, 0x2d, 0xc9, 0x72, 0xe9, 0xa1, 0x94, 0x42, + 0x59, 0x00, 0x22, 0xd8, 0xa9, 0xc9, 0xf5, 0x5a, 0x47, 0xd2, 0x0b, 0xa4, 0xfd, 0x4c, 0x5c, 0xd7, + 0x60, 0xf9, 0xa5, 0xe9, 0x2d, 0xca, 0x40, 0xb2, 0x54, 0xad, 0x2a, 0x0f, 0xda, 0xdd, 0x9a, 0x2c, + 0x2d, 0x20, 0x09, 0xd2, 0x72, 0xad, 0xd9, 0x7e, 0x50, 0xf3, 0x28, 0x1c, 0x5a, 0x86, 0x0c, 0x61, + 0x68, 0xb5, 0x5b, 0x1e, 0x69, 0x11, 0xad, 0x80, 0xe4, 0x31, 0x85, 0x54, 0x7e, 0x4d, 0xf8, 0xf0, + 0x57, 0xeb, 0x0b, 0xd7, 0xdb, 0x27, 0xbe, 0x72, 0x58, 0x4b, 0x41, 0x8e, 0x7e, 0xbf, 0xd6, 0xaa, + 0xd6, 0x5b, 0x77, 0xa4, 0x05, 0xf2, 0xd2, 0xe9, 0x96, 0xee, 0x90, 0x17, 0x9e, 0xa8, 0xaf, 0xb4, + 0x9b, 0xcd, 0x7a, 0xb7, 0x5b, 0xab, 0x4a, 0x1c, 0x59, 0x2b, 0x95, 0xdb, 0x32, 0x79, 0x59, 0x64, + 0x80, 0xe5, 0xaf, 0x3c, 0xfd, 0xdb, 0xfa, 0xc2, 0xd3, 0xe3, 0x75, 0xee, 0xd9, 0xf1, 0x3a, 0xf7, + 0xc9, 0xf1, 0x3a, 0xf7, 0xd7, 0xe3, 0x75, 0xee, 0xe7, 0xcf, 0xd7, 0x17, 0x9e, 0x3d, 0x5f, 0x5f, + 0xf8, 0xe4, 0xf9, 0xfa, 0xc2, 0xa3, 0x84, 0xe7, 0xd3, 0x5e, 0x9c, 0xfe, 0x6e, 0x76, 0xe3, 0x5f, + 0x01, 0x00, 0x00, 0xff, 0xff, 0xa9, 0xcb, 0xcd, 0x18, 0x0f, 0x1c, 0x00, 0x00, } diff --git a/pkg/roachpb/data.proto b/pkg/roachpb/data.proto index 09d9208178ec..0730d2a4ca97 100644 --- a/pkg/roachpb/data.proto +++ b/pkg/roachpb/data.proto @@ -148,7 +148,8 @@ message MergeTrigger { // before it stopped responding to requests altogether (in anticipation of // being subsumed). It is suitable for use as the timestamp cache's low water // mark for the keys previously owned by the subsumed range. - util.hlc.Timestamp freeze_start = 5 [(gogoproto.nullable) = false]; + util.hlc.Timestamp freeze_start = 5 [(gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/util/hlc.ClockTimestamp"]; } // ReplicaChangeType is a parameter of ChangeReplicasTrigger. @@ -285,7 +286,8 @@ message ObservedTimestamp { option (gogoproto.populate) = true; int32 node_id = 1 [(gogoproto.customname) = "NodeID", (gogoproto.casttype) = "NodeID"]; - util.hlc.Timestamp timestamp = 2 [(gogoproto.nullable) = false]; + util.hlc.Timestamp timestamp = 2 [(gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/util/hlc.ClockTimestamp"]; } // A Transaction is a unit of work performed on the database. @@ -310,6 +312,10 @@ message Transaction { // The last time that the transaction's record was sent a heartbeat by its // coordinator to indicate client activity. Concurrent transactions will // avoid aborting a transaction if it observes recent-enough activity. + // + // NOTE: this could use a ClockTimestamp type, but doing so results in a + // large diff that doesn't seem worth it, given that we never feed this + // timestamp back into a clock. util.hlc.Timestamp last_heartbeat = 5 [(gogoproto.nullable) = false]; // This flag is set if the transaction's timestamp was "leaked" beyond the // transaction (e.g. via cluster_logical_timestamp()). If true, this prevents @@ -435,7 +441,7 @@ message TransactionRecord { option (gogoproto.populate) = true; // See comments on Transaction proto. - storage.enginepb.TxnMeta meta = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true]; + storage.enginepb.TxnMeta meta = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true]; TransactionStatus status = 4; util.hlc.Timestamp last_heartbeat = 5 [(gogoproto.nullable) = false]; repeated Span lock_spans = 11 [(gogoproto.nullable) = false]; @@ -525,7 +531,8 @@ message Lease { // to help with the rollout (such that a lease applied by some nodes before // the rollout and some nodes after the rollout is serialized the same). // TODO(andrei): Make this non-nullable after the rollout. - util.hlc.Timestamp proposed_ts = 5 [(gogoproto.customname) = "ProposedTS"]; + util.hlc.Timestamp proposed_ts = 5 [(gogoproto.customname) = "ProposedTS", + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/util/hlc.ClockTimestamp"]; // The epoch of the lease holder's node liveness entry. If this value is // non-zero, the expiration field is ignored. diff --git a/pkg/roachpb/data_test.go b/pkg/roachpb/data_test.go index d439bc9f2906..b6ae8d733c2e 100644 --- a/pkg/roachpb/data_test.go +++ b/pkg/roachpb/data_test.go @@ -40,15 +40,27 @@ import ( "go.etcd.io/etcd/raft/v3/raftpb" ) -func makeTS(walltime int64, logical int32) hlc.Timestamp { - return hlc.Timestamp{ +func makeClockTS(walltime int64, logical int32) hlc.ClockTimestamp { + return hlc.ClockTimestamp{ WallTime: walltime, Logical: logical, } } +func makeClockTSWithFlag(walltime int64, logical int32) hlc.ClockTimestamp { + return hlc.ClockTimestamp{ + WallTime: walltime, + Logical: logical, + Flags: uint32(hlc.TimestampFlag_SYNTHETIC), + } +} + +func makeTS(walltime int64, logical int32) hlc.Timestamp { + return makeClockTS(walltime, logical).ToTimestamp() +} + func makeTSWithFlag(walltime int64, logical int32) hlc.Timestamp { - return makeTS(walltime, logical).SetFlag(hlc.TimestampFlag_SYNTHETIC) + return makeClockTSWithFlag(walltime, logical).ToTimestamp() } // TestKeyNext tests that the method for creating lexicographic @@ -408,9 +420,9 @@ func TestTransactionObservedTimestamp(t *testing.T) { rng, seed := randutil.NewPseudoRand() t.Logf("running with seed %d", seed) ids := append([]int{109, 104, 102, 108, 1000}, rand.Perm(100)...) - timestamps := make(map[NodeID]hlc.Timestamp, len(ids)) + timestamps := make(map[NodeID]hlc.ClockTimestamp, len(ids)) for i := 0; i < len(ids); i++ { - timestamps[NodeID(i)] = hlc.Timestamp{WallTime: rng.Int63()} + timestamps[NodeID(i)] = hlc.ClockTimestamp{WallTime: rng.Int63()} } for i, n := range ids { nodeID := NodeID(n) @@ -418,7 +430,7 @@ func TestTransactionObservedTimestamp(t *testing.T) { t.Fatalf("%d: false positive hit %s in %v", nodeID, ts, ids[:i+1]) } txn.UpdateObservedTimestamp(nodeID, timestamps[nodeID]) - txn.UpdateObservedTimestamp(nodeID, hlc.MaxTimestamp) // should be noop + txn.UpdateObservedTimestamp(nodeID, hlc.MaxClockTimestamp) // should be noop if exp, act := i+1, len(txn.ObservedTimestamps); act != exp { t.Fatalf("%d: expected %d entries, got %d: %v", nodeID, exp, act, txn.ObservedTimestamps) } @@ -432,7 +444,7 @@ func TestTransactionObservedTimestamp(t *testing.T) { } var emptyTxn Transaction - ts := hlc.Timestamp{WallTime: 1, Logical: 2} + ts := hlc.ClockTimestamp{WallTime: 1, Logical: 2} emptyTxn.UpdateObservedTimestamp(NodeID(1), ts) if actTS, _ := emptyTxn.GetObservedTimestamp(NodeID(1)); actTS != ts { t.Fatalf("unexpected: %s (wanted %s)", actTS, ts) @@ -445,12 +457,12 @@ func TestFastPathObservedTimestamp(t *testing.T) { if _, ok := txn.GetObservedTimestamp(nodeID); ok { t.Errorf("fetched observed timestamp where none should exist") } - expTS := hlc.Timestamp{WallTime: 10} + expTS := hlc.ClockTimestamp{WallTime: 10} txn.UpdateObservedTimestamp(nodeID, expTS) if ts, ok := txn.GetObservedTimestamp(nodeID); !ok || !ts.Equal(expTS) { t.Errorf("expected %s; got %s", expTS, ts) } - expTS = hlc.Timestamp{WallTime: 9} + expTS = hlc.ClockTimestamp{WallTime: 9} txn.UpdateObservedTimestamp(nodeID, expTS) if ts, ok := txn.GetObservedTimestamp(nodeID); !ok || !ts.Equal(expTS) { t.Errorf("expected %s; got %s", expTS, ts) @@ -472,7 +484,7 @@ var nonZeroTxn = Transaction{ LastHeartbeat: makeTSWithFlag(1, 2), ReadTimestamp: makeTSWithFlag(20, 22), MaxTimestamp: makeTSWithFlag(40, 41), - ObservedTimestamps: []ObservedTimestamp{{NodeID: 1, Timestamp: makeTSWithFlag(1, 2)}}, + ObservedTimestamps: []ObservedTimestamp{{NodeID: 1, Timestamp: makeClockTSWithFlag(1, 2)}}, WriteTooOld: true, LockSpans: []Span{{Key: []byte("a"), EndKey: []byte("b")}}, InFlightWrites: []SequencedWrite{{Key: []byte("c"), Sequence: 1}}, @@ -923,20 +935,22 @@ func TestLeaseEquivalence(t *testing.T) { ts1 := makeTS(1, 1) ts2 := makeTS(2, 1) ts3 := makeTS(3, 1) + clockTS1 := makeClockTS(1, 1) + clockTS2 := makeClockTS(2, 1) epoch1 := Lease{Replica: r1, Start: ts1, Epoch: 1} epoch2 := Lease{Replica: r1, Start: ts1, Epoch: 2} - expire1 := Lease{Replica: r1, Start: ts1, Expiration: ts2.Clone()} - expire2 := Lease{Replica: r1, Start: ts1, Expiration: ts3.Clone()} + expire1 := Lease{Replica: r1, Start: ts1, Expiration: &ts2} + expire2 := Lease{Replica: r1, Start: ts1, Expiration: &ts3} epoch2TS2 := Lease{Replica: r2, Start: ts2, Epoch: 2} - expire2TS2 := Lease{Replica: r2, Start: ts2, Expiration: ts3.Clone()} + expire2TS2 := Lease{Replica: r2, Start: ts2, Expiration: &ts3} - proposed1 := Lease{Replica: r1, Start: ts1, Epoch: 1, ProposedTS: ts1.Clone()} - proposed2 := Lease{Replica: r1, Start: ts1, Epoch: 2, ProposedTS: ts1.Clone()} - proposed3 := Lease{Replica: r1, Start: ts1, Epoch: 1, ProposedTS: ts2.Clone()} + proposed1 := Lease{Replica: r1, Start: ts1, Epoch: 1, ProposedTS: &clockTS1} + proposed2 := Lease{Replica: r1, Start: ts1, Epoch: 2, ProposedTS: &clockTS1} + proposed3 := Lease{Replica: r1, Start: ts1, Epoch: 1, ProposedTS: &clockTS2} - stasis1 := Lease{Replica: r1, Start: ts1, Epoch: 1, DeprecatedStartStasis: ts1.Clone()} - stasis2 := Lease{Replica: r1, Start: ts1, Epoch: 1, DeprecatedStartStasis: ts2.Clone()} + stasis1 := Lease{Replica: r1, Start: ts1, Epoch: 1, DeprecatedStartStasis: &ts1} + stasis2 := Lease{Replica: r1, Start: ts1, Epoch: 1, DeprecatedStartStasis: &ts2} r1Voter, r1Learner := r1, r1 r1Voter.Type = ReplicaTypeVoterFull() @@ -984,7 +998,7 @@ func TestLeaseEquivalence(t *testing.T) { // Similar potential bug triggers, but these were actually handled correctly. DeprecatedStartStasis: new(hlc.Timestamp), - ProposedTS: &hlc.Timestamp{WallTime: 10}, + ProposedTS: &hlc.ClockTimestamp{WallTime: 10}, } postPRLease := prePRLease postPRLease.DeprecatedStartStasis = nil @@ -1001,7 +1015,7 @@ func TestLeaseEqual(t *testing.T) { Expiration *hlc.Timestamp Replica ReplicaDescriptor DeprecatedStartStasis *hlc.Timestamp - ProposedTS *hlc.Timestamp + ProposedTS *hlc.ClockTimestamp Epoch int64 Sequence LeaseSequence } @@ -1040,13 +1054,14 @@ func TestLeaseEqual(t *testing.T) { t.Fatalf("expectedly compared equal") } - ts := hlc.Timestamp{Logical: 1} + clockTS := hlc.ClockTimestamp{Logical: 1} + ts := clockTS.ToTimestamp() testCases := []Lease{ {Start: ts}, {Expiration: &ts}, {Replica: ReplicaDescriptor{NodeID: 1}}, {DeprecatedStartStasis: &ts}, - {ProposedTS: &ts}, + {ProposedTS: &clockTS}, {Epoch: 1}, {Sequence: 1}, } @@ -1616,7 +1631,7 @@ func TestUpdateObservedTimestamps(t *testing.T) { f := func(nodeID NodeID, walltime int64) ObservedTimestamp { return ObservedTimestamp{ NodeID: nodeID, - Timestamp: hlc.Timestamp{ + Timestamp: hlc.ClockTimestamp{ WallTime: walltime, }, } diff --git a/pkg/roachpb/errors.pb.go b/pkg/roachpb/errors.pb.go index 1dca723a70d7..5accc89482ad 100644 --- a/pkg/roachpb/errors.pb.go +++ b/pkg/roachpb/errors.pb.go @@ -27,6 +27,7 @@ import errorspb "github.com/cockroachdb/errors/errorspb" import hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" import github_com_cockroachdb_cockroach_pkg_util_uuid "github.com/cockroachdb/cockroach/pkg/util/uuid" +import github_com_cockroachdb_cockroach_pkg_util_hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" import io "io" @@ -139,7 +140,7 @@ func (x *TransactionAbortedReason) UnmarshalJSON(data []byte) error { return nil } func (TransactionAbortedReason) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_errors_06bf79933074ee74, []int{0} + return fileDescriptor_errors_b3c9ed361eb5091b, []int{0} } // TransactionRetryReason specifies what caused a transaction retry. @@ -190,7 +191,7 @@ func (x *TransactionRetryReason) UnmarshalJSON(data []byte) error { return nil } func (TransactionRetryReason) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_errors_06bf79933074ee74, []int{1} + return fileDescriptor_errors_b3c9ed361eb5091b, []int{1} } // TransactionRestart indicates how an error should be handled in a @@ -241,7 +242,7 @@ func (x *TransactionRestart) UnmarshalJSON(data []byte) error { return nil } func (TransactionRestart) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_errors_06bf79933074ee74, []int{2} + return fileDescriptor_errors_b3c9ed361eb5091b, []int{2} } // Reason specifies what caused the error. @@ -280,7 +281,7 @@ func (x *TransactionStatusError_Reason) UnmarshalJSON(data []byte) error { return nil } func (TransactionStatusError_Reason) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_errors_06bf79933074ee74, []int{9, 0} + return fileDescriptor_errors_b3c9ed361eb5091b, []int{9, 0} } // Reason specifies what caused the error. @@ -336,7 +337,7 @@ func (x *RangeFeedRetryError_Reason) UnmarshalJSON(data []byte) error { return nil } func (RangeFeedRetryError_Reason) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_errors_06bf79933074ee74, []int{27, 0} + return fileDescriptor_errors_b3c9ed361eb5091b, []int{27, 0} } // A NotLeaseHolderError indicates that the current range is not the lease @@ -367,7 +368,7 @@ func (m *NotLeaseHolderError) Reset() { *m = NotLeaseHolderError{} } func (m *NotLeaseHolderError) String() string { return proto.CompactTextString(m) } func (*NotLeaseHolderError) ProtoMessage() {} func (*NotLeaseHolderError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_06bf79933074ee74, []int{0} + return fileDescriptor_errors_b3c9ed361eb5091b, []int{0} } func (m *NotLeaseHolderError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -402,7 +403,7 @@ func (m *NodeUnavailableError) Reset() { *m = NodeUnavailableError{} } func (m *NodeUnavailableError) String() string { return proto.CompactTextString(m) } func (*NodeUnavailableError) ProtoMessage() {} func (*NodeUnavailableError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_06bf79933074ee74, []int{1} + return fileDescriptor_errors_b3c9ed361eb5091b, []int{1} } func (m *NodeUnavailableError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -436,7 +437,7 @@ func (m *UnsupportedRequestError) Reset() { *m = UnsupportedRequestError func (m *UnsupportedRequestError) String() string { return proto.CompactTextString(m) } func (*UnsupportedRequestError) ProtoMessage() {} func (*UnsupportedRequestError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_06bf79933074ee74, []int{2} + return fileDescriptor_errors_b3c9ed361eb5091b, []int{2} } func (m *UnsupportedRequestError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -473,7 +474,7 @@ func (m *RangeNotFoundError) Reset() { *m = RangeNotFoundError{} } func (m *RangeNotFoundError) String() string { return proto.CompactTextString(m) } func (*RangeNotFoundError) ProtoMessage() {} func (*RangeNotFoundError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_06bf79933074ee74, []int{3} + return fileDescriptor_errors_b3c9ed361eb5091b, []int{3} } func (m *RangeNotFoundError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -524,7 +525,7 @@ func (m *RangeKeyMismatchError) Reset() { *m = RangeKeyMismatchError{} } func (m *RangeKeyMismatchError) String() string { return proto.CompactTextString(m) } func (*RangeKeyMismatchError) ProtoMessage() {} func (*RangeKeyMismatchError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_06bf79933074ee74, []int{4} + return fileDescriptor_errors_b3c9ed361eb5091b, []int{4} } func (m *RangeKeyMismatchError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -569,7 +570,7 @@ type ReadWithinUncertaintyIntervalError struct { func (m *ReadWithinUncertaintyIntervalError) Reset() { *m = ReadWithinUncertaintyIntervalError{} } func (*ReadWithinUncertaintyIntervalError) ProtoMessage() {} func (*ReadWithinUncertaintyIntervalError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_06bf79933074ee74, []int{5} + return fileDescriptor_errors_b3c9ed361eb5091b, []int{5} } func (m *ReadWithinUncertaintyIntervalError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -612,7 +613,7 @@ func (m *TransactionAbortedError) Reset() { *m = TransactionAbortedError func (m *TransactionAbortedError) String() string { return proto.CompactTextString(m) } func (*TransactionAbortedError) ProtoMessage() {} func (*TransactionAbortedError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_06bf79933074ee74, []int{6} + return fileDescriptor_errors_b3c9ed361eb5091b, []int{6} } func (m *TransactionAbortedError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -648,7 +649,7 @@ func (m *TransactionPushError) Reset() { *m = TransactionPushError{} } func (m *TransactionPushError) String() string { return proto.CompactTextString(m) } func (*TransactionPushError) ProtoMessage() {} func (*TransactionPushError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_06bf79933074ee74, []int{7} + return fileDescriptor_errors_b3c9ed361eb5091b, []int{7} } func (m *TransactionPushError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -684,7 +685,7 @@ func (m *TransactionRetryError) Reset() { *m = TransactionRetryError{} } func (m *TransactionRetryError) String() string { return proto.CompactTextString(m) } func (*TransactionRetryError) ProtoMessage() {} func (*TransactionRetryError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_06bf79933074ee74, []int{8} + return fileDescriptor_errors_b3c9ed361eb5091b, []int{8} } func (m *TransactionRetryError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -724,7 +725,7 @@ func (m *TransactionStatusError) Reset() { *m = TransactionStatusError{} func (m *TransactionStatusError) String() string { return proto.CompactTextString(m) } func (*TransactionStatusError) ProtoMessage() {} func (*TransactionStatusError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_06bf79933074ee74, []int{9} + return fileDescriptor_errors_b3c9ed361eb5091b, []int{9} } func (m *TransactionStatusError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -762,7 +763,7 @@ func (m *WriteIntentError) Reset() { *m = WriteIntentError{} } func (m *WriteIntentError) String() string { return proto.CompactTextString(m) } func (*WriteIntentError) ProtoMessage() {} func (*WriteIntentError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_06bf79933074ee74, []int{10} + return fileDescriptor_errors_b3c9ed361eb5091b, []int{10} } func (m *WriteIntentError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -807,7 +808,7 @@ func (m *WriteTooOldError) Reset() { *m = WriteTooOldError{} } func (m *WriteTooOldError) String() string { return proto.CompactTextString(m) } func (*WriteTooOldError) ProtoMessage() {} func (*WriteTooOldError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_06bf79933074ee74, []int{11} + return fileDescriptor_errors_b3c9ed361eb5091b, []int{11} } func (m *WriteTooOldError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -843,7 +844,7 @@ func (m *OpRequiresTxnError) Reset() { *m = OpRequiresTxnError{} } func (m *OpRequiresTxnError) String() string { return proto.CompactTextString(m) } func (*OpRequiresTxnError) ProtoMessage() {} func (*OpRequiresTxnError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_06bf79933074ee74, []int{12} + return fileDescriptor_errors_b3c9ed361eb5091b, []int{12} } func (m *OpRequiresTxnError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -880,7 +881,7 @@ func (m *ConditionFailedError) Reset() { *m = ConditionFailedError{} } func (m *ConditionFailedError) String() string { return proto.CompactTextString(m) } func (*ConditionFailedError) ProtoMessage() {} func (*ConditionFailedError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_06bf79933074ee74, []int{13} + return fileDescriptor_errors_b3c9ed361eb5091b, []int{13} } func (m *ConditionFailedError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -917,7 +918,7 @@ func (m *LeaseRejectedError) Reset() { *m = LeaseRejectedError{} } func (m *LeaseRejectedError) String() string { return proto.CompactTextString(m) } func (*LeaseRejectedError) ProtoMessage() {} func (*LeaseRejectedError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_06bf79933074ee74, []int{14} + return fileDescriptor_errors_b3c9ed361eb5091b, []int{14} } func (m *LeaseRejectedError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -955,7 +956,7 @@ func (m *AmbiguousResultError) Reset() { *m = AmbiguousResultError{} } func (m *AmbiguousResultError) String() string { return proto.CompactTextString(m) } func (*AmbiguousResultError) ProtoMessage() {} func (*AmbiguousResultError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_06bf79933074ee74, []int{15} + return fileDescriptor_errors_b3c9ed361eb5091b, []int{15} } func (m *AmbiguousResultError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -989,7 +990,7 @@ func (m *RaftGroupDeletedError) Reset() { *m = RaftGroupDeletedError{} } func (m *RaftGroupDeletedError) String() string { return proto.CompactTextString(m) } func (*RaftGroupDeletedError) ProtoMessage() {} func (*RaftGroupDeletedError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_06bf79933074ee74, []int{16} + return fileDescriptor_errors_b3c9ed361eb5091b, []int{16} } func (m *RaftGroupDeletedError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1027,7 +1028,7 @@ func (m *ReplicaCorruptionError) Reset() { *m = ReplicaCorruptionError{} func (m *ReplicaCorruptionError) String() string { return proto.CompactTextString(m) } func (*ReplicaCorruptionError) ProtoMessage() {} func (*ReplicaCorruptionError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_06bf79933074ee74, []int{17} + return fileDescriptor_errors_b3c9ed361eb5091b, []int{17} } func (m *ReplicaCorruptionError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1064,7 +1065,7 @@ func (m *ReplicaTooOldError) Reset() { *m = ReplicaTooOldError{} } func (m *ReplicaTooOldError) String() string { return proto.CompactTextString(m) } func (*ReplicaTooOldError) ProtoMessage() {} func (*ReplicaTooOldError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_06bf79933074ee74, []int{18} + return fileDescriptor_errors_b3c9ed361eb5091b, []int{18} } func (m *ReplicaTooOldError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1099,7 +1100,7 @@ func (m *StoreNotFoundError) Reset() { *m = StoreNotFoundError{} } func (m *StoreNotFoundError) String() string { return proto.CompactTextString(m) } func (*StoreNotFoundError) ProtoMessage() {} func (*StoreNotFoundError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_06bf79933074ee74, []int{19} + return fileDescriptor_errors_b3c9ed361eb5091b, []int{19} } func (m *StoreNotFoundError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1144,7 +1145,7 @@ type UnhandledRetryableError struct { func (m *UnhandledRetryableError) Reset() { *m = UnhandledRetryableError{} } func (*UnhandledRetryableError) ProtoMessage() {} func (*UnhandledRetryableError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_06bf79933074ee74, []int{20} + return fileDescriptor_errors_b3c9ed361eb5091b, []int{20} } func (m *UnhandledRetryableError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1196,7 +1197,7 @@ func (m *TransactionRetryWithProtoRefreshError) Reset() { *m = Transacti func (m *TransactionRetryWithProtoRefreshError) String() string { return proto.CompactTextString(m) } func (*TransactionRetryWithProtoRefreshError) ProtoMessage() {} func (*TransactionRetryWithProtoRefreshError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_06bf79933074ee74, []int{21} + return fileDescriptor_errors_b3c9ed361eb5091b, []int{21} } func (m *TransactionRetryWithProtoRefreshError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1234,7 +1235,7 @@ func (m *TxnAlreadyEncounteredErrorError) Reset() { *m = TxnAlreadyEncou func (m *TxnAlreadyEncounteredErrorError) String() string { return proto.CompactTextString(m) } func (*TxnAlreadyEncounteredErrorError) ProtoMessage() {} func (*TxnAlreadyEncounteredErrorError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_06bf79933074ee74, []int{22} + return fileDescriptor_errors_b3c9ed361eb5091b, []int{22} } func (m *TxnAlreadyEncounteredErrorError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1271,7 +1272,7 @@ func (m *IntegerOverflowError) Reset() { *m = IntegerOverflowError{} } func (m *IntegerOverflowError) String() string { return proto.CompactTextString(m) } func (*IntegerOverflowError) ProtoMessage() {} func (*IntegerOverflowError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_06bf79933074ee74, []int{23} + return fileDescriptor_errors_b3c9ed361eb5091b, []int{23} } func (m *IntegerOverflowError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1307,7 +1308,7 @@ func (m *BatchTimestampBeforeGCError) Reset() { *m = BatchTimestampBefor func (m *BatchTimestampBeforeGCError) String() string { return proto.CompactTextString(m) } func (*BatchTimestampBeforeGCError) ProtoMessage() {} func (*BatchTimestampBeforeGCError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_06bf79933074ee74, []int{24} + return fileDescriptor_errors_b3c9ed361eb5091b, []int{24} } func (m *BatchTimestampBeforeGCError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1346,7 +1347,7 @@ func (m *IntentMissingError) Reset() { *m = IntentMissingError{} } func (m *IntentMissingError) String() string { return proto.CompactTextString(m) } func (*IntentMissingError) ProtoMessage() {} func (*IntentMissingError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_06bf79933074ee74, []int{25} + return fileDescriptor_errors_b3c9ed361eb5091b, []int{25} } func (m *IntentMissingError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1383,7 +1384,7 @@ func (m *MergeInProgressError) Reset() { *m = MergeInProgressError{} } func (m *MergeInProgressError) String() string { return proto.CompactTextString(m) } func (*MergeInProgressError) ProtoMessage() {} func (*MergeInProgressError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_06bf79933074ee74, []int{26} + return fileDescriptor_errors_b3c9ed361eb5091b, []int{26} } func (m *MergeInProgressError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1418,7 +1419,7 @@ func (m *RangeFeedRetryError) Reset() { *m = RangeFeedRetryError{} } func (m *RangeFeedRetryError) String() string { return proto.CompactTextString(m) } func (*RangeFeedRetryError) ProtoMessage() {} func (*RangeFeedRetryError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_06bf79933074ee74, []int{27} + return fileDescriptor_errors_b3c9ed361eb5091b, []int{27} } func (m *RangeFeedRetryError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1457,7 +1458,7 @@ func (m *IndeterminateCommitError) Reset() { *m = IndeterminateCommitErr func (m *IndeterminateCommitError) String() string { return proto.CompactTextString(m) } func (*IndeterminateCommitError) ProtoMessage() {} func (*IndeterminateCommitError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_06bf79933074ee74, []int{28} + return fileDescriptor_errors_b3c9ed361eb5091b, []int{28} } func (m *IndeterminateCommitError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1520,7 +1521,7 @@ func (m *ErrorDetail) Reset() { *m = ErrorDetail{} } func (m *ErrorDetail) String() string { return proto.CompactTextString(m) } func (*ErrorDetail) ProtoMessage() {} func (*ErrorDetail) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_06bf79933074ee74, []int{29} + return fileDescriptor_errors_b3c9ed361eb5091b, []int{29} } func (m *ErrorDetail) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2447,7 +2448,7 @@ func (m *ErrPosition) Reset() { *m = ErrPosition{} } func (m *ErrPosition) String() string { return proto.CompactTextString(m) } func (*ErrPosition) ProtoMessage() {} func (*ErrPosition) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_06bf79933074ee74, []int{30} + return fileDescriptor_errors_b3c9ed361eb5091b, []int{30} } func (m *ErrPosition) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2503,13 +2504,13 @@ type Error struct { Index *ErrPosition `protobuf:"bytes,7,opt,name=index" json:"index,omitempty"` // now is the current time at the node sending the response, // which can be used by the receiver to update its local HLC. - Now hlc.Timestamp `protobuf:"bytes,8,opt,name=now" json:"now"` + Now github_com_cockroachdb_cockroach_pkg_util_hlc.ClockTimestamp `protobuf:"bytes,8,opt,name=now,casttype=github.com/cockroachdb/cockroach/pkg/util/hlc.ClockTimestamp" json:"now"` } func (m *Error) Reset() { *m = Error{} } func (*Error) ProtoMessage() {} func (*Error) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_06bf79933074ee74, []int{31} + return fileDescriptor_errors_b3c9ed361eb5091b, []int{31} } func (m *Error) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8946,194 +8947,195 @@ var ( ErrIntOverflowErrors = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/errors.proto", fileDescriptor_errors_06bf79933074ee74) } +func init() { proto.RegisterFile("roachpb/errors.proto", fileDescriptor_errors_b3c9ed361eb5091b) } -var fileDescriptor_errors_06bf79933074ee74 = []byte{ - // 2971 bytes of a gzipped FileDescriptorProto +var fileDescriptor_errors_b3c9ed361eb5091b = []byte{ + // 2987 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x59, 0xcf, 0x6f, 0x1b, 0xc7, 0xf5, 0x27, 0x29, 0x4a, 0xa2, 0x9e, 0x7e, 0x78, 0x3d, 0x96, 0xe5, 0x95, 0x1c, 0x53, 0xca, 0xda, - 0x4e, 0x6c, 0x07, 0xa1, 0xbe, 0x70, 0xbe, 0x06, 0xbe, 0xc9, 0x37, 0x39, 0xf0, 0xc7, 0x4a, 0x5c, - 0x89, 0xbf, 0xba, 0xa4, 0x22, 0x3b, 0x46, 0x31, 0x59, 0x71, 0x47, 0xd4, 0xd6, 0xe4, 0x2e, 0x33, - 0xbb, 0x94, 0x25, 0xa0, 0x87, 0xa2, 0xbd, 0x14, 0x2d, 0x50, 0xf4, 0xd6, 0x1e, 0x0b, 0x04, 0x3d, - 0x14, 0x28, 0x8a, 0xfe, 0x05, 0x3d, 0xfb, 0x98, 0x63, 0x50, 0x14, 0x46, 0xeb, 0xf4, 0xd6, 0xff, - 0x20, 0xa7, 0x62, 0x7e, 0x2c, 0xb9, 0x14, 0x97, 0x8a, 0x92, 0x1b, 0xf7, 0xcd, 0x7b, 0x6f, 0xde, - 0xbc, 0x99, 0xf9, 0xbc, 0xcf, 0x1b, 0xc2, 0x2a, 0xf5, 0xac, 0xf6, 0x49, 0xff, 0x68, 0x9b, 0x50, - 0xea, 0x51, 0x3f, 0xd7, 0xa7, 0x5e, 0xe0, 0xa1, 0xeb, 0x6d, 0xaf, 0xfd, 0x82, 0x8f, 0xe4, 0x5e, - 0x9c, 0xe6, 0x5e, 0x9c, 0xf6, 0x8f, 0x36, 0x6e, 0x0a, 0x85, 0x0b, 0x9a, 0x1b, 0x28, 0xb4, 0xb7, - 0xad, 0xc0, 0x92, 0xb2, 0xb5, 0x50, 0xd6, 0x23, 0x81, 0x15, 0x91, 0xab, 0x83, 0xc0, 0xe9, 0x6e, - 0x9f, 0x74, 0xdb, 0xdb, 0x81, 0xd3, 0x23, 0x7e, 0x60, 0xf5, 0xfa, 0x72, 0x64, 0xb5, 0xe3, 0x75, - 0x3c, 0xfe, 0x73, 0x9b, 0xfd, 0x12, 0x52, 0xed, 0xaf, 0x29, 0xb8, 0x51, 0xf3, 0x82, 0x0a, 0xb1, - 0x7c, 0x52, 0xf6, 0xba, 0x36, 0xa1, 0x3a, 0x9b, 0x1a, 0x95, 0x60, 0x9e, 0x92, 0x7e, 0xd7, 0x69, - 0x5b, 0x6a, 0x72, 0x2b, 0xf9, 0x60, 0xf1, 0xf1, 0xbd, 0xdc, 0x28, 0x5e, 0x39, 0x77, 0xce, 0x14, - 0x1a, 0x25, 0xe2, 0xb7, 0xa9, 0xd3, 0x0f, 0x3c, 0x5a, 0x48, 0xbf, 0x7a, 0xbd, 0x99, 0x30, 0x43, - 0x53, 0xb4, 0x0b, 0x4b, 0x5d, 0xe6, 0x19, 0x9f, 0x70, 0xd7, 0x6a, 0xea, 0xea, 0xae, 0xcc, 0xc5, - 0xee, 0x28, 0x26, 0xf4, 0x04, 0x32, 0xd4, 0x72, 0x3b, 0x04, 0x3b, 0xb6, 0x3a, 0xb3, 0x95, 0x7c, - 0x30, 0x53, 0xd8, 0x60, 0x33, 0xbd, 0x79, 0xbd, 0x39, 0x6f, 0x32, 0xb9, 0x51, 0xfa, 0x76, 0xf4, - 0xd3, 0x9c, 0xe7, 0xba, 0x86, 0x8d, 0x72, 0x30, 0xcb, 0xbd, 0xa8, 0x69, 0x3e, 0xb1, 0x1a, 0x33, - 0x31, 0x5f, 0xb9, 0x29, 0xd4, 0xd0, 0x5d, 0x80, 0xf6, 0xc0, 0x0f, 0xbc, 0x1e, 0xee, 0xf9, 0x1d, - 0x75, 0x76, 0x2b, 0xf9, 0x60, 0x41, 0x2e, 0x69, 0x41, 0xc8, 0xab, 0x7e, 0x47, 0x5b, 0x83, 0xd5, - 0x9a, 0x67, 0x93, 0x03, 0xd7, 0x3a, 0xb5, 0x9c, 0xae, 0x75, 0xd4, 0x25, 0x3c, 0x65, 0xda, 0x3a, - 0xdc, 0x3a, 0x70, 0xfd, 0x41, 0xbf, 0xef, 0xd1, 0x80, 0xd8, 0x26, 0xf9, 0x62, 0x40, 0xfc, 0x40, - 0x0c, 0xfd, 0x3c, 0x09, 0x88, 0x07, 0x57, 0xf3, 0x82, 0x1d, 0x6f, 0xe0, 0xda, 0x22, 0xc9, 0xd1, - 0x55, 0x25, 0xaf, 0xbe, 0xaa, 0x27, 0x90, 0xf1, 0x03, 0x8f, 0x72, 0xb3, 0xd4, 0xb8, 0x59, 0x93, - 0xc9, 0x85, 0x99, 0xfc, 0x69, 0xce, 0x73, 0x5d, 0xc3, 0xd6, 0x7e, 0x37, 0x03, 0x37, 0xb9, 0xaf, - 0x7d, 0x72, 0x5e, 0x75, 0xfc, 0x9e, 0x15, 0xb4, 0x4f, 0x44, 0x1c, 0x1f, 0xc0, 0x75, 0x2a, 0xc2, - 0xc5, 0x7e, 0x60, 0xd1, 0x00, 0xbf, 0x20, 0xe7, 0x3c, 0xa0, 0xa5, 0xc2, 0xfc, 0xb7, 0xaf, 0x37, - 0x67, 0xf6, 0xc9, 0xb9, 0x79, 0x4d, 0x6a, 0x34, 0x99, 0xc2, 0x3e, 0x39, 0x47, 0xdb, 0x10, 0x8a, - 0x30, 0x71, 0x6d, 0x6e, 0x92, 0x1a, 0x37, 0x59, 0x96, 0xe3, 0xba, 0x6b, 0x33, 0x83, 0x13, 0xb8, - 0x6d, 0x93, 0x3e, 0x25, 0x6d, 0x2b, 0x20, 0x36, 0xee, 0xc9, 0x08, 0x88, 0x8d, 0xf9, 0xba, 0xf8, - 0xb6, 0x2e, 0x3e, 0xd6, 0xe2, 0xce, 0x06, 0x1b, 0x9f, 0x38, 0x64, 0xeb, 0x23, 0x67, 0xd5, 0xa1, - 0x2f, 0xae, 0x8a, 0x3e, 0x87, 0x8d, 0xc8, 0x4c, 0xfe, 0xa0, 0xd3, 0x21, 0x7e, 0x30, 0x9c, 0x28, - 0x7d, 0xd5, 0x89, 0x4c, 0x75, 0xe4, 0xa5, 0x19, 0x3a, 0x11, 0x33, 0x54, 0x60, 0x8e, 0x3b, 0xf3, - 0xd5, 0xd9, 0xad, 0x99, 0x07, 0x8b, 0x8f, 0xdf, 0x9a, 0xe6, 0xcd, 0x70, 0x8f, 0xbd, 0xc2, 0x9a, - 0xdc, 0x9e, 0x15, 0x61, 0x63, 0xb8, 0x01, 0xa1, 0xae, 0xd5, 0x35, 0xa5, 0x0f, 0xed, 0x3f, 0x29, - 0xd0, 0x4c, 0x62, 0xd9, 0x87, 0x4e, 0x70, 0xe2, 0xb8, 0x07, 0x6e, 0x9b, 0xd0, 0xc0, 0x72, 0xdc, - 0xe0, 0x9c, 0x6b, 0x9e, 0x5a, 0x5d, 0xb1, 0x4d, 0x7b, 0xb0, 0x42, 0x89, 0x65, 0xe3, 0xe1, 0xcd, - 0x96, 0x57, 0xf3, 0x4e, 0x64, 0x72, 0x76, 0xfd, 0x73, 0x27, 0xdd, 0x76, 0xae, 0x15, 0x2a, 0xc9, - 0x74, 0x2d, 0x33, 0xd3, 0xa1, 0x10, 0x99, 0x80, 0xc8, 0x99, 0xe3, 0x07, 0x8e, 0xdb, 0x89, 0xf8, - 0x4b, 0x5d, 0xdd, 0xdf, 0xf5, 0xd0, 0x7c, 0xe4, 0xb3, 0x00, 0xcb, 0x3d, 0xeb, 0x2c, 0xe2, 0x6e, - 0xe6, 0x0a, 0xee, 0xcc, 0xa5, 0x9e, 0x75, 0x36, 0xf2, 0xf1, 0x1c, 0x6e, 0x78, 0x47, 0x3e, 0xa1, - 0xa7, 0x24, 0xb2, 0x4e, 0x5f, 0x4d, 0xf3, 0x2c, 0xc7, 0x01, 0x47, 0x5d, 0x6a, 0x5f, 0x8c, 0x0f, - 0x79, 0x17, 0x07, 0xfc, 0x8f, 0xd2, 0xbf, 0xff, 0xc3, 0x66, 0x42, 0xb3, 0xe1, 0x56, 0x8b, 0x5a, - 0xae, 0x6f, 0xb5, 0x03, 0xc7, 0x73, 0xf3, 0x47, 0xfc, 0xba, 0x8a, 0x0c, 0x1b, 0x30, 0x47, 0x89, - 0xe5, 0x7b, 0x2e, 0xcf, 0xec, 0xca, 0xe3, 0xf7, 0x72, 0x13, 0x20, 0x9d, 0x9b, 0xb4, 0x35, 0xb9, - 0x89, 0x9c, 0x57, 0x3a, 0xd0, 0x9e, 0xc3, 0x6a, 0x44, 0xb3, 0x31, 0xf0, 0xe5, 0x5d, 0x2b, 0x02, - 0xf4, 0x07, 0xfe, 0x09, 0x21, 0x38, 0x38, 0x73, 0xe5, 0x06, 0x66, 0x63, 0xd6, 0x15, 0x31, 0x0e, - 0x21, 0x48, 0xd8, 0xb5, 0xce, 0x5c, 0xed, 0x17, 0x49, 0xb8, 0x19, 0x51, 0x30, 0x49, 0x40, 0xcf, - 0x85, 0xfb, 0xdd, 0x0b, 0x2b, 0x78, 0x78, 0xf9, 0x0a, 0xb8, 0x65, 0x5c, 0xfc, 0xe8, 0x6d, 0x58, - 0x20, 0x67, 0x01, 0xb5, 0x38, 0x12, 0xa6, 0x22, 0x48, 0x98, 0xe1, 0x62, 0x06, 0x84, 0x7f, 0x4b, - 0xc2, 0x5a, 0xc4, 0x57, 0x33, 0xb0, 0x82, 0x81, 0x2f, 0xc2, 0x58, 0x83, 0x19, 0x66, 0x97, 0x8c, - 0xd8, 0x31, 0x01, 0xaa, 0x0d, 0xc3, 0x4b, 0xf1, 0xf0, 0xfe, 0xe7, 0xf2, 0xf0, 0x22, 0x2e, 0x73, - 0xb1, 0x59, 0xfe, 0x18, 0xe6, 0x84, 0x1c, 0x21, 0x58, 0x31, 0xf5, 0x7c, 0xb3, 0x5e, 0xc3, 0x07, - 0xb5, 0xfd, 0x5a, 0xfd, 0xb0, 0xa6, 0x24, 0x90, 0x0a, 0xab, 0x52, 0xd6, 0x7a, 0x5a, 0xc3, 0xc5, - 0x7a, 0xb5, 0x6a, 0xb4, 0x5a, 0x7a, 0x49, 0x49, 0x69, 0xe9, 0x4c, 0x52, 0x49, 0x6a, 0x4d, 0x50, - 0x0e, 0xa9, 0x13, 0x10, 0x76, 0xcd, 0x5c, 0x01, 0xd5, 0xe8, 0x43, 0x98, 0x77, 0xf8, 0xa7, 0xaf, - 0x26, 0xf9, 0xa1, 0x5b, 0x8f, 0xd9, 0x1c, 0x61, 0x10, 0x56, 0x3b, 0xa9, 0xbf, 0x97, 0xce, 0xa4, - 0x94, 0x19, 0xed, 0x8f, 0x49, 0xe9, 0xb5, 0xe5, 0x79, 0xf5, 0xae, 0x3c, 0x58, 0x79, 0x58, 0xf8, - 0x41, 0xb7, 0x76, 0x64, 0x85, 0x6a, 0xa0, 0x58, 0xed, 0x60, 0x60, 0x75, 0x7f, 0xd8, 0x7d, 0xbd, - 0x26, 0x8c, 0x87, 0x62, 0x6d, 0x15, 0x50, 0xbd, 0xcf, 0xaa, 0x94, 0x43, 0x89, 0xdf, 0x3a, 0x73, - 0x45, 0xa5, 0x6a, 0xc2, 0x6a, 0xd1, 0x73, 0x6d, 0x87, 0x65, 0x7f, 0xc7, 0x72, 0xba, 0xe1, 0xcd, - 0xf8, 0x7f, 0x58, 0x92, 0xb3, 0x9f, 0x5a, 0xdd, 0x01, 0x91, 0x6b, 0x88, 0x2b, 0xa8, 0x9f, 0xb2, - 0x71, 0x73, 0x51, 0x68, 0xf3, 0x0f, 0xed, 0x2f, 0x49, 0x40, 0xa2, 0xce, 0x92, 0x9f, 0x90, 0xf6, - 0xf0, 0xb6, 0x65, 0x61, 0xbe, 0x47, 0x7c, 0xdf, 0xea, 0x90, 0xb1, 0x83, 0x12, 0x0a, 0xd1, 0xc7, - 0xb0, 0x20, 0x2b, 0x08, 0xb1, 0xe5, 0x52, 0xa7, 0x56, 0xf0, 0x30, 0x5f, 0x43, 0x03, 0xf4, 0x11, - 0x64, 0x42, 0x88, 0x92, 0x40, 0xf4, 0x5d, 0xc6, 0x43, 0x7d, 0xed, 0x0b, 0x58, 0xcd, 0xf7, 0x8e, - 0x9c, 0xce, 0xc0, 0x1b, 0xf8, 0x26, 0xf1, 0x07, 0xdd, 0xe0, 0x6a, 0x11, 0x7f, 0x08, 0x8b, 0x2f, - 0xa9, 0xd5, 0xef, 0x13, 0x1b, 0x13, 0x4a, 0x63, 0x62, 0x0e, 0xcf, 0x38, 0x77, 0x67, 0x82, 0x54, - 0xd6, 0x29, 0xd5, 0x6e, 0xb1, 0xe2, 0x7c, 0x1c, 0xec, 0x52, 0x6f, 0xd0, 0x2f, 0x91, 0x2e, 0x09, - 0xb3, 0xa4, 0x61, 0x58, 0x93, 0xe4, 0xa8, 0xe8, 0x51, 0x3a, 0xe8, 0xb3, 0x9d, 0x11, 0xd1, 0xb0, - 0x2b, 0xca, 0x7e, 0xe0, 0x8b, 0x57, 0x2d, 0xc3, 0xc5, 0x55, 0xbf, 0x83, 0x34, 0x58, 0xe8, 0x53, - 0xaf, 0x4d, 0x7c, 0x5f, 0xa6, 0x30, 0x33, 0x04, 0x93, 0x50, 0xac, 0x35, 0x01, 0xc9, 0x09, 0xa2, - 0x27, 0xf6, 0x13, 0x00, 0xc9, 0xe2, 0x42, 0x76, 0x32, 0x5b, 0xc8, 0xca, 0x3a, 0xb6, 0x20, 0xf5, - 0x39, 0xd1, 0x18, 0x7d, 0xb0, 0xec, 0x8b, 0x9f, 0xb6, 0xb6, 0x0f, 0x88, 0x13, 0x90, 0x09, 0xc2, - 0x33, 0x64, 0x2e, 0xc9, 0xab, 0x33, 0x97, 0x26, 0x63, 0x56, 0x27, 0x96, 0x6b, 0x77, 0x19, 0xd8, - 0x06, 0xf4, 0x7c, 0x48, 0xba, 0xd0, 0x63, 0x48, 0xf7, 0x75, 0x4a, 0x63, 0xce, 0xe3, 0x58, 0xaa, - 0xe5, 0xaa, 0xb9, 0xae, 0x2c, 0x03, 0xff, 0x4e, 0xc2, 0xfd, 0x8b, 0x48, 0xc8, 0x0a, 0x70, 0x83, - 0xf1, 0x62, 0x93, 0x1c, 0x53, 0x12, 0x42, 0xf6, 0x34, 0x30, 0x7b, 0x0e, 0x73, 0xc1, 0x99, 0x1b, - 0xb2, 0xb0, 0xa5, 0x42, 0x89, 0x0d, 0xfd, 0xfd, 0xf5, 0xe6, 0x07, 0x1d, 0x27, 0x38, 0x19, 0x1c, - 0xe5, 0xda, 0x5e, 0x6f, 0x7b, 0x18, 0x8f, 0x7d, 0x34, 0xfa, 0xbd, 0xdd, 0x7f, 0xd1, 0xd9, 0xe6, - 0x44, 0x7d, 0x30, 0x70, 0xec, 0xdc, 0xc1, 0x81, 0x51, 0x7a, 0xf3, 0x7a, 0x73, 0xb6, 0x75, 0xe6, - 0x1a, 0x25, 0x73, 0x36, 0x38, 0x73, 0x0d, 0x1b, 0xed, 0xc0, 0x62, 0x30, 0x8a, 0x4e, 0x9e, 0xe0, - 0xab, 0x15, 0x8a, 0xa8, 0xa1, 0xb6, 0x03, 0x9b, 0xad, 0x33, 0x37, 0xdf, 0x65, 0xe5, 0xff, 0x5c, - 0x77, 0xdb, 0xde, 0x80, 0x71, 0x0a, 0x79, 0xb8, 0xc4, 0xfa, 0xee, 0x02, 0xf4, 0x29, 0x39, 0xc5, - 0xfc, 0xd4, 0x8c, 0x2d, 0x73, 0x81, 0xc9, 0xc5, 0x31, 0xfc, 0x75, 0x12, 0x56, 0x19, 0xec, 0x75, - 0x08, 0xad, 0x9f, 0x12, 0x7a, 0xdc, 0xf5, 0x5e, 0x0a, 0xeb, 0x75, 0x98, 0x89, 0xa1, 0x8b, 0x4c, - 0x86, 0x1e, 0xc2, 0x72, 0x7b, 0x40, 0x29, 0x71, 0x03, 0x89, 0x1a, 0x82, 0xad, 0x0a, 0xdf, 0x4b, - 0x72, 0x88, 0x43, 0x04, 0x7a, 0x1f, 0xae, 0x39, 0x6e, 0x9b, 0x92, 0xde, 0x48, 0x79, 0x26, 0xa2, - 0xbc, 0x32, 0x1c, 0x14, 0x88, 0xf2, 0x65, 0x12, 0x6e, 0x17, 0x18, 0xe5, 0x1b, 0xc1, 0x1c, 0x39, - 0xf6, 0x28, 0xd9, 0x2d, 0x0e, 0xf1, 0xb6, 0xf5, 0x83, 0xf0, 0x76, 0xc4, 0x44, 0x98, 0x8b, 0x13, - 0x76, 0x08, 0xbc, 0xae, 0xfd, 0x7d, 0x80, 0x76, 0x64, 0xa5, 0xf5, 0x00, 0x89, 0x4a, 0x51, 0x75, - 0x7c, 0xdf, 0x71, 0x3b, 0x22, 0xb6, 0x8f, 0x61, 0xe9, 0x25, 0xf5, 0xdc, 0x0e, 0x16, 0x75, 0x43, - 0x86, 0x37, 0xbd, 0xcc, 0x98, 0x8b, 0x5c, 0x5d, 0x7c, 0x84, 0xe9, 0x4e, 0x4d, 0xa6, 0x9b, 0x35, - 0x26, 0x55, 0x42, 0x19, 0xe7, 0x6c, 0x50, 0xaf, 0x43, 0x89, 0x2f, 0x2a, 0xa7, 0xf6, 0x9b, 0x14, - 0xdc, 0xe0, 0x64, 0x74, 0x87, 0xc8, 0xfb, 0x23, 0x02, 0xd9, 0xbf, 0xc0, 0x15, 0xde, 0x8f, 0xb9, - 0x3d, 0x31, 0x76, 0xf1, 0x95, 0xf8, 0x4f, 0xc9, 0x61, 0x29, 0xde, 0x80, 0x35, 0x59, 0x76, 0x4d, - 0xbd, 0x51, 0x31, 0x8a, 0x79, 0x6c, 0xea, 0xd5, 0xfa, 0xa7, 0x7a, 0x49, 0x49, 0xa0, 0x35, 0x40, - 0xe1, 0x58, 0xbe, 0xb6, 0xab, 0xe3, 0x66, 0xa3, 0x62, 0xb4, 0x94, 0x24, 0xba, 0x05, 0x37, 0xc6, - 0xe4, 0x55, 0xdd, 0xdc, 0x65, 0x95, 0x3a, 0x52, 0xc3, 0xcd, 0xfc, 0x4e, 0x0b, 0x37, 0x6b, 0xf9, - 0x46, 0xb3, 0x5c, 0x6f, 0x29, 0x33, 0x28, 0x0b, 0x1b, 0x72, 0xa4, 0x52, 0xdf, 0x35, 0x8a, 0xf9, - 0x0a, 0xae, 0x37, 0x9a, 0xb8, 0x6a, 0x34, 0x9b, 0x46, 0x6d, 0x57, 0x49, 0x47, 0x2c, 0x9b, 0x95, - 0xfa, 0x21, 0x2e, 0xd6, 0x6b, 0xcd, 0x83, 0xaa, 0x6e, 0x2a, 0xb3, 0x9a, 0x05, 0xaa, 0xe1, 0xda, - 0x24, 0x20, 0xb4, 0xe7, 0xb8, 0x56, 0x40, 0x8a, 0x5e, 0xaf, 0xe7, 0x48, 0x88, 0xd7, 0x61, 0xd1, - 0x0f, 0xac, 0x0e, 0xe7, 0xc5, 0xdf, 0x93, 0xa0, 0x81, 0x34, 0x64, 0x0c, 0xed, 0xd5, 0x2a, 0x2c, - 0x72, 0x87, 0x25, 0x12, 0x58, 0x4e, 0x17, 0x99, 0xa0, 0xb8, 0x5e, 0x80, 0xc7, 0xba, 0x61, 0xe1, - 0xfb, 0x9d, 0x98, 0xac, 0xc7, 0x74, 0xe4, 0xe5, 0x84, 0xb9, 0xe2, 0x8e, 0x89, 0x51, 0x1d, 0xae, - 0x89, 0xf6, 0x91, 0x79, 0x3e, 0x66, 0x28, 0x2b, 0xcf, 0xe9, 0xfd, 0x69, 0x1b, 0x39, 0x86, 0xc6, - 0x65, 0xd6, 0x14, 0x44, 0xa5, 0xe8, 0x29, 0x20, 0xe1, 0xf0, 0x05, 0x39, 0x1f, 0x36, 0x68, 0x12, - 0x7a, 0x1e, 0x4c, 0xf3, 0x79, 0xb1, 0x9b, 0x2c, 0x27, 0x4c, 0x85, 0x5e, 0x18, 0x40, 0x3f, 0x4b, - 0xc2, 0x16, 0xef, 0x5d, 0x5e, 0xf2, 0x16, 0x07, 0x0f, 0x46, 0x3d, 0x0e, 0xbf, 0x06, 0xac, 0xc9, - 0x91, 0x8d, 0xd9, 0x93, 0xb8, 0x89, 0xbe, 0xb3, 0x39, 0x2a, 0x27, 0xcc, 0x3b, 0xf4, 0x32, 0x2d, - 0xf4, 0x63, 0xb8, 0x11, 0xc1, 0x45, 0x6c, 0x09, 0xee, 0xce, 0x9b, 0xfc, 0xc5, 0xc7, 0x8f, 0xae, - 0x44, 0xf4, 0xc3, 0x99, 0x50, 0x30, 0x31, 0x84, 0x5a, 0xa0, 0x44, 0xdd, 0x33, 0xae, 0xae, 0xce, - 0x71, 0xdf, 0xef, 0x5e, 0xee, 0x7b, 0xd8, 0x1a, 0x94, 0x13, 0xe6, 0xb5, 0x60, 0x5c, 0x8e, 0x0e, - 0xe1, 0x7a, 0xd4, 0x2b, 0x65, 0x97, 0x50, 0x9d, 0x9f, 0xba, 0x21, 0xb1, 0x3d, 0x01, 0xdb, 0x90, - 0xe0, 0xc2, 0x00, 0xfa, 0x0c, 0xa2, 0x8b, 0x60, 0x6d, 0x7f, 0x30, 0xf0, 0xd5, 0x0c, 0xf7, 0xfc, - 0xf0, 0xca, 0xa4, 0xbc, 0x9c, 0x30, 0xa3, 0xf1, 0x89, 0x11, 0x54, 0x66, 0x00, 0xe7, 0x04, 0x24, - 0x04, 0xb8, 0x05, 0xee, 0xf5, 0x6e, 0x8c, 0xd7, 0x8b, 0xec, 0xbb, 0x9c, 0x60, 0x60, 0x37, 0x94, - 0x21, 0x03, 0x96, 0x85, 0xa7, 0xc0, 0xf3, 0x30, 0xc3, 0x61, 0xb8, 0xdc, 0x55, 0x84, 0xc0, 0x0c, - 0x5d, 0x09, 0x19, 0xbb, 0x2c, 0x5e, 0x1f, 0x53, 0xc9, 0x77, 0xf9, 0xdd, 0x5e, 0x9c, 0x7a, 0x59, - 0x26, 0x89, 0x31, 0xbb, 0x2c, 0x5e, 0x54, 0xca, 0x36, 0xbc, 0x1d, 0x32, 0x65, 0x7c, 0xcc, 0xa9, - 0xb2, 0xba, 0x34, 0x75, 0xc3, 0xe3, 0x48, 0x35, 0xdb, 0xf0, 0xf6, 0xb8, 0x1c, 0xd5, 0x60, 0x45, - 0x60, 0x04, 0x95, 0x54, 0x59, 0x5d, 0x9e, 0x1a, 0xe5, 0x24, 0xa5, 0x66, 0x51, 0x76, 0xa3, 0x52, - 0x16, 0xa5, 0xeb, 0xd9, 0x04, 0x0f, 0x46, 0xaf, 0x55, 0xea, 0xca, 0xd4, 0x28, 0xe3, 0xde, 0xb5, - 0x58, 0x94, 0xee, 0xb8, 0x5c, 0x00, 0xc5, 0x71, 0x80, 0x3b, 0x8c, 0xad, 0x62, 0x5b, 0xd0, 0x55, - 0x55, 0xb9, 0x04, 0x28, 0x62, 0x98, 0xad, 0x00, 0x8a, 0xf1, 0x01, 0x76, 0x2e, 0x43, 0xda, 0xd9, - 0x1e, 0xd2, 0x5d, 0xf5, 0xfa, 0xd4, 0x73, 0x19, 0x4f, 0x8d, 0xd9, 0xb9, 0xa4, 0x17, 0x47, 0x38, - 0x5e, 0x4a, 0xdf, 0xe1, 0x79, 0x42, 0xd3, 0xf1, 0x72, 0x82, 0x12, 0x73, 0xbc, 0x8c, 0x4a, 0x59, - 0x72, 0xad, 0xb0, 0x4d, 0xc0, 0x94, 0xf7, 0x09, 0xea, 0xc6, 0xd4, 0xe4, 0xc6, 0x75, 0x14, 0x2c, - 0xb9, 0xd6, 0xb8, 0x9c, 0x85, 0x29, 0x48, 0xf2, 0x08, 0xd6, 0x6f, 0x4f, 0x0d, 0x73, 0x92, 0x64, - 0xb3, 0x30, 0xfd, 0xa8, 0x14, 0xfd, 0x2a, 0x09, 0xf7, 0x26, 0x50, 0x84, 0x23, 0x31, 0xe6, 0x8f, - 0xc0, 0x98, 0x0a, 0xb6, 0xab, 0xbe, 0xc5, 0xa7, 0xf9, 0xbf, 0x2b, 0x00, 0x4b, 0x2c, 0x51, 0x2e, - 0x27, 0xcc, 0xad, 0xe0, 0x3b, 0x14, 0x59, 0xce, 0x1c, 0x41, 0x23, 0xb1, 0x27, 0x79, 0xa4, 0xba, - 0x39, 0x35, 0x67, 0x71, 0x8c, 0x93, 0xe5, 0xcc, 0x19, 0x97, 0x33, 0x70, 0x1f, 0x8c, 0xde, 0x5e, - 0xb1, 0xec, 0x02, 0xd5, 0xad, 0xa9, 0xe0, 0x3e, 0xe5, 0xa5, 0x96, 0x81, 0xfb, 0x60, 0x62, 0x08, - 0x3d, 0x07, 0x65, 0xd8, 0x74, 0xe3, 0x23, 0xce, 0x34, 0x55, 0x8d, 0xfb, 0xce, 0xc5, 0xf8, 0xbe, - 0x84, 0x98, 0x72, 0x8c, 0x1f, 0x1f, 0x41, 0x2f, 0xe1, 0x0e, 0x6b, 0x23, 0x2c, 0x41, 0xd1, 0x31, - 0x19, 0x71, 0x74, 0xc9, 0xc8, 0xef, 0xf2, 0x99, 0x1e, 0xc7, 0x6d, 0xcb, 0xe5, 0xcc, 0xbe, 0x9c, - 0x30, 0x37, 0x82, 0xa9, 0x2a, 0x0c, 0x6b, 0x04, 0x42, 0xb3, 0x5a, 0xcf, 0xf8, 0xa9, 0x7a, 0x6f, - 0xea, 0x39, 0x9b, 0xe4, 0xb1, 0xec, 0x9c, 0x39, 0x51, 0x29, 0x3a, 0x80, 0xeb, 0x3d, 0xc6, 0x3f, - 0xb1, 0xe3, 0xb2, 0x83, 0xc5, 0x19, 0xa8, 0x7a, 0x7f, 0xea, 0xde, 0xc6, 0x71, 0x55, 0x96, 0x9f, - 0xde, 0xb8, 0x1c, 0xfd, 0x48, 0xd2, 0x9c, 0x63, 0xc2, 0x77, 0x96, 0x55, 0xc0, 0x77, 0xa6, 0x32, - 0xa7, 0x18, 0xbe, 0xca, 0x98, 0xd3, 0xd0, 0x81, 0xa8, 0x7e, 0x9f, 0xc3, 0xaa, 0x13, 0x25, 0x80, - 0xb8, 0xcd, 0x19, 0xa0, 0xfa, 0x2e, 0xf7, 0xfb, 0x5e, 0xec, 0xfa, 0xe3, 0xf9, 0x62, 0x39, 0x61, - 0xde, 0x70, 0x26, 0xc7, 0x0a, 0xf3, 0x30, 0xcb, 0xbb, 0x98, 0xbd, 0x74, 0xe6, 0x9a, 0xa2, 0xec, - 0xa5, 0x33, 0x37, 0x94, 0xd5, 0xbd, 0x74, 0x66, 0x55, 0xb9, 0xb9, 0x97, 0xce, 0xdc, 0x54, 0xd6, - 0xf6, 0xd2, 0x99, 0x35, 0xe5, 0xd6, 0x5e, 0x3a, 0x73, 0x4b, 0x51, 0xf7, 0xd2, 0x19, 0x55, 0x59, - 0xdf, 0x4b, 0x67, 0xd6, 0x95, 0x8d, 0xbd, 0x74, 0xe6, 0x8e, 0x92, 0xdd, 0x4b, 0x67, 0xb2, 0xca, - 0xe6, 0x5e, 0x3a, 0xf3, 0xb6, 0xa2, 0x69, 0x0f, 0x39, 0x93, 0x6c, 0x78, 0x3e, 0xaf, 0x13, 0x68, - 0x03, 0x66, 0xd9, 0x84, 0x67, 0xb2, 0x27, 0x17, 0xd4, 0x53, 0x88, 0xb4, 0x57, 0x69, 0x98, 0x0d, - 0x9f, 0xf4, 0x2f, 0xbc, 0x54, 0xac, 0xcb, 0x46, 0xfb, 0x7a, 0xe4, 0xd9, 0x5c, 0x28, 0x8c, 0x9e, - 0x2f, 0x7e, 0x3a, 0x4e, 0x91, 0x28, 0xe1, 0xff, 0x06, 0x70, 0x02, 0xb8, 0x12, 0x7b, 0x2a, 0xc6, - 0x60, 0x81, 0x2b, 0x17, 0xee, 0xc9, 0x79, 0xde, 0x1a, 0xcd, 0x33, 0xa9, 0x35, 0xc6, 0xa0, 0xa4, - 0x0c, 0x15, 0x61, 0x79, 0xe0, 0x92, 0xb3, 0xbe, 0xe7, 0x13, 0x9b, 0xd7, 0xe7, 0xf4, 0x55, 0xb8, - 0xb7, 0xb9, 0x34, 0x34, 0x62, 0x55, 0x79, 0x1b, 0x16, 0x3d, 0xea, 0x74, 0x1c, 0x17, 0xb3, 0x9a, - 0xc5, 0xd9, 0xdd, 0x6c, 0x61, 0x85, 0xc5, 0xf4, 0xed, 0xeb, 0xcd, 0x39, 0x56, 0xdf, 0x8c, 0x92, - 0x09, 0x42, 0x85, 0x7d, 0xa1, 0x06, 0xcc, 0xd9, 0x9c, 0xa2, 0x4b, 0xb6, 0x96, 0x9d, 0xf6, 0x84, - 0x20, 0x88, 0x7c, 0x41, 0x95, 0xeb, 0x53, 0x46, 0xeb, 0x13, 0x23, 0xa6, 0xf4, 0x83, 0xfe, 0x37, - 0xdc, 0xa0, 0xf9, 0xcb, 0x1c, 0x86, 0xfb, 0x29, 0xb7, 0x0e, 0x3d, 0x81, 0x19, 0xd7, 0x7b, 0x29, - 0x19, 0xd8, 0x95, 0x1a, 0x4d, 0xa6, 0x8f, 0xf6, 0x61, 0x99, 0x01, 0x86, 0x3d, 0x04, 0x0b, 0x41, - 0xb6, 0xb6, 0x22, 0x0e, 0xc2, 0x3f, 0x15, 0x73, 0xba, 0x50, 0x8c, 0x3e, 0x88, 0x2c, 0x91, 0x88, - 0x4c, 0x3c, 0x8c, 0x88, 0x67, 0xcc, 0x47, 0xff, 0x48, 0x81, 0x3a, 0xed, 0xa9, 0x9b, 0xb5, 0x56, - 0xf9, 0x42, 0xdd, 0x6c, 0xe1, 0x89, 0x27, 0xd7, 0xfb, 0xf0, 0xf6, 0xd8, 0x08, 0xff, 0xd0, 0x4b, - 0xd8, 0xd4, 0x8b, 0x75, 0xb3, 0x84, 0x77, 0xea, 0x07, 0xb5, 0x92, 0x92, 0x64, 0xbd, 0xdb, 0x98, - 0x5a, 0xb1, 0x62, 0xe8, 0x35, 0xf6, 0xb5, 0xa7, 0x17, 0x59, 0x6f, 0xb7, 0x09, 0xb7, 0xc7, 0xc6, - 0x1b, 0x07, 0xcd, 0xb2, 0x6e, 0x86, 0xde, 0x94, 0x34, 0xba, 0x0d, 0xb7, 0x26, 0xe7, 0xc1, 0xcd, - 0x46, 0xbe, 0xa6, 0xcc, 0xa2, 0x3c, 0x7c, 0x32, 0x3e, 0x58, 0x31, 0xf5, 0x7c, 0xe9, 0xd9, 0xe8, - 0x05, 0x18, 0xd7, 0x4d, 0x6c, 0xd6, 0x2b, 0x15, 0xbd, 0x84, 0x0b, 0xf9, 0xe2, 0x3e, 0x6e, 0xd4, - 0x9b, 0x4d, 0xa3, 0x50, 0xd1, 0x79, 0xc3, 0x9a, 0x7f, 0xa6, 0xcc, 0xa1, 0x77, 0xe1, 0xee, 0x98, - 0x8b, 0x9a, 0x7e, 0x88, 0x2b, 0x7a, 0xbe, 0xa9, 0xe3, 0x86, 0xa9, 0x7f, 0xaa, 0xd7, 0x5a, 0x4d, - 0xdc, 0x7a, 0x5a, 0x53, 0x32, 0xe8, 0x21, 0xdc, 0x1f, 0x53, 0x6c, 0x19, 0x55, 0xbd, 0xd9, 0xca, - 0x57, 0x1b, 0xb8, 0x98, 0x2f, 0x96, 0x75, 0xb9, 0x24, 0xbd, 0xa4, 0xcc, 0x6f, 0xa4, 0x7f, 0xf9, - 0x65, 0x36, 0xa1, 0xb1, 0xf4, 0xa6, 0x1e, 0xfd, 0x79, 0xfc, 0xed, 0x3c, 0xf2, 0x0e, 0x2f, 0xfa, - 0xd6, 0x96, 0xf9, 0x6c, 0x32, 0xb9, 0xbc, 0x49, 0x66, 0x23, 0x87, 0xa6, 0xd1, 0xd2, 0x71, 0xab, - 0x5e, 0xc7, 0xf5, 0x0a, 0x4b, 0x27, 0xef, 0xaa, 0xd9, 0x40, 0x53, 0x37, 0x8d, 0x7c, 0xc5, 0xf8, - 0x2c, 0x5f, 0xa8, 0xe8, 0xca, 0x0c, 0xba, 0x03, 0xeb, 0x42, 0x9e, 0x6f, 0x3e, 0xab, 0x15, 0xa5, - 0xd9, 0x4e, 0xde, 0xa8, 0x1c, 0x98, 0xba, 0x32, 0x8b, 0x34, 0xc8, 0x8a, 0x61, 0x91, 0x18, 0x5c, - 0xd2, 0xf3, 0xa5, 0x8a, 0x51, 0xd3, 0xb1, 0xfe, 0xb4, 0xa8, 0xeb, 0x25, 0xbd, 0xa4, 0xcc, 0x89, - 0xa0, 0x1f, 0x7d, 0x04, 0x68, 0xf2, 0x16, 0xa3, 0x0c, 0xa4, 0x6b, 0xf5, 0x9a, 0xae, 0x24, 0xd0, - 0x22, 0xcc, 0xb3, 0x44, 0xd6, 0x77, 0x76, 0x94, 0x24, 0x5a, 0x86, 0x05, 0xa3, 0x5a, 0xd5, 0x4b, - 0x46, 0xbe, 0xa5, 0x2b, 0xa9, 0xc2, 0xc3, 0x57, 0xff, 0xca, 0x26, 0x5e, 0xbd, 0xc9, 0x26, 0xbf, - 0x7a, 0x93, 0x4d, 0x7e, 0xfd, 0x26, 0x9b, 0xfc, 0xe7, 0x9b, 0x6c, 0xf2, 0xb7, 0xdf, 0x64, 0x13, - 0x5f, 0x7d, 0x93, 0x4d, 0x7c, 0xfd, 0x4d, 0x36, 0xf1, 0xd9, 0xbc, 0xbc, 0xd7, 0xff, 0x0d, 0x00, - 0x00, 0xff, 0xff, 0xfc, 0x55, 0xd0, 0x11, 0x2a, 0x1f, 0x00, 0x00, + 0x4e, 0x6c, 0x07, 0xa1, 0xbe, 0x70, 0xbe, 0x01, 0x9a, 0xd4, 0x39, 0xf0, 0xc7, 0x4a, 0xa4, 0xc4, + 0x5f, 0x5d, 0x52, 0xb1, 0x1d, 0xa3, 0x98, 0xac, 0xb8, 0x23, 0x6a, 0x6b, 0x72, 0x97, 0x99, 0xdd, + 0x95, 0x25, 0xa0, 0x87, 0xa2, 0xbd, 0x04, 0x2d, 0x50, 0xf4, 0xd6, 0x1e, 0x0b, 0x04, 0x3d, 0x14, + 0x28, 0x8a, 0xfe, 0x05, 0x3d, 0xfb, 0x98, 0x63, 0x50, 0x14, 0x46, 0xeb, 0xf4, 0xd6, 0xff, 0x20, + 0xa7, 0x62, 0x7e, 0x2c, 0xb9, 0x14, 0x97, 0xb2, 0x92, 0x1b, 0xf7, 0xcd, 0x7b, 0x6f, 0xde, 0xbc, + 0x99, 0xf9, 0xbc, 0xcf, 0x1b, 0xc2, 0x2a, 0x75, 0xcd, 0xce, 0xf1, 0xe0, 0x70, 0x9b, 0x50, 0xea, + 0x52, 0x2f, 0x37, 0xa0, 0xae, 0xef, 0xa2, 0xab, 0x1d, 0xb7, 0xf3, 0x9c, 0x8f, 0xe4, 0x9e, 0x9f, + 0xe4, 0x9e, 0x9f, 0x0c, 0x0e, 0x37, 0xae, 0x0b, 0x85, 0x73, 0x9a, 0x1b, 0x28, 0xb4, 0xb7, 0x4c, + 0xdf, 0x94, 0xb2, 0xb5, 0x50, 0xd6, 0x27, 0xbe, 0x19, 0x91, 0xab, 0x81, 0x6f, 0xf7, 0xb6, 0x8f, + 0x7b, 0x9d, 0x6d, 0xdf, 0xee, 0x13, 0xcf, 0x37, 0xfb, 0x03, 0x39, 0xb2, 0xda, 0x75, 0xbb, 0x2e, + 0xff, 0xb9, 0xcd, 0x7e, 0x09, 0xa9, 0xf6, 0xb7, 0x14, 0x5c, 0xab, 0xbb, 0x7e, 0x95, 0x98, 0x1e, + 0x29, 0xbb, 0x3d, 0x8b, 0x50, 0x9d, 0x4d, 0x8d, 0x4a, 0x30, 0x4f, 0xc9, 0xa0, 0x67, 0x77, 0x4c, + 0x35, 0xb9, 0x95, 0xbc, 0xb7, 0xf8, 0xf0, 0x4e, 0x6e, 0x14, 0xaf, 0x9c, 0x3b, 0x67, 0x08, 0x8d, + 0x12, 0xf1, 0x3a, 0xd4, 0x1e, 0xf8, 0x2e, 0x2d, 0xa4, 0x5f, 0xbe, 0xda, 0x4c, 0x18, 0xa1, 0x29, + 0xda, 0x85, 0xa5, 0x1e, 0xf3, 0x8c, 0x8f, 0xb9, 0x6b, 0x35, 0x75, 0x79, 0x57, 0xc6, 0x62, 0x6f, + 0x14, 0x13, 0xfa, 0x10, 0x32, 0xd4, 0x74, 0xba, 0x04, 0xdb, 0x96, 0x3a, 0xb3, 0x95, 0xbc, 0x37, + 0x53, 0xd8, 0x60, 0x33, 0xbd, 0x7e, 0xb5, 0x39, 0x6f, 0x30, 0x79, 0xa5, 0xf4, 0xdd, 0xe8, 0xa7, + 0x31, 0xcf, 0x75, 0x2b, 0x16, 0xca, 0xc1, 0x2c, 0xf7, 0xa2, 0xa6, 0xf9, 0xc4, 0x6a, 0xcc, 0xc4, + 0x7c, 0xe5, 0x86, 0x50, 0x43, 0xb7, 0x01, 0x3a, 0x81, 0xe7, 0xbb, 0x7d, 0xdc, 0xf7, 0xba, 0xea, + 0xec, 0x56, 0xf2, 0xde, 0x82, 0x5c, 0xd2, 0x82, 0x90, 0xd7, 0xbc, 0xae, 0xb6, 0x06, 0xab, 0x75, + 0xd7, 0x22, 0x07, 0x8e, 0x79, 0x62, 0xda, 0x3d, 0xf3, 0xb0, 0x47, 0x78, 0xca, 0xb4, 0x75, 0xb8, + 0x71, 0xe0, 0x78, 0xc1, 0x60, 0xe0, 0x52, 0x9f, 0x58, 0x06, 0xf9, 0x22, 0x20, 0x9e, 0x2f, 0x86, + 0x7e, 0x99, 0x04, 0xc4, 0x83, 0xab, 0xbb, 0xfe, 0x8e, 0x1b, 0x38, 0x96, 0x48, 0x72, 0x74, 0x55, + 0xc9, 0xcb, 0xaf, 0xea, 0x43, 0xc8, 0x78, 0xbe, 0x4b, 0xb9, 0x59, 0x6a, 0xdc, 0xac, 0xc5, 0xe4, + 0xc2, 0x4c, 0xfe, 0x34, 0xe6, 0xb9, 0x6e, 0xc5, 0xd2, 0x7e, 0x3f, 0x03, 0xd7, 0xb9, 0xaf, 0x7d, + 0x72, 0x56, 0xb3, 0xbd, 0xbe, 0xe9, 0x77, 0x8e, 0x45, 0x1c, 0x1f, 0xc0, 0x55, 0x2a, 0xc2, 0xc5, + 0x9e, 0x6f, 0x52, 0x1f, 0x3f, 0x27, 0x67, 0x3c, 0xa0, 0xa5, 0xc2, 0xfc, 0x77, 0xaf, 0x36, 0x67, + 0xf6, 0xc9, 0x99, 0x71, 0x45, 0x6a, 0xb4, 0x98, 0xc2, 0x3e, 0x39, 0x43, 0xdb, 0x10, 0x8a, 0x30, + 0x71, 0x2c, 0x6e, 0x92, 0x1a, 0x37, 0x59, 0x96, 0xe3, 0xba, 0x63, 0x31, 0x83, 0x63, 0xb8, 0x69, + 0x91, 0x01, 0x25, 0x1d, 0xd3, 0x27, 0x16, 0xee, 0xcb, 0x08, 0x88, 0x85, 0xf9, 0xba, 0xf8, 0xb6, + 0x2e, 0x3e, 0xd4, 0xe2, 0xce, 0x06, 0x1b, 0x9f, 0x38, 0x64, 0xeb, 0x23, 0x67, 0xb5, 0xa1, 0x2f, + 0xae, 0x8a, 0x3e, 0x87, 0x8d, 0xc8, 0x4c, 0x5e, 0xd0, 0xed, 0x12, 0xcf, 0x1f, 0x4e, 0x94, 0xbe, + 0xec, 0x44, 0x86, 0x3a, 0xf2, 0xd2, 0x0a, 0x9d, 0x88, 0x19, 0xaa, 0x30, 0xc7, 0x9d, 0x79, 0xea, + 0xec, 0xd6, 0xcc, 0xbd, 0xc5, 0x87, 0x6f, 0x4d, 0xf3, 0x56, 0x71, 0x8e, 0xdc, 0xc2, 0x9a, 0xdc, + 0x9e, 0x15, 0x61, 0x53, 0x71, 0x7c, 0x42, 0x1d, 0xb3, 0x67, 0x48, 0x1f, 0xda, 0x7f, 0x53, 0xa0, + 0x19, 0xc4, 0xb4, 0x1e, 0xdb, 0xfe, 0xb1, 0xed, 0x1c, 0x38, 0x1d, 0x42, 0x7d, 0xd3, 0x76, 0xfc, + 0x33, 0xae, 0x79, 0x62, 0xf6, 0xc4, 0x36, 0xed, 0xc1, 0x0a, 0x25, 0xa6, 0x85, 0x87, 0x37, 0x5b, + 0x5e, 0xcd, 0x5b, 0x91, 0xc9, 0xd9, 0xf5, 0xcf, 0x1d, 0xf7, 0x3a, 0xb9, 0x76, 0xa8, 0x24, 0xd3, + 0xb5, 0xcc, 0x4c, 0x87, 0x42, 0x64, 0x00, 0x22, 0xa7, 0xb6, 0xe7, 0xdb, 0x4e, 0x37, 0xe2, 0x2f, + 0x75, 0x79, 0x7f, 0x57, 0x43, 0xf3, 0x91, 0xcf, 0x02, 0x2c, 0xf7, 0xcd, 0xd3, 0x88, 0xbb, 0x99, + 0x4b, 0xb8, 0x33, 0x96, 0xfa, 0xe6, 0xe9, 0xc8, 0xc7, 0x33, 0xb8, 0xe6, 0x1e, 0x7a, 0x84, 0x9e, + 0x90, 0xc8, 0x3a, 0x3d, 0x35, 0xcd, 0xb3, 0x1c, 0x07, 0x1c, 0x0d, 0xa9, 0x7d, 0x3e, 0x3e, 0xe4, + 0x9e, 0x1f, 0xf0, 0x3e, 0x4e, 0xff, 0xe1, 0x8f, 0x9b, 0x09, 0xcd, 0x82, 0x1b, 0x6d, 0x6a, 0x3a, + 0x9e, 0xd9, 0xf1, 0x6d, 0xd7, 0xc9, 0x1f, 0xf2, 0xeb, 0x2a, 0x32, 0x5c, 0x81, 0x39, 0x4a, 0x4c, + 0xcf, 0x75, 0x78, 0x66, 0x57, 0x1e, 0xbe, 0x97, 0x9b, 0x00, 0xe9, 0xdc, 0xa4, 0xad, 0xc1, 0x4d, + 0xe4, 0xbc, 0xd2, 0x81, 0xf6, 0x0c, 0x56, 0x23, 0x9a, 0xcd, 0xc0, 0x93, 0x77, 0xad, 0x08, 0x30, + 0x08, 0xbc, 0x63, 0x42, 0xb0, 0x7f, 0xea, 0xc8, 0x0d, 0xcc, 0xc6, 0xac, 0x2b, 0x62, 0x1c, 0x42, + 0x90, 0xb0, 0x6b, 0x9f, 0x3a, 0xda, 0xaf, 0x92, 0x70, 0x3d, 0xa2, 0x60, 0x10, 0x9f, 0x9e, 0x09, + 0xf7, 0xbb, 0xe7, 0x56, 0x70, 0xff, 0xe2, 0x15, 0x70, 0xcb, 0xb8, 0xf8, 0xd1, 0xdb, 0xb0, 0x40, + 0x4e, 0x7d, 0x6a, 0x72, 0x24, 0x4c, 0x45, 0x90, 0x30, 0xc3, 0xc5, 0x0c, 0x08, 0xff, 0x9e, 0x84, + 0xb5, 0x88, 0xaf, 0x96, 0x6f, 0xfa, 0x81, 0x27, 0xc2, 0x58, 0x83, 0x19, 0x66, 0x97, 0x8c, 0xd8, + 0x31, 0x01, 0xaa, 0x0f, 0xc3, 0x4b, 0xf1, 0xf0, 0xfe, 0xef, 0xe2, 0xf0, 0x22, 0x2e, 0x73, 0xb1, + 0x59, 0x7e, 0x04, 0x73, 0x42, 0x8e, 0x10, 0xac, 0x18, 0x7a, 0xbe, 0xd5, 0xa8, 0xe3, 0x83, 0xfa, + 0x7e, 0xbd, 0xf1, 0xb8, 0xae, 0x24, 0x90, 0x0a, 0xab, 0x52, 0xd6, 0x7e, 0x52, 0xc7, 0xc5, 0x46, + 0xad, 0x56, 0x69, 0xb7, 0xf5, 0x92, 0x92, 0xd2, 0xd2, 0x99, 0xa4, 0x92, 0xd4, 0x5a, 0xa0, 0x3c, + 0xa6, 0xb6, 0x4f, 0xd8, 0x35, 0x73, 0x04, 0x54, 0xa3, 0x8f, 0x60, 0xde, 0xe6, 0x9f, 0x9e, 0x9a, + 0xe4, 0x87, 0x6e, 0x3d, 0x66, 0x73, 0x84, 0x41, 0x58, 0xed, 0xa4, 0xfe, 0x5e, 0x3a, 0x93, 0x52, + 0x66, 0xb4, 0x3f, 0x25, 0xa5, 0xd7, 0xb6, 0xeb, 0x36, 0x7a, 0xf2, 0x60, 0xe5, 0x61, 0xe1, 0x07, + 0xdd, 0xda, 0x91, 0x15, 0xaa, 0x83, 0x62, 0x76, 0xfc, 0xc0, 0xec, 0xfd, 0xb0, 0xfb, 0x7a, 0x45, + 0x18, 0x0f, 0xc5, 0xda, 0x2a, 0xa0, 0xc6, 0x80, 0x55, 0x29, 0x9b, 0x12, 0xaf, 0x7d, 0xea, 0x88, + 0x4a, 0xd5, 0x82, 0xd5, 0xa2, 0xeb, 0x58, 0x36, 0xcb, 0xfe, 0x8e, 0x69, 0xf7, 0xc2, 0x9b, 0xf1, + 0x63, 0x58, 0x92, 0xb3, 0x9f, 0x98, 0xbd, 0x80, 0xc8, 0x35, 0xc4, 0x15, 0xd4, 0x4f, 0xd9, 0xb8, + 0xb1, 0x28, 0xb4, 0xf9, 0x87, 0xf6, 0xd7, 0x24, 0x20, 0x51, 0x67, 0xc9, 0xcf, 0x48, 0x67, 0x78, + 0xdb, 0xb2, 0x30, 0xdf, 0x27, 0x9e, 0x67, 0x76, 0xc9, 0xd8, 0x41, 0x09, 0x85, 0xe8, 0x11, 0x2c, + 0xc8, 0x0a, 0x42, 0x2c, 0xb9, 0xd4, 0xa9, 0x15, 0x3c, 0xcc, 0xd7, 0xd0, 0x00, 0x7d, 0x0c, 0x99, + 0x10, 0xa2, 0x24, 0x10, 0xbd, 0xc9, 0x78, 0xa8, 0xaf, 0x7d, 0x01, 0xab, 0xf9, 0xfe, 0xa1, 0xdd, + 0x0d, 0xdc, 0xc0, 0x33, 0x88, 0x17, 0xf4, 0xfc, 0xcb, 0x45, 0xfc, 0x11, 0x2c, 0xbe, 0xa0, 0xe6, + 0x60, 0x40, 0x2c, 0x4c, 0x28, 0x8d, 0x89, 0x39, 0x3c, 0xe3, 0xdc, 0x9d, 0x01, 0x52, 0x59, 0xa7, + 0x54, 0xbb, 0xc1, 0x8a, 0xf3, 0x91, 0xbf, 0x4b, 0xdd, 0x60, 0x50, 0x22, 0x3d, 0x12, 0x66, 0x49, + 0xc3, 0xb0, 0x26, 0xc9, 0x51, 0xd1, 0xa5, 0x34, 0x18, 0xb0, 0x9d, 0x11, 0xd1, 0xb0, 0x2b, 0xca, + 0x7e, 0xe0, 0xf3, 0x57, 0x2d, 0xc3, 0xc5, 0x35, 0xaf, 0x8b, 0x34, 0x58, 0x18, 0x50, 0xb7, 0x43, + 0x3c, 0x4f, 0xa6, 0x30, 0x33, 0x04, 0x93, 0x50, 0xac, 0xb5, 0x00, 0xc9, 0x09, 0xa2, 0x27, 0xf6, + 0x13, 0x00, 0xc9, 0xe2, 0x42, 0x76, 0x32, 0x5b, 0xc8, 0xca, 0x3a, 0xb6, 0x20, 0xf5, 0x39, 0xd1, + 0x18, 0x7d, 0xb0, 0xec, 0x8b, 0x9f, 0x96, 0xb6, 0x0f, 0x88, 0x13, 0x90, 0x09, 0xc2, 0x33, 0x64, + 0x2e, 0xc9, 0xcb, 0x33, 0x97, 0x16, 0x63, 0x56, 0xc7, 0xa6, 0x63, 0xf5, 0x18, 0xd8, 0xfa, 0xf4, + 0x6c, 0x48, 0xba, 0xd0, 0x43, 0x48, 0x0f, 0x74, 0x4a, 0x63, 0xce, 0xe3, 0x58, 0xaa, 0xe5, 0xaa, + 0xb9, 0xae, 0x2c, 0x03, 0xff, 0x49, 0xc2, 0xdd, 0xf3, 0x48, 0xc8, 0x0a, 0x70, 0x93, 0xf1, 0x62, + 0x83, 0x1c, 0x51, 0x12, 0x42, 0xf6, 0x34, 0x30, 0x7b, 0x06, 0x73, 0xfe, 0xa9, 0x13, 0xb2, 0xb0, + 0xa5, 0x42, 0x89, 0x0d, 0xfd, 0xe3, 0xd5, 0xe6, 0x07, 0x5d, 0xdb, 0x3f, 0x0e, 0x0e, 0x73, 0x1d, + 0xb7, 0xbf, 0x3d, 0x8c, 0xc7, 0x3a, 0x1c, 0xfd, 0xde, 0x1e, 0x3c, 0xef, 0x6e, 0x73, 0xa2, 0x1e, + 0x04, 0xb6, 0x95, 0x3b, 0x38, 0xa8, 0x94, 0x5e, 0xbf, 0xda, 0x9c, 0x6d, 0x9f, 0x3a, 0x95, 0x92, + 0x31, 0xeb, 0x9f, 0x3a, 0x15, 0x0b, 0xed, 0xc0, 0xa2, 0x3f, 0x8a, 0x4e, 0x9e, 0xe0, 0xcb, 0x15, + 0x8a, 0xa8, 0xa1, 0xb6, 0x03, 0x9b, 0xed, 0x53, 0x27, 0xdf, 0x63, 0xe5, 0xff, 0x4c, 0x77, 0x3a, + 0x6e, 0xc0, 0x38, 0x85, 0x3c, 0x5c, 0x62, 0x7d, 0xb7, 0x01, 0x06, 0x94, 0x9c, 0x60, 0x7e, 0x6a, + 0xc6, 0x96, 0xb9, 0xc0, 0xe4, 0xe2, 0x18, 0xfe, 0x26, 0x09, 0xab, 0x0c, 0xf6, 0xba, 0x84, 0x36, + 0x4e, 0x08, 0x3d, 0xea, 0xb9, 0x2f, 0x84, 0xf5, 0x3a, 0xcc, 0xc4, 0xd0, 0x45, 0x26, 0x43, 0xf7, + 0x61, 0xb9, 0x13, 0x50, 0x4a, 0x1c, 0x5f, 0xa2, 0x86, 0x60, 0xab, 0xc2, 0xf7, 0x92, 0x1c, 0xe2, + 0x10, 0x81, 0xde, 0x87, 0x2b, 0xb6, 0xd3, 0xa1, 0xa4, 0x3f, 0x52, 0x9e, 0x89, 0x28, 0xaf, 0x0c, + 0x07, 0x05, 0xa2, 0x7c, 0x95, 0x84, 0x9b, 0x05, 0x46, 0xf9, 0x46, 0x30, 0x47, 0x8e, 0x5c, 0x4a, + 0x76, 0x8b, 0x43, 0xbc, 0x6d, 0xff, 0x20, 0xbc, 0x1d, 0x31, 0x11, 0xe6, 0xe2, 0x98, 0x1d, 0x02, + 0xb7, 0x67, 0x7d, 0x1f, 0xa0, 0x1d, 0x59, 0x69, 0x7d, 0x40, 0xa2, 0x52, 0xd4, 0x6c, 0xcf, 0xb3, + 0x9d, 0xae, 0x88, 0xed, 0x11, 0x2c, 0xbd, 0xa0, 0xae, 0xd3, 0xc5, 0xa2, 0x6e, 0xc8, 0xf0, 0xa6, + 0x97, 0x19, 0x63, 0x91, 0xab, 0x8b, 0x8f, 0x30, 0xdd, 0xa9, 0xc9, 0x74, 0xb3, 0xc6, 0xa4, 0x46, + 0x28, 0xe3, 0x9c, 0x4d, 0xea, 0x76, 0x29, 0xf1, 0x44, 0xe5, 0xd4, 0x7e, 0x9b, 0x82, 0x6b, 0x9c, + 0x8c, 0xee, 0x10, 0x79, 0x7f, 0x44, 0x20, 0xfb, 0xe7, 0xb8, 0xc2, 0xfb, 0x31, 0xb7, 0x27, 0xc6, + 0x2e, 0xbe, 0x12, 0xff, 0x39, 0x39, 0x2c, 0xc5, 0x1b, 0xb0, 0x26, 0xcb, 0xae, 0xa1, 0x37, 0xab, + 0x95, 0x62, 0x1e, 0x1b, 0x7a, 0xad, 0xf1, 0xa9, 0x5e, 0x52, 0x12, 0x68, 0x0d, 0x50, 0x38, 0x96, + 0xaf, 0xef, 0xea, 0xb8, 0xd5, 0xac, 0x56, 0xda, 0x4a, 0x12, 0xdd, 0x80, 0x6b, 0x63, 0xf2, 0x9a, + 0x6e, 0xec, 0xb2, 0x4a, 0x1d, 0xa9, 0xe1, 0x46, 0x7e, 0xa7, 0x8d, 0x5b, 0xf5, 0x7c, 0xb3, 0x55, + 0x6e, 0xb4, 0x95, 0x19, 0x94, 0x85, 0x0d, 0x39, 0x52, 0x6d, 0xec, 0x56, 0x8a, 0xf9, 0x2a, 0x6e, + 0x34, 0x5b, 0xb8, 0x56, 0x69, 0xb5, 0x2a, 0xf5, 0x5d, 0x25, 0x1d, 0xb1, 0x6c, 0x55, 0x1b, 0x8f, + 0x71, 0xb1, 0x51, 0x6f, 0x1d, 0xd4, 0x74, 0x43, 0x99, 0xd5, 0x4c, 0x50, 0x2b, 0x8e, 0x45, 0x7c, + 0x42, 0xfb, 0xb6, 0x63, 0xfa, 0xa4, 0xe8, 0xf6, 0xfb, 0xb6, 0x84, 0x78, 0x1d, 0x16, 0x3d, 0xdf, + 0xec, 0x72, 0x5e, 0xfc, 0x3d, 0x09, 0x1a, 0x48, 0x43, 0xc6, 0xd0, 0x5e, 0xae, 0xc2, 0x22, 0x77, + 0x58, 0x22, 0xbe, 0x69, 0xf7, 0x90, 0x01, 0x8a, 0xe3, 0xfa, 0x78, 0xac, 0x1b, 0x16, 0xbe, 0xdf, + 0x89, 0xc9, 0x7a, 0x4c, 0x47, 0x5e, 0x4e, 0x18, 0x2b, 0xce, 0x98, 0x18, 0x35, 0xe0, 0x8a, 0x68, + 0x1f, 0x99, 0xe7, 0x23, 0x86, 0xb2, 0xf2, 0x9c, 0xde, 0x9d, 0xb6, 0x91, 0x63, 0x68, 0x5c, 0x66, + 0x4d, 0x41, 0x54, 0x8a, 0x9e, 0x00, 0x12, 0x0e, 0x9f, 0x93, 0xb3, 0x61, 0x83, 0x26, 0xa1, 0xe7, + 0xde, 0x34, 0x9f, 0xe7, 0xbb, 0xc9, 0x72, 0xc2, 0x50, 0xe8, 0xb9, 0x01, 0xf4, 0x8b, 0x24, 0x6c, + 0xf1, 0xde, 0xe5, 0x05, 0x6f, 0x71, 0x70, 0x30, 0xea, 0x71, 0xf8, 0x35, 0x60, 0x4d, 0x8e, 0x6c, + 0xcc, 0x3e, 0x8c, 0x9b, 0xe8, 0x8d, 0xcd, 0x51, 0x39, 0x61, 0xdc, 0xa2, 0x17, 0x69, 0xa1, 0x9f, + 0xc2, 0xb5, 0x08, 0x2e, 0x62, 0x53, 0x70, 0x77, 0xde, 0xe4, 0x2f, 0x3e, 0x7c, 0x70, 0x29, 0xa2, + 0x1f, 0xce, 0x84, 0xfc, 0x89, 0x21, 0xd4, 0x06, 0x25, 0xea, 0x9e, 0x71, 0x75, 0x75, 0x8e, 0xfb, + 0x7e, 0xf7, 0x62, 0xdf, 0xc3, 0xd6, 0xa0, 0x9c, 0x30, 0xae, 0xf8, 0xe3, 0x72, 0xf4, 0x18, 0xae, + 0x46, 0xbd, 0x52, 0x76, 0x09, 0xd5, 0xf9, 0xa9, 0x1b, 0x12, 0xdb, 0x13, 0xb0, 0x0d, 0xf1, 0xcf, + 0x0d, 0xa0, 0xcf, 0x20, 0xba, 0x08, 0xd6, 0xf6, 0xfb, 0x81, 0xa7, 0x66, 0xb8, 0xe7, 0xfb, 0x97, + 0x26, 0xe5, 0xe5, 0x84, 0x11, 0x8d, 0x4f, 0x8c, 0xa0, 0x32, 0x03, 0x38, 0xdb, 0x27, 0x21, 0xc0, + 0x2d, 0x70, 0xaf, 0xb7, 0x63, 0xbc, 0x9e, 0x67, 0xdf, 0xe5, 0x04, 0x03, 0xbb, 0xa1, 0x0c, 0x55, + 0x60, 0x59, 0x78, 0xf2, 0x5d, 0x17, 0x33, 0x1c, 0x86, 0x8b, 0x5d, 0x45, 0x08, 0xcc, 0xd0, 0x95, + 0x90, 0xb1, 0xcb, 0xe2, 0x0e, 0x30, 0x95, 0x7c, 0x97, 0xdf, 0xed, 0xc5, 0xa9, 0x97, 0x65, 0x92, + 0x18, 0xb3, 0xcb, 0xe2, 0x46, 0xa5, 0x6c, 0xc3, 0x3b, 0x21, 0x53, 0xc6, 0x47, 0x9c, 0x2a, 0xab, + 0x4b, 0x53, 0x37, 0x3c, 0x8e, 0x54, 0xb3, 0x0d, 0xef, 0x8c, 0xcb, 0x51, 0x1d, 0x56, 0x04, 0x46, + 0x50, 0x49, 0x95, 0xd5, 0xe5, 0xa9, 0x51, 0x4e, 0x52, 0x6a, 0x16, 0x65, 0x2f, 0x2a, 0x65, 0x51, + 0x3a, 0xae, 0x45, 0x70, 0x30, 0x7a, 0xad, 0x52, 0x57, 0xa6, 0x46, 0x19, 0xf7, 0xae, 0xc5, 0xa2, + 0x74, 0xc6, 0xe5, 0x02, 0x28, 0x8e, 0x7c, 0xdc, 0x65, 0x6c, 0x15, 0x5b, 0x82, 0xae, 0xaa, 0xca, + 0x05, 0x40, 0x11, 0xc3, 0x6c, 0x05, 0x50, 0x8c, 0x0f, 0xb0, 0x73, 0x19, 0xd2, 0xce, 0xce, 0x90, + 0xee, 0xaa, 0x57, 0xa7, 0x9e, 0xcb, 0x78, 0x6a, 0xcc, 0xce, 0x25, 0x3d, 0x3f, 0xc2, 0xf1, 0x52, + 0xfa, 0x0e, 0xcf, 0x13, 0x9a, 0x8e, 0x97, 0x13, 0x94, 0x98, 0xe3, 0x65, 0x54, 0xca, 0x92, 0x6b, + 0x86, 0x6d, 0x02, 0xa6, 0xbc, 0x4f, 0x50, 0x37, 0xa6, 0x26, 0x37, 0xae, 0xa3, 0x60, 0xc9, 0x35, + 0xc7, 0xe5, 0x2c, 0x4c, 0x41, 0x92, 0x47, 0xb0, 0x7e, 0x73, 0x6a, 0x98, 0x93, 0x24, 0x9b, 0x85, + 0xe9, 0x45, 0xa5, 0xe8, 0xd7, 0x49, 0xb8, 0x33, 0x81, 0x22, 0x1c, 0x89, 0x31, 0x7f, 0x04, 0xc6, + 0x54, 0xb0, 0x5d, 0xf5, 0x2d, 0x3e, 0xcd, 0x8f, 0x2e, 0x01, 0x2c, 0xb1, 0x44, 0xb9, 0x9c, 0x30, + 0xb6, 0xfc, 0x37, 0x28, 0xb2, 0x9c, 0xd9, 0x82, 0x46, 0x62, 0x57, 0xf2, 0x48, 0x75, 0x73, 0x6a, + 0xce, 0xe2, 0x18, 0x27, 0xcb, 0x99, 0x3d, 0x2e, 0x67, 0xe0, 0x1e, 0x8c, 0xde, 0x5e, 0xb1, 0xec, + 0x02, 0xd5, 0xad, 0xa9, 0xe0, 0x3e, 0xe5, 0xa5, 0x96, 0x81, 0x7b, 0x30, 0x31, 0x84, 0x9e, 0x81, + 0x32, 0x6c, 0xba, 0xf1, 0x21, 0x67, 0x9a, 0xaa, 0xc6, 0x7d, 0xe7, 0x62, 0x7c, 0x5f, 0x40, 0x4c, + 0x39, 0xc6, 0x8f, 0x8f, 0xa0, 0x17, 0x70, 0x8b, 0xb5, 0x11, 0xa6, 0xa0, 0xe8, 0x98, 0x8c, 0x38, + 0xba, 0x64, 0xe4, 0xb7, 0xf9, 0x4c, 0x0f, 0xe3, 0xb6, 0xe5, 0x62, 0x66, 0x5f, 0x4e, 0x18, 0x1b, + 0xfe, 0x54, 0x15, 0x86, 0x35, 0x02, 0xa1, 0x59, 0xad, 0x67, 0xfc, 0x54, 0xbd, 0x33, 0xf5, 0x9c, + 0x4d, 0xf2, 0x58, 0x76, 0xce, 0xec, 0xa8, 0x14, 0x1d, 0xc0, 0xd5, 0x3e, 0xe3, 0x9f, 0xd8, 0x76, + 0xd8, 0xc1, 0xe2, 0x0c, 0x54, 0xbd, 0x3b, 0x75, 0x6f, 0xe3, 0xb8, 0x2a, 0xcb, 0x4f, 0x7f, 0x5c, + 0x8e, 0x7e, 0x22, 0x69, 0xce, 0x11, 0xe1, 0x3b, 0xcb, 0x2a, 0xe0, 0x3b, 0x53, 0x99, 0x53, 0x0c, + 0x5f, 0x65, 0xcc, 0x69, 0xe8, 0x40, 0x54, 0xbf, 0xcf, 0x61, 0xd5, 0x8e, 0x12, 0x40, 0xdc, 0xe1, + 0x0c, 0x50, 0x7d, 0x97, 0xfb, 0x7d, 0x2f, 0x76, 0xfd, 0xf1, 0x7c, 0xb1, 0x9c, 0x30, 0xae, 0xd9, + 0x93, 0x63, 0x85, 0x79, 0x98, 0xe5, 0x5d, 0xcc, 0x5e, 0x3a, 0x73, 0x45, 0x51, 0xf6, 0xd2, 0x99, + 0x6b, 0xca, 0xea, 0x5e, 0x3a, 0xb3, 0xaa, 0x5c, 0xdf, 0x4b, 0x67, 0xae, 0x2b, 0x6b, 0x7b, 0xe9, + 0xcc, 0x9a, 0x72, 0x63, 0x2f, 0x9d, 0xb9, 0xa1, 0xa8, 0x7b, 0xe9, 0x8c, 0xaa, 0xac, 0xef, 0xa5, + 0x33, 0xeb, 0xca, 0xc6, 0x5e, 0x3a, 0x73, 0x4b, 0xc9, 0xee, 0xa5, 0x33, 0x59, 0x65, 0x73, 0x2f, + 0x9d, 0x79, 0x5b, 0xd1, 0xb4, 0xfb, 0x9c, 0x49, 0x36, 0x5d, 0x8f, 0xd7, 0x09, 0xb4, 0x01, 0xb3, + 0x6c, 0xc2, 0x53, 0xd9, 0x93, 0x0b, 0xea, 0x29, 0x44, 0xda, 0x97, 0xb3, 0x30, 0x1b, 0x3e, 0xe9, + 0x9f, 0x7b, 0xa9, 0x58, 0x97, 0x8d, 0xf6, 0xd5, 0xc8, 0xb3, 0xb9, 0x50, 0x18, 0x3d, 0x5f, 0xfc, + 0x7c, 0x9c, 0x22, 0x51, 0xc2, 0xff, 0x0d, 0xe0, 0x04, 0x70, 0x25, 0xf6, 0x54, 0x8c, 0xc1, 0x02, + 0x57, 0x2e, 0xdc, 0x91, 0xf3, 0xbc, 0x35, 0x9a, 0x67, 0x52, 0x6b, 0x8c, 0x41, 0x49, 0x19, 0x2a, + 0xc2, 0x72, 0xe0, 0x90, 0xd3, 0x81, 0xeb, 0x11, 0x8b, 0xd7, 0xe7, 0xf4, 0x65, 0xb8, 0xb7, 0xb1, + 0x34, 0x34, 0x62, 0x55, 0x79, 0x1b, 0x16, 0x5d, 0x6a, 0x77, 0x6d, 0x07, 0xb3, 0x9a, 0xc5, 0xd9, + 0xdd, 0x6c, 0x61, 0x85, 0xc5, 0xf4, 0xdd, 0xab, 0xcd, 0x39, 0x56, 0xdf, 0x2a, 0x25, 0x03, 0x84, + 0x0a, 0xfb, 0x42, 0x4d, 0x98, 0xb3, 0x38, 0x45, 0x97, 0x6c, 0x2d, 0x3b, 0xed, 0x09, 0x41, 0x10, + 0xf9, 0x82, 0x2a, 0xd7, 0xa7, 0x8c, 0xd6, 0x27, 0x46, 0x0c, 0xe9, 0x07, 0xfd, 0x7f, 0xb8, 0x41, + 0xf3, 0x17, 0x39, 0x0c, 0xf7, 0x53, 0x6e, 0x1d, 0x0a, 0x60, 0xc6, 0x71, 0x5f, 0x48, 0x06, 0xf6, + 0x86, 0x46, 0xb3, 0x24, 0xd7, 0xf3, 0xe8, 0xf2, 0x0f, 0x0d, 0xcc, 0x41, 0xb1, 0xe7, 0x76, 0x9e, + 0x8f, 0x1e, 0xde, 0xd9, 0x7c, 0x68, 0x1f, 0x96, 0x19, 0xe0, 0x58, 0x43, 0xb0, 0x11, 0x64, 0x6d, + 0x2b, 0x12, 0x40, 0xf8, 0xa7, 0x64, 0x4e, 0x17, 0x8a, 0xd1, 0x07, 0x95, 0x25, 0x12, 0x91, 0x89, + 0x87, 0x15, 0xf1, 0x0c, 0xfa, 0xe0, 0x9f, 0x29, 0x50, 0xa7, 0x3d, 0x95, 0xb3, 0xd6, 0x2c, 0x5f, + 0x68, 0x18, 0x6d, 0x3c, 0xf1, 0x64, 0x7b, 0x17, 0xde, 0x1e, 0x1b, 0xe1, 0x1f, 0x7a, 0x09, 0x1b, + 0x7a, 0xb1, 0x61, 0x94, 0xf0, 0x4e, 0xe3, 0xa0, 0x5e, 0x52, 0x92, 0xac, 0xf7, 0x1b, 0x53, 0x2b, + 0x56, 0x2b, 0x7a, 0x9d, 0x7d, 0xed, 0xe9, 0x45, 0xd6, 0x1b, 0x6e, 0xc2, 0xcd, 0xb1, 0xf1, 0xe6, + 0x41, 0xab, 0xac, 0x1b, 0xa1, 0x37, 0x25, 0x8d, 0x6e, 0xc2, 0x8d, 0xc9, 0x79, 0x70, 0xab, 0x99, + 0xaf, 0x2b, 0xb3, 0x28, 0x0f, 0x9f, 0x8c, 0x0f, 0x56, 0x0d, 0x3d, 0x5f, 0x7a, 0x3a, 0x7a, 0x41, + 0xc6, 0x0d, 0x03, 0x1b, 0x8d, 0x6a, 0x55, 0x2f, 0xe1, 0x42, 0xbe, 0xb8, 0x8f, 0x9b, 0x8d, 0x56, + 0xab, 0x52, 0xa8, 0xea, 0xbc, 0xe1, 0xcd, 0x3f, 0x55, 0xe6, 0xd0, 0xbb, 0x70, 0x7b, 0xcc, 0x45, + 0x5d, 0x7f, 0x8c, 0xab, 0x7a, 0xbe, 0xa5, 0xe3, 0xa6, 0xa1, 0x7f, 0xaa, 0xd7, 0xdb, 0x2d, 0xdc, + 0x7e, 0x52, 0x57, 0x32, 0xe8, 0x3e, 0xdc, 0x1d, 0x53, 0x6c, 0x57, 0x6a, 0x7a, 0xab, 0x9d, 0xaf, + 0x35, 0x71, 0x31, 0x5f, 0x2c, 0xeb, 0x72, 0x49, 0x7a, 0x49, 0x99, 0xdf, 0x48, 0x7f, 0xf9, 0x55, + 0x36, 0xa1, 0xb1, 0xf4, 0xa6, 0x1e, 0xfc, 0x65, 0xfc, 0xed, 0x3d, 0xf2, 0x8e, 0x2f, 0xfa, 0xde, + 0xb6, 0xf1, 0x74, 0x32, 0xb9, 0xbc, 0xc9, 0x66, 0x23, 0x8f, 0x8d, 0x4a, 0x5b, 0xc7, 0xed, 0x46, + 0x03, 0x37, 0xaa, 0x2c, 0x9d, 0xbc, 0x2b, 0x67, 0x03, 0x2d, 0xdd, 0xa8, 0xe4, 0xab, 0x95, 0xcf, + 0xf2, 0x85, 0xaa, 0xae, 0xcc, 0xa0, 0x5b, 0xb0, 0x2e, 0xe4, 0xf9, 0xd6, 0xd3, 0x7a, 0x51, 0x9a, + 0xed, 0xe4, 0x2b, 0xd5, 0x03, 0x43, 0x57, 0x66, 0x91, 0x06, 0x59, 0x31, 0x2c, 0x12, 0x83, 0x4b, + 0x7a, 0xbe, 0x54, 0xad, 0xd4, 0x75, 0xac, 0x3f, 0x29, 0xea, 0x7a, 0x49, 0x2f, 0x29, 0x73, 0x22, + 0xe8, 0x07, 0x1f, 0x03, 0x9a, 0x44, 0x01, 0x94, 0x81, 0x74, 0xbd, 0x51, 0xd7, 0x95, 0x04, 0x5a, + 0x84, 0x79, 0x96, 0xc8, 0xc6, 0xce, 0x8e, 0x92, 0x44, 0xcb, 0xb0, 0x50, 0xa9, 0xd5, 0xf4, 0x52, + 0x25, 0xdf, 0xd6, 0x95, 0x54, 0xe1, 0xfe, 0xcb, 0x7f, 0x67, 0x13, 0x2f, 0x5f, 0x67, 0x93, 0x5f, + 0xbf, 0xce, 0x26, 0xbf, 0x79, 0x9d, 0x4d, 0xfe, 0xeb, 0x75, 0x36, 0xf9, 0xbb, 0x6f, 0xb3, 0x89, + 0xaf, 0xbf, 0xcd, 0x26, 0xbe, 0xf9, 0x36, 0x9b, 0xf8, 0x6c, 0x5e, 0xe2, 0xc2, 0xff, 0x02, 0x00, + 0x00, 0xff, 0xff, 0x14, 0x07, 0xfe, 0xb0, 0x6a, 0x1f, 0x00, 0x00, } diff --git a/pkg/roachpb/errors.proto b/pkg/roachpb/errors.proto index 35f462c48244..f25f249e6370 100644 --- a/pkg/roachpb/errors.proto +++ b/pkg/roachpb/errors.proto @@ -579,7 +579,8 @@ message Error { // now is the current time at the node sending the response, // which can be used by the receiver to update its local HLC. - optional util.hlc.Timestamp now = 8 [(gogoproto.nullable) = false]; + optional util.hlc.Timestamp now = 8 [(gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/util/hlc.ClockTimestamp"]; reserved 2; } diff --git a/pkg/roachpb/errors_test.go b/pkg/roachpb/errors_test.go index fd1f8e4b6ac0..64cccba63726 100644 --- a/pkg/roachpb/errors_test.go +++ b/pkg/roachpb/errors_test.go @@ -101,7 +101,7 @@ func TestReadWithinUncertaintyIntervalError(t *testing.T) { hlc.Timestamp{WallTime: 1}, hlc.Timestamp{WallTime: 2}, &Transaction{ MaxTimestamp: hlc.Timestamp{WallTime: 3}, - ObservedTimestamps: []ObservedTimestamp{{NodeID: 12, Timestamp: hlc.Timestamp{WallTime: 4}}}, + ObservedTimestamps: []ObservedTimestamp{{NodeID: 12, Timestamp: hlc.ClockTimestamp{WallTime: 4}}}, }) expNew := "ReadWithinUncertaintyIntervalError: read at time 0.000000001,0 encountered " + "previous write with future timestamp 0.000000002,0 within uncertainty interval " + @@ -139,7 +139,7 @@ func TestErrorRedaction(t *testing.T) { hlc.Timestamp{WallTime: 1}, hlc.Timestamp{WallTime: 2}, &Transaction{ MaxTimestamp: hlc.Timestamp{WallTime: 3}, - ObservedTimestamps: []ObservedTimestamp{{NodeID: 12, Timestamp: hlc.Timestamp{WallTime: 4}}}, + ObservedTimestamps: []ObservedTimestamp{{NodeID: 12, Timestamp: hlc.ClockTimestamp{WallTime: 4}}}, })) txn := MakeTransaction("foo", Key("bar"), 1, hlc.Timestamp{WallTime: 1}, 1) txn.ID = uuid.Nil diff --git a/pkg/server/admin_test.go b/pkg/server/admin_test.go index 7526151c1695..d9acddbcf8d4 100644 --- a/pkg/server/admin_test.go +++ b/pkg/server/admin_test.go @@ -1369,7 +1369,7 @@ func TestHealthAPI(t *testing.T) { defer ts.nodeLiveness.PauseAllHeartbeatsForTest()() self, ok := ts.nodeLiveness.Self() assert.True(t, ok) - s.Clock().Update(self.Expiration.ToTimestamp().Add(1, 0)) + s.Clock().Update(self.Expiration.ToTimestamp().Add(1, 0).UnsafeToClockTimestamp()) var resp serverpb.HealthResponse testutils.SucceedsSoon(t, func() error { diff --git a/pkg/server/server_test.go b/pkg/server/server_test.go index 2315e5e65443..aa3fe7182ab9 100644 --- a/pkg/server/server_test.go +++ b/pkg/server/server_test.go @@ -407,8 +407,8 @@ func TestMultiRangeScanDeleteRange(t *testing.T) { t.Errorf("expected %d keys to be deleted, but got %d instead", writes, dr.Keys) } - now := s.Clock().Now() - txnProto := roachpb.MakeTransaction("MyTxn", nil, 0, now, 0) + now := s.Clock().NowAsClockTimestamp() + txnProto := roachpb.MakeTransaction("MyTxn", nil, 0, now.ToTimestamp(), 0) txn := kv.NewTxnFromProto(ctx, db, s.NodeID(), now, kv.RootTxn, &txnProto) scan := roachpb.NewScan(writes[0], writes[len(writes)-1].Next(), false) diff --git a/pkg/sql/distsql_running.go b/pkg/sql/distsql_running.go index 0b965485ad02..278cd82fcdb0 100644 --- a/pkg/sql/distsql_running.go +++ b/pkg/sql/distsql_running.go @@ -530,7 +530,7 @@ var receiverSyncPool = sync.Pool{ // timestamp. Usually wraps an hlc.Clock. type clockUpdater interface { // Update updates this ClockUpdater with the observed hlc.Timestamp. - Update(observedTS hlc.Timestamp) + Update(observedTS hlc.ClockTimestamp) } // MakeDistSQLReceiver creates a DistSQLReceiver. diff --git a/pkg/sql/flowinfra/cluster_test.go b/pkg/sql/flowinfra/cluster_test.go index 23a82417ef8e..f001207c7b7d 100644 --- a/pkg/sql/flowinfra/cluster_test.go +++ b/pkg/sql/flowinfra/cluster_test.go @@ -90,12 +90,12 @@ func TestClusterFlow(t *testing.T) { ctx := tracing.ContextWithSpan(context.Background(), sp) defer sp.Finish() - now := tc.Server(0).Clock().Now() + now := tc.Server(0).Clock().NowAsClockTimestamp() txnProto := roachpb.MakeTransaction( "cluster-test", nil, // baseKey roachpb.NormalUserPriority, - now, + now.ToTimestamp(), 0, // maxOffset ) txn := kv.NewTxnFromProto(ctx, kvDB, tc.Server(0).NodeID(), now, kv.RootTxn, &txnProto) @@ -416,12 +416,12 @@ func TestLimitedBufferingDeadlock(t *testing.T) { Type: descpb.InnerJoin, } - now := tc.Server(0).Clock().Now() + now := tc.Server(0).Clock().NowAsClockTimestamp() txnProto := roachpb.MakeTransaction( "deadlock-test", nil, // baseKey roachpb.NormalUserPriority, - now, + now.ToTimestamp(), 0, // maxOffset ) txn := kv.NewTxnFromProto( @@ -736,12 +736,12 @@ func BenchmarkInfrastructure(b *testing.B) { } return execinfrapb.StreamEndpointSpec_REMOTE } - now := tc.Server(0).Clock().Now() + now := tc.Server(0).Clock().NowAsClockTimestamp() txnProto := roachpb.MakeTransaction( "cluster-test", nil, // baseKey roachpb.NormalUserPriority, - now, + now.ToTimestamp(), 0, // maxOffset ) txn := kv.NewTxnFromProto( diff --git a/pkg/sql/sem/tree/timeconv_test.go b/pkg/sql/sem/tree/timeconv_test.go index af1fce473baf..62bffa7fe429 100644 --- a/pkg/sql/sem/tree/timeconv_test.go +++ b/pkg/sql/sem/tree/timeconv_test.go @@ -56,12 +56,12 @@ func TestClusterTimestampConversion(t *testing.T) { db := kv.NewDB(testutils.MakeAmbientCtx(), senderFactory, clock, stopper) for _, d := range testData { - ts := hlc.Timestamp{WallTime: d.walltime, Logical: d.logical} + ts := hlc.ClockTimestamp{WallTime: d.walltime, Logical: d.logical} txnProto := roachpb.MakeTransaction( "test", nil, // baseKey roachpb.NormalUserPriority, - ts, + ts.ToTimestamp(), 0, /* maxOffsetNs */ ) diff --git a/pkg/sql/txn_restart_test.go b/pkg/sql/txn_restart_test.go index 93c69055ca68..d7350e5d707c 100644 --- a/pkg/sql/txn_restart_test.go +++ b/pkg/sql/txn_restart_test.go @@ -1162,7 +1162,7 @@ func TestReacquireLeaseOnRestart(t *testing.T) { if req, ok := union.GetInner().(*roachpb.ScanRequest); ok { if bytes.Contains(req.Key, testKey) && !kv.TestingIsRangeLookupRequest(req) { atomic.AddInt32(&clockUpdate, 1) - now := c.Now() + now := c.NowAsClockTimestamp() now.WallTime += advancement.Nanoseconds() c.Update(now) break @@ -1198,9 +1198,9 @@ func TestReacquireLeaseOnRestart(t *testing.T) { // Return ReadWithinUncertaintyIntervalError to update the transaction timestamp on retry. txn := args.Hdr.Txn txn.ResetObservedTimestamps() - now := s.Clock().Now() + now := s.Clock().NowAsClockTimestamp() txn.UpdateObservedTimestamp(s.(*server.TestServer).Gossip().NodeID.Get(), now) - return roachpb.NewErrorWithTxn(roachpb.NewReadWithinUncertaintyIntervalError(now, now, txn), txn) + return roachpb.NewErrorWithTxn(roachpb.NewReadWithinUncertaintyIntervalError(now.ToTimestamp(), now.ToTimestamp(), txn), txn) } } return nil @@ -1269,9 +1269,9 @@ func TestFlushUncommitedDescriptorCacheOnRestart(t *testing.T) { // Return ReadWithinUncertaintyIntervalError. txn := args.Hdr.Txn txn.ResetObservedTimestamps() - now := s.Clock().Now() + now := s.Clock().NowAsClockTimestamp() txn.UpdateObservedTimestamp(s.(*server.TestServer).Gossip().NodeID.Get(), now) - return roachpb.NewErrorWithTxn(roachpb.NewReadWithinUncertaintyIntervalError(now, now, txn), txn) + return roachpb.NewErrorWithTxn(roachpb.NewReadWithinUncertaintyIntervalError(now.ToTimestamp(), now.ToTimestamp(), txn), txn) } } return nil @@ -1331,7 +1331,7 @@ func TestDistSQLRetryableError(t *testing.T) { hlc.Timestamp{}, nil) errTxn := fArgs.Hdr.Txn.Clone() - errTxn.UpdateObservedTimestamp(roachpb.NodeID(2), hlc.Timestamp{}) + errTxn.UpdateObservedTimestamp(roachpb.NodeID(2), hlc.ClockTimestamp{}) pErr := roachpb.NewErrorWithTxn(err, errTxn) pErr.OriginNode = 2 return pErr diff --git a/pkg/storage/enginepb/mvcc3.pb.go b/pkg/storage/enginepb/mvcc3.pb.go index ba8e0cd7aa4e..55de6d8ac862 100644 --- a/pkg/storage/enginepb/mvcc3.pb.go +++ b/pkg/storage/enginepb/mvcc3.pb.go @@ -111,6 +111,10 @@ type TxnMeta struct { // txn record was not yet written. In that case, the pusher uses this field // as an indication of a timestamp when the pushee's coordinator is known // to have been alive. + // + // NOTE: this could use a ClockTimestamp type, but doing so results in a + // large diff that doesn't seem worth it, given that we never feed this + // timestamp back into a clock. MinTimestamp hlc.Timestamp `protobuf:"bytes,9,opt,name=min_timestamp,json=minTimestamp,proto3" json:"min_timestamp"` // The transaction's priority, ratcheted on transaction pushes. Priority TxnPriority `protobuf:"varint,6,opt,name=priority,proto3,casttype=TxnPriority" json:"priority,omitempty"` @@ -125,7 +129,7 @@ type TxnMeta struct { func (m *TxnMeta) Reset() { *m = TxnMeta{} } func (*TxnMeta) ProtoMessage() {} func (*TxnMeta) Descriptor() ([]byte, []int) { - return fileDescriptor_mvcc3_7509a297d4cf653a, []int{0} + return fileDescriptor_mvcc3_4bb1b6bffd8f789e, []int{0} } func (m *TxnMeta) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -161,7 +165,7 @@ func (m *IgnoredSeqNumRange) Reset() { *m = IgnoredSeqNumRange{} } func (m *IgnoredSeqNumRange) String() string { return proto.CompactTextString(m) } func (*IgnoredSeqNumRange) ProtoMessage() {} func (*IgnoredSeqNumRange) Descriptor() ([]byte, []int) { - return fileDescriptor_mvcc3_7509a297d4cf653a, []int{1} + return fileDescriptor_mvcc3_4bb1b6bffd8f789e, []int{1} } func (m *IgnoredSeqNumRange) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -211,7 +215,7 @@ func (m *MVCCStatsDelta) Reset() { *m = MVCCStatsDelta{} } func (m *MVCCStatsDelta) String() string { return proto.CompactTextString(m) } func (*MVCCStatsDelta) ProtoMessage() {} func (*MVCCStatsDelta) Descriptor() ([]byte, []int) { - return fileDescriptor_mvcc3_7509a297d4cf653a, []int{2} + return fileDescriptor_mvcc3_4bb1b6bffd8f789e, []int{2} } func (m *MVCCStatsDelta) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -262,7 +266,7 @@ func (m *MVCCPersistentStats) Reset() { *m = MVCCPersistentStats{} } func (m *MVCCPersistentStats) String() string { return proto.CompactTextString(m) } func (*MVCCPersistentStats) ProtoMessage() {} func (*MVCCPersistentStats) Descriptor() ([]byte, []int) { - return fileDescriptor_mvcc3_7509a297d4cf653a, []int{3} + return fileDescriptor_mvcc3_4bb1b6bffd8f789e, []int{3} } func (m *MVCCPersistentStats) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -307,7 +311,7 @@ func (m *RangeAppliedState) Reset() { *m = RangeAppliedState{} } func (m *RangeAppliedState) String() string { return proto.CompactTextString(m) } func (*RangeAppliedState) ProtoMessage() {} func (*RangeAppliedState) Descriptor() ([]byte, []int) { - return fileDescriptor_mvcc3_7509a297d4cf653a, []int{4} + return fileDescriptor_mvcc3_4bb1b6bffd8f789e, []int{4} } func (m *RangeAppliedState) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -345,7 +349,7 @@ func (m *MVCCWriteValueOp) Reset() { *m = MVCCWriteValueOp{} } func (m *MVCCWriteValueOp) String() string { return proto.CompactTextString(m) } func (*MVCCWriteValueOp) ProtoMessage() {} func (*MVCCWriteValueOp) Descriptor() ([]byte, []int) { - return fileDescriptor_mvcc3_7509a297d4cf653a, []int{5} + return fileDescriptor_mvcc3_4bb1b6bffd8f789e, []int{5} } func (m *MVCCWriteValueOp) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -383,7 +387,7 @@ func (m *MVCCWriteIntentOp) Reset() { *m = MVCCWriteIntentOp{} } func (m *MVCCWriteIntentOp) String() string { return proto.CompactTextString(m) } func (*MVCCWriteIntentOp) ProtoMessage() {} func (*MVCCWriteIntentOp) Descriptor() ([]byte, []int) { - return fileDescriptor_mvcc3_7509a297d4cf653a, []int{6} + return fileDescriptor_mvcc3_4bb1b6bffd8f789e, []int{6} } func (m *MVCCWriteIntentOp) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -419,7 +423,7 @@ func (m *MVCCUpdateIntentOp) Reset() { *m = MVCCUpdateIntentOp{} } func (m *MVCCUpdateIntentOp) String() string { return proto.CompactTextString(m) } func (*MVCCUpdateIntentOp) ProtoMessage() {} func (*MVCCUpdateIntentOp) Descriptor() ([]byte, []int) { - return fileDescriptor_mvcc3_7509a297d4cf653a, []int{7} + return fileDescriptor_mvcc3_4bb1b6bffd8f789e, []int{7} } func (m *MVCCUpdateIntentOp) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -458,7 +462,7 @@ func (m *MVCCCommitIntentOp) Reset() { *m = MVCCCommitIntentOp{} } func (m *MVCCCommitIntentOp) String() string { return proto.CompactTextString(m) } func (*MVCCCommitIntentOp) ProtoMessage() {} func (*MVCCCommitIntentOp) Descriptor() ([]byte, []int) { - return fileDescriptor_mvcc3_7509a297d4cf653a, []int{8} + return fileDescriptor_mvcc3_4bb1b6bffd8f789e, []int{8} } func (m *MVCCCommitIntentOp) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -498,7 +502,7 @@ func (m *MVCCAbortIntentOp) Reset() { *m = MVCCAbortIntentOp{} } func (m *MVCCAbortIntentOp) String() string { return proto.CompactTextString(m) } func (*MVCCAbortIntentOp) ProtoMessage() {} func (*MVCCAbortIntentOp) Descriptor() ([]byte, []int) { - return fileDescriptor_mvcc3_7509a297d4cf653a, []int{9} + return fileDescriptor_mvcc3_4bb1b6bffd8f789e, []int{9} } func (m *MVCCAbortIntentOp) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -534,7 +538,7 @@ func (m *MVCCAbortTxnOp) Reset() { *m = MVCCAbortTxnOp{} } func (m *MVCCAbortTxnOp) String() string { return proto.CompactTextString(m) } func (*MVCCAbortTxnOp) ProtoMessage() {} func (*MVCCAbortTxnOp) Descriptor() ([]byte, []int) { - return fileDescriptor_mvcc3_7509a297d4cf653a, []int{10} + return fileDescriptor_mvcc3_4bb1b6bffd8f789e, []int{10} } func (m *MVCCAbortTxnOp) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -573,7 +577,7 @@ func (m *MVCCLogicalOp) Reset() { *m = MVCCLogicalOp{} } func (m *MVCCLogicalOp) String() string { return proto.CompactTextString(m) } func (*MVCCLogicalOp) ProtoMessage() {} func (*MVCCLogicalOp) Descriptor() ([]byte, []int) { - return fileDescriptor_mvcc3_7509a297d4cf653a, []int{11} + return fileDescriptor_mvcc3_4bb1b6bffd8f789e, []int{11} } func (m *MVCCLogicalOp) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4232,10 +4236,10 @@ var ( ) func init() { - proto.RegisterFile("storage/enginepb/mvcc3.proto", fileDescriptor_mvcc3_7509a297d4cf653a) + proto.RegisterFile("storage/enginepb/mvcc3.proto", fileDescriptor_mvcc3_4bb1b6bffd8f789e) } -var fileDescriptor_mvcc3_7509a297d4cf653a = []byte{ +var fileDescriptor_mvcc3_4bb1b6bffd8f789e = []byte{ // 1191 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xcc, 0x57, 0x41, 0x6f, 0x1a, 0xc7, 0x17, 0x67, 0xd9, 0xc5, 0x86, 0x07, 0xb6, 0x61, 0x12, 0xe9, 0x8f, 0xf2, 0x4f, 0x80, 0x72, 0xa8, diff --git a/pkg/storage/enginepb/mvcc3.proto b/pkg/storage/enginepb/mvcc3.proto index bd8f3e8bf20e..19a91e647a92 100644 --- a/pkg/storage/enginepb/mvcc3.proto +++ b/pkg/storage/enginepb/mvcc3.proto @@ -106,6 +106,10 @@ message TxnMeta { // txn record was not yet written. In that case, the pusher uses this field // as an indication of a timestamp when the pushee's coordinator is known // to have been alive. + // + // NOTE: this could use a ClockTimestamp type, but doing so results in a + // large diff that doesn't seem worth it, given that we never feed this + // timestamp back into a clock. util.hlc.Timestamp min_timestamp = 9 [(gogoproto.nullable) = false]; // The transaction's priority, ratcheted on transaction pushes. int32 priority = 6 [(gogoproto.casttype) = "TxnPriority"]; diff --git a/pkg/testutils/lint/lint_test.go b/pkg/testutils/lint/lint_test.go index bcedec4aa3a7..5c681f9b0ad0 100644 --- a/pkg/testutils/lint/lint_test.go +++ b/pkg/testutils/lint/lint_test.go @@ -1020,6 +1020,7 @@ func TestLint(t *testing.T) { ":!util/protoutil/marshal.go", ":!util/protoutil/marshaler.go", ":!util/encoding/encoding.go", + ":!util/hlc/timestamp.go", ":!sql/types/types_jsonpb.go", ) if err != nil { diff --git a/pkg/util/hlc/hlc.go b/pkg/util/hlc/hlc.go index d54ef4382442..b870292e0b7a 100644 --- a/pkg/util/hlc/hlc.go +++ b/pkg/util/hlc/hlc.go @@ -78,7 +78,7 @@ type Clock struct { // be updated atomically, even though it is protected by a mutex - this // enables a fast path for reading the wall time without grabbing the // lock. - timestamp Timestamp + timestamp ClockTimestamp // isMonitoringForwardClockJumps is a flag to ensure that only one jump monitoring // goroutine is running per clock @@ -278,12 +278,20 @@ func (c *Clock) checkPhysicalClock(ctx context.Context, oldTime, newTime int64) } } -// Now returns a timestamp associated with an event from -// the local machine that may be sent to other members -// of the distributed network. This is the counterpart -// of Update, which is passed a timestamp received from -// another member of the distributed network. +// Now returns a timestamp associated with an event from the local +// machine that may be sent to other members of the distributed network. func (c *Clock) Now() Timestamp { + return c.NowAsClockTimestamp().ToTimestamp() +} + +// NowAsClockTimestamp is like Now, but returns a ClockTimestamp instead +// of a raw Timestamp. +// +// This is the counterpart of Update, which is passed a ClockTimestamp +// received from another member of the distributed network. As such, +// callers that intend to use the returned timestamp to update a peer's +// HLC clock should use this method. +func (c *Clock) NowAsClockTimestamp() ClockTimestamp { physicalClock := c.getPhysicalClockAndCheck(context.TODO()) c.mu.Lock() defer c.mu.Unlock() @@ -334,7 +342,7 @@ func (c *Clock) PhysicalTime() time.Time { // the maximum clock offset. To receive an error response instead of forcing the // update in case the remote timestamp is too far into the future, use // UpdateAndCheckMaxOffset() instead. -func (c *Clock) Update(rt Timestamp) { +func (c *Clock) Update(rt ClockTimestamp) { // Fast path to avoid grabbing the mutex if the remote time is behind. This // requires c.mu.timestamp.WallTime to be written atomically, even though @@ -366,7 +374,7 @@ func (c *Clock) Update(rt Timestamp) { // UpdateAndCheckMaxOffset is like Update, but also takes the wall time into account and // returns an error in the event that the supplied remote timestamp exceeds // the wall clock time by more than the maximum clock offset. -func (c *Clock) UpdateAndCheckMaxOffset(ctx context.Context, rt Timestamp) error { +func (c *Clock) UpdateAndCheckMaxOffset(ctx context.Context, rt ClockTimestamp) error { var err error physicalClock := c.getPhysicalClockAndCheck(ctx) @@ -377,7 +385,7 @@ func (c *Clock) UpdateAndCheckMaxOffset(ctx context.Context, rt Timestamp) error } if physicalClock > rt.WallTime { - c.Update(Timestamp{WallTime: physicalClock}) + c.Update(ClockTimestamp{WallTime: physicalClock}) } else { c.Update(rt) } diff --git a/pkg/util/hlc/hlc_test.go b/pkg/util/hlc/hlc_test.go index 68d489cf38e9..4c41ac847f2e 100644 --- a/pkg/util/hlc/hlc_test.go +++ b/pkg/util/hlc/hlc_test.go @@ -303,37 +303,37 @@ func TestHLCClock(t *testing.T) { wallClock int64 event Event // If this is a receive event, this holds the "input" timestamp. - input *Timestamp + input *ClockTimestamp // The expected timestamp generated from the input. - expected Timestamp + expected ClockTimestamp }{ // A few valid steps to warm up. - {5, SEND, nil, Timestamp{WallTime: 5, Logical: 0}}, - {6, SEND, nil, Timestamp{WallTime: 6, Logical: 0}}, - {10, RECV, &Timestamp{WallTime: 10, Logical: 5}, Timestamp{WallTime: 10, Logical: 6}}, + {5, SEND, nil, ClockTimestamp{WallTime: 5, Logical: 0}}, + {6, SEND, nil, ClockTimestamp{WallTime: 6, Logical: 0}}, + {10, RECV, &ClockTimestamp{WallTime: 10, Logical: 5}, ClockTimestamp{WallTime: 10, Logical: 6}}, // Our clock mysteriously jumps back. - {7, SEND, nil, Timestamp{WallTime: 10, Logical: 7}}, + {7, SEND, nil, ClockTimestamp{WallTime: 10, Logical: 7}}, // Wall clocks coincide, but the local logical clock wins. - {8, RECV, &Timestamp{WallTime: 10, Logical: 4}, Timestamp{WallTime: 10, Logical: 9}}, + {8, RECV, &ClockTimestamp{WallTime: 10, Logical: 4}, ClockTimestamp{WallTime: 10, Logical: 9}}, // Wall clocks coincide, but the remote logical clock wins. - {10, RECV, &Timestamp{WallTime: 10, Logical: 99}, Timestamp{WallTime: 10, Logical: 100}}, + {10, RECV, &ClockTimestamp{WallTime: 10, Logical: 99}, ClockTimestamp{WallTime: 10, Logical: 100}}, // The physical clock has caught up and takes over. - {11, RECV, &Timestamp{WallTime: 10, Logical: 31}, Timestamp{WallTime: 11, Logical: 1}}, - {11, SEND, nil, Timestamp{WallTime: 11, Logical: 2}}, + {11, RECV, &ClockTimestamp{WallTime: 10, Logical: 31}, ClockTimestamp{WallTime: 11, Logical: 1}}, + {11, SEND, nil, ClockTimestamp{WallTime: 11, Logical: 2}}, } - var current Timestamp + var current ClockTimestamp for i, step := range expectedHistory { m.Set(step.wallClock) switch step.event { case SEND: - current = c.Now() + current = c.NowAsClockTimestamp() case RECV: fallthrough default: - previous := c.Now() + previous := c.NowAsClockTimestamp() c.Update(*step.input) - current = c.Now() + current = c.NowAsClockTimestamp() if current == previous { t.Errorf("%d: clock not updated", i) } @@ -480,7 +480,7 @@ func TestHLCEnforceWallTimeWithinBoundsInUpdate(t *testing.T) { c := NewClock(m.UnixNano, time.Nanosecond) c.mu.wallTimeUpperBound = test.wallTimeUpperBound fatal = false - err := c.UpdateAndCheckMaxOffset(ctx, Timestamp{WallTime: test.messageWallTime}) + err := c.UpdateAndCheckMaxOffset(ctx, ClockTimestamp{WallTime: test.messageWallTime}) a.Nil(err) a.Equal(test.isFatal, fatal) }) @@ -603,15 +603,15 @@ func BenchmarkUpdate(b *testing.B) { // benchmarking the contention of the benchmark synchronization rather than // the HLC. r := rand.New(rand.NewSource(34704832098)) - timestamps := make([][]Timestamp, concurrency) + timestamps := make([][]ClockTimestamp, concurrency) for w := 0; w < concurrency; w++ { - timestamps[w] = make([]Timestamp, updates/concurrency) + timestamps[w] = make([]ClockTimestamp, updates/concurrency) wallTime := 0 for i := 0; i < updates/concurrency; i++ { if r.Float64() < advanceChance { wallTime += r.Intn(advanceMax + 1) } - timestamps[w][i] = Timestamp{WallTime: int64(wallTime)} + timestamps[w][i] = ClockTimestamp{WallTime: int64(wallTime)} } } diff --git a/pkg/util/hlc/timestamp.go b/pkg/util/hlc/timestamp.go index faa51406da53..27470c5cc9bc 100644 --- a/pkg/util/hlc/timestamp.go +++ b/pkg/util/hlc/timestamp.go @@ -30,6 +30,8 @@ var ( MaxTimestamp = Timestamp{WallTime: math.MaxInt64, Logical: math.MaxInt32} // MinTimestamp is the min value allowed for Timestamp. MinTimestamp = Timestamp{WallTime: 0, Logical: 1} + // MaxClockTimestamp is the max value allowed for ClockTimestamp. + MaxClockTimestamp = ClockTimestamp{WallTime: math.MaxInt64, Logical: math.MaxInt32} ) // EqOrdering returns whether the receiver sorts equally to the parameter. @@ -363,6 +365,92 @@ func (t LegacyTimestamp) Less(s LegacyTimestamp) bool { return t.ToTimestamp().Less(s.ToTimestamp()) } +// String implements the fmt.Formatter interface. func (t LegacyTimestamp) String() string { return t.ToTimestamp().String() } + +// ClockTimestamp is a Timestamp with the added capability of being able to +// update a peer's HLC clock. It possesses this capability because the clock +// timestamp itself is guaranteed to have come from an HLC clock somewhere in +// the system. As such, a clock timestamp is an promise that some node in the +// system has a clock with a reading equal to or above its value. +// +// ClockTimestamp is the statically typed version of a Timestamp with its +// Synthetic flag set to false. +type ClockTimestamp Timestamp + +// TryToClockTimestamp attempts to downcast a Timestamp into a ClockTimestamp. +// Returns the result and a boolean indicating whether the cast succeeded. +func (t Timestamp) TryToClockTimestamp() (ClockTimestamp, bool) { + if t.IsFlagSet(TimestampFlag_SYNTHETIC) { + return ClockTimestamp{}, false + } + return ClockTimestamp(t), true +} + +// UnsafeToClockTimestamp converts a Timestamp to a ClockTimestamp, regardless +// of whether such a cast would be legal according to the Synthetic flag. The +// method should only be used in tests. +func (t Timestamp) UnsafeToClockTimestamp() ClockTimestamp { + // TODO(nvanbenschoten): unset the Synthetic flag here. + return ClockTimestamp(t) +} + +// ToTimestamp upcasts a ClockTimestamp into a Timestamp. +func (t ClockTimestamp) ToTimestamp() Timestamp { + return Timestamp(t) +} + +// Less returns whether the receiver is less than the parameter. +func (t ClockTimestamp) Less(s ClockTimestamp) bool { return Timestamp(t).Less(Timestamp(s)) } + +// String implements the fmt.Formatter interface. +func (t ClockTimestamp) String() string { return t.ToTimestamp().String() } + +// SafeValue implements the redact.SafeValue interface. +func (t ClockTimestamp) SafeValue() {} + +// IsEmpty retruns true if t is an empty ClockTimestamp. +func (t ClockTimestamp) IsEmpty() bool { return Timestamp(t).IsEmpty() } + +// Forward is like Timestamp.Forward, but for ClockTimestamps. +func (t *ClockTimestamp) Forward(s ClockTimestamp) bool { return (*Timestamp)(t).Forward(Timestamp(s)) } + +// Reset implements the protoutil.Message interface. +func (t *ClockTimestamp) Reset() { (*Timestamp)(t).Reset() } + +// ProtoMessage implements the protoutil.Message interface. +func (t *ClockTimestamp) ProtoMessage() {} + +// MarshalTo implements the protoutil.Message interface. +func (t *ClockTimestamp) MarshalTo(data []byte) (int, error) { return (*Timestamp)(t).MarshalTo(data) } + +// Unmarshal implements the protoutil.Message interface. +func (t *ClockTimestamp) Unmarshal(data []byte) error { return (*Timestamp)(t).Unmarshal(data) } + +// Size implements the protoutil.Message interface. +func (t *ClockTimestamp) Size() int { return (*Timestamp)(t).Size() } + +// Equal is needed for the gogoproto.equal option. +func (t *ClockTimestamp) Equal(that interface{}) bool { + switch v := that.(type) { + case nil: + return t == nil + case ClockTimestamp: + return (*Timestamp)(t).Equal((Timestamp)(v)) + case *ClockTimestamp: + return (*Timestamp)(t).Equal((*Timestamp)(v)) + default: + return false + } +} + +// NewPopulatedClockTimestamp is needed for the gogoproto.populate option. +func NewPopulatedClockTimestamp(r randyTimestamp, easy bool) *ClockTimestamp { + return (*ClockTimestamp)(NewPopulatedTimestamp(r, easy)) +} + +// Ignore unused warnings. The function is called, but by generated functions +// that themselves are unused. +var _ = NewPopulatedClockTimestamp diff --git a/pkg/util/hlc/timestamp.pb.go b/pkg/util/hlc/timestamp.pb.go index 687dcf3cc27f..e727fcd03317 100644 --- a/pkg/util/hlc/timestamp.pb.go +++ b/pkg/util/hlc/timestamp.pb.go @@ -54,7 +54,7 @@ func (x TimestampFlag) String() string { return proto.EnumName(TimestampFlag_name, int32(x)) } func (TimestampFlag) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_timestamp_7c076e9f3a1546ea, []int{0} + return fileDescriptor_timestamp_3d06174efd2169e7, []int{0} } // Timestamp represents a state of the hybrid logical clock. @@ -74,13 +74,21 @@ type Timestamp struct { // considered when performing structural equality checks (e.g. using the // == operator). Consider use of the EqOrdering method when testing for // equality. + // + // TODO(nvanbenschoten): use a bool to shave off a + // byte when set. This will allow the flag to serve as the dynamically + // typed version of ClockTimestamp. See TryToClockTimestamp. + // + // Should look like: + // bool synthetic = 3; + // Flags uint32 `protobuf:"varint,3,opt,name=flags,proto3" json:"flags,omitempty"` } func (m *Timestamp) Reset() { *m = Timestamp{} } func (*Timestamp) ProtoMessage() {} func (*Timestamp) Descriptor() ([]byte, []int) { - return fileDescriptor_timestamp_7c076e9f3a1546ea, []int{0} + return fileDescriptor_timestamp_3d06174efd2169e7, []int{0} } func (m *Timestamp) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -513,10 +521,10 @@ var ( ) func init() { - proto.RegisterFile("util/hlc/timestamp.proto", fileDescriptor_timestamp_7c076e9f3a1546ea) + proto.RegisterFile("util/hlc/timestamp.proto", fileDescriptor_timestamp_3d06174efd2169e7) } -var fileDescriptor_timestamp_7c076e9f3a1546ea = []byte{ +var fileDescriptor_timestamp_3d06174efd2169e7 = []byte{ // 247 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x92, 0x28, 0x2d, 0xc9, 0xcc, 0xd1, 0xcf, 0xc8, 0x49, 0xd6, 0x2f, 0xc9, 0xcc, 0x4d, 0x2d, 0x2e, 0x49, 0xcc, 0x2d, 0xd0, 0x2b, diff --git a/pkg/util/hlc/timestamp.proto b/pkg/util/hlc/timestamp.proto index e38713ecc81b..c1947b4c2ed8 100644 --- a/pkg/util/hlc/timestamp.proto +++ b/pkg/util/hlc/timestamp.proto @@ -36,6 +36,14 @@ message Timestamp { // considered when performing structural equality checks (e.g. using the // == operator). Consider use of the EqOrdering method when testing for // equality. + // + // TODO(nvanbenschoten): use a bool to shave off a + // byte when set. This will allow the flag to serve as the dynamically + // typed version of ClockTimestamp. See TryToClockTimestamp. + // + // Should look like: + // bool synthetic = 3; + // uint32 flags = 3; }