From 7336ae1b3c98f59c95bd2ec15b926e108deca425 Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Tue, 22 Dec 2020 17:15:54 -0500 Subject: [PATCH 1/7] hlc: strongly type ClockTimestamp as specialization of Timestamp MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This commit splits off a new hlc.ClockTimestamp type from the existing hlc.Timestamp type through a type alias. While the two types share the same memory and proto representation, they have different purposes and properties. Timestamp serves the role of representing an arbitrary timestamp, one that makes no claim about real time. ClockTimestamp serves the role of representing a real timestamp pulled from one of the HLC clocks in the system. Because of this, it has the added capability to update a peer's HLC clock. 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. The commit also pushes towards a world where the ClockTimestamp specialization is maintained through a combination of static and dynamic typing. While the primary mechanisms that use ClockTimestamps will use static typing, Timestamp will also carry a bit indicating whether it can be downcast to a ClockTimestamp. This bit will replace the current flag structure. So instead of an interaction like the one introduced in #57077 checking whether the value has a "synthetic" flag set, it will instead check whether the value has a "clock timestamp". This serves as an alternative to an approach like the one in #58213, where we split the types in the other direction, keeping Timestamp to represent a clock timestamp and introduce a new enginepb.TxnTimestamp to represent an arbitrary MVCC timestamp. That change resulted in a significantly larger diff, as it misjudged the extremely small percent of all Timestamp usages which care about the capability of updating remote HLC clocks. It also forced all users (tests, etc.) of timestamps to address this question front-and-center, which had benefits but was also a burden on all uses of timestamps. The original intention of this change was to follow it up by inverting the synthetic flag on Timestamps, replacing an "is_synthetic" bit with a "from_clock" bit. While inverting the flag optimized the encoded size of non-clock (currently synthetic) timestamps at the expense of the encoded size of clock timestamps by 2 bytes, it came with major benefits. By making clock timestamps opt-in instead of opt-out, we more closely match the capability model we're trying to establish here with static typing, where a clock timestamp can do everything a normal timestamp can, but can also be used to update an HLC clock. The opt-in nature mitigated the risk of bugs that forget to set this flag correctly. Instead of risking a capability escalation where a non-clock timestamp is incorrectly interpreted as a clock timestamp and used to update an HLC clock, we risked a much less harmful capability de-escalation where a clock timestamp loses its ability to update an HLC clock. We could then much more carefully audit the cases where the flag needs to be unset, such as in the Timestamp.Add and Timestamp.Forward methods. Unfortunately, this "from_clock" approach (attempted in 4e34e20) came with serious complications as well, which became clear only after making the change. Chiefly, the approach made the mixed-version migration of this field significantly more difficult. With v20.2 nodes unaware of the flag, v21.1 nodes would need to find a way to either auto-assign it for all timestamps coming from v21.1 nodes, or to work around its absence. But this latter idea touches on a second complication – the absence of the field resulted (intentionally) in an MVCC encoding that v20.2 nodes would be unable to decode. So v21.1 nodes would need to be very careful to always set the "from_clock" flag on all timestamps when in a mixed version cluster. But this all made the migration towards not setting the flag once in a v21.1-only cluster even more difficult. In the end, opting into "is_synthetic" is a lot easier to work with than opting into "from_clock", so that's how the rest of this PR will operate. --- docs/generated/redact_safe.md | 1 + pkg/kv/client_test.go | 4 +- .../kvcoord/dist_sender_server_test.go | 4 +- pkg/kv/kvclient/kvcoord/dist_sender_test.go | 10 +- .../kvclient/kvcoord/txn_coord_sender_test.go | 8 +- .../txn_interceptor_span_refresher_test.go | 2 +- .../kvserver/batcheval/cmd_lease_request.go | 2 +- pkg/kv/kvserver/batcheval/cmd_subsume.go | 4 +- pkg/kv/kvserver/client_replica_test.go | 2 +- pkg/kv/kvserver/closed_timestamp_test.go | 2 +- .../concurrency/concurrency_manager_test.go | 2 +- .../intentresolver/intent_resolver_test.go | 2 +- pkg/kv/kvserver/kvserverpb/proposer_kv.pb.go | 208 +-- pkg/kv/kvserver/kvserverpb/proposer_kv.proto | 2 +- pkg/kv/kvserver/observedts/limit.go | 3 +- pkg/kv/kvserver/observedts/limit_test.go | 2 +- pkg/kv/kvserver/replica.go | 2 +- .../replica_application_state_machine.go | 10 +- pkg/kv/kvserver/replica_gc_queue.go | 2 +- pkg/kv/kvserver/replica_init.go | 2 +- pkg/kv/kvserver/replica_range_lease.go | 25 +- pkg/kv/kvserver/replica_test.go | 12 +- pkg/kv/kvserver/store_merge.go | 4 +- pkg/kv/kvserver/store_send.go | 29 +- pkg/kv/kvserver/store_test.go | 5 +- pkg/kv/txn.go | 12 +- pkg/kv/txn_test.go | 2 +- pkg/roachpb/api.pb.go | 1324 +++++++++-------- pkg/roachpb/api.proto | 12 +- pkg/roachpb/data.go | 23 +- pkg/roachpb/data.pb.go | 381 ++--- pkg/roachpb/data.proto | 15 +- pkg/roachpb/data_test.go | 61 +- pkg/roachpb/errors.pb.go | 452 +++--- pkg/roachpb/errors.proto | 3 +- pkg/roachpb/errors_test.go | 4 +- pkg/server/admin_test.go | 2 +- pkg/server/server_test.go | 4 +- pkg/sql/distsql_running.go | 2 +- pkg/sql/flowinfra/cluster_test.go | 12 +- pkg/sql/sem/tree/timeconv_test.go | 4 +- pkg/sql/txn_restart_test.go | 12 +- pkg/storage/enginepb/mvcc3.pb.go | 32 +- pkg/storage/enginepb/mvcc3.proto | 4 + pkg/testutils/lint/lint_test.go | 1 + pkg/util/hlc/hlc.go | 26 +- pkg/util/hlc/hlc_test.go | 36 +- pkg/util/hlc/timestamp.go | 90 +- pkg/util/hlc/timestamp.pb.go | 16 +- pkg/util/hlc/timestamp.proto | 8 + 50 files changed, 1543 insertions(+), 1344 deletions(-) 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 8fd5fd9907b8..3a69c7b51c2a 100644 --- a/pkg/kv/kvserver/client_replica_test.go +++ b/pkg/kv/kvserver/client_replica_test.go @@ -953,7 +953,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 1367bf2221b2..ce1f23fddd67 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 c3bdbf3fc174..42b96fee0b69 100644 --- a/pkg/kv/kvserver/replica_application_state_machine.go +++ b/pkg/kv/kvserver/replica_application_state_machine.go @@ -376,9 +376,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. @@ -464,7 +464,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 f8f5c035995a..eda36ced8ffc 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -323,7 +323,7 @@ func (tc *testContext) Sender() kv.Sender { tc.Fatal(err) } } - tc.Clock().Update(ba.Timestamp) + tc.Clock().Update(ba.Timestamp.UnsafeToClockTimestamp()) return ba }) } @@ -823,7 +823,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) } @@ -1653,7 +1653,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") } @@ -2161,10 +2161,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() @@ -2261,7 +2261,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..e49dd6419afc 100644 --- a/pkg/kv/kvserver/store_send.go +++ b/pkg/kv/kvserver/store_send.go @@ -85,14 +85,15 @@ 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 instead of updating the local clock and proceeding. + if err := s.cfg.Clock.UpdateAndCheckMaxOffset(ctx, baClockTS); err != nil { + return nil, roachpb.NewError(err) + } } } @@ -119,7 +120,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 +130,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 +140,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 +157,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 f1cc4c44f6b8..f0fae176a747 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 fc6357bfbf50..2e4f5b611a5a 100644 --- a/pkg/kv/txn.go +++ b/pkg/kv/txn.go @@ -97,12 +97,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(), ) @@ -123,7 +123,7 @@ func NewTxnFromProto( ctx context.Context, db *DB, gatewayNodeID roachpb.NodeID, - now hlc.Timestamp, + now hlc.ClockTimestamp, typ TxnType, proto *roachpb.Transaction, ) *Txn { @@ -1134,8 +1134,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, @@ -1144,7 +1144,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 63a4dbd08674..75d478cccee9 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_dc2dcd6ae5bf551d, []int{0} + return fileDescriptor_api_08826d2d66ba29e3, []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_dc2dcd6ae5bf551d, []int{1} + return fileDescriptor_api_08826d2d66ba29e3, []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_dc2dcd6ae5bf551d, []int{2} + return fileDescriptor_api_08826d2d66ba29e3, []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_dc2dcd6ae5bf551d, []int{3} + return fileDescriptor_api_08826d2d66ba29e3, []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_dc2dcd6ae5bf551d, []int{4} + return fileDescriptor_api_08826d2d66ba29e3, []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_dc2dcd6ae5bf551d, []int{5} + return fileDescriptor_api_08826d2d66ba29e3, []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_dc2dcd6ae5bf551d, []int{1, 0} + return fileDescriptor_api_08826d2d66ba29e3, []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_dc2dcd6ae5bf551d, []int{25, 0} + return fileDescriptor_api_08826d2d66ba29e3, []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_dc2dcd6ae5bf551d, []int{0} + return fileDescriptor_api_08826d2d66ba29e3, []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_dc2dcd6ae5bf551d, []int{1} + return fileDescriptor_api_08826d2d66ba29e3, []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_dc2dcd6ae5bf551d, []int{2} + return fileDescriptor_api_08826d2d66ba29e3, []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_dc2dcd6ae5bf551d, []int{3} + return fileDescriptor_api_08826d2d66ba29e3, []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_dc2dcd6ae5bf551d, []int{4} + return fileDescriptor_api_08826d2d66ba29e3, []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_dc2dcd6ae5bf551d, []int{5} + return fileDescriptor_api_08826d2d66ba29e3, []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_dc2dcd6ae5bf551d, []int{6} + return fileDescriptor_api_08826d2d66ba29e3, []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_dc2dcd6ae5bf551d, []int{7} + return fileDescriptor_api_08826d2d66ba29e3, []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_dc2dcd6ae5bf551d, []int{8} + return fileDescriptor_api_08826d2d66ba29e3, []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_dc2dcd6ae5bf551d, []int{9} + return fileDescriptor_api_08826d2d66ba29e3, []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_dc2dcd6ae5bf551d, []int{10} + return fileDescriptor_api_08826d2d66ba29e3, []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_dc2dcd6ae5bf551d, []int{11} + return fileDescriptor_api_08826d2d66ba29e3, []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_dc2dcd6ae5bf551d, []int{12} + return fileDescriptor_api_08826d2d66ba29e3, []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_dc2dcd6ae5bf551d, []int{13} + return fileDescriptor_api_08826d2d66ba29e3, []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_dc2dcd6ae5bf551d, []int{14} + return fileDescriptor_api_08826d2d66ba29e3, []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_dc2dcd6ae5bf551d, []int{15} + return fileDescriptor_api_08826d2d66ba29e3, []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_dc2dcd6ae5bf551d, []int{16} + return fileDescriptor_api_08826d2d66ba29e3, []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_dc2dcd6ae5bf551d, []int{17} + return fileDescriptor_api_08826d2d66ba29e3, []int{17} } func (m *ClearRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1126,7 +1127,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_dc2dcd6ae5bf551d, []int{18} + return fileDescriptor_api_08826d2d66ba29e3, []int{18} } func (m *RevertRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1160,7 +1161,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_dc2dcd6ae5bf551d, []int{19} + return fileDescriptor_api_08826d2d66ba29e3, []int{19} } func (m *RevertRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1211,7 +1212,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_dc2dcd6ae5bf551d, []int{20} + return fileDescriptor_api_08826d2d66ba29e3, []int{20} } func (m *ScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1263,7 +1264,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_dc2dcd6ae5bf551d, []int{21} + return fileDescriptor_api_08826d2d66ba29e3, []int{21} } func (m *ScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1314,7 +1315,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_dc2dcd6ae5bf551d, []int{22} + return fileDescriptor_api_08826d2d66ba29e3, []int{22} } func (m *ReverseScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1366,7 +1367,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_dc2dcd6ae5bf551d, []int{23} + return fileDescriptor_api_08826d2d66ba29e3, []int{23} } func (m *ReverseScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1419,7 +1420,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_dc2dcd6ae5bf551d, []int{24} + return fileDescriptor_api_08826d2d66ba29e3, []int{24} } func (m *CheckConsistencyRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1456,7 +1457,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_dc2dcd6ae5bf551d, []int{25} + return fileDescriptor_api_08826d2d66ba29e3, []int{25} } func (m *CheckConsistencyResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1500,7 +1501,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_dc2dcd6ae5bf551d, []int{25, 0} + return fileDescriptor_api_08826d2d66ba29e3, []int{25, 0} } func (m *CheckConsistencyResponse_Result) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1548,7 +1549,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_dc2dcd6ae5bf551d, []int{26} + return fileDescriptor_api_08826d2d66ba29e3, []int{26} } func (m *RecomputeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1584,7 +1585,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_dc2dcd6ae5bf551d, []int{27} + return fileDescriptor_api_08826d2d66ba29e3, []int{27} } func (m *RecomputeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1695,7 +1696,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_dc2dcd6ae5bf551d, []int{28} + return fileDescriptor_api_08826d2d66ba29e3, []int{28} } func (m *EndTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1741,7 +1742,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_dc2dcd6ae5bf551d, []int{29} + return fileDescriptor_api_08826d2d66ba29e3, []int{29} } func (m *EndTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1802,7 +1803,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_dc2dcd6ae5bf551d, []int{30} + return fileDescriptor_api_08826d2d66ba29e3, []int{30} } func (m *AdminSplitRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1837,7 +1838,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_dc2dcd6ae5bf551d, []int{31} + return fileDescriptor_api_08826d2d66ba29e3, []int{31} } func (m *AdminSplitResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1876,7 +1877,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_dc2dcd6ae5bf551d, []int{32} + return fileDescriptor_api_08826d2d66ba29e3, []int{32} } func (m *AdminUnsplitRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1911,7 +1912,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_dc2dcd6ae5bf551d, []int{33} + return fileDescriptor_api_08826d2d66ba29e3, []int{33} } func (m *AdminUnsplitResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1954,7 +1955,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_dc2dcd6ae5bf551d, []int{34} + return fileDescriptor_api_08826d2d66ba29e3, []int{34} } func (m *AdminMergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1989,7 +1990,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_dc2dcd6ae5bf551d, []int{35} + return fileDescriptor_api_08826d2d66ba29e3, []int{35} } func (m *AdminMergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2027,7 +2028,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_dc2dcd6ae5bf551d, []int{36} + return fileDescriptor_api_08826d2d66ba29e3, []int{36} } func (m *AdminTransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2060,7 +2061,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_dc2dcd6ae5bf551d, []int{37} + return fileDescriptor_api_08826d2d66ba29e3, []int{37} } func (m *AdminTransferLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2095,7 +2096,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_dc2dcd6ae5bf551d, []int{38} + return fileDescriptor_api_08826d2d66ba29e3, []int{38} } func (m *ReplicationChange) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2153,7 +2154,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_dc2dcd6ae5bf551d, []int{39} + return fileDescriptor_api_08826d2d66ba29e3, []int{39} } func (m *AdminChangeReplicasRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2188,7 +2189,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_dc2dcd6ae5bf551d, []int{40} + return fileDescriptor_api_08826d2d66ba29e3, []int{40} } func (m *AdminChangeReplicasResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2225,7 +2226,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_dc2dcd6ae5bf551d, []int{41} + return fileDescriptor_api_08826d2d66ba29e3, []int{41} } func (m *AdminRelocateRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2258,7 +2259,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_dc2dcd6ae5bf551d, []int{42} + return fileDescriptor_api_08826d2d66ba29e3, []int{42} } func (m *AdminRelocateRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2290,14 +2291,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_dc2dcd6ae5bf551d, []int{43} + return fileDescriptor_api_08826d2d66ba29e3, []int{43} } func (m *HeartbeatTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2334,7 +2338,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_dc2dcd6ae5bf551d, []int{44} + return fileDescriptor_api_08826d2d66ba29e3, []int{44} } func (m *HeartbeatTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2372,7 +2376,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_dc2dcd6ae5bf551d, []int{45} + return fileDescriptor_api_08826d2d66ba29e3, []int{45} } func (m *GCRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2406,7 +2410,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_dc2dcd6ae5bf551d, []int{45, 0} + return fileDescriptor_api_08826d2d66ba29e3, []int{45, 0} } func (m *GCRequest_GCKey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2440,7 +2444,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_dc2dcd6ae5bf551d, []int{46} + return fileDescriptor_api_08826d2d66ba29e3, []int{46} } func (m *GCResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2509,7 +2513,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_dc2dcd6ae5bf551d, []int{47} + return fileDescriptor_api_08826d2d66ba29e3, []int{47} } func (m *PushTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2552,7 +2556,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_dc2dcd6ae5bf551d, []int{48} + return fileDescriptor_api_08826d2d66ba29e3, []int{48} } func (m *PushTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2599,7 +2603,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_dc2dcd6ae5bf551d, []int{49} + return fileDescriptor_api_08826d2d66ba29e3, []int{49} } func (m *RecoverTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2635,7 +2639,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_dc2dcd6ae5bf551d, []int{50} + return fileDescriptor_api_08826d2d66ba29e3, []int{50} } func (m *RecoverTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2679,7 +2683,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_dc2dcd6ae5bf551d, []int{51} + return fileDescriptor_api_08826d2d66ba29e3, []int{51} } func (m *QueryTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2723,7 +2727,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_dc2dcd6ae5bf551d, []int{52} + return fileDescriptor_api_08826d2d66ba29e3, []int{52} } func (m *QueryTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2783,7 +2787,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_dc2dcd6ae5bf551d, []int{53} + return fileDescriptor_api_08826d2d66ba29e3, []int{53} } func (m *QueryIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2819,7 +2823,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_dc2dcd6ae5bf551d, []int{54} + return fileDescriptor_api_08826d2d66ba29e3, []int{54} } func (m *QueryIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2866,7 +2870,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_dc2dcd6ae5bf551d, []int{55} + return fileDescriptor_api_08826d2d66ba29e3, []int{55} } func (m *ResolveIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2901,7 +2905,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_dc2dcd6ae5bf551d, []int{56} + return fileDescriptor_api_08826d2d66ba29e3, []int{56} } func (m *ResolveIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2952,7 +2956,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_dc2dcd6ae5bf551d, []int{57} + return fileDescriptor_api_08826d2d66ba29e3, []int{57} } func (m *ResolveIntentRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2987,7 +2991,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_dc2dcd6ae5bf551d, []int{58} + return fileDescriptor_api_08826d2d66ba29e3, []int{58} } func (m *ResolveIntentRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3024,7 +3028,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_dc2dcd6ae5bf551d, []int{59} + return fileDescriptor_api_08826d2d66ba29e3, []int{59} } func (m *MergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3058,7 +3062,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_dc2dcd6ae5bf551d, []int{60} + return fileDescriptor_api_08826d2d66ba29e3, []int{60} } func (m *MergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3103,7 +3107,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_dc2dcd6ae5bf551d, []int{61} + return fileDescriptor_api_08826d2d66ba29e3, []int{61} } func (m *TruncateLogRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3137,7 +3141,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_dc2dcd6ae5bf551d, []int{62} + return fileDescriptor_api_08826d2d66ba29e3, []int{62} } func (m *TruncateLogResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3174,14 +3178,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_dc2dcd6ae5bf551d, []int{63} + return fileDescriptor_api_08826d2d66ba29e3, []int{63} } func (m *RequestLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3230,7 +3234,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_dc2dcd6ae5bf551d, []int{64} + return fileDescriptor_api_08826d2d66ba29e3, []int{64} } func (m *TransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3267,7 +3271,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_dc2dcd6ae5bf551d, []int{65} + return fileDescriptor_api_08826d2d66ba29e3, []int{65} } func (m *LeaseInfoRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3304,7 +3308,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_dc2dcd6ae5bf551d, []int{66} + return fileDescriptor_api_08826d2d66ba29e3, []int{66} } func (m *LeaseInfoResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3339,7 +3343,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_dc2dcd6ae5bf551d, []int{67} + return fileDescriptor_api_08826d2d66ba29e3, []int{67} } func (m *RequestLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3394,7 +3398,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_dc2dcd6ae5bf551d, []int{68} + return fileDescriptor_api_08826d2d66ba29e3, []int{68} } func (m *ComputeChecksumRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3431,7 +3435,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_dc2dcd6ae5bf551d, []int{69} + return fileDescriptor_api_08826d2d66ba29e3, []int{69} } func (m *ComputeChecksumResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3471,7 +3475,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_dc2dcd6ae5bf551d, []int{70} + return fileDescriptor_api_08826d2d66ba29e3, []int{70} } func (m *ExternalStorage) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3505,7 +3509,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_dc2dcd6ae5bf551d, []int{70, 0} + return fileDescriptor_api_08826d2d66ba29e3, []int{70, 0} } func (m *ExternalStorage_LocalFilePath) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3538,7 +3542,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_dc2dcd6ae5bf551d, []int{70, 1} + return fileDescriptor_api_08826d2d66ba29e3, []int{70, 1} } func (m *ExternalStorage_Http) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3580,7 +3584,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_dc2dcd6ae5bf551d, []int{70, 2} + return fileDescriptor_api_08826d2d66ba29e3, []int{70, 2} } func (m *ExternalStorage_S3) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3619,7 +3623,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_dc2dcd6ae5bf551d, []int{70, 3} + return fileDescriptor_api_08826d2d66ba29e3, []int{70, 3} } func (m *ExternalStorage_GCS) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3655,7 +3659,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_dc2dcd6ae5bf551d, []int{70, 4} + return fileDescriptor_api_08826d2d66ba29e3, []int{70, 4} } func (m *ExternalStorage_Azure) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3694,7 +3698,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_dc2dcd6ae5bf551d, []int{70, 5} + return fileDescriptor_api_08826d2d66ba29e3, []int{70, 5} } func (m *ExternalStorage_Workload) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3736,7 +3740,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_dc2dcd6ae5bf551d, []int{70, 6} + return fileDescriptor_api_08826d2d66ba29e3, []int{70, 6} } func (m *ExternalStorage_FileTable) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3776,7 +3780,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_dc2dcd6ae5bf551d, []int{71} + return fileDescriptor_api_08826d2d66ba29e3, []int{71} } func (m *WriteBatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3810,7 +3814,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_dc2dcd6ae5bf551d, []int{72} + return fileDescriptor_api_08826d2d66ba29e3, []int{72} } func (m *WriteBatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3846,7 +3850,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_dc2dcd6ae5bf551d, []int{73} + return fileDescriptor_api_08826d2d66ba29e3, []int{73} } func (m *FileEncryptionOptions) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3916,7 +3920,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_dc2dcd6ae5bf551d, []int{74} + return fileDescriptor_api_08826d2d66ba29e3, []int{74} } func (m *ExportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3968,7 +3972,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_dc2dcd6ae5bf551d, []int{75} + return fileDescriptor_api_08826d2d66ba29e3, []int{75} } func (m *BulkOpSummary) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4004,7 +4008,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_dc2dcd6ae5bf551d, []int{76} + return fileDescriptor_api_08826d2d66ba29e3, []int{76} } func (m *ExportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4044,7 +4048,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_dc2dcd6ae5bf551d, []int{76, 0} + return fileDescriptor_api_08826d2d66ba29e3, []int{76, 0} } func (m *ExportResponse_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4095,7 +4099,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_dc2dcd6ae5bf551d, []int{77} + return fileDescriptor_api_08826d2d66ba29e3, []int{77} } func (m *ImportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4130,7 +4134,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_dc2dcd6ae5bf551d, []int{77, 0} + return fileDescriptor_api_08826d2d66ba29e3, []int{77, 0} } func (m *ImportRequest_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4166,7 +4170,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_dc2dcd6ae5bf551d, []int{77, 1} + return fileDescriptor_api_08826d2d66ba29e3, []int{77, 1} } func (m *ImportRequest_TableRekey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4201,7 +4205,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_dc2dcd6ae5bf551d, []int{78} + return fileDescriptor_api_08826d2d66ba29e3, []int{78} } func (m *ImportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4239,7 +4243,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_dc2dcd6ae5bf551d, []int{79} + return fileDescriptor_api_08826d2d66ba29e3, []int{79} } func (m *AdminScatterRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4276,7 +4280,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_dc2dcd6ae5bf551d, []int{80} + return fileDescriptor_api_08826d2d66ba29e3, []int{80} } func (m *AdminScatterResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4309,7 +4313,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_dc2dcd6ae5bf551d, []int{80, 0} + return fileDescriptor_api_08826d2d66ba29e3, []int{80, 0} } func (m *AdminScatterResponse_Range) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4354,7 +4358,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_dc2dcd6ae5bf551d, []int{81} + return fileDescriptor_api_08826d2d66ba29e3, []int{81} } func (m *AdminVerifyProtectedTimestampRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4392,7 +4396,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_dc2dcd6ae5bf551d, []int{82} + return fileDescriptor_api_08826d2d66ba29e3, []int{82} } func (m *AdminVerifyProtectedTimestampResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4445,7 +4449,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_dc2dcd6ae5bf551d, []int{83} + return fileDescriptor_api_08826d2d66ba29e3, []int{83} } func (m *AddSSTableRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4479,7 +4483,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_dc2dcd6ae5bf551d, []int{84} + return fileDescriptor_api_08826d2d66ba29e3, []int{84} } func (m *AddSSTableResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4523,7 +4527,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_dc2dcd6ae5bf551d, []int{85} + return fileDescriptor_api_08826d2d66ba29e3, []int{85} } func (m *RefreshRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4557,7 +4561,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_dc2dcd6ae5bf551d, []int{86} + return fileDescriptor_api_08826d2d66ba29e3, []int{86} } func (m *RefreshResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4596,7 +4600,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_dc2dcd6ae5bf551d, []int{87} + return fileDescriptor_api_08826d2d66ba29e3, []int{87} } func (m *RefreshRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4630,7 +4634,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_dc2dcd6ae5bf551d, []int{88} + return fileDescriptor_api_08826d2d66ba29e3, []int{88} } func (m *RefreshRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4679,7 +4683,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_dc2dcd6ae5bf551d, []int{89} + return fileDescriptor_api_08826d2d66ba29e3, []int{89} } func (m *SubsumeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4721,14 +4725,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_dc2dcd6ae5bf551d, []int{90} + return fileDescriptor_api_08826d2d66ba29e3, []int{90} } func (m *SubsumeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4763,7 +4767,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_dc2dcd6ae5bf551d, []int{91} + return fileDescriptor_api_08826d2d66ba29e3, []int{91} } func (m *RangeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4805,7 +4809,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_dc2dcd6ae5bf551d, []int{92} + return fileDescriptor_api_08826d2d66ba29e3, []int{92} } func (m *RangeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4848,7 +4852,7 @@ func (m *MigrateRequest) Reset() { *m = MigrateRequest{} } func (m *MigrateRequest) String() string { return proto.CompactTextString(m) } func (*MigrateRequest) ProtoMessage() {} func (*MigrateRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_dc2dcd6ae5bf551d, []int{93} + return fileDescriptor_api_08826d2d66ba29e3, []int{93} } func (m *MigrateRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4882,7 +4886,7 @@ func (m *MigrateResponse) Reset() { *m = MigrateResponse{} } func (m *MigrateResponse) String() string { return proto.CompactTextString(m) } func (*MigrateResponse) ProtoMessage() {} func (*MigrateResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_dc2dcd6ae5bf551d, []int{94} + return fileDescriptor_api_08826d2d66ba29e3, []int{94} } func (m *MigrateResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4966,7 +4970,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_dc2dcd6ae5bf551d, []int{95} + return fileDescriptor_api_08826d2d66ba29e3, []int{95} } func (m *RequestUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -6447,7 +6451,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_dc2dcd6ae5bf551d, []int{96} + return fileDescriptor_api_08826d2d66ba29e3, []int{96} } func (m *ResponseUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8012,7 +8016,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_dc2dcd6ae5bf551d, []int{97} + return fileDescriptor_api_08826d2d66ba29e3, []int{97} } func (m *Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8050,7 +8054,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_dc2dcd6ae5bf551d, []int{98} + return fileDescriptor_api_08826d2d66ba29e3, []int{98} } func (m *ClientRangeInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8086,7 +8090,7 @@ type BatchRequest struct { func (m *BatchRequest) Reset() { *m = BatchRequest{} } func (*BatchRequest) ProtoMessage() {} func (*BatchRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_dc2dcd6ae5bf551d, []int{99} + return fileDescriptor_api_08826d2d66ba29e3, []int{99} } func (m *BatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8123,7 +8127,7 @@ type BatchResponse struct { func (m *BatchResponse) Reset() { *m = BatchResponse{} } func (*BatchResponse) ProtoMessage() {} func (*BatchResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_dc2dcd6ae5bf551d, []int{100} + return fileDescriptor_api_08826d2d66ba29e3, []int{100} } func (m *BatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8176,7 +8180,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"` @@ -8202,7 +8206,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_dc2dcd6ae5bf551d, []int{100, 0} + return fileDescriptor_api_08826d2d66ba29e3, []int{100, 0} } func (m *BatchResponse_Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8240,7 +8244,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_dc2dcd6ae5bf551d, []int{101} + return fileDescriptor_api_08826d2d66ba29e3, []int{101} } func (m *RangeLookupRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8278,7 +8282,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_dc2dcd6ae5bf551d, []int{102} + return fileDescriptor_api_08826d2d66ba29e3, []int{102} } func (m *RangeLookupResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8317,7 +8321,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_dc2dcd6ae5bf551d, []int{103} + return fileDescriptor_api_08826d2d66ba29e3, []int{103} } func (m *RangeFeedRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8358,7 +8362,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_dc2dcd6ae5bf551d, []int{104} + return fileDescriptor_api_08826d2d66ba29e3, []int{104} } func (m *RangeFeedValue) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8399,7 +8403,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_dc2dcd6ae5bf551d, []int{105} + return fileDescriptor_api_08826d2d66ba29e3, []int{105} } func (m *RangeFeedCheckpoint) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8436,7 +8440,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_dc2dcd6ae5bf551d, []int{106} + return fileDescriptor_api_08826d2d66ba29e3, []int{106} } func (m *RangeFeedError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8473,7 +8477,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_dc2dcd6ae5bf551d, []int{107} + return fileDescriptor_api_08826d2d66ba29e3, []int{107} } func (m *RangeFeedEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8517,7 +8521,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_dc2dcd6ae5bf551d, []int{108} + return fileDescriptor_api_08826d2d66ba29e3, []int{108} } func (m *ResetQuorumRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8549,7 +8553,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_dc2dcd6ae5bf551d, []int{109} + return fileDescriptor_api_08826d2d66ba29e3, []int{109} } func (m *ResetQuorumResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8590,7 +8594,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_dc2dcd6ae5bf551d, []int{110} + return fileDescriptor_api_08826d2d66ba29e3, []int{110} } func (m *GossipSubscriptionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8630,7 +8634,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_dc2dcd6ae5bf551d, []int{111} + return fileDescriptor_api_08826d2d66ba29e3, []int{111} } func (m *GossipSubscriptionEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8666,7 +8670,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_dc2dcd6ae5bf551d, []int{112} + return fileDescriptor_api_08826d2d66ba29e3, []int{112} } func (m *JoinNodeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8705,7 +8709,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_dc2dcd6ae5bf551d, []int{113} + return fileDescriptor_api_08826d2d66ba29e3, []int{113} } func (m *JoinNodeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8748,7 +8752,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_dc2dcd6ae5bf551d, []int{114} + return fileDescriptor_api_08826d2d66ba29e3, []int{114} } func (m *ContentionEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -27373,7 +27377,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 @@ -39114,521 +39118,523 @@ var ( ErrIntOverflowApi = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_dc2dcd6ae5bf551d) } - -var fileDescriptor_api_dc2dcd6ae5bf551d = []byte{ - // 8203 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0xbd, 0x6b, 0x6c, 0x23, 0x59, - 0x76, 0x1f, 0xae, 0x22, 0x29, 0x89, 0x3c, 0x14, 0x1f, 0xba, 0xea, 0x07, 0x9b, 0x33, 0xd3, 0xea, - 0xae, 0x7e, 0xf7, 0xce, 0x48, 0xd3, 0xdd, 0x3b, 0x9e, 0xf1, 0xf4, 0x78, 0xd6, 0x12, 0xc5, 0x6e, - 0x52, 0x6a, 0xa9, 0xd5, 0x45, 0xaa, 0xdb, 0x33, 0x5e, 0xff, 0x6b, 0x4b, 0x55, 0x57, 0x54, 0xad, - 0xc8, 0x2a, 0x76, 0x55, 0x51, 0x8f, 0x01, 0xfe, 0x40, 0x1c, 0x1b, 0x89, 0x81, 0x00, 0x8b, 0xfd, - 0xe0, 0x20, 0x6b, 0x38, 0x89, 0xd7, 0x71, 0x1c, 0x07, 0x08, 0xf2, 0x00, 0x12, 0xc4, 0x41, 0x90, - 0xd8, 0x5f, 0x0c, 0x64, 0x11, 0x38, 0xc8, 0xfa, 0x53, 0x8c, 0x00, 0x51, 0x6c, 0x6d, 0xbe, 0x25, - 0x08, 0x82, 0x20, 0x40, 0x80, 0xf9, 0x10, 0x04, 0xf7, 0x51, 0x2f, 0xb2, 0xf8, 0x50, 0x6f, 0x0d, - 0x32, 0x80, 0xbf, 0x48, 0xac, 0x53, 0xf7, 0x9c, 0xba, 0xf7, 0xdc, 0x7b, 0xcf, 0x3d, 0xbf, 0x5b, - 0xe7, 0x9e, 0x82, 0x79, 0xcb, 0x54, 0xd4, 0xfd, 0xee, 0xee, 0xb2, 0xd2, 0xd5, 0x97, 0xba, 0x96, - 0xe9, 0x98, 0x68, 0x5e, 0x35, 0xd5, 0x03, 0x4a, 0x5e, 0xe2, 0x37, 0xcb, 0xf7, 0x0f, 0x0e, 0x97, - 0x0f, 0x0e, 0x6d, 0x6c, 0x1d, 0x62, 0x6b, 0x59, 0x35, 0x0d, 0xb5, 0x67, 0x59, 0xd8, 0x50, 0x4f, - 0x96, 0xdb, 0xa6, 0x7a, 0x40, 0xff, 0xe8, 0x46, 0x8b, 0xb1, 0x97, 0x91, 0x2b, 0x51, 0x53, 0x1c, - 0x85, 0xd3, 0x2e, 0xb8, 0x34, 0x6c, 0x59, 0xa6, 0x65, 0x73, 0xea, 0x25, 0x97, 0xda, 0xc1, 0x8e, - 0x12, 0x28, 0xfd, 0x96, 0xed, 0x98, 0x96, 0xd2, 0xc2, 0xcb, 0xd8, 0x68, 0xe9, 0x06, 0x26, 0x05, - 0x0e, 0x55, 0x95, 0xdf, 0x7c, 0x3b, 0xf2, 0xe6, 0x23, 0x7e, 0xb7, 0xd4, 0x73, 0xf4, 0xf6, 0xf2, - 0x7e, 0x5b, 0x5d, 0x76, 0xf4, 0x0e, 0xb6, 0x1d, 0xa5, 0xd3, 0xe5, 0x77, 0xee, 0xd3, 0x3b, 0x8e, - 0xa5, 0xa8, 0xba, 0xd1, 0x72, 0xff, 0x77, 0x77, 0x97, 0x2d, 0xac, 0x9a, 0x96, 0x86, 0x35, 0xd9, - 0xee, 0x2a, 0x86, 0x5b, 0xdd, 0x96, 0xd9, 0x32, 0xe9, 0xcf, 0x65, 0xf2, 0x8b, 0x53, 0xaf, 0xb6, - 0x4c, 0xb3, 0xd5, 0xc6, 0xcb, 0xf4, 0x6a, 0xb7, 0xb7, 0xb7, 0xac, 0xf5, 0x2c, 0xc5, 0xd1, 0x4d, - 0xce, 0x25, 0xfe, 0x73, 0x01, 0x72, 0x12, 0x7e, 0xdd, 0xc3, 0xb6, 0x53, 0xc3, 0x8a, 0x86, 0x2d, - 0x74, 0x05, 0x92, 0x07, 0xf8, 0xa4, 0x94, 0xbc, 0x26, 0xdc, 0x9d, 0x5b, 0x9d, 0xfd, 0xf2, 0x74, - 0x31, 0xb9, 0x81, 0x4f, 0x24, 0x42, 0x43, 0xd7, 0x60, 0x16, 0x1b, 0x9a, 0x4c, 0x6e, 0xa7, 0xc2, - 0xb7, 0x67, 0xb0, 0xa1, 0x6d, 0xe0, 0x13, 0xf4, 0x6d, 0x48, 0xdb, 0x44, 0x9a, 0xa1, 0xe2, 0xd2, - 0xf4, 0x35, 0xe1, 0xee, 0xf4, 0xea, 0xcf, 0x7f, 0x79, 0xba, 0xf8, 0x49, 0x4b, 0x77, 0xf6, 0x7b, - 0xbb, 0x4b, 0xaa, 0xd9, 0x59, 0xf6, 0xfa, 0x49, 0xdb, 0xf5, 0x7f, 0x2f, 0x77, 0x0f, 0x5a, 0xcb, - 0xfd, 0x3a, 0x5a, 0x6a, 0x1e, 0x1b, 0x0d, 0xfc, 0x5a, 0xf2, 0x24, 0xae, 0xa7, 0xd2, 0x42, 0x31, - 0xb1, 0x9e, 0x4a, 0x27, 0x8a, 0x49, 0xf1, 0x77, 0x92, 0x90, 0x97, 0xb0, 0xdd, 0x35, 0x0d, 0x1b, - 0xf3, 0x9a, 0xbf, 0x0f, 0x49, 0xe7, 0xd8, 0xa0, 0x35, 0xcf, 0x3e, 0xbc, 0xba, 0x34, 0x30, 0x22, - 0x96, 0x9a, 0x96, 0x62, 0xd8, 0x8a, 0x4a, 0x9a, 0x2f, 0x91, 0xa2, 0xe8, 0x23, 0xc8, 0x5a, 0xd8, - 0xee, 0x75, 0x30, 0x55, 0x24, 0x6d, 0x54, 0xf6, 0xe1, 0xe5, 0x08, 0xce, 0x46, 0x57, 0x31, 0x24, - 0x60, 0x65, 0xc9, 0x6f, 0x74, 0x05, 0xd2, 0x46, 0xaf, 0x43, 0x54, 0x61, 0xd3, 0x86, 0x26, 0xa5, - 0x59, 0xa3, 0xd7, 0xd9, 0xc0, 0x27, 0x36, 0xfa, 0x05, 0xb8, 0xa4, 0xe1, 0xae, 0x85, 0x55, 0xc5, - 0xc1, 0x9a, 0x6c, 0x29, 0x46, 0x0b, 0xcb, 0xba, 0xb1, 0x67, 0xda, 0xa5, 0x99, 0x6b, 0xc9, 0xbb, - 0xd9, 0x87, 0x6f, 0x47, 0xc8, 0x97, 0x48, 0xa9, 0xba, 0xb1, 0x67, 0xae, 0xa6, 0x7e, 0x74, 0xba, - 0x38, 0x25, 0x5d, 0xf0, 0x25, 0x78, 0xb7, 0x6c, 0xd4, 0x80, 0x1c, 0xaf, 0xae, 0x85, 0x15, 0xdb, - 0x34, 0x4a, 0xb3, 0xd7, 0x84, 0xbb, 0xf9, 0x87, 0x4b, 0x51, 0x02, 0x43, 0xaa, 0x21, 0x97, 0xbd, - 0x0e, 0x96, 0x28, 0x97, 0x34, 0x67, 0x05, 0xae, 0xd0, 0x5b, 0x90, 0x21, 0x2d, 0xd9, 0x3d, 0x71, - 0xb0, 0x5d, 0x4a, 0xd3, 0xa6, 0x90, 0xa6, 0xad, 0x92, 0x6b, 0xf1, 0x53, 0x98, 0x0b, 0xb2, 0x22, - 0x04, 0x79, 0xa9, 0xda, 0xd8, 0xd9, 0xac, 0xca, 0x3b, 0x5b, 0x1b, 0x5b, 0xcf, 0x5f, 0x6d, 0x15, - 0xa7, 0xd0, 0x05, 0x28, 0x72, 0xda, 0x46, 0xf5, 0x33, 0xf9, 0x59, 0x7d, 0xb3, 0xde, 0x2c, 0x0a, - 0xe5, 0xd4, 0xaf, 0xfd, 0xce, 0xd5, 0x29, 0x71, 0x1b, 0xe0, 0x29, 0x76, 0xf8, 0x00, 0x43, 0xab, - 0x30, 0xb3, 0x4f, 0xeb, 0x53, 0x12, 0xa8, 0xa6, 0xaf, 0x45, 0x56, 0x3c, 0x30, 0x18, 0x57, 0xd3, - 0x44, 0x1b, 0x3f, 0x3e, 0x5d, 0x14, 0x24, 0xce, 0x29, 0xfe, 0xa1, 0x00, 0x59, 0x2a, 0x92, 0xb5, - 0x0f, 0x55, 0xfa, 0x64, 0x5e, 0x1f, 0xab, 0x8c, 0x41, 0xa1, 0x68, 0x09, 0xa6, 0x0f, 0x95, 0x76, - 0x0f, 0x97, 0x12, 0x54, 0x46, 0x29, 0x42, 0xc6, 0x4b, 0x72, 0x5f, 0x62, 0xc5, 0xd0, 0x63, 0x98, - 0xd3, 0x0d, 0x07, 0x1b, 0x8e, 0xcc, 0xd8, 0x92, 0x63, 0xd8, 0xb2, 0xac, 0x34, 0xbd, 0x10, 0xff, - 0x99, 0x00, 0xb0, 0xdd, 0x8b, 0x53, 0x29, 0xe8, 0x9b, 0x13, 0xd6, 0x9f, 0x8f, 0x2e, 0xde, 0x8a, - 0x4b, 0x30, 0xa3, 0x1b, 0x6d, 0xdd, 0x60, 0xf5, 0x4f, 0x4b, 0xfc, 0x0a, 0x5d, 0x80, 0xe9, 0xdd, - 0xb6, 0x6e, 0x68, 0x74, 0x3e, 0xa4, 0x25, 0x76, 0x21, 0x4a, 0x90, 0xa5, 0xb5, 0x8e, 0x51, 0xef, - 0xe2, 0x69, 0x02, 0x2e, 0x56, 0x4c, 0x43, 0xd3, 0xc9, 0x94, 0x54, 0xda, 0x5f, 0x0b, 0xad, 0xac, - 0x43, 0x60, 0xf2, 0xc9, 0xf8, 0xb8, 0x3b, 0x61, 0x1f, 0x23, 0x9f, 0xab, 0x7a, 0xdc, 0xa5, 0xb4, - 0x68, 0x4d, 0xa2, 0x6f, 0xc2, 0x65, 0xa5, 0xdd, 0x36, 0x8f, 0x64, 0x7d, 0x4f, 0xd6, 0x4c, 0x6c, - 0xcb, 0x86, 0xe9, 0xc8, 0xf8, 0x58, 0xb7, 0x1d, 0x6a, 0x4a, 0xd2, 0xd2, 0x02, 0xbd, 0x5d, 0xdf, - 0x5b, 0x33, 0xb1, 0xbd, 0x65, 0x3a, 0x55, 0x72, 0x8b, 0xcc, 0x53, 0x52, 0x19, 0x36, 0x4f, 0x67, - 0x88, 0xf9, 0x95, 0xd2, 0xf8, 0xb8, 0x4b, 0xe7, 0x69, 0xa0, 0x2b, 0x67, 0x83, 0x5d, 0x29, 0xfe, - 0x12, 0x5c, 0xea, 0xd7, 0x6f, 0x9c, 0xfd, 0xf7, 0xc7, 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, 0xf9, 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, 0x21, - 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, 0x58, 0x08, 0x55, 0x2c, 0xde, 0xee, 0x4b, 0xd1, 0x3a, 0x25, 0xae, 0x25, 0x83, 0x3e, 0x1c, - 0x25, 0x8a, 0x3f, 0x10, 0x60, 0xbe, 0xd2, 0xc6, 0x8a, 0x15, 0xbb, 0x46, 0xbe, 0x05, 0x69, 0x0d, - 0x2b, 0x1a, 0x6d, 0x32, 0x9b, 0xd8, 0xef, 0x04, 0xa4, 0x10, 0x4f, 0x77, 0x69, 0xbf, 0xad, 0x2e, - 0x35, 0x5d, 0x1f, 0x98, 0xcf, 0x6e, 0x8f, 0x49, 0xfc, 0x0c, 0x50, 0xb0, 0x66, 0x71, 0x0e, 0x84, - 0xff, 0x25, 0x00, 0x92, 0xf0, 0x21, 0xb6, 0x9c, 0xd8, 0x9b, 0xbd, 0x06, 0x59, 0x47, 0xb1, 0x5a, - 0xd8, 0x91, 0x89, 0x77, 0x7f, 0x9e, 0x96, 0x03, 0xe3, 0x23, 0x64, 0xd4, 0x84, 0x3b, 0xd8, 0x50, - 0x76, 0xdb, 0x98, 0x4a, 0x91, 0x77, 0xcd, 0x9e, 0xa1, 0xc9, 0xba, 0x83, 0x2d, 0xc5, 0x31, 0x2d, - 0xd9, 0xec, 0x3a, 0x7a, 0x47, 0xff, 0x82, 0x3a, 0xf6, 0x7c, 0xa8, 0xdd, 0x60, 0xc5, 0x09, 0xf3, - 0x2a, 0x29, 0x5c, 0xe7, 0x65, 0x9f, 0x07, 0x8a, 0x8a, 0x9f, 0xc3, 0x42, 0xa8, 0xd5, 0x71, 0xaa, - 0xf4, 0x7f, 0x08, 0x90, 0x6d, 0xa8, 0x8a, 0x11, 0xa7, 0x2e, 0x3f, 0x85, 0xac, 0xad, 0x2a, 0x86, - 0xbc, 0x67, 0x5a, 0x1d, 0xc5, 0xa1, 0x13, 0x27, 0x1f, 0xd2, 0xa5, 0xe7, 0xaf, 0xab, 0x8a, 0xf1, - 0x84, 0x16, 0x92, 0xc0, 0xf6, 0x7e, 0xa3, 0x17, 0x90, 0x3d, 0xc0, 0x27, 0x32, 0xc7, 0x7e, 0x74, - 0xb5, 0xcd, 0x3f, 0x7c, 0x3f, 0xc0, 0x7f, 0x70, 0xb8, 0xe4, 0x42, 0xc6, 0xa5, 0x00, 0x64, 0x5c, - 0x22, 0x1c, 0x4b, 0x0d, 0xc7, 0xc2, 0x46, 0xcb, 0xd9, 0x97, 0xe0, 0x00, 0x9f, 0x3c, 0x63, 0x32, - 0xd8, 0x74, 0x5d, 0x4f, 0xa5, 0x93, 0xc5, 0x94, 0xf8, 0xbf, 0x05, 0x98, 0x63, 0x4d, 0x8e, 0x73, - 0xba, 0x7e, 0x00, 0x29, 0xcb, 0x3c, 0x62, 0xd3, 0x35, 0xfb, 0xf0, 0xad, 0x08, 0x11, 0x1b, 0xf8, - 0x24, 0xb8, 0x1e, 0xd2, 0xe2, 0x68, 0x15, 0xb8, 0xd7, 0x29, 0x53, 0xee, 0xe4, 0xa4, 0xdc, 0xc0, - 0xb8, 0x24, 0x22, 0xe3, 0x0e, 0x14, 0x76, 0x15, 0x47, 0xdd, 0x97, 0x2d, 0x5e, 0x49, 0xb2, 0x76, - 0x26, 0xef, 0xce, 0x49, 0x79, 0x4a, 0x76, 0xab, 0x6e, 0x93, 0x96, 0xb3, 0xf9, 0x63, 0xe3, 0xbf, - 0x60, 0x7d, 0xfe, 0x7f, 0x04, 0x3e, 0x87, 0xdc, 0x96, 0xff, 0x45, 0xeb, 0xfa, 0xdf, 0x48, 0xc0, - 0xe5, 0xca, 0x3e, 0x56, 0x0f, 0x2a, 0xa6, 0x61, 0xeb, 0xb6, 0x43, 0x74, 0x17, 0x67, 0xff, 0xbf, - 0x05, 0x99, 0x23, 0xdd, 0xd9, 0x97, 0x35, 0x7d, 0x6f, 0x8f, 0x5a, 0xcf, 0xb4, 0x94, 0x26, 0x84, - 0x35, 0x7d, 0x6f, 0x0f, 0x3d, 0x82, 0x54, 0xc7, 0xd4, 0x98, 0x73, 0x9e, 0x7f, 0xb8, 0x18, 0x21, - 0x9e, 0x56, 0xcd, 0xee, 0x75, 0x36, 0x4d, 0x0d, 0x4b, 0xb4, 0x30, 0xba, 0x0a, 0xa0, 0x12, 0x6a, - 0xd7, 0xd4, 0x0d, 0x87, 0xaf, 0xbe, 0x01, 0x0a, 0xaa, 0x41, 0xc6, 0xc1, 0x56, 0x47, 0x37, 0x14, - 0x07, 0x97, 0xa6, 0xa9, 0xf2, 0x6e, 0x46, 0x56, 0xbc, 0xdb, 0xd6, 0x55, 0x65, 0x0d, 0xdb, 0xaa, - 0xa5, 0x77, 0x1d, 0xd3, 0xe2, 0x5a, 0xf4, 0x99, 0xc5, 0xef, 0xa5, 0xa0, 0x34, 0xa8, 0x9b, 0x38, - 0x47, 0xc8, 0x36, 0xcc, 0x10, 0x34, 0xdf, 0x76, 0xf8, 0x18, 0x79, 0x38, 0x4c, 0x05, 0x11, 0x35, - 0xa0, 0xbb, 0x02, 0x6d, 0x87, 0x57, 0x9b, 0xcb, 0x29, 0xff, 0x2b, 0x01, 0x66, 0xd8, 0x0d, 0xf4, - 0x00, 0xd2, 0x7c, 0xfb, 0x42, 0xa3, 0x75, 0x4c, 0xae, 0x5e, 0x3a, 0x3b, 0x5d, 0x9c, 0x65, 0x3b, - 0x12, 0x6b, 0x5f, 0xfa, 0x3f, 0xa5, 0x59, 0x5a, 0xae, 0xae, 0x91, 0xde, 0xb2, 0x1d, 0xc5, 0x72, - 0xe8, 0x26, 0x51, 0x82, 0xa1, 0x14, 0x4a, 0xd8, 0xc0, 0x27, 0x68, 0x1d, 0x66, 0x6c, 0x47, 0x71, - 0x7a, 0x36, 0xef, 0xaf, 0x73, 0x55, 0xb6, 0x41, 0x39, 0x25, 0x2e, 0x81, 0xb8, 0x4f, 0x1a, 0x76, - 0x14, 0xbd, 0x4d, 0x3b, 0x30, 0x23, 0xf1, 0x2b, 0xf1, 0x37, 0x05, 0x98, 0x61, 0x45, 0xd1, 0x65, - 0x58, 0x90, 0x56, 0xb6, 0x9e, 0x56, 0xe5, 0xfa, 0xd6, 0x5a, 0xb5, 0x59, 0x95, 0x36, 0xeb, 0x5b, - 0x2b, 0xcd, 0x6a, 0x71, 0x0a, 0x5d, 0x02, 0xe4, 0xde, 0xa8, 0x3c, 0xdf, 0x6a, 0xd4, 0x1b, 0xcd, - 0xea, 0x56, 0xb3, 0x28, 0xd0, 0x9d, 0x0c, 0x4a, 0x0f, 0x50, 0x13, 0xe8, 0x26, 0x5c, 0xeb, 0xa7, - 0xca, 0x8d, 0xe6, 0x4a, 0xb3, 0x21, 0x57, 0x1b, 0xcd, 0xfa, 0xe6, 0x4a, 0xb3, 0xba, 0x56, 0x4c, - 0x8e, 0x28, 0x45, 0x1e, 0x22, 0x49, 0xd5, 0x4a, 0xb3, 0x98, 0x12, 0x1d, 0xb8, 0x28, 0x61, 0xd5, - 0xec, 0x74, 0x7b, 0x0e, 0x26, 0xb5, 0xb4, 0xe3, 0x9c, 0x29, 0x97, 0x61, 0x56, 0xb3, 0x4e, 0x64, - 0xab, 0x67, 0xf0, 0x79, 0x32, 0xa3, 0x59, 0x27, 0x52, 0xcf, 0x10, 0xff, 0x89, 0x00, 0x97, 0xfa, - 0x1f, 0x1b, 0xe7, 0x20, 0x7c, 0x01, 0x59, 0x45, 0xd3, 0xb0, 0x26, 0x6b, 0xb8, 0xed, 0x28, 0xdc, - 0xc5, 0xb9, 0x1f, 0x90, 0xc4, 0xb7, 0xf6, 0x96, 0xbc, 0xad, 0xbd, 0xcd, 0x97, 0x95, 0x0a, 0xad, - 0xc8, 0x1a, 0xe1, 0x70, 0xcd, 0x0f, 0x15, 0x42, 0x29, 0xe2, 0x7f, 0x4b, 0x41, 0xae, 0x6a, 0x68, - 0xcd, 0xe3, 0x58, 0xd7, 0x92, 0x4b, 0x30, 0xa3, 0x9a, 0x9d, 0x8e, 0xee, 0xb8, 0x0a, 0x62, 0x57, - 0xe8, 0x67, 0x03, 0xae, 0x69, 0x72, 0x02, 0x07, 0xcd, 0x77, 0x4a, 0xd1, 0x77, 0xe0, 0x32, 0xb1, - 0x9a, 0x96, 0xa1, 0xb4, 0x65, 0x26, 0x4d, 0x76, 0x2c, 0xbd, 0xd5, 0xc2, 0x16, 0xdf, 0x4e, 0xbc, - 0x1b, 0x51, 0xcf, 0x3a, 0xe7, 0xa8, 0x50, 0x86, 0x26, 0x2b, 0x2f, 0x5d, 0xd4, 0xa3, 0xc8, 0xe8, - 0x13, 0x00, 0xb2, 0x14, 0xd1, 0x2d, 0x4a, 0x9b, 0xdb, 0xa3, 0x61, 0x7b, 0x94, 0xae, 0x09, 0x22, - 0x0c, 0xe4, 0xda, 0x46, 0xcb, 0x04, 0x87, 0xbc, 0xee, 0xe9, 0x16, 0x96, 0x1f, 0x74, 0x55, 0xba, - 0x71, 0x90, 0x5e, 0xcd, 0x9f, 0x9d, 0x2e, 0x82, 0xc4, 0xc8, 0x0f, 0xb6, 0x2b, 0x04, 0x97, 0xb0, - 0xdf, 0x5d, 0x15, 0xbd, 0x82, 0x7b, 0x81, 0xfd, 0x0f, 0xb2, 0xf2, 0xf2, 0x66, 0x29, 0x8e, 0xbc, - 0xaf, 0xb7, 0xf6, 0xb1, 0x25, 0x7b, 0xdb, 0xd4, 0x74, 0xbf, 0x30, 0x2d, 0xdd, 0xf4, 0x19, 0x2a, - 0x8a, 0xc1, 0x6a, 0xbf, 0xe2, 0xd4, 0x68, 0x61, 0x4f, 0x67, 0x44, 0xf9, 0x5d, 0x53, 0xb7, 0x4d, - 0xa3, 0x94, 0x61, 0xca, 0x67, 0x57, 0xe8, 0x1e, 0x14, 0x9d, 0x63, 0x43, 0xde, 0xc7, 0x8a, 0xe5, - 0xec, 0x62, 0xc5, 0x21, 0xab, 0x34, 0xd0, 0x12, 0x05, 0xe7, 0xd8, 0xa8, 0x05, 0xc8, 0xe8, 0x05, - 0x14, 0x75, 0x43, 0xde, 0x6b, 0xeb, 0xad, 0x7d, 0x47, 0x3e, 0xb2, 0x74, 0x07, 0xdb, 0xa5, 0x79, - 0xaa, 0x90, 0xa8, 0x71, 0xdb, 0xe0, 0xfb, 0xc6, 0xda, 0x2b, 0x52, 0x92, 0xab, 0x26, 0xaf, 0x1b, - 0x4f, 0x28, 0x3f, 0x25, 0xda, 0xeb, 0xa9, 0xf4, 0x6c, 0x31, 0x2d, 0xfe, 0x27, 0x01, 0xf2, 0xee, - 0x70, 0x8b, 0x73, 0x66, 0xdc, 0x85, 0xa2, 0x69, 0x60, 0xb9, 0xbb, 0xaf, 0xd8, 0x98, 0xeb, 0x91, - 0x2f, 0x38, 0x79, 0xd3, 0xc0, 0xdb, 0x84, 0xcc, 0xd4, 0x85, 0xb6, 0x61, 0xde, 0x76, 0x94, 0x96, - 0x6e, 0xb4, 0x02, 0xea, 0x9d, 0x9e, 0x1c, 0x2c, 0x14, 0x39, 0xb7, 0x47, 0x0f, 0x79, 0x29, 0x7f, - 0x22, 0xc0, 0xfc, 0x8a, 0xd6, 0xd1, 0x8d, 0x46, 0xb7, 0xad, 0xc7, 0xba, 0x07, 0x71, 0x13, 0x32, - 0x36, 0x91, 0xe9, 0x1b, 0x7c, 0x1f, 0x51, 0xa6, 0xe9, 0x1d, 0x62, 0xf9, 0x9f, 0x41, 0x01, 0x1f, - 0x77, 0x75, 0xf6, 0xea, 0x81, 0x01, 0xa1, 0xd4, 0xe4, 0x6d, 0xcb, 0xfb, 0xbc, 0xe4, 0x16, 0x6f, - 0xd3, 0x67, 0x80, 0x82, 0x4d, 0x8a, 0x13, 0xbb, 0x7c, 0x06, 0x0b, 0x54, 0xf4, 0x8e, 0x61, 0xc7, - 0xac, 0x2f, 0xf1, 0x17, 0xe1, 0x42, 0x58, 0x74, 0x9c, 0xf5, 0x7e, 0xc5, 0x7b, 0x79, 0x13, 0x5b, - 0xb1, 0x82, 0x58, 0x4f, 0xd7, 0x5c, 0x70, 0x9c, 0x75, 0xfe, 0x55, 0x01, 0xae, 0x50, 0xd9, 0xf4, - 0xed, 0xcc, 0x1e, 0xb6, 0x9e, 0x61, 0xc5, 0x8e, 0x15, 0x81, 0xdf, 0x80, 0x19, 0x86, 0xa4, 0xe9, - 0xf8, 0x9c, 0x5e, 0xcd, 0x12, 0xcf, 0xa5, 0xe1, 0x98, 0x16, 0xf1, 0x5c, 0xf8, 0x2d, 0x51, 0x81, - 0x72, 0x54, 0x2d, 0xe2, 0x6c, 0xe9, 0xdf, 0x16, 0x60, 0x9e, 0x3b, 0x8d, 0x64, 0x28, 0x57, 0xf6, - 0x89, 0xcf, 0x84, 0xaa, 0x90, 0x55, 0xe9, 0x2f, 0xd9, 0x39, 0xe9, 0x62, 0x2a, 0x3f, 0x3f, 0xca, - 0xdf, 0x64, 0x6c, 0xcd, 0x93, 0x2e, 0x26, 0x4e, 0xab, 0xfb, 0x9b, 0x28, 0x2a, 0xd0, 0xc8, 0x91, - 0x1e, 0x2b, 0x9d, 0x47, 0xb4, 0xac, 0xeb, 0xfa, 0x71, 0x1d, 0xfc, 0xd3, 0x24, 0x57, 0x02, 0x7b, - 0x06, 0x2f, 0x1e, 0xab, 0x8f, 0xf2, 0x79, 0xe8, 0xe5, 0x58, 0xb0, 0xe1, 0x89, 0x73, 0x34, 0x3c, - 0xb0, 0x43, 0xef, 0x53, 0xd1, 0x67, 0x10, 0xd8, 0x83, 0x97, 0x59, 0x9b, 0x5c, 0xf4, 0x73, 0x1e, - 0x75, 0xcc, 0xfb, 0x52, 0x18, 0xdd, 0x46, 0x15, 0x48, 0xe3, 0xe3, 0xae, 0xac, 0x61, 0x5b, 0xe5, - 0x86, 0x4b, 0x1c, 0xf6, 0x16, 0x6f, 0x00, 0x0f, 0xcc, 0xe2, 0xe3, 0x2e, 0x21, 0xa2, 0x1d, 0xb2, - 0x7a, 0xb9, 0xae, 0x02, 0xad, 0xb6, 0x3d, 0x1e, 0x5e, 0xf8, 0x23, 0x85, 0x8b, 0x2b, 0x78, 0x5e, - 0x02, 0x13, 0x21, 0xfe, 0x50, 0x80, 0xb7, 0x22, 0x7b, 0x2d, 0xce, 0x85, 0xec, 0x13, 0x48, 0xd1, - 0xc6, 0x27, 0xce, 0xd9, 0x78, 0xca, 0x25, 0xfe, 0xae, 0x3b, 0xc7, 0x25, 0xdc, 0x36, 0x89, 0x62, - 0xbf, 0x82, 0x5d, 0xb6, 0x59, 0xb7, 0xc3, 0x13, 0xe7, 0xee, 0x70, 0x97, 0xd5, 0x33, 0x02, 0x7d, - 0xd5, 0x8c, 0xd3, 0x08, 0x7c, 0x5f, 0x80, 0x05, 0xcf, 0xa7, 0x89, 0xd9, 0xbd, 0xfd, 0x00, 0x92, - 0x86, 0x79, 0x74, 0x9e, 0x2d, 0x46, 0x52, 0x9e, 0x2c, 0x49, 0xe1, 0x1a, 0xc5, 0xd9, 0xde, 0x7f, - 0x9d, 0x80, 0xcc, 0xd3, 0x4a, 0x9c, 0xad, 0xfc, 0x84, 0x6f, 0x5f, 0xb3, 0x89, 0x1d, 0x35, 0x14, - 0xbd, 0xe7, 0x2d, 0x3d, 0xad, 0x6c, 0xe0, 0x13, 0x77, 0x28, 0x12, 0x2e, 0xb4, 0x02, 0x19, 0x67, - 0xdf, 0xc2, 0xf6, 0xbe, 0xd9, 0xd6, 0xce, 0xe3, 0x83, 0xf8, 0x5c, 0x65, 0x0c, 0xd3, 0x54, 0xae, - 0x1b, 0x2a, 0x21, 0x44, 0x84, 0x4a, 0x90, 0xc7, 0x78, 0x6e, 0x5c, 0xe2, 0x3c, 0x8f, 0x09, 0xf8, - 0x6f, 0xd3, 0xc5, 0x19, 0xf1, 0x05, 0x00, 0x69, 0x4e, 0x9c, 0x5d, 0xf2, 0x57, 0x92, 0x90, 0xdf, - 0xee, 0xd9, 0xfb, 0x31, 0x8f, 0xbe, 0x0a, 0x40, 0xb7, 0x67, 0x53, 0x7c, 0x70, 0x6c, 0xf0, 0x36, - 0x8f, 0x89, 0xc2, 0x70, 0x1b, 0xcd, 0xf8, 0x9a, 0xc7, 0x06, 0xaa, 0x71, 0x21, 0x58, 0xf6, 0x43, - 0x39, 0x6e, 0x8c, 0x42, 0x92, 0xcd, 0x63, 0x63, 0x13, 0x7b, 0x10, 0x92, 0x49, 0xc2, 0x44, 0xd2, - 0x27, 0x30, 0x4b, 0x2e, 0x64, 0xc7, 0x3c, 0x4f, 0x37, 0xcf, 0x10, 0x9e, 0xa6, 0x89, 0x1e, 0x43, - 0x86, 0x71, 0x93, 0xa5, 0x69, 0x86, 0x2e, 0x4d, 0x51, 0x6d, 0xe1, 0x6a, 0xa4, 0x8b, 0x52, 0x9a, - 0xb2, 0x92, 0x85, 0xe8, 0x02, 0x4c, 0xef, 0x99, 0x96, 0xea, 0xbe, 0x8c, 0x65, 0x17, 0xac, 0x3f, - 0xd7, 0x53, 0xe9, 0x74, 0x31, 0xb3, 0x9e, 0x4a, 0x67, 0x8a, 0x20, 0xfe, 0xa6, 0x00, 0x05, 0xaf, - 0x23, 0xe2, 0xb4, 0xd6, 0x95, 0x90, 0x16, 0xcf, 0xdf, 0x15, 0x44, 0x81, 0xe2, 0xbf, 0xa1, 0xee, - 0x8a, 0x6a, 0x1e, 0xd2, 0x9e, 0x89, 0x73, 0xa4, 0x3c, 0x66, 0x81, 0x3a, 0x89, 0xf3, 0xf6, 0x2e, - 0x8d, 0xd9, 0x79, 0x00, 0x17, 0xf4, 0x0e, 0xb1, 0xe3, 0xba, 0xd3, 0x3e, 0xe1, 0x98, 0xca, 0xc1, - 0xee, 0x5b, 0xdf, 0x05, 0xff, 0x5e, 0xc5, 0xbd, 0x25, 0xfe, 0x5d, 0xba, 0x3b, 0xed, 0xb7, 0x24, - 0x4e, 0x55, 0xd7, 0x21, 0x67, 0x31, 0xd1, 0xc4, 0xe7, 0x38, 0xa7, 0xb6, 0xe7, 0x3c, 0x56, 0xa2, - 0xf0, 0xdf, 0x4e, 0x40, 0xe1, 0x45, 0x0f, 0x5b, 0x27, 0x5f, 0x27, 0x75, 0xdf, 0x86, 0xc2, 0x91, - 0xa2, 0x3b, 0xf2, 0x9e, 0x69, 0xc9, 0xbd, 0xae, 0xa6, 0x38, 0x6e, 0xb4, 0x48, 0x8e, 0x90, 0x9f, - 0x98, 0xd6, 0x0e, 0x25, 0x22, 0x0c, 0xe8, 0xc0, 0x30, 0x8f, 0x0c, 0x99, 0x90, 0x29, 0x8a, 0x3d, - 0x36, 0xf8, 0x96, 0xf1, 0xea, 0x87, 0xff, 0xf1, 0x74, 0xf1, 0xd1, 0x44, 0x31, 0x60, 0x34, 0xde, - 0xad, 0xd7, 0xd3, 0xb5, 0xa5, 0x9d, 0x9d, 0xfa, 0x9a, 0x54, 0xa4, 0x22, 0x5f, 0x31, 0x89, 0xcd, - 0x63, 0xc3, 0x16, 0xff, 0x5e, 0x02, 0x8a, 0xbe, 0x8e, 0xe2, 0xec, 0xc8, 0x2a, 0x64, 0x5f, 0xf7, - 0xb0, 0xa5, 0xbf, 0x41, 0x37, 0x02, 0x67, 0x24, 0x66, 0xe7, 0x73, 0x98, 0x0b, 0x69, 0x20, 0xf9, - 0xd3, 0x69, 0x20, 0x7b, 0xe4, 0x37, 0x1e, 0xdd, 0x87, 0x79, 0xe7, 0xd8, 0x90, 0x59, 0xf4, 0x1f, - 0x8b, 0x18, 0x71, 0x83, 0x19, 0x0a, 0x0e, 0xd1, 0x07, 0xa1, 0xd3, 0x68, 0x11, 0x5b, 0xfc, 0x43, - 0x01, 0x10, 0x55, 0x54, 0x9d, 0xed, 0xe9, 0x7f, 0x5d, 0xc6, 0xd3, 0x5d, 0x28, 0xd2, 0x78, 0x4a, - 0x59, 0xdf, 0x93, 0x3b, 0xba, 0x6d, 0xeb, 0x46, 0x8b, 0x0f, 0xa8, 0x3c, 0xa5, 0xd7, 0xf7, 0x36, - 0x19, 0x55, 0xfc, 0xff, 0x61, 0x21, 0xd4, 0x80, 0x38, 0x3b, 0xfb, 0x3a, 0xcc, 0xed, 0xb1, 0x57, - 0xa8, 0x54, 0x38, 0xdf, 0x0e, 0xcc, 0x52, 0x1a, 0x7b, 0x9e, 0xf8, 0x5f, 0x13, 0x70, 0x41, 0xc2, - 0xb6, 0xd9, 0x3e, 0xc4, 0xf1, 0xab, 0xb0, 0x06, 0xfc, 0x5d, 0x8b, 0xfc, 0x46, 0x9a, 0xcc, 0x30, - 0x66, 0xb6, 0xcc, 0x85, 0xf7, 0xd4, 0x6f, 0x8e, 0x1e, 0xb1, 0x83, 0xbb, 0xe8, 0x7c, 0x4f, 0x2e, - 0x15, 0xda, 0x93, 0x33, 0xa1, 0xa0, 0xb7, 0x0c, 0x93, 0xd8, 0x34, 0x1b, 0xbf, 0x36, 0x7a, 0x1d, - 0x17, 0xa9, 0x2c, 0x8d, 0xaa, 0x64, 0x9d, 0xb1, 0x34, 0xf0, 0xeb, 0xad, 0x5e, 0x87, 0xfa, 0xce, - 0xab, 0x97, 0x48, 0x7d, 0xcf, 0x4e, 0x17, 0xf3, 0xa1, 0x7b, 0xb6, 0x94, 0xd7, 0xbd, 0x6b, 0x22, - 0x5d, 0xfc, 0x36, 0x5c, 0xec, 0x53, 0x76, 0x9c, 0x1e, 0xcf, 0xbf, 0x4c, 0xc2, 0x95, 0xb0, 0xf8, - 0xb8, 0xf1, 0xc7, 0xd7, 0xbd, 0x43, 0x6b, 0x90, 0xeb, 0xe8, 0xc6, 0x9b, 0x6d, 0x2d, 0xce, 0x75, - 0x74, 0xc3, 0xdf, 0xc6, 0x8d, 0x18, 0x1a, 0x33, 0x5f, 0xe9, 0xd0, 0x50, 0xa0, 0x1c, 0xd5, 0x77, - 0x71, 0x8e, 0x8f, 0x5f, 0x13, 0x60, 0x2e, 0xee, 0x3d, 0xb3, 0x37, 0x8b, 0x62, 0x13, 0x9b, 0x90, - 0xfb, 0x0a, 0x36, 0xd9, 0x7e, 0x5b, 0x00, 0xd4, 0xb4, 0x7a, 0x06, 0x01, 0xb5, 0xcf, 0xcc, 0x56, - 0x9c, 0xcd, 0xbc, 0x00, 0xd3, 0xba, 0xa1, 0xe1, 0x63, 0xda, 0xcc, 0x94, 0xc4, 0x2e, 0x42, 0xaf, - 0x0e, 0x93, 0x13, 0xbd, 0x3a, 0x14, 0x3f, 0x87, 0x85, 0x50, 0x15, 0xe3, 0x6c, 0xff, 0xdf, 0x4f, - 0xc0, 0x02, 0x6f, 0x48, 0xec, 0xdb, 0x8b, 0xdf, 0x84, 0xe9, 0x36, 0x91, 0x39, 0xa2, 0x9f, 0xe9, - 0x33, 0xdd, 0x7e, 0xa6, 0x85, 0xd1, 0xcf, 0x01, 0x74, 0x2d, 0x7c, 0x28, 0x33, 0xd6, 0xe4, 0x44, - 0xac, 0x19, 0xc2, 0x41, 0x09, 0xe8, 0x17, 0xa0, 0x40, 0xe6, 0x73, 0xd7, 0x32, 0xbb, 0xa6, 0x4d, - 0x5c, 0x16, 0x7b, 0x32, 0x94, 0x33, 0x7f, 0x76, 0xba, 0x98, 0xdb, 0xd4, 0x8d, 0x6d, 0xce, 0xd8, - 0x6c, 0x48, 0xc4, 0x30, 0x78, 0x97, 0xb6, 0xf8, 0xef, 0x05, 0xb8, 0xf0, 0x95, 0x6d, 0xc5, 0xfe, - 0xbf, 0xd0, 0x95, 0xf8, 0x12, 0x8a, 0xf4, 0x47, 0xdd, 0xd8, 0x33, 0xe3, 0xdc, 0x14, 0xff, 0x9e, - 0x00, 0xf3, 0x01, 0xc1, 0x71, 0xfa, 0x27, 0x6f, 0xa4, 0x27, 0xf1, 0x17, 0x89, 0xc7, 0x12, 0x1c, - 0xe4, 0x71, 0x4e, 0xa1, 0xdf, 0x4f, 0xc0, 0xa5, 0x0a, 0x7b, 0x85, 0xec, 0xc6, 0x54, 0xc4, 0x39, - 0x32, 0x4a, 0x30, 0x7b, 0x88, 0x2d, 0x5b, 0x37, 0xd9, 0xea, 0x99, 0x93, 0xdc, 0x4b, 0x54, 0x86, - 0xb4, 0x6d, 0x28, 0x5d, 0x7b, 0xdf, 0x74, 0xdf, 0x9d, 0x79, 0xd7, 0x5e, 0xfc, 0xc7, 0xf4, 0x9b, - 0xc7, 0x7f, 0xcc, 0x8c, 0x8e, 0xff, 0x98, 0xfd, 0x29, 0xe2, 0x3f, 0xf8, 0x8b, 0xaa, 0x7f, 0x2b, - 0xc0, 0xe5, 0x01, 0xcd, 0xc5, 0x39, 0x5a, 0xbe, 0x0b, 0x59, 0x95, 0x0b, 0x26, 0xf6, 0x96, 0xbd, - 0x85, 0xab, 0x93, 0x62, 0x6f, 0x08, 0x3b, 0xce, 0x4e, 0x17, 0xc1, 0xad, 0x6a, 0x7d, 0x8d, 0x2b, - 0x87, 0xfc, 0xd6, 0xc4, 0x5f, 0xc9, 0x41, 0xa1, 0x7a, 0xcc, 0x76, 0xa0, 0x1b, 0x6c, 0x95, 0x47, - 0x4f, 0x20, 0xdd, 0xb5, 0xcc, 0x43, 0xdd, 0x6d, 0x46, 0x3e, 0xf4, 0xf2, 0xdf, 0x6d, 0x46, 0x1f, - 0xd7, 0x36, 0xe7, 0x90, 0x3c, 0x5e, 0xd4, 0x84, 0xcc, 0x33, 0x53, 0x55, 0xda, 0x4f, 0xf4, 0xb6, - 0x3b, 0xf2, 0xdf, 0x1f, 0x2f, 0x68, 0xc9, 0xe3, 0xd9, 0x56, 0x9c, 0x7d, 0xb7, 0x13, 0x3c, 0x22, - 0xaa, 0x43, 0xba, 0xe6, 0x38, 0x5d, 0x72, 0x93, 0xdb, 0x8e, 0x3b, 0x13, 0x08, 0x25, 0x2c, 0x6e, - 0x04, 0xaa, 0xcb, 0x8e, 0x9a, 0x30, 0xff, 0x94, 0x9e, 0xa7, 0xaa, 0xb4, 0xcd, 0x9e, 0x56, 0x31, - 0x8d, 0x3d, 0xbd, 0xc5, 0xed, 0xee, 0xed, 0x09, 0x64, 0x3e, 0xad, 0x34, 0xa4, 0x41, 0x01, 0x68, - 0x05, 0xd2, 0x8d, 0x47, 0x5c, 0x18, 0x73, 0xcb, 0x6e, 0x4d, 0x20, 0xac, 0xf1, 0x48, 0xf2, 0xd8, - 0xd0, 0x3a, 0x64, 0x57, 0xbe, 0xe8, 0x59, 0x98, 0x4b, 0x99, 0x19, 0x1a, 0x79, 0xd0, 0x2f, 0x85, - 0x72, 0x49, 0x41, 0x66, 0xd4, 0x80, 0xfc, 0x2b, 0xd3, 0x3a, 0x68, 0x9b, 0x8a, 0xdb, 0xc2, 0x59, - 0x2a, 0xee, 0x1b, 0x13, 0x88, 0x73, 0x19, 0xa5, 0x3e, 0x11, 0xe8, 0xdb, 0x50, 0x20, 0x9d, 0xd1, - 0x54, 0x76, 0xdb, 0x6e, 0x25, 0xd3, 0x54, 0xea, 0xbb, 0x13, 0x48, 0xf5, 0x38, 0xdd, 0x57, 0x20, - 0x7d, 0xa2, 0xca, 0x12, 0xe4, 0x42, 0x83, 0x00, 0x21, 0x48, 0x75, 0x49, 0x7f, 0x0b, 0x34, 0x36, - 0x88, 0xfe, 0x46, 0xef, 0xc1, 0xac, 0x61, 0x6a, 0xd8, 0x9d, 0x21, 0xb9, 0xd5, 0x0b, 0x67, 0xa7, - 0x8b, 0x33, 0x5b, 0xa6, 0xc6, 0x1c, 0x12, 0xfe, 0x4b, 0x9a, 0x21, 0x85, 0xea, 0x5a, 0xf9, 0x1a, - 0xa4, 0x48, 0xbf, 0x13, 0xc3, 0xb4, 0xab, 0xd8, 0x78, 0xc7, 0xd2, 0xb9, 0x34, 0xf7, 0xb2, 0xfc, - 0x8f, 0x13, 0x90, 0x68, 0x3c, 0x22, 0x2e, 0xf7, 0x6e, 0x4f, 0x3d, 0xc0, 0x0e, 0xbf, 0xcf, 0xaf, - 0xa8, 0x2b, 0x6e, 0xe1, 0x3d, 0x9d, 0x79, 0x46, 0x19, 0x89, 0x5f, 0xa1, 0x77, 0x00, 0x14, 0x55, - 0xc5, 0xb6, 0x2d, 0xbb, 0xe7, 0xec, 0x32, 0x52, 0x86, 0x51, 0x36, 0xf0, 0x09, 0x61, 0xb3, 0xb1, - 0x6a, 0x61, 0xc7, 0x0d, 0x6c, 0x62, 0x57, 0x84, 0xcd, 0xc1, 0x9d, 0xae, 0xec, 0x98, 0x07, 0xd8, - 0xa0, 0xe3, 0x24, 0x43, 0x4c, 0x4d, 0xa7, 0xdb, 0x24, 0x04, 0x62, 0x25, 0xb1, 0xa1, 0xf9, 0x26, - 0x2d, 0x23, 0x79, 0xd7, 0x44, 0xa4, 0x85, 0x5b, 0x3a, 0x3f, 0x30, 0x96, 0x91, 0xf8, 0x15, 0xd1, - 0x92, 0xd2, 0x73, 0xf6, 0x69, 0x4f, 0x64, 0x24, 0xfa, 0x1b, 0xdd, 0x86, 0x02, 0x8b, 0x85, 0x94, - 0xb1, 0xa1, 0xca, 0xd4, 0xb8, 0x66, 0xe8, 0xed, 0x1c, 0x23, 0x57, 0x0d, 0x95, 0x98, 0x52, 0xf4, - 0x08, 0x38, 0x41, 0x3e, 0xe8, 0xd8, 0x44, 0xa7, 0x40, 0x4a, 0xad, 0x16, 0xce, 0x4e, 0x17, 0xb3, - 0x0d, 0x7a, 0x63, 0x63, 0xb3, 0x51, 0x5f, 0x93, 0xb2, 0xac, 0xd4, 0x46, 0xc7, 0xae, 0x6b, 0xe5, - 0x5f, 0x17, 0x20, 0xf9, 0xb4, 0xd2, 0x38, 0xb7, 0xca, 0xdc, 0x8a, 0x26, 0x03, 0x15, 0xbd, 0x03, - 0x85, 0x5d, 0xbd, 0xdd, 0xd6, 0x8d, 0x16, 0xf1, 0x82, 0xbe, 0x8b, 0x55, 0x57, 0x61, 0x79, 0x4e, - 0xde, 0x66, 0x54, 0x74, 0x0d, 0xb2, 0xaa, 0x85, 0x35, 0x6c, 0x38, 0xba, 0xd2, 0xb6, 0xb9, 0xe6, - 0x82, 0xa4, 0xf2, 0x2f, 0x0b, 0x30, 0x4d, 0x67, 0x00, 0x7a, 0x1b, 0x32, 0xaa, 0x69, 0x38, 0x8a, - 0x6e, 0x70, 0x53, 0x96, 0x91, 0x7c, 0xc2, 0xd0, 0xea, 0x5d, 0x87, 0x39, 0x45, 0x55, 0xcd, 0x9e, - 0xe1, 0xc8, 0x86, 0xd2, 0xc1, 0xbc, 0x9a, 0x59, 0x4e, 0xdb, 0x52, 0x3a, 0x18, 0x2d, 0x82, 0x7b, - 0xe9, 0x1d, 0x9f, 0xcc, 0x48, 0xc0, 0x49, 0x1b, 0xf8, 0xa4, 0xfc, 0x47, 0x02, 0xa4, 0xdd, 0x39, - 0x43, 0xaa, 0xd1, 0xc2, 0x06, 0x0b, 0xd8, 0x76, 0xab, 0xe1, 0x11, 0xfa, 0x97, 0xca, 0x8c, 0xbf, - 0x54, 0x5e, 0x80, 0x69, 0x87, 0x4c, 0x0b, 0x5e, 0x03, 0x76, 0x41, 0xb7, 0xa3, 0xdb, 0x4a, 0x8b, - 0xed, 0xc6, 0x65, 0x24, 0x76, 0x41, 0x1a, 0xc3, 0x43, 0x6a, 0x99, 0x46, 0xf8, 0x15, 0xa9, 0x29, - 0x0b, 0xfc, 0xdc, 0xc5, 0x2d, 0xdd, 0xa0, 0x63, 0x29, 0x29, 0x01, 0x25, 0xad, 0x12, 0x0a, 0x7a, - 0x0b, 0x32, 0xac, 0x00, 0x36, 0x34, 0x3a, 0xa0, 0x92, 0x52, 0x9a, 0x12, 0xaa, 0x86, 0x56, 0xc6, - 0x90, 0xf1, 0x26, 0x27, 0xe9, 0xb6, 0x9e, 0xed, 0x29, 0x92, 0xfe, 0x46, 0xef, 0xc3, 0x85, 0xd7, - 0x3d, 0xa5, 0xad, 0xef, 0xd1, 0x8d, 0x36, 0x1a, 0xd1, 0x4e, 0x75, 0xc6, 0x5a, 0x82, 0xbc, 0x7b, - 0x54, 0x02, 0x55, 0x9d, 0x3b, 0x97, 0x93, 0xfe, 0x5c, 0x16, 0x7f, 0x4f, 0x80, 0x79, 0x16, 0xd5, - 0xc3, 0x82, 0x51, 0xe3, 0xf3, 0x43, 0x3e, 0x86, 0x8c, 0xa6, 0x38, 0x0a, 0x3b, 0x10, 0x9a, 0x18, - 0x79, 0x20, 0xd4, 0x3b, 0xa0, 0xa0, 0x38, 0x0a, 0x3d, 0x14, 0x8a, 0x20, 0x45, 0x7e, 0xb3, 0xb3, - 0xb3, 0x12, 0xfd, 0x2d, 0x7e, 0x06, 0x28, 0x58, 0xd1, 0x38, 0x3d, 0xb2, 0x7b, 0x70, 0x91, 0xe8, - 0xba, 0x6a, 0xa8, 0xd6, 0x49, 0xd7, 0xd1, 0x4d, 0xe3, 0x39, 0xfd, 0x6b, 0xa3, 0x62, 0xe0, 0xbd, - 0x14, 0x7d, 0x1d, 0x25, 0xfe, 0xc1, 0x0c, 0xe4, 0xaa, 0xc7, 0x5d, 0xd3, 0x8a, 0x75, 0x17, 0x6b, - 0x15, 0x66, 0x39, 0xd0, 0x1f, 0xf1, 0x5e, 0xb8, 0xcf, 0x98, 0xbb, 0xaf, 0x5c, 0x39, 0x23, 0x5a, - 0x05, 0x60, 0x01, 0xa3, 0x34, 0x28, 0x28, 0x79, 0x8e, 0x37, 0x65, 0x94, 0x8d, 0x1e, 0x8e, 0xd8, - 0x82, 0x6c, 0xe7, 0x50, 0x55, 0xe5, 0x3d, 0xbd, 0xed, 0xf0, 0xb8, 0xbb, 0xe8, 0x10, 0xf1, 0xcd, - 0x97, 0x95, 0xca, 0x13, 0x5a, 0x88, 0x85, 0xc0, 0xf9, 0xd7, 0x12, 0x10, 0x09, 0xec, 0x37, 0x7a, - 0x17, 0xf8, 0x41, 0x1d, 0xd9, 0x76, 0xcf, 0xe4, 0xad, 0xe6, 0xce, 0x4e, 0x17, 0x33, 0x12, 0xa5, - 0x36, 0x1a, 0x4d, 0x29, 0xc3, 0x0a, 0x34, 0x6c, 0x07, 0xdd, 0x80, 0x9c, 0xd9, 0xd1, 0x1d, 0xd9, - 0x75, 0x92, 0xb8, 0x47, 0x39, 0x47, 0x88, 0xae, 0x13, 0x75, 0x9e, 0xf3, 0x1b, 0xb3, 0x13, 0x9f, - 0xdf, 0x40, 0x7f, 0x55, 0x80, 0x4b, 0x5c, 0x91, 0xf2, 0x2e, 0x8d, 0x71, 0x57, 0xda, 0xba, 0x73, - 0x22, 0x1f, 0x1c, 0x96, 0xd2, 0xd4, 0x6f, 0xfd, 0xd9, 0xc8, 0x0e, 0x09, 0x8c, 0x83, 0x25, 0xb7, - 0x5b, 0x4e, 0x9e, 0x71, 0xe6, 0x8d, 0xc3, 0xaa, 0xe1, 0x58, 0x27, 0xab, 0x97, 0xcf, 0x4e, 0x17, - 0x17, 0x06, 0xef, 0xbe, 0x94, 0x16, 0xec, 0x41, 0x16, 0x54, 0x03, 0xc0, 0xde, 0x38, 0xa4, 0x2b, - 0x46, 0xb4, 0xff, 0x11, 0x39, 0x60, 0xa5, 0x00, 0x2f, 0xba, 0x0b, 0x45, 0x7e, 0x5e, 0x66, 0x4f, - 0x6f, 0x63, 0xd9, 0xd6, 0xbf, 0xc0, 0x74, 0x6d, 0x49, 0x4a, 0x79, 0x46, 0x27, 0x22, 0x1a, 0xfa, - 0x17, 0xb8, 0xfc, 0x5d, 0x28, 0x0d, 0xab, 0x7d, 0x70, 0x0a, 0x64, 0xd8, 0x1b, 0xd9, 0x8f, 0xc2, - 0xdb, 0x31, 0x13, 0x0c, 0x55, 0xbe, 0x25, 0xf3, 0x71, 0xe2, 0x23, 0x41, 0xfc, 0x07, 0x09, 0xc8, - 0xad, 0xf6, 0xda, 0x07, 0xcf, 0xbb, 0x8d, 0x5e, 0xa7, 0xa3, 0x58, 0x27, 0xc4, 0x0c, 0x32, 0x43, - 0x41, 0x2a, 0x28, 0x30, 0x33, 0x48, 0x2d, 0x81, 0xfe, 0x05, 0x26, 0x8b, 0x53, 0xf0, 0x0c, 0x38, - 0x8b, 0xe1, 0xa7, 0x6d, 0x08, 0x1c, 0xec, 0x36, 0x8f, 0x6c, 0xf4, 0x11, 0x94, 0x02, 0x05, 0xe9, - 0xde, 0x89, 0x8c, 0x0d, 0xc7, 0xd2, 0x31, 0xdb, 0xff, 0x4b, 0x4a, 0x81, 0x78, 0x99, 0x3a, 0xb9, - 0x5d, 0x65, 0x77, 0x51, 0x13, 0xe6, 0x48, 0xc1, 0x13, 0x99, 0x2e, 0x21, 0xee, 0xfe, 0xec, 0x83, - 0x88, 0x66, 0x85, 0xea, 0xbd, 0x44, 0xf5, 0x53, 0xa1, 0x3c, 0xf4, 0xa7, 0x94, 0xc5, 0x3e, 0xa5, - 0xfc, 0x29, 0x14, 0xfb, 0x0b, 0x04, 0x75, 0x99, 0x62, 0xba, 0xbc, 0x10, 0xd4, 0x65, 0x32, 0xa0, - 0xa7, 0xf5, 0x54, 0x3a, 0x55, 0x9c, 0x16, 0xff, 0x3c, 0x09, 0x79, 0x77, 0x98, 0xc5, 0x09, 0x74, - 0x56, 0x61, 0x9a, 0x0c, 0x0a, 0x37, 0xc6, 0xe3, 0xf6, 0x88, 0xd1, 0xcd, 0xa3, 0xc6, 0xc9, 0x60, - 0x71, 0x41, 0x32, 0x65, 0x8d, 0xc3, 0xe0, 0x94, 0x7f, 0x39, 0x01, 0x29, 0x8a, 0x2d, 0x1e, 0x40, - 0x8a, 0x2e, 0x14, 0xc2, 0x24, 0x0b, 0x05, 0x2d, 0xea, 0x2d, 0x67, 0x89, 0x80, 0x6b, 0x4a, 0x7c, - 0xbe, 0x7d, 0xe5, 0x83, 0x07, 0x0f, 0xa9, 0xb1, 0x99, 0x93, 0xf8, 0x15, 0x5a, 0xa5, 0x61, 0x47, - 0xa6, 0xe5, 0x60, 0x8d, 0xfb, 0xf4, 0xd7, 0xc6, 0xf5, 0xaf, 0xbb, 0x28, 0xb9, 0x7c, 0xe8, 0x0a, - 0x24, 0x89, 0x15, 0x9b, 0x65, 0x41, 0x0a, 0x67, 0xa7, 0x8b, 0x49, 0x62, 0xbf, 0x08, 0x0d, 0x2d, - 0x43, 0x36, 0x6c, 0x32, 0x88, 0x07, 0x47, 0x0d, 0x63, 0x60, 0xba, 0x43, 0xdb, 0x9b, 0x5a, 0x0c, - 0xcf, 0xf2, 0x3e, 0xfe, 0xef, 0x29, 0xc8, 0xd5, 0x3b, 0x71, 0x2f, 0x29, 0x2b, 0xe1, 0x1e, 0x8e, - 0x02, 0x42, 0xa1, 0x87, 0x46, 0x74, 0x70, 0x68, 0x05, 0x4f, 0x9e, 0x6f, 0x05, 0xaf, 0x13, 0x4f, - 0x99, 0x27, 0x75, 0x48, 0x0e, 0xc1, 0x3c, 0xe1, 0xe7, 0x53, 0x3f, 0x45, 0x22, 0x3c, 0xfe, 0x39, - 0x0a, 0x1a, 0x68, 0xf2, 0x29, 0x75, 0xc8, 0xd9, 0x28, 0x9b, 0x99, 0x7c, 0x94, 0xcd, 0x62, 0x43, - 0xa3, 0x8b, 0x5a, 0xd8, 0xa2, 0xce, 0xbe, 0xb9, 0x45, 0x2d, 0x3b, 0x7c, 0xb0, 0x7e, 0x0c, 0x49, - 0x4d, 0x77, 0x3b, 0x67, 0xf2, 0xa5, 0x9a, 0x30, 0x8d, 0x19, 0xb5, 0xa9, 0xe0, 0xa8, 0x65, 0xa3, - 0xa4, 0x5c, 0x07, 0xf0, 0x75, 0x83, 0xae, 0xc1, 0x8c, 0xd9, 0xd6, 0xdc, 0x83, 0x24, 0xb9, 0xd5, - 0xcc, 0xd9, 0xe9, 0xe2, 0xf4, 0xf3, 0xb6, 0x56, 0x5f, 0x93, 0xa6, 0xcd, 0xb6, 0x56, 0xd7, 0x68, - 0x46, 0x0d, 0x7c, 0x24, 0x7b, 0x51, 0x66, 0x73, 0xd2, 0xac, 0x81, 0x8f, 0xd6, 0xb0, 0xad, 0xf2, - 0x01, 0xf7, 0x5b, 0x02, 0xe4, 0x5d, 0xdd, 0xc7, 0x6b, 0x54, 0xd2, 0x7a, 0x87, 0x4f, 0xb2, 0xe4, - 0xf9, 0x26, 0x99, 0xcb, 0xc7, 0x0f, 0xed, 0xfe, 0xaa, 0xc0, 0xe3, 0x86, 0x1b, 0xaa, 0xe2, 0x10, - 0xa7, 0x22, 0xc6, 0x89, 0x71, 0x0f, 0x8a, 0x96, 0x62, 0x68, 0x66, 0x47, 0xff, 0x02, 0xb3, 0x8d, - 0x50, 0x9b, 0xbf, 0xb5, 0x2c, 0x78, 0x74, 0xba, 0xeb, 0x67, 0x8b, 0x7f, 0x94, 0xe0, 0x31, 0xc6, - 0x5e, 0x35, 0xe2, 0x54, 0xd7, 0x77, 0x60, 0xbe, 0x3f, 0xbd, 0x89, 0x3b, 0x5b, 0xdf, 0x8b, 0x90, - 0x17, 0x55, 0x11, 0x16, 0x2b, 0xe8, 0x06, 0xae, 0xf7, 0xa5, 0x3a, 0xb1, 0x51, 0x05, 0xb2, 0xc1, - 0xac, 0x29, 0xc9, 0x89, 0xb3, 0xa6, 0x80, 0xe5, 0xe5, 0x4a, 0x29, 0xff, 0x3c, 0x4c, 0xd3, 0xdb, - 0x6f, 0x60, 0xa2, 0x79, 0x6f, 0xfe, 0x59, 0x02, 0x6e, 0xd2, 0xda, 0xbf, 0xc4, 0x96, 0xbe, 0x77, - 0xb2, 0x6d, 0x99, 0x0e, 0x56, 0x1d, 0xac, 0xf9, 0xa7, 0x40, 0x62, 0xb5, 0x7b, 0x99, 0xae, 0xfb, - 0x80, 0x73, 0xc5, 0x8b, 0x79, 0x5c, 0x68, 0x03, 0x0a, 0x3c, 0x32, 0x40, 0x69, 0xeb, 0x87, 0x58, - 0x56, 0x9c, 0xf3, 0xac, 0x6e, 0x39, 0xc6, 0xbb, 0x42, 0x58, 0x57, 0x1c, 0xa4, 0x41, 0x86, 0x0b, - 0xd3, 0x35, 0x9e, 0xec, 0xe7, 0xe9, 0x4f, 0xb7, 0xa1, 0x98, 0x66, 0xe1, 0x09, 0xf5, 0x35, 0x29, - 0xcd, 0x24, 0xd7, 0x35, 0xf1, 0x3f, 0x08, 0x70, 0x6b, 0x8c, 0x8a, 0xe3, 0x1c, 0xba, 0x65, 0x48, - 0x1f, 0x92, 0x07, 0xe9, 0x5c, 0xc7, 0x69, 0xc9, 0xbb, 0x46, 0x9b, 0x90, 0xdb, 0x53, 0xf4, 0xb6, - 0x3f, 0xa4, 0x87, 0x87, 0x17, 0x46, 0x47, 0xba, 0xce, 0x31, 0x76, 0x36, 0x86, 0xc5, 0xdf, 0x48, - 0xc0, 0xfc, 0x8a, 0xa6, 0x35, 0x1a, 0xdc, 0x06, 0xc6, 0x37, 0x52, 0x5c, 0x90, 0x99, 0xf0, 0x41, - 0x26, 0x7a, 0x0f, 0x90, 0xa6, 0xdb, 0x2c, 0xa9, 0x88, 0xbd, 0xaf, 0x68, 0xe6, 0x91, 0x1f, 0x56, - 0x31, 0xef, 0xde, 0x69, 0xb8, 0x37, 0x50, 0x03, 0x28, 0xda, 0x91, 0x6d, 0x47, 0xf1, 0xde, 0x1b, - 0xdd, 0x9a, 0xe8, 0xb8, 0x16, 0x83, 0x41, 0xde, 0xa5, 0x94, 0x21, 0x72, 0xe8, 0x4f, 0xe2, 0xb7, - 0xeb, 0xa4, 0xe9, 0x8e, 0xac, 0xd8, 0xee, 0xd9, 0x1c, 0x96, 0xce, 0x24, 0xcf, 0xe8, 0x2b, 0x36, - 0x3b, 0x72, 0xc3, 0x0e, 0x13, 0xf8, 0xaa, 0x89, 0x13, 0x12, 0xff, 0x1d, 0x01, 0xf2, 0x12, 0xde, - 0xb3, 0xb0, 0x1d, 0xeb, 0xa6, 0xc0, 0x13, 0x98, 0xb3, 0x98, 0x54, 0x79, 0xcf, 0x32, 0x3b, 0xe7, - 0x99, 0x57, 0x59, 0xce, 0xf8, 0xc4, 0x32, 0x3b, 0xdc, 0xb0, 0xbc, 0x84, 0x82, 0x57, 0xc7, 0x38, - 0x1b, 0xff, 0x7b, 0xf4, 0x28, 0x32, 0x13, 0x1c, 0x77, 0x7c, 0x43, 0xbc, 0x1a, 0xa0, 0x2f, 0xaa, - 0x82, 0x15, 0x8d, 0x53, 0x0d, 0xff, 0x45, 0x80, 0x7c, 0xa3, 0xb7, 0xcb, 0xb2, 0x56, 0xc5, 0xa7, - 0x81, 0x2a, 0x64, 0xda, 0x78, 0xcf, 0x91, 0xdf, 0x28, 0x0c, 0x3e, 0x4d, 0x58, 0xe9, 0x21, 0x80, - 0xa7, 0x00, 0x16, 0x3d, 0xbe, 0x46, 0xe5, 0x24, 0xcf, 0x29, 0x27, 0x43, 0x79, 0x09, 0x59, 0xfc, - 0x87, 0x09, 0x28, 0x78, 0xcd, 0x8c, 0xd3, 0x4a, 0xbe, 0x0a, 0x59, 0x87, 0xe4, 0x79, 0xac, 0xc3, - 0x3c, 0x0f, 0xe9, 0x88, 0xb6, 0x10, 0x4b, 0xb0, 0x40, 0x1d, 0x17, 0x59, 0xe9, 0x76, 0xdb, 0xba, - 0x0b, 0x77, 0xa9, 0xfd, 0x49, 0x49, 0xf3, 0xf4, 0xd6, 0x0a, 0xbb, 0x43, 0x81, 0x2e, 0x19, 0x73, - 0x7b, 0x16, 0xc6, 0x5f, 0x60, 0x99, 0x22, 0xaf, 0xf3, 0x84, 0xac, 0x64, 0x19, 0x63, 0x83, 0xf0, - 0xf1, 0x31, 0xf7, 0x0a, 0xe6, 0xa9, 0x4e, 0xe3, 0x3e, 0x76, 0x2b, 0xfe, 0xad, 0x04, 0xa0, 0xa0, - 0xe4, 0xaf, 0xae, 0x2f, 0x12, 0xf1, 0xf5, 0xc5, 0xbb, 0x80, 0x58, 0xd0, 0xa2, 0x2d, 0x77, 0xb1, - 0x25, 0xdb, 0x58, 0x35, 0x79, 0xe6, 0x24, 0x41, 0x2a, 0xf2, 0x3b, 0xdb, 0xd8, 0x6a, 0x50, 0x3a, - 0x7a, 0x0c, 0xe0, 0x7b, 0x64, 0x7c, 0xc1, 0x18, 0xe9, 0x90, 0x49, 0x19, 0xcf, 0x15, 0x13, 0xbf, - 0x2f, 0x40, 0x7e, 0x53, 0x6f, 0x59, 0x4a, 0xac, 0x29, 0x81, 0xd0, 0xc7, 0xe1, 0x5d, 0xf0, 0xec, - 0xc3, 0x72, 0x54, 0x80, 0x0d, 0x2b, 0xe1, 0x62, 0x2b, 0xce, 0x40, 0x2c, 0xb0, 0x57, 0xa3, 0x38, - 0x4d, 0xcf, 0xbf, 0x2b, 0xc3, 0x1c, 0xaf, 0xf7, 0x8e, 0xa1, 0x9b, 0x06, 0x7a, 0x00, 0xc9, 0x16, - 0x7f, 0xcb, 0x91, 0x8d, 0xdc, 0x91, 0xf4, 0xb3, 0xe3, 0xd5, 0xa6, 0x24, 0x52, 0x96, 0xb0, 0x74, - 0x7b, 0x4e, 0x84, 0x0f, 0xe8, 0xc7, 0x9c, 0x07, 0x59, 0xba, 0x3d, 0x07, 0x35, 0xa0, 0xa0, 0xfa, - 0x59, 0xbe, 0x64, 0xc2, 0x9e, 0x1c, 0x8a, 0x17, 0x23, 0xf3, 0xad, 0xd5, 0xa6, 0xa4, 0xbc, 0x1a, - 0xba, 0x81, 0x2a, 0xc1, 0xe4, 0x52, 0xa9, 0x81, 0x80, 0x36, 0xff, 0x28, 0x73, 0x38, 0xb1, 0x55, - 0x6d, 0x2a, 0x90, 0x83, 0x0a, 0x7d, 0x0c, 0x33, 0x1a, 0x4d, 0x63, 0xc4, 0x27, 0x6f, 0x54, 0x47, - 0x87, 0xb2, 0x45, 0xd5, 0xa6, 0x24, 0xce, 0x81, 0xd6, 0x61, 0x8e, 0xfd, 0x62, 0x1e, 0x19, 0x07, - 0xd1, 0xb7, 0x86, 0x4b, 0x08, 0xac, 0x79, 0xb5, 0x29, 0x29, 0xab, 0xf9, 0x54, 0xf4, 0x4d, 0x48, - 0xd9, 0xaa, 0xe2, 0xc2, 0xe8, 0xab, 0x43, 0xb2, 0x87, 0xf8, 0xcc, 0xb4, 0x34, 0x7a, 0xcc, 0xf2, - 0x5d, 0x3a, 0xc7, 0xee, 0x8e, 0x66, 0x54, 0xf5, 0x43, 0xa7, 0xd4, 0x49, 0xf5, 0x31, 0x25, 0xa0, - 0xa7, 0x90, 0x55, 0x88, 0x6b, 0x2b, 0xd3, 0x73, 0x9e, 0x74, 0x0b, 0x33, 0x3a, 0x8e, 0x60, 0xe0, - 0x5c, 0x6e, 0x8d, 0x1e, 0x85, 0x77, 0x89, 0xbe, 0xa0, 0x0e, 0xb6, 0x5a, 0xb8, 0x94, 0x1d, 0x2d, - 0x28, 0x18, 0xc6, 0xe6, 0x09, 0xa2, 0x44, 0xe2, 0xe2, 0x7a, 0x87, 0xac, 0x69, 0xa3, 0xe6, 0x86, - 0xbe, 0xb9, 0x8e, 0x38, 0xa1, 0x54, 0x9b, 0x92, 0xe6, 0xf6, 0x03, 0x64, 0xb4, 0x04, 0x89, 0x96, - 0x5a, 0xca, 0x0d, 0x35, 0x06, 0xde, 0x29, 0x9c, 0xda, 0x94, 0x94, 0x68, 0xa9, 0xe8, 0x53, 0x48, - 0xb3, 0x23, 0x15, 0xc7, 0x46, 0x29, 0x3f, 0x74, 0x8e, 0x85, 0x0f, 0xa6, 0xd4, 0xa6, 0x24, 0x7a, - 0x8a, 0x83, 0x3c, 0x6f, 0x1b, 0xf2, 0x16, 0x8b, 0x03, 0x74, 0xa3, 0x76, 0x8b, 0x43, 0xdf, 0xe6, - 0x47, 0x05, 0xee, 0xd6, 0x28, 0xc8, 0x09, 0xd0, 0xd1, 0x77, 0xe0, 0x42, 0x58, 0x22, 0x1f, 0x69, - 0xf3, 0x43, 0xdf, 0x4c, 0x0f, 0x0d, 0x22, 0xad, 0x4d, 0x49, 0xc8, 0x1a, 0xb8, 0x89, 0x3e, 0x84, - 0x69, 0xd6, 0x6b, 0x88, 0x8a, 0x8c, 0x0a, 0x50, 0xe9, 0xeb, 0x30, 0x56, 0x9e, 0x0c, 0x7e, 0x87, - 0x07, 0xc3, 0xc9, 0x6d, 0xb3, 0x55, 0x5a, 0x18, 0x3a, 0xf8, 0x07, 0xc3, 0xfa, 0xc8, 0xe0, 0x77, - 0x7c, 0x2a, 0xe9, 0x77, 0x8b, 0xdd, 0xe1, 0x11, 0x54, 0x17, 0x86, 0xf6, 0x7b, 0x44, 0x8c, 0x5c, - 0x8d, 0x1e, 0x53, 0xf0, 0xc9, 0xa4, 0x6a, 0x16, 0x4b, 0x78, 0x23, 0xd3, 0x39, 0x75, 0x71, 0x68, - 0xd5, 0x06, 0x33, 0x02, 0xd5, 0xa8, 0x3b, 0xe8, 0x51, 0xd1, 0x4b, 0x28, 0xf2, 0xb4, 0x14, 0xfe, - 0xeb, 0x93, 0x4b, 0x54, 0xde, 0xbd, 0x48, 0xd3, 0x15, 0x15, 0x7e, 0x54, 0x9b, 0x92, 0x0a, 0x6a, - 0xf8, 0x0e, 0xfa, 0x0c, 0xe6, 0xa9, 0x3c, 0x59, 0xf5, 0x33, 0x89, 0x94, 0x4a, 0x03, 0x79, 0x29, - 0x86, 0x27, 0x1d, 0x71, 0x25, 0x17, 0xd5, 0xbe, 0x5b, 0x64, 0x18, 0xeb, 0x86, 0xee, 0x50, 0x2b, - 0x5b, 0x1e, 0x3a, 0x8c, 0xc3, 0x59, 0x11, 0xc9, 0x30, 0xd6, 0x19, 0x85, 0x0c, 0x63, 0x87, 0x87, - 0xd7, 0xf1, 0xee, 0x78, 0x7b, 0xe8, 0x30, 0x8e, 0x8a, 0xc3, 0x23, 0xc3, 0xd8, 0x09, 0xd2, 0xc9, - 0x30, 0x66, 0x06, 0xa2, 0x4f, 0xee, 0x3b, 0x43, 0x87, 0xf1, 0xd0, 0xf3, 0xd6, 0x64, 0x18, 0x2b, - 0x03, 0x37, 0xd1, 0x1a, 0x00, 0xf3, 0xdc, 0xe8, 0xfa, 0x7f, 0x75, 0xe8, 0x62, 0xd0, 0x1f, 0x66, - 0x47, 0x16, 0x83, 0xb6, 0x4b, 0x23, 0x86, 0x8c, 0xe2, 0x42, 0x99, 0xbe, 0x2d, 0x2e, 0x2d, 0x0e, - 0x35, 0x64, 0x03, 0x6f, 0x76, 0x89, 0x21, 0x3b, 0xf2, 0x88, 0x64, 0x55, 0x61, 0xdb, 0xdb, 0xa5, - 0x6b, 0xc3, 0xcd, 0x72, 0xf0, 0x2d, 0x17, 0x35, 0xcb, 0x94, 0x80, 0x56, 0x20, 0x43, 0xdc, 0x9b, - 0x13, 0x6a, 0x86, 0xae, 0x0f, 0x75, 0xbf, 0xfb, 0xce, 0xe1, 0xd4, 0xa6, 0xa4, 0xf4, 0x6b, 0x4e, - 0x22, 0x8f, 0x67, 0x1b, 0x7f, 0x25, 0x71, 0xe8, 0xe3, 0x43, 0x9b, 0xc4, 0xe4, 0xf1, 0x8c, 0x03, - 0xa9, 0x70, 0x91, 0xf5, 0x15, 0x3f, 0xfa, 0x6c, 0xf1, 0xd3, 0xba, 0xa5, 0x1b, 0x54, 0xd4, 0xd0, - 0x1d, 0xb4, 0xc8, 0x13, 0xd9, 0xb5, 0x29, 0x69, 0x41, 0x19, 0xbc, 0x4b, 0x26, 0x3c, 0x5f, 0x7a, - 0xd8, 0xbe, 0x5b, 0xe9, 0xe6, 0xd0, 0x09, 0x1f, 0xb1, 0x5d, 0x49, 0x26, 0xbc, 0x12, 0x20, 0xb3, - 0x05, 0x48, 0x93, 0x6d, 0x9b, 0xc5, 0x16, 0xdc, 0x1a, 0xb1, 0x00, 0xf5, 0x6d, 0x78, 0xb0, 0x05, - 0x48, 0x6b, 0x30, 0x4e, 0x22, 0x48, 0x6d, 0x63, 0xc5, 0xe2, 0x66, 0xf6, 0xf6, 0x50, 0x41, 0x03, - 0x09, 0x08, 0x89, 0x20, 0xd5, 0x23, 0x12, 0x87, 0xc7, 0x72, 0xf3, 0xd9, 0x70, 0xdf, 0xf8, 0xce, - 0x50, 0x87, 0x27, 0x32, 0xe1, 0x0e, 0x71, 0x78, 0xac, 0xd0, 0x0d, 0xf4, 0x73, 0x30, 0xcb, 0x91, - 0x6a, 0xe9, 0xee, 0x08, 0x17, 0x30, 0xb8, 0xb9, 0x40, 0xe6, 0x35, 0xe7, 0x61, 0x56, 0x96, 0x21, - 0x64, 0xd6, 0xbc, 0x7b, 0x23, 0xac, 0xec, 0x00, 0x48, 0x67, 0x56, 0xd6, 0x27, 0x13, 0x2b, 0xcb, - 0xc6, 0x29, 0x5f, 0xeb, 0xee, 0x0f, 0xb5, 0xb2, 0x83, 0xa7, 0x7c, 0x88, 0x95, 0x7d, 0xed, 0x53, - 0x49, 0xcb, 0x6c, 0x86, 0x14, 0x4b, 0xdf, 0x18, 0xda, 0xb2, 0x30, 0x64, 0x26, 0x2d, 0xe3, 0x3c, - 0xa4, 0xdb, 0x98, 0xf7, 0xcf, 0x34, 0xfd, 0xee, 0xf0, 0xfc, 0x02, 0xfd, 0xf8, 0xaa, 0xe6, 0xee, - 0xc9, 0x32, 0x0d, 0x7b, 0x86, 0xca, 0xe2, 0xe7, 0xab, 0xb9, 0xa6, 0xde, 0x1b, 0x6d, 0xa8, 0xa2, - 0x0e, 0x8d, 0x7b, 0x86, 0x2a, 0x74, 0x93, 0x56, 0x95, 0x1d, 0xa9, 0xa3, 0xf3, 0x7b, 0x69, 0x44, - 0x2a, 0x84, 0xbe, 0x83, 0x8d, 0xb4, 0xaa, 0x1e, 0xd1, 0x9f, 0x42, 0x3d, 0x96, 0xa7, 0xa3, 0xb4, - 0x3c, 0x7a, 0x0a, 0x85, 0x33, 0x85, 0x78, 0x53, 0x88, 0x93, 0xbd, 0x35, 0xd3, 0xf5, 0x30, 0xde, - 0x1f, 0xbd, 0x66, 0xf6, 0xbb, 0x16, 0x6c, 0xcd, 0xe4, 0x3e, 0xc5, 0x5f, 0x16, 0xe0, 0x1a, 0xab, - 0x1b, 0xdd, 0xbc, 0x3c, 0x91, 0xbd, 0x2d, 0xe0, 0xc0, 0xf1, 0x8e, 0x07, 0xf4, 0x01, 0x1f, 0x0e, - 0xab, 0xee, 0x98, 0x2d, 0xed, 0xda, 0x94, 0xf4, 0x8e, 0x32, 0xaa, 0x1c, 0x19, 0x52, 0x1d, 0x86, - 0xa0, 0x4a, 0x0f, 0x87, 0x0e, 0xa9, 0x30, 0xea, 0x23, 0x43, 0x8a, 0xf3, 0xac, 0xce, 0xf2, 0x17, - 0xc8, 0xde, 0x71, 0xd8, 0x42, 0xb1, 0xb8, 0x9e, 0x4a, 0x5f, 0x2e, 0x96, 0xd6, 0x53, 0xe9, 0x2b, - 0xc5, 0xf2, 0x7a, 0x2a, 0xfd, 0x56, 0xf1, 0x6d, 0xf1, 0x1f, 0x95, 0x21, 0xe7, 0x82, 0x2e, 0x06, - 0xa8, 0x1e, 0x06, 0x01, 0xd5, 0xd5, 0x61, 0x80, 0x8a, 0xc3, 0x34, 0x8e, 0xa8, 0x1e, 0x06, 0x11, - 0xd5, 0xd5, 0x61, 0x88, 0xca, 0xe7, 0x21, 0x90, 0xaa, 0x39, 0x0c, 0x52, 0xdd, 0x9b, 0x00, 0x52, - 0x79, 0xa2, 0xfa, 0x31, 0xd5, 0xda, 0x20, 0xa6, 0xba, 0x39, 0x1a, 0x53, 0x79, 0xa2, 0x02, 0xa0, - 0xea, 0x71, 0x1f, 0xa8, 0xba, 0x3e, 0x02, 0x54, 0x79, 0xfc, 0x2e, 0xaa, 0xda, 0x88, 0x44, 0x55, - 0xb7, 0xc7, 0xa1, 0x2a, 0x4f, 0x4e, 0x08, 0x56, 0x7d, 0x10, 0x82, 0x55, 0x8b, 0x43, 0x61, 0x95, - 0xc7, 0xcd, 0x70, 0xd5, 0x27, 0xfd, 0xb8, 0xea, 0xfa, 0x08, 0x5c, 0xe5, 0xb7, 0x80, 0x03, 0xab, - 0x5a, 0x14, 0xb0, 0xba, 0x35, 0x06, 0x58, 0x79, 0x52, 0x82, 0xc8, 0xaa, 0x16, 0x85, 0xac, 0x6e, - 0x8d, 0x41, 0x56, 0x7d, 0x92, 0x18, 0xb4, 0xda, 0x8a, 0x86, 0x56, 0x77, 0xc6, 0x42, 0x2b, 0x4f, - 0x5a, 0x18, 0x5b, 0x2d, 0x07, 0xb0, 0xd5, 0x3b, 0x43, 0xb0, 0x95, 0xc7, 0x4a, 0xc0, 0xd5, 0xb7, - 0x06, 0xc0, 0x95, 0x38, 0x0a, 0x5c, 0x79, 0xbc, 0x1e, 0xba, 0x7a, 0x31, 0x04, 0x5d, 0xdd, 0x1d, - 0x8f, 0xae, 0x3c, 0x61, 0x7d, 0xf0, 0x4a, 0x19, 0x09, 0xaf, 0xde, 0x9b, 0x10, 0x5e, 0x79, 0xd2, - 0xa3, 0xf0, 0xd5, 0x47, 0x61, 0x7c, 0x75, 0x6d, 0x38, 0xbe, 0xf2, 0xc4, 0x70, 0x80, 0xb5, 0x11, - 0x09, 0xb0, 0x6e, 0x8f, 0x03, 0x58, 0xfe, 0x3c, 0x08, 0x22, 0xac, 0xad, 0x68, 0x84, 0x75, 0x67, - 0x2c, 0xc2, 0xf2, 0xbb, 0x3f, 0x04, 0xb1, 0x36, 0x22, 0x21, 0xd6, 0xed, 0x71, 0x10, 0xcb, 0xaf, - 0x5c, 0x10, 0x63, 0xbd, 0x1a, 0x8a, 0xb1, 0xee, 0x4f, 0x82, 0xb1, 0x3c, 0xa1, 0x03, 0x20, 0xeb, - 0xf3, 0xe1, 0x20, 0xeb, 0x1b, 0xe7, 0xc8, 0xec, 0x18, 0x89, 0xb2, 0xbe, 0x35, 0x80, 0xb2, 0xc4, - 0x51, 0x28, 0xcb, 0x1f, 0xcf, 0x2e, 0xcc, 0x52, 0x46, 0x82, 0xa2, 0xf7, 0x26, 0x04, 0x45, 0xfe, - 0xe0, 0x8b, 0x40, 0x45, 0xd5, 0x08, 0x54, 0x74, 0x73, 0x34, 0x2a, 0xf2, 0xcd, 0xb9, 0x0f, 0x8b, - 0x6a, 0x51, 0xb0, 0xe8, 0xd6, 0x18, 0x58, 0xe4, 0x5b, 0xa1, 0x00, 0x2e, 0x7a, 0xdc, 0x87, 0x8b, - 0xae, 0x8f, 0x8d, 0x8f, 0x0a, 0x00, 0xa3, 0xd5, 0x41, 0x60, 0x74, 0x63, 0x24, 0x30, 0xf2, 0x24, - 0xf8, 0xc8, 0xe8, 0x71, 0x1f, 0x32, 0xba, 0x3e, 0x02, 0x19, 0xf9, 0x15, 0xe0, 0xd0, 0x48, 0x1b, - 0x0d, 0x8d, 0x96, 0x26, 0x85, 0x46, 0x9e, 0xe0, 0x48, 0x6c, 0xb4, 0x15, 0x8d, 0x8d, 0xee, 0x4c, - 0x18, 0xba, 0x30, 0x00, 0x8e, 0x6a, 0x51, 0xe0, 0xe8, 0xd6, 0x18, 0x70, 0x14, 0x5c, 0x43, 0x3c, - 0x74, 0x54, 0x8b, 0x42, 0x47, 0xb7, 0xc6, 0xa0, 0x23, 0x5f, 0x52, 0x00, 0x1e, 0x35, 0x87, 0xc1, - 0xa3, 0x7b, 0x13, 0xc0, 0x23, 0xdf, 0x79, 0xe9, 0xc3, 0x47, 0x9f, 0xf6, 0xe3, 0x23, 0x71, 0x14, - 0x3e, 0xf2, 0x67, 0xa4, 0x0b, 0x90, 0xb6, 0xa2, 0x01, 0xd2, 0x9d, 0xb1, 0x00, 0x29, 0x68, 0x24, - 0x03, 0x08, 0x69, 0x23, 0x12, 0x21, 0xdd, 0x1e, 0x87, 0x90, 0x7c, 0x23, 0x19, 0x84, 0x48, 0x9f, - 0xf6, 0x43, 0x24, 0x71, 0x14, 0x44, 0xf2, 0x1b, 0xe7, 0x62, 0xa4, 0x5a, 0x14, 0x46, 0xba, 0x35, - 0x06, 0x23, 0xf9, 0x9d, 0x17, 0x00, 0x49, 0xca, 0x48, 0x90, 0xf4, 0xde, 0x84, 0x20, 0xa9, 0xcf, - 0x70, 0x85, 0x51, 0x52, 0x2d, 0x0a, 0x25, 0xdd, 0x1a, 0x83, 0x92, 0x02, 0x95, 0xf5, 0x61, 0xd2, - 0x56, 0x34, 0x4c, 0xba, 0x33, 0x16, 0x26, 0xf5, 0xcd, 0x26, 0x17, 0x27, 0x6d, 0x44, 0xe2, 0xa4, - 0xdb, 0xe3, 0x70, 0x52, 0xdf, 0xc2, 0xc7, 0x9d, 0x83, 0x5f, 0x99, 0x1c, 0x28, 0x7d, 0x74, 0x7e, - 0xa0, 0xe4, 0x3d, 0x73, 0x0c, 0x52, 0xfa, 0xb4, 0x1f, 0x29, 0x89, 0xa3, 0x90, 0x92, 0x3f, 0xb2, - 0xce, 0x07, 0x95, 0xd6, 0x53, 0xe9, 0xb7, 0x8b, 0xef, 0x88, 0x7f, 0x6d, 0x16, 0x66, 0x6a, 0x5e, - 0x44, 0x91, 0xdf, 0x4a, 0xe1, 0x4d, 0x32, 0x50, 0xa1, 0x35, 0x32, 0xe3, 0xa9, 0xdd, 0x1c, 0x9f, - 0x54, 0x70, 0x30, 0xed, 0x1d, 0x67, 0x7d, 0x83, 0xa3, 0xe0, 0xe8, 0x03, 0xc8, 0xf5, 0x6c, 0x6c, - 0xc9, 0x5d, 0x4b, 0x37, 0x2d, 0xdd, 0x61, 0xe7, 0x65, 0x84, 0xd5, 0xe2, 0x97, 0xa7, 0x8b, 0x73, - 0x3b, 0x36, 0xb6, 0xb6, 0x39, 0x5d, 0x9a, 0xeb, 0x05, 0xae, 0xdc, 0x0f, 0x80, 0x4d, 0x4f, 0xfe, - 0x01, 0xb0, 0x17, 0x50, 0xb4, 0xb0, 0xa2, 0x85, 0x3c, 0x18, 0x96, 0xed, 0x29, 0x7a, 0xcc, 0xd1, - 0xf3, 0x6c, 0x6e, 0x49, 0x9a, 0xf5, 0xa9, 0x60, 0x85, 0x89, 0xe8, 0x01, 0x5c, 0xec, 0x28, 0xc7, - 0x34, 0xfa, 0x54, 0x76, 0x9d, 0x42, 0x1a, 0x51, 0xca, 0xbe, 0xad, 0x85, 0x3a, 0xca, 0x31, 0xfd, - 0x9a, 0x18, 0xbb, 0x45, 0x3f, 0x05, 0x72, 0x0b, 0xf2, 0x9a, 0x6e, 0x3b, 0xba, 0xa1, 0x3a, 0x3c, - 0xcb, 0x2f, 0xcb, 0x90, 0x9b, 0x73, 0xa9, 0x2c, 0x95, 0xef, 0x7d, 0x98, 0xe7, 0xc7, 0x12, 0x02, - 0x2f, 0x63, 0x79, 0xa6, 0x5c, 0x76, 0xc3, 0x7b, 0xff, 0x8a, 0x2a, 0x50, 0x68, 0x29, 0x0e, 0x3e, - 0x52, 0x4e, 0x64, 0xf7, 0xd0, 0x5b, 0x96, 0x26, 0xbf, 0x7c, 0xeb, 0xec, 0x74, 0x31, 0xf7, 0x94, - 0xdd, 0x1a, 0x38, 0xfb, 0x96, 0x6b, 0x05, 0x6e, 0x68, 0xe8, 0x0e, 0x14, 0x14, 0xfb, 0xc4, 0x50, - 0xa9, 0x7a, 0xb0, 0x61, 0xf7, 0x6c, 0x0a, 0x49, 0xd2, 0x52, 0x9e, 0x92, 0x2b, 0x2e, 0x15, 0x5d, - 0x87, 0x39, 0x1e, 0xb3, 0xcf, 0x3e, 0x4f, 0x54, 0xa0, 0x4d, 0xe5, 0xdf, 0xbd, 0x60, 0x5f, 0x28, - 0x7a, 0x0c, 0x65, 0x9e, 0xc5, 0xff, 0x48, 0xb1, 0x34, 0x99, 0x6a, 0xdd, 0x1f, 0x9f, 0x45, 0x2a, - 0xf6, 0x32, 0xcb, 0xda, 0x4f, 0x0a, 0x10, 0x55, 0xfb, 0x73, 0x68, 0x0b, 0xe6, 0xd5, 0xb6, 0xee, - 0x21, 0x08, 0xd6, 0xf2, 0xf9, 0xa1, 0xb3, 0xa9, 0x42, 0xcb, 0xfa, 0x2f, 0xa3, 0x0b, 0x6a, 0x98, - 0x80, 0x1a, 0x40, 0xf3, 0xea, 0xc8, 0x5d, 0xb3, 0xad, 0xab, 0x27, 0x14, 0x3c, 0x84, 0xb3, 0x91, - 0x8f, 0xfc, 0x26, 0xc0, 0x2b, 0x45, 0x77, 0xb6, 0x29, 0xa7, 0x04, 0x47, 0xde, 0x6f, 0x96, 0x49, - 0x78, 0x3d, 0x95, 0x9e, 0x2b, 0xe6, 0xd6, 0x53, 0xe9, 0x7c, 0xb1, 0x20, 0xfe, 0x75, 0x01, 0x0a, - 0x7d, 0x75, 0x41, 0x35, 0xb8, 0xa8, 0x79, 0x53, 0x45, 0xe6, 0x87, 0xba, 0x74, 0xd3, 0xe0, 0x09, - 0xd6, 0x17, 0xbe, 0x3c, 0x5d, 0x2c, 0xd0, 0xd2, 0x4f, 0xbd, 0x5b, 0xd2, 0x05, 0x9f, 0xc3, 0xa7, - 0xa2, 0x8f, 0x20, 0xcf, 0xdc, 0x4f, 0xef, 0x8b, 0x7b, 0x34, 0x1a, 0x7f, 0x75, 0xfe, 0xcb, 0xd3, - 0xc5, 0x1c, 0xf5, 0x39, 0xdd, 0x04, 0xc8, 0x52, 0xae, 0x1d, 0xbc, 0x14, 0x7f, 0x5d, 0x80, 0xb9, - 0xd0, 0xc1, 0xa9, 0xc7, 0x7d, 0x2f, 0xbf, 0xaf, 0x44, 0xe3, 0xd6, 0x61, 0xa1, 0x8b, 0x69, 0x3e, - 0xce, 0xdd, 0x38, 0xd0, 0xc5, 0xe1, 0xb8, 0x87, 0xee, 0xe2, 0xb8, 0x41, 0x31, 0x2e, 0xdb, 0xc7, - 0xa9, 0x1f, 0xfc, 0x70, 0x71, 0x4a, 0xfc, 0xfd, 0x14, 0xe4, 0xc2, 0xc7, 0xa4, 0xea, 0x7d, 0xf5, - 0x8a, 0x5a, 0x57, 0x42, 0x1c, 0x4b, 0x23, 0x12, 0x44, 0x66, 0xfc, 0x2f, 0x19, 0xb0, 0x6a, 0x5e, - 0x1b, 0xf1, 0x8a, 0x3f, 0x58, 0x4f, 0x9f, 0xb1, 0xfc, 0xbd, 0xa4, 0x67, 0x5f, 0x97, 0x60, 0x9a, - 0x26, 0x2c, 0xe2, 0x55, 0x2b, 0xf5, 0x8f, 0x1e, 0xe2, 0x6b, 0x93, 0xfb, 0x12, 0x2b, 0x46, 0xec, - 0x71, 0xf3, 0x8d, 0x32, 0x02, 0xfa, 0xd3, 0xe0, 0xfc, 0x1f, 0x38, 0xe4, 0x19, 0x21, 0xa7, 0xcf, - 0x97, 0x11, 0x12, 0xfd, 0x12, 0x14, 0x54, 0xb3, 0xdd, 0x66, 0x6b, 0x25, 0xb3, 0x48, 0x83, 0x39, - 0x5e, 0xa8, 0x08, 0xfe, 0x4d, 0xca, 0x25, 0xef, 0xdb, 0x94, 0x4b, 0x12, 0xff, 0x36, 0x65, 0x20, - 0xaa, 0x36, 0xef, 0x09, 0x63, 0x86, 0xac, 0x2f, 0xc0, 0x77, 0xf6, 0x4d, 0x02, 0x7c, 0x59, 0x50, - 0x38, 0x1f, 0x39, 0x7f, 0x22, 0xf0, 0x10, 0x9c, 0x67, 0xa6, 0x79, 0xd0, 0xf3, 0x02, 0x73, 0xcb, - 0xc1, 0xfc, 0x8c, 0xe9, 0x2f, 0x4f, 0x17, 0x53, 0x92, 0x97, 0xa0, 0x31, 0xca, 0xf2, 0x27, 0x7e, - 0x3a, 0xcb, 0x7f, 0x1d, 0xe6, 0xba, 0x16, 0xde, 0xc3, 0x8e, 0xba, 0x2f, 0x1b, 0xbd, 0x0e, 0x3f, - 0xbf, 0x93, 0x75, 0x69, 0x5b, 0xbd, 0x0e, 0xba, 0x07, 0x45, 0xaf, 0x08, 0xc7, 0xe8, 0x6e, 0x02, - 0x2f, 0x97, 0xce, 0x11, 0xbd, 0xf8, 0x3f, 0x05, 0x58, 0x08, 0xb5, 0x89, 0xcf, 0x89, 0x75, 0xc8, - 0xfa, 0xe6, 0xc0, 0x2e, 0x09, 0xe7, 0x0c, 0x53, 0x0d, 0x32, 0x23, 0x19, 0x2e, 0xb9, 0x8f, 0xa5, - 0xd9, 0xfb, 0x7d, 0xb1, 0x89, 0x73, 0x8a, 0xbd, 0xe8, 0xcb, 0x59, 0x0b, 0x3c, 0xc0, 0x9b, 0x24, - 0xc9, 0x89, 0x26, 0x89, 0xf8, 0x5b, 0x02, 0x14, 0xe9, 0x03, 0x9e, 0x60, 0xac, 0xc5, 0x62, 0x9d, - 0xdc, 0xf0, 0xef, 0xc4, 0xe4, 0x27, 0x74, 0x42, 0x5f, 0x1c, 0x49, 0x86, 0xbf, 0x38, 0x22, 0xfe, - 0x50, 0x80, 0xbc, 0x57, 0x43, 0xf6, 0x95, 0xbf, 0x11, 0x69, 0x40, 0xdf, 0xec, 0x4b, 0x76, 0x6e, - 0xbe, 0x93, 0x89, 0x3e, 0x3c, 0x18, 0xcc, 0x77, 0xc2, 0xbe, 0xc0, 0xf6, 0x37, 0xdd, 0x91, 0x43, - 0xaa, 0x58, 0xf1, 0xf3, 0x5a, 0xbc, 0xc1, 0x61, 0x25, 0x89, 0x7e, 0x20, 0xd5, 0x6c, 0x1f, 0xb2, - 0x14, 0x33, 0x13, 0x99, 0x2d, 0xc4, 0x03, 0xcf, 0x80, 0xef, 0xe6, 0x69, 0xcd, 0x06, 0xfd, 0x74, - 0x2a, 0xfb, 0x6d, 0x8b, 0x4f, 0x02, 0x0a, 0xa4, 0x9d, 0x4f, 0xb4, 0x34, 0x91, 0x29, 0x75, 0xb5, - 0xc4, 0xc6, 0xca, 0x1f, 0x07, 0x7b, 0xa2, 0x7a, 0x48, 0x50, 0xdc, 0x23, 0x48, 0x1e, 0x2a, 0xed, - 0x51, 0x11, 0x5c, 0xa1, 0x9e, 0x93, 0x48, 0x69, 0xf4, 0x24, 0x94, 0x0e, 0x24, 0x31, 0x1c, 0x71, - 0x0c, 0xaa, 0x34, 0x94, 0x36, 0xe4, 0xc3, 0xf0, 0x58, 0x1f, 0xf9, 0xf8, 0xe0, 0xa0, 0xff, 0x38, - 0xf5, 0xa3, 0x1f, 0x2e, 0x0a, 0xe2, 0x27, 0x80, 0x24, 0x6c, 0x63, 0xe7, 0x45, 0xcf, 0xb4, 0xfc, - 0xd4, 0x2a, 0xb7, 0xfb, 0x3e, 0xc1, 0x32, 0xbd, 0x9a, 0x3d, 0x8b, 0x4a, 0x9e, 0x74, 0x11, 0x16, - 0x42, 0xdc, 0xcc, 0x58, 0x88, 0x1f, 0xc2, 0x95, 0xa7, 0xa6, 0x6d, 0xeb, 0x5d, 0x02, 0x5d, 0xe9, - 0xac, 0x24, 0x4b, 0x83, 0x67, 0x1e, 0xd3, 0x5d, 0xba, 0x89, 0x61, 0x30, 0x33, 0x92, 0x91, 0xbc, - 0x6b, 0xf1, 0x0f, 0x04, 0xb8, 0x3c, 0xc8, 0xc9, 0xb4, 0x1c, 0x75, 0xb6, 0x72, 0x56, 0x35, 0xfd, - 0x4c, 0x7a, 0xe3, 0x47, 0xab, 0x5b, 0x9c, 0xb8, 0x98, 0xfc, 0x99, 0x72, 0x47, 0xa1, 0xe6, 0x83, - 0x9f, 0xf3, 0xce, 0x73, 0xf2, 0x26, 0xa3, 0xfa, 0x96, 0x24, 0x35, 0x99, 0x25, 0x69, 0x42, 0x61, - 0xdd, 0xd4, 0x0d, 0xe2, 0xc9, 0xba, 0xed, 0x5d, 0x81, 0xfc, 0xae, 0x6e, 0x28, 0xd6, 0x89, 0xec, - 0x06, 0x0e, 0x0a, 0xe3, 0x02, 0x07, 0xa5, 0x1c, 0xe3, 0xe0, 0x97, 0xe2, 0x8f, 0x05, 0x28, 0xfa, - 0x62, 0xb9, 0x45, 0x7e, 0x17, 0x40, 0x6d, 0xf7, 0x6c, 0x07, 0x5b, 0x6e, 0x2f, 0xcd, 0xb1, 0x38, - 0xf9, 0x0a, 0xa3, 0xd6, 0xd7, 0xa4, 0x0c, 0x2f, 0x50, 0xd7, 0xd0, 0x8d, 0x70, 0x1a, 0x8a, 0xe9, - 0x55, 0x38, 0x1b, 0x48, 0x3e, 0x41, 0xba, 0xdd, 0x76, 0x4c, 0xcb, 0xc3, 0x4c, 0xbc, 0xdb, 0xdd, - 0xa4, 0xf5, 0xf4, 0xf4, 0x34, 0x29, 0xb7, 0x02, 0x79, 0xb2, 0xdc, 0x1f, 0x62, 0xaf, 0x49, 0xa9, - 0xf1, 0x4d, 0x62, 0x1c, 0x6e, 0x93, 0x7e, 0x97, 0x38, 0xa9, 0xac, 0x37, 0xbc, 0x1e, 0x1e, 0x61, - 0xd1, 0x7e, 0x26, 0x98, 0xfc, 0x71, 0xb2, 0xf4, 0x98, 0xd4, 0x13, 0xf9, 0x16, 0xa4, 0xdd, 0x4f, - 0x4f, 0xf3, 0x09, 0x72, 0x65, 0x89, 0x7d, 0x9b, 0x7a, 0xc9, 0xfd, 0x36, 0xf5, 0xd2, 0x1a, 0x2f, - 0xc0, 0xcc, 0xf8, 0x0f, 0xfe, 0xf3, 0xa2, 0x20, 0x79, 0x4c, 0xf7, 0x1b, 0x64, 0x84, 0x0f, 0xac, - 0xc2, 0x28, 0x0f, 0x10, 0xf8, 0x5a, 0x0f, 0xff, 0x1a, 0xf1, 0xca, 0x9a, 0xbc, 0xb3, 0x55, 0x79, - 0xbe, 0xb9, 0x59, 0x6f, 0x36, 0xab, 0x6b, 0x45, 0x01, 0x15, 0x61, 0x2e, 0xf4, 0xad, 0x9f, 0x04, - 0xfb, 0x3e, 0xf1, 0xfd, 0x9f, 0x01, 0xf0, 0x3f, 0x1b, 0x46, 0x64, 0x6d, 0x54, 0x3f, 0x93, 0x5f, - 0xae, 0x3c, 0xdb, 0xa9, 0x36, 0x8a, 0x53, 0x08, 0x41, 0x7e, 0x75, 0xa5, 0x59, 0xa9, 0xc9, 0x52, - 0xb5, 0xb1, 0xfd, 0x7c, 0xab, 0x51, 0x75, 0xbf, 0x6b, 0x7c, 0x7f, 0x0d, 0xe6, 0x82, 0x89, 0x85, - 0xd0, 0x02, 0x14, 0x2a, 0xb5, 0x6a, 0x65, 0x43, 0x7e, 0x59, 0x5f, 0x91, 0x5f, 0xec, 0x54, 0x77, - 0xaa, 0xc5, 0x29, 0x5a, 0x35, 0x4a, 0x7c, 0xb2, 0xf3, 0xec, 0x59, 0x51, 0x40, 0x05, 0xc8, 0xb2, - 0x6b, 0xfa, 0x5d, 0xa0, 0x62, 0xe2, 0xfe, 0x26, 0x64, 0x03, 0x09, 0x84, 0xc9, 0xe3, 0xb6, 0x77, - 0x1a, 0x35, 0xb9, 0x59, 0xdf, 0xac, 0x36, 0x9a, 0x2b, 0x9b, 0xdb, 0x4c, 0x06, 0xa5, 0xad, 0xac, - 0x3e, 0x97, 0x9a, 0x45, 0xc1, 0xbb, 0x6e, 0x3e, 0xdf, 0xa9, 0xd4, 0xdc, 0x66, 0x88, 0xa9, 0x74, - 0xb2, 0x98, 0xbc, 0xff, 0x97, 0x04, 0xb8, 0x3c, 0x24, 0xc9, 0x0e, 0xca, 0xc2, 0xec, 0x8e, 0x41, - 0xb3, 0xa9, 0x16, 0xa7, 0x50, 0x2e, 0x90, 0x67, 0xa7, 0x28, 0xa0, 0x34, 0xcb, 0x74, 0x52, 0x4c, - 0xa0, 0x19, 0x48, 0x34, 0x1e, 0x15, 0x93, 0xa4, 0xa6, 0x81, 0x34, 0x35, 0xc5, 0x14, 0xca, 0xf0, - 0x04, 0x19, 0xc5, 0x69, 0x34, 0xe7, 0xe7, 0xa9, 0x28, 0xce, 0x10, 0x51, 0x5e, 0xbe, 0x87, 0xe2, - 0xec, 0xfd, 0xeb, 0x10, 0x38, 0x53, 0x8f, 0x00, 0x66, 0x9e, 0x29, 0x0e, 0xb6, 0x9d, 0xe2, 0x14, - 0x9a, 0x85, 0xe4, 0x4a, 0xbb, 0x5d, 0x14, 0x1e, 0xfe, 0x8b, 0x14, 0xa4, 0xdd, 0xef, 0xdf, 0xa0, - 0x67, 0x30, 0xcd, 0xb6, 0xae, 0x17, 0x87, 0x7b, 0xf6, 0x74, 0xf2, 0x96, 0xaf, 0x8d, 0x73, 0xfd, - 0xc5, 0x29, 0xf4, 0xff, 0x41, 0x36, 0xe0, 0x31, 0xa1, 0xa1, 0xdb, 0x6f, 0x21, 0x2f, 0xb1, 0x7c, - 0x7b, 0x5c, 0x31, 0x4f, 0xfe, 0x2b, 0xc8, 0x78, 0x16, 0x1c, 0xdd, 0x18, 0x65, 0xdf, 0x5d, 0xd9, - 0xa3, 0x17, 0x01, 0x32, 0xd7, 0xc4, 0xa9, 0xf7, 0x05, 0x64, 0x01, 0x1a, 0x34, 0xb6, 0x28, 0x2a, - 0x20, 0x62, 0xa8, 0x35, 0x2f, 0xdf, 0x9f, 0xa8, 0xb4, 0xff, 0x4c, 0xa2, 0x2c, 0x7f, 0xc5, 0x88, - 0x56, 0xd6, 0xc0, 0x7a, 0x14, 0xad, 0xac, 0x88, 0x85, 0x67, 0x0a, 0xbd, 0x80, 0x14, 0xb1, 0x94, - 0x28, 0xca, 0x87, 0xec, 0xb3, 0xcc, 0xe5, 0x1b, 0x23, 0xcb, 0xb8, 0x22, 0x57, 0xef, 0xfd, 0xe8, - 0xcf, 0xaf, 0x4e, 0xfd, 0xe8, 0xec, 0xaa, 0xf0, 0xe3, 0xb3, 0xab, 0xc2, 0x9f, 0x9e, 0x5d, 0x15, - 0xfe, 0xec, 0xec, 0xaa, 0xf0, 0xfd, 0x9f, 0x5c, 0x9d, 0xfa, 0xf1, 0x4f, 0xae, 0x4e, 0xfd, 0xe9, - 0x4f, 0xae, 0x4e, 0x7d, 0x3e, 0xcb, 0xb9, 0x77, 0x67, 0xa8, 0x51, 0x79, 0xf4, 0x7f, 0x03, 0x00, - 0x00, 0xff, 0xff, 0x33, 0xea, 0xe1, 0xb7, 0x2b, 0x80, 0x00, 0x00, +func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_08826d2d66ba29e3) } + +var fileDescriptor_api_08826d2d66ba29e3 = []byte{ + // 8230 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x7d, 0x5d, 0x6c, 0x23, 0xc9, + 0x76, 0x9e, 0x9a, 0xa4, 0x24, 0xf2, 0x50, 0xfc, 0x51, 0x69, 0x7e, 0xb8, 0xdc, 0xdd, 0xd1, 0x4c, + 0xcf, 0xff, 0xdc, 0x5d, 0x69, 0x67, 0xe6, 0xae, 0x77, 0xbd, 0xb3, 0xde, 0x6b, 0x89, 0xe2, 0x0c, + 0x29, 0x8d, 0x34, 0x9a, 0x26, 0x35, 0x93, 0x5d, 0x5f, 0xa7, 0x6f, 0xab, 0xbb, 0x44, 0xf5, 0x15, + 0xd9, 0xcd, 0xe9, 0x6e, 0xea, 0x67, 0x81, 0x00, 0x71, 0x6c, 0x38, 0x06, 0x02, 0x18, 0xf7, 0xc1, + 0x41, 0xae, 0xe1, 0x24, 0xbe, 0x8e, 0xe3, 0xf8, 0x21, 0x40, 0x12, 0x20, 0x41, 0x12, 0x04, 0x89, + 0xfd, 0x62, 0x20, 0x17, 0x81, 0x93, 0x5c, 0x3f, 0xc5, 0x08, 0x10, 0xc5, 0xd6, 0xcd, 0x4b, 0x90, + 0xc0, 0x08, 0x82, 0x00, 0x06, 0xf6, 0x21, 0x08, 0xea, 0xa7, 0xff, 0xc8, 0xe6, 0x8f, 0x66, 0x7b, + 0xe3, 0x05, 0xee, 0x8b, 0xc4, 0x3e, 0x55, 0xe7, 0x74, 0xd5, 0xa9, 0xaa, 0x53, 0xe7, 0xab, 0x3a, + 0x55, 0x0d, 0xf3, 0x96, 0xa9, 0xa8, 0xfb, 0xdd, 0xdd, 0x65, 0xa5, 0xab, 0x2f, 0x75, 0x2d, 0xd3, + 0x31, 0xd1, 0xbc, 0x6a, 0xaa, 0x07, 0x94, 0xbc, 0xc4, 0x13, 0xcb, 0xf7, 0x0e, 0x0e, 0x97, 0x0f, + 0x0e, 0x6d, 0x6c, 0x1d, 0x62, 0x6b, 0x59, 0x35, 0x0d, 0xb5, 0x67, 0x59, 0xd8, 0x50, 0x4f, 0x96, + 0xdb, 0xa6, 0x7a, 0x40, 0xff, 0xe8, 0x46, 0x8b, 0xb1, 0x97, 0x91, 0x2b, 0x51, 0x53, 0x1c, 0x85, + 0xd3, 0x2e, 0xb8, 0x34, 0x6c, 0x59, 0xa6, 0x65, 0x73, 0xea, 0x25, 0x97, 0xda, 0xc1, 0x8e, 0x12, + 0xc8, 0xfd, 0xa6, 0xed, 0x98, 0x96, 0xd2, 0xc2, 0xcb, 0xd8, 0x68, 0xe9, 0x06, 0x26, 0x19, 0x0e, + 0x55, 0x95, 0x27, 0xbe, 0x15, 0x99, 0xf8, 0x90, 0xa7, 0x96, 0x7a, 0x8e, 0xde, 0x5e, 0xde, 0x6f, + 0xab, 0xcb, 0x8e, 0xde, 0xc1, 0xb6, 0xa3, 0x74, 0xba, 0x3c, 0xe5, 0x1e, 0x4d, 0x71, 0x2c, 0x45, + 0xd5, 0x8d, 0x96, 0xfb, 0xbf, 0xbb, 0xbb, 0x6c, 0x61, 0xd5, 0xb4, 0x34, 0xac, 0xc9, 0x76, 0x57, + 0x31, 0xdc, 0xe2, 0xb6, 0xcc, 0x96, 0x49, 0x7f, 0x2e, 0x93, 0x5f, 0x9c, 0x7a, 0xa5, 0x65, 0x9a, + 0xad, 0x36, 0x5e, 0xa6, 0x4f, 0xbb, 0xbd, 0xbd, 0x65, 0xad, 0x67, 0x29, 0x8e, 0x6e, 0x72, 0x2e, + 0xf1, 0x9f, 0x0b, 0x90, 0x93, 0xf0, 0xab, 0x1e, 0xb6, 0x9d, 0x1a, 0x56, 0x34, 0x6c, 0xa1, 0x37, + 0x20, 0x79, 0x80, 0x4f, 0x4a, 0xc9, 0xab, 0xc2, 0x9d, 0xb9, 0xd5, 0xd9, 0x2f, 0x4e, 0x17, 0x93, + 0x1b, 0xf8, 0x44, 0x22, 0x34, 0x74, 0x15, 0x66, 0xb1, 0xa1, 0xc9, 0x24, 0x39, 0x15, 0x4e, 0x9e, + 0xc1, 0x86, 0xb6, 0x81, 0x4f, 0xd0, 0xb7, 0x21, 0x6d, 0x13, 0x69, 0x86, 0x8a, 0x4b, 0xd3, 0x57, + 0x85, 0x3b, 0xd3, 0xab, 0x3f, 0xfb, 0xc5, 0xe9, 0xe2, 0xc7, 0x2d, 0xdd, 0xd9, 0xef, 0xed, 0x2e, + 0xa9, 0x66, 0x67, 0xd9, 0x6b, 0x27, 0x6d, 0xd7, 0xff, 0xbd, 0xdc, 0x3d, 0x68, 0x2d, 0xf7, 0xeb, + 0x68, 0xa9, 0x79, 0x6c, 0x34, 0xf0, 0x2b, 0xc9, 0x93, 0xb8, 0x9e, 0x4a, 0x0b, 0xc5, 0xc4, 0x7a, + 0x2a, 0x9d, 0x28, 0x26, 0xc5, 0xdf, 0x4e, 0x42, 0x5e, 0xc2, 0x76, 0xd7, 0x34, 0x6c, 0xcc, 0x4b, + 0xfe, 0x1e, 0x24, 0x9d, 0x63, 0x83, 0x96, 0x3c, 0xfb, 0xe0, 0xca, 0xd2, 0x40, 0x8f, 0x58, 0x6a, + 0x5a, 0x8a, 0x61, 0x2b, 0x2a, 0xa9, 0xbe, 0x44, 0xb2, 0xa2, 0x0f, 0x21, 0x6b, 0x61, 0xbb, 0xd7, + 0xc1, 0x54, 0x91, 0xb4, 0x52, 0xd9, 0x07, 0x97, 0x23, 0x38, 0x1b, 0x5d, 0xc5, 0x90, 0x80, 0xe5, + 0x25, 0xbf, 0xd1, 0x1b, 0x90, 0x36, 0x7a, 0x1d, 0xa2, 0x0a, 0x9b, 0x56, 0x34, 0x29, 0xcd, 0x1a, + 0xbd, 0xce, 0x06, 0x3e, 0xb1, 0xd1, 0x5f, 0x82, 0x4b, 0x1a, 0xee, 0x5a, 0x58, 0x55, 0x1c, 0xac, + 0xc9, 0x96, 0x62, 0xb4, 0xb0, 0xac, 0x1b, 0x7b, 0xa6, 0x5d, 0x9a, 0xb9, 0x9a, 0xbc, 0x93, 0x7d, + 0xf0, 0x56, 0x84, 0x7c, 0x89, 0xe4, 0xaa, 0x1b, 0x7b, 0xe6, 0x6a, 0xea, 0x87, 0xa7, 0x8b, 0x53, + 0xd2, 0x05, 0x5f, 0x82, 0x97, 0x64, 0xa3, 0x06, 0xe4, 0x78, 0x71, 0x2d, 0xac, 0xd8, 0xa6, 0x51, + 0x9a, 0xbd, 0x2a, 0xdc, 0xc9, 0x3f, 0x58, 0x8a, 0x12, 0x18, 0x52, 0x0d, 0x79, 0xec, 0x75, 0xb0, + 0x44, 0xb9, 0xa4, 0x39, 0x2b, 0xf0, 0x84, 0xde, 0x84, 0x0c, 0xa9, 0xc9, 0xee, 0x89, 0x83, 0xed, + 0x52, 0x9a, 0x56, 0x85, 0x54, 0x6d, 0x95, 0x3c, 0x8b, 0x9f, 0xc0, 0x5c, 0x90, 0x15, 0x21, 0xc8, + 0x4b, 0xd5, 0xc6, 0xce, 0x66, 0x55, 0xde, 0xd9, 0xda, 0xd8, 0x7a, 0xf6, 0x72, 0xab, 0x38, 0x85, + 0x2e, 0x40, 0x91, 0xd3, 0x36, 0xaa, 0x9f, 0xca, 0x4f, 0xeb, 0x9b, 0xf5, 0x66, 0x51, 0x28, 0xa7, + 0x7e, 0xe5, 0xb7, 0xaf, 0x4c, 0x89, 0xdb, 0x00, 0x4f, 0xb0, 0xc3, 0x3b, 0x18, 0x5a, 0x85, 0x99, + 0x7d, 0x5a, 0x9e, 0x92, 0x40, 0x35, 0x7d, 0x35, 0xb2, 0xe0, 0x81, 0xce, 0xb8, 0x9a, 0x26, 0xda, + 0xf8, 0xd1, 0xe9, 0xa2, 0x20, 0x71, 0x4e, 0xf1, 0xf7, 0x05, 0xc8, 0x52, 0x91, 0xac, 0x7e, 0xa8, + 0xd2, 0x27, 0xf3, 0xda, 0x58, 0x65, 0x0c, 0x0a, 0x45, 0x4b, 0x30, 0x7d, 0xa8, 0xb4, 0x7b, 0xb8, + 0x94, 0xa0, 0x32, 0x4a, 0x11, 0x32, 0x5e, 0x90, 0x74, 0x89, 0x65, 0x43, 0x8f, 0x60, 0x4e, 0x37, + 0x1c, 0x6c, 0x38, 0x32, 0x63, 0x4b, 0x8e, 0x61, 0xcb, 0xb2, 0xdc, 0xf4, 0x41, 0xfc, 0x67, 0x02, + 0xc0, 0x76, 0x2f, 0x4e, 0xa5, 0xa0, 0x6f, 0x4e, 0x58, 0x7e, 0xde, 0xbb, 0x78, 0x2d, 0x2e, 0xc1, + 0x8c, 0x6e, 0xb4, 0x75, 0x83, 0x95, 0x3f, 0x2d, 0xf1, 0x27, 0x74, 0x01, 0xa6, 0x77, 0xdb, 0xba, + 0xa1, 0xd1, 0xf1, 0x90, 0x96, 0xd8, 0x83, 0x28, 0x41, 0x96, 0x96, 0x3a, 0x46, 0xbd, 0x8b, 0xa7, + 0x09, 0xb8, 0x58, 0x31, 0x0d, 0x4d, 0x27, 0x43, 0x52, 0x69, 0x7f, 0x2d, 0xb4, 0xb2, 0x0e, 0x81, + 0xc1, 0x27, 0xe3, 0xe3, 0xee, 0x84, 0x6d, 0x8c, 0x7c, 0xae, 0xea, 0x71, 0x97, 0xd2, 0xa2, 0x35, + 0x89, 0xbe, 0x09, 0x97, 0x95, 0x76, 0xdb, 0x3c, 0x92, 0xf5, 0x3d, 0x59, 0x33, 0xb1, 0x2d, 0x1b, + 0xa6, 0x23, 0xe3, 0x63, 0xdd, 0x76, 0xa8, 0x29, 0x49, 0x4b, 0x0b, 0x34, 0xb9, 0xbe, 0xb7, 0x66, + 0x62, 0x7b, 0xcb, 0x74, 0xaa, 0x24, 0x89, 0x8c, 0x53, 0x52, 0x18, 0x36, 0x4e, 0x67, 0x88, 0xf9, + 0x95, 0xd2, 0xf8, 0xb8, 0x4b, 0xc7, 0x69, 0xa0, 0x29, 0x67, 0x83, 0x4d, 0x29, 0xfe, 0x3c, 0x5c, + 0xea, 0xd7, 0x6f, 0x9c, 0xed, 0xf7, 0x87, 0x02, 0xe4, 0xeb, 0x86, 0xee, 0x7c, 0x2d, 0x1a, 0xce, + 0x53, 0x76, 0x32, 0xa8, 0xec, 0x7b, 0x50, 0xdc, 0x53, 0xf4, 0xf6, 0x33, 0xa3, 0x69, 0x76, 0x76, + 0x6d, 0xc7, 0x34, 0xb0, 0xcd, 0x5b, 0x63, 0x80, 0x2e, 0xbe, 0x80, 0x82, 0x57, 0x9b, 0x38, 0xd5, + 0xe4, 0x40, 0xb1, 0x6e, 0xa8, 0x16, 0xee, 0x60, 0x23, 0x56, 0x3d, 0xbd, 0x05, 0x19, 0xdd, 0x95, + 0x4b, 0x75, 0x95, 0x94, 0x7c, 0x82, 0xd8, 0x83, 0xf9, 0xc0, 0x5b, 0xe3, 0x34, 0x97, 0x64, 0xca, + 0xc0, 0x47, 0xb2, 0xdf, 0x46, 0x64, 0xca, 0xc0, 0x47, 0xcc, 0xbc, 0x35, 0x20, 0xb7, 0x86, 0xdb, + 0xd8, 0xc1, 0x71, 0x5a, 0xfd, 0x1d, 0xc8, 0xbb, 0x42, 0xe3, 0x6c, 0x98, 0xbf, 0x25, 0x00, 0xe2, + 0x72, 0xc9, 0x2c, 0x1b, 0x67, 0xdb, 0x2c, 0x12, 0xd7, 0xc2, 0xe9, 0x59, 0x06, 0xf3, 0x11, 0x58, + 0x9f, 0x04, 0x46, 0xa2, 0x6e, 0x82, 0x3f, 0x64, 0x53, 0xc1, 0x21, 0xcb, 0xdd, 0x9b, 0x23, 0x58, + 0x08, 0x15, 0x2c, 0xde, 0xe6, 0x4b, 0xd1, 0x32, 0x25, 0xae, 0x26, 0x83, 0x3e, 0x1c, 0x25, 0x8a, + 0xdf, 0x17, 0x60, 0xbe, 0xd2, 0xc6, 0x8a, 0x15, 0xbb, 0x46, 0xbe, 0x05, 0x69, 0x0d, 0x2b, 0x1a, + 0xad, 0x32, 0x1b, 0xd8, 0x6f, 0x07, 0xa4, 0x10, 0x4f, 0x77, 0x69, 0xbf, 0xad, 0x2e, 0x35, 0x5d, + 0x1f, 0x98, 0x8f, 0x6e, 0x8f, 0x49, 0xfc, 0x14, 0x50, 0xb0, 0x64, 0x71, 0x76, 0x84, 0xff, 0x23, + 0x00, 0x92, 0xf0, 0x21, 0xb6, 0x9c, 0xd8, 0xab, 0xbd, 0x06, 0x59, 0x47, 0xb1, 0x5a, 0xd8, 0x91, + 0x89, 0x77, 0x7f, 0x9e, 0x9a, 0x03, 0xe3, 0x23, 0x64, 0xd4, 0x84, 0xdb, 0xd8, 0x50, 0x76, 0xdb, + 0x98, 0x4a, 0x91, 0x77, 0xcd, 0x9e, 0xa1, 0xc9, 0xba, 0x83, 0x2d, 0xc5, 0x31, 0x2d, 0xd9, 0xec, + 0x3a, 0x7a, 0x47, 0xff, 0x9c, 0x3a, 0xf6, 0xbc, 0xab, 0x5d, 0x67, 0xd9, 0x09, 0xf3, 0x2a, 0xc9, + 0x5c, 0xe7, 0x79, 0x9f, 0x05, 0xb2, 0x8a, 0x9f, 0xc1, 0x42, 0xa8, 0xd6, 0x71, 0xaa, 0xf4, 0x7f, + 0x09, 0x90, 0x6d, 0xa8, 0x8a, 0x11, 0xa7, 0x2e, 0x3f, 0x81, 0xac, 0xad, 0x2a, 0x86, 0xbc, 0x67, + 0x5a, 0x1d, 0xc5, 0xa1, 0x03, 0x27, 0x1f, 0xd2, 0xa5, 0xe7, 0xaf, 0xab, 0x8a, 0xf1, 0x98, 0x66, + 0x92, 0xc0, 0xf6, 0x7e, 0xa3, 0xe7, 0x90, 0x3d, 0xc0, 0x27, 0x32, 0xc7, 0x7e, 0x74, 0xb6, 0xcd, + 0x3f, 0x78, 0x2f, 0xc0, 0x7f, 0x70, 0xb8, 0xe4, 0x42, 0xc6, 0xa5, 0x00, 0x64, 0x5c, 0x22, 0x1c, + 0x4b, 0x0d, 0xc7, 0xc2, 0x46, 0xcb, 0xd9, 0x97, 0xe0, 0x00, 0x9f, 0x3c, 0x65, 0x32, 0xd8, 0x70, + 0x5d, 0x4f, 0xa5, 0x93, 0xc5, 0x94, 0xf8, 0xe7, 0x02, 0xcc, 0xb1, 0x2a, 0xc7, 0x39, 0x5c, 0xdf, + 0x87, 0x94, 0x65, 0x1e, 0xb1, 0xe1, 0x9a, 0x7d, 0xf0, 0x66, 0x84, 0x88, 0x0d, 0x7c, 0x12, 0x9c, + 0x0f, 0x69, 0x76, 0xb4, 0x0a, 0xdc, 0xeb, 0x94, 0x29, 0x77, 0x72, 0x52, 0x6e, 0x60, 0x5c, 0x12, + 0x91, 0x71, 0x1b, 0x0a, 0xbb, 0x8a, 0xa3, 0xee, 0xcb, 0x16, 0x2f, 0x24, 0x99, 0x3b, 0x93, 0x77, + 0xe6, 0xa4, 0x3c, 0x25, 0xbb, 0x45, 0xb7, 0x49, 0xcd, 0xd9, 0xf8, 0xb1, 0xf1, 0x4f, 0x58, 0x9b, + 0xff, 0x5f, 0x81, 0x8f, 0x21, 0xb7, 0xe6, 0x3f, 0x69, 0x4d, 0xff, 0xeb, 0x09, 0xb8, 0x5c, 0xd9, + 0xc7, 0xea, 0x41, 0xc5, 0x34, 0x6c, 0xdd, 0x76, 0x88, 0xee, 0xe2, 0x6c, 0xff, 0x37, 0x21, 0x73, + 0xa4, 0x3b, 0xfb, 0xb2, 0xa6, 0xef, 0xed, 0x51, 0xeb, 0x99, 0x96, 0xd2, 0x84, 0xb0, 0xa6, 0xef, + 0xed, 0xa1, 0x87, 0x90, 0xea, 0x98, 0x1a, 0x73, 0xce, 0xf3, 0x0f, 0x16, 0x23, 0xc4, 0xd3, 0xa2, + 0xd9, 0xbd, 0xce, 0xa6, 0xa9, 0x61, 0x89, 0x66, 0x46, 0x57, 0x00, 0x54, 0x42, 0xed, 0x9a, 0xba, + 0xe1, 0xf0, 0xd9, 0x37, 0x40, 0x41, 0x35, 0xc8, 0x38, 0xd8, 0xea, 0xe8, 0x86, 0xe2, 0xe0, 0xd2, + 0x34, 0x55, 0xde, 0x8d, 0xc8, 0x82, 0x77, 0xdb, 0xba, 0xaa, 0xac, 0x61, 0x5b, 0xb5, 0xf4, 0xae, + 0x63, 0x5a, 0x5c, 0x8b, 0x3e, 0xb3, 0xf8, 0xab, 0x29, 0x28, 0x0d, 0xea, 0x26, 0xce, 0x1e, 0xb2, + 0x0d, 0x33, 0x04, 0xcd, 0xb7, 0x1d, 0xde, 0x47, 0x1e, 0x0c, 0x53, 0x41, 0x44, 0x09, 0xe8, 0xaa, + 0x40, 0xdb, 0xe1, 0xc5, 0xe6, 0x72, 0xca, 0xff, 0x5a, 0x80, 0x19, 0x96, 0x80, 0xee, 0x43, 0x9a, + 0x2f, 0x5f, 0x68, 0xb4, 0x8c, 0xc9, 0xd5, 0x4b, 0x67, 0xa7, 0x8b, 0xb3, 0x6c, 0x45, 0x62, 0xed, + 0x0b, 0xff, 0xa7, 0x34, 0x4b, 0xf3, 0xd5, 0x35, 0xd2, 0x5a, 0xb6, 0xa3, 0x58, 0x0e, 0x5d, 0x24, + 0x4a, 0x30, 0x94, 0x42, 0x09, 0x1b, 0xf8, 0x04, 0xad, 0xc3, 0x8c, 0xed, 0x28, 0x4e, 0xcf, 0xe6, + 0xed, 0x75, 0xae, 0xc2, 0x36, 0x28, 0xa7, 0xc4, 0x25, 0x10, 0xf7, 0x49, 0xc3, 0x8e, 0xa2, 0xb7, + 0x69, 0x03, 0x66, 0x24, 0xfe, 0x24, 0xfe, 0x86, 0x00, 0x33, 0x2c, 0x2b, 0xba, 0x0c, 0x0b, 0xd2, + 0xca, 0xd6, 0x93, 0xaa, 0x5c, 0xdf, 0x5a, 0xab, 0x36, 0xab, 0xd2, 0x66, 0x7d, 0x6b, 0xa5, 0x59, + 0x2d, 0x4e, 0xa1, 0x4b, 0x80, 0xdc, 0x84, 0xca, 0xb3, 0xad, 0x46, 0xbd, 0xd1, 0xac, 0x6e, 0x35, + 0x8b, 0x02, 0x5d, 0xc9, 0xa0, 0xf4, 0x00, 0x35, 0x81, 0x6e, 0xc0, 0xd5, 0x7e, 0xaa, 0xdc, 0x68, + 0xae, 0x34, 0x1b, 0x72, 0xb5, 0xd1, 0xac, 0x6f, 0xae, 0x34, 0xab, 0x6b, 0xc5, 0xe4, 0x88, 0x5c, + 0xe4, 0x25, 0x92, 0x54, 0xad, 0x34, 0x8b, 0x29, 0xd1, 0x81, 0x8b, 0x12, 0x56, 0xcd, 0x4e, 0xb7, + 0xe7, 0x60, 0x52, 0x4a, 0x3b, 0xce, 0x91, 0x72, 0x19, 0x66, 0x35, 0xeb, 0x44, 0xb6, 0x7a, 0x06, + 0x1f, 0x27, 0x33, 0x9a, 0x75, 0x22, 0xf5, 0x0c, 0xf1, 0x9f, 0x08, 0x70, 0xa9, 0xff, 0xb5, 0x71, + 0x76, 0xc2, 0xe7, 0x90, 0x55, 0x34, 0x0d, 0x6b, 0xb2, 0x86, 0xdb, 0x8e, 0xc2, 0x5d, 0x9c, 0x7b, + 0x01, 0x49, 0x7c, 0x69, 0x6f, 0xc9, 0x5b, 0xda, 0xdb, 0x7c, 0x51, 0xa9, 0xd0, 0x82, 0xac, 0x11, + 0x0e, 0xd7, 0xfc, 0x50, 0x21, 0x94, 0x22, 0xfe, 0xcf, 0x14, 0xe4, 0xaa, 0x86, 0xd6, 0x3c, 0x8e, + 0x75, 0x2e, 0xb9, 0x04, 0x33, 0xaa, 0xd9, 0xe9, 0xe8, 0x8e, 0xab, 0x20, 0xf6, 0x84, 0x7e, 0x3a, + 0xe0, 0x9a, 0x26, 0x27, 0x70, 0xd0, 0x7c, 0xa7, 0x14, 0x7d, 0x07, 0x2e, 0x13, 0xab, 0x69, 0x19, + 0x4a, 0x5b, 0x66, 0xd2, 0x64, 0xc7, 0xd2, 0x5b, 0x2d, 0x6c, 0xf1, 0xe5, 0xc4, 0x3b, 0x11, 0xe5, + 0xac, 0x73, 0x8e, 0x0a, 0x65, 0x68, 0xb2, 0xfc, 0xd2, 0x45, 0x3d, 0x8a, 0x8c, 0x3e, 0x06, 0x20, + 0x53, 0x11, 0x5d, 0xa2, 0xb4, 0xb9, 0x3d, 0x1a, 0xb6, 0x46, 0xe9, 0x9a, 0x20, 0xc2, 0x40, 0x9e, + 0x6d, 0xb4, 0x4c, 0x70, 0xc8, 0xab, 0x9e, 0x6e, 0x61, 0xf9, 0x7e, 0x57, 0xa5, 0x0b, 0x07, 0xe9, + 0xd5, 0xfc, 0xd9, 0xe9, 0x22, 0x48, 0x8c, 0x7c, 0x7f, 0xbb, 0x42, 0x70, 0x09, 0xfb, 0xdd, 0x55, + 0xd1, 0x4b, 0xb8, 0x1b, 0x58, 0xff, 0x20, 0x33, 0x2f, 0xaf, 0x96, 0xe2, 0xc8, 0xfb, 0x7a, 0x6b, + 0x1f, 0x5b, 0xb2, 0xb7, 0x4c, 0x4d, 0xd7, 0x0b, 0xd3, 0xd2, 0x0d, 0x9f, 0xa1, 0xa2, 0x18, 0xac, + 0xf4, 0x2b, 0x4e, 0x8d, 0x66, 0xf6, 0x74, 0x46, 0x94, 0xdf, 0x35, 0x75, 0xdb, 0x34, 0x4a, 0x19, + 0xa6, 0x7c, 0xf6, 0x84, 0xee, 0x42, 0xd1, 0x39, 0x36, 0xe4, 0x7d, 0xac, 0x58, 0xce, 0x2e, 0x56, + 0x1c, 0x32, 0x4b, 0x03, 0xcd, 0x51, 0x70, 0x8e, 0x8d, 0x5a, 0x80, 0x8c, 0x9e, 0x43, 0x51, 0x37, + 0xe4, 0xbd, 0xb6, 0xde, 0xda, 0x77, 0xe4, 0x23, 0x4b, 0x77, 0xb0, 0x5d, 0x9a, 0xa7, 0x0a, 0x89, + 0xea, 0xb7, 0x0d, 0xbe, 0x6e, 0xac, 0xbd, 0x24, 0x39, 0xb9, 0x6a, 0xf2, 0xba, 0xf1, 0x98, 0xf2, + 0x53, 0xa2, 0xbd, 0x9e, 0x4a, 0xcf, 0x16, 0xd3, 0xe2, 0x7f, 0x11, 0x20, 0xef, 0x76, 0xb7, 0x38, + 0x47, 0xc6, 0x1d, 0x28, 0x9a, 0x06, 0x96, 0xbb, 0xfb, 0x8a, 0x8d, 0xb9, 0x1e, 0xf9, 0x84, 0x93, + 0x37, 0x0d, 0xbc, 0x4d, 0xc8, 0x4c, 0x5d, 0x68, 0x1b, 0xe6, 0x6d, 0x47, 0x69, 0xe9, 0x46, 0x2b, + 0xa0, 0xde, 0xe9, 0xc9, 0xc1, 0x42, 0x91, 0x73, 0x7b, 0xf4, 0x90, 0x97, 0xf2, 0x47, 0x02, 0xcc, + 0xaf, 0x68, 0x1d, 0xdd, 0x68, 0x74, 0xdb, 0x7a, 0xac, 0x6b, 0x10, 0x37, 0x20, 0x63, 0x13, 0x99, + 0xbe, 0xc1, 0xf7, 0x11, 0x65, 0x9a, 0xa6, 0x10, 0xcb, 0xff, 0x14, 0x0a, 0xf8, 0xb8, 0xab, 0xb3, + 0xad, 0x07, 0x06, 0x84, 0x52, 0x93, 0xd7, 0x2d, 0xef, 0xf3, 0x92, 0x24, 0x5e, 0xa7, 0x4f, 0x01, + 0x05, 0xab, 0x14, 0x27, 0x76, 0xf9, 0x14, 0x16, 0xa8, 0xe8, 0x1d, 0xc3, 0x8e, 0x59, 0x5f, 0xe2, + 0xcf, 0xc1, 0x85, 0xb0, 0xe8, 0x38, 0xcb, 0xfd, 0x92, 0xb7, 0xf2, 0x26, 0xb6, 0x62, 0x05, 0xb1, + 0x9e, 0xae, 0xb9, 0xe0, 0x38, 0xcb, 0xfc, 0x4b, 0x02, 0xbc, 0x41, 0x65, 0xd3, 0xdd, 0x99, 0x3d, + 0x6c, 0x3d, 0xc5, 0x8a, 0x1d, 0x2b, 0x02, 0xbf, 0x0e, 0x33, 0x0c, 0x49, 0xd3, 0xfe, 0x39, 0xbd, + 0x9a, 0x25, 0x9e, 0x4b, 0xc3, 0x31, 0x2d, 0xe2, 0xb9, 0xf0, 0x24, 0x51, 0x81, 0x72, 0x54, 0x29, + 0xe2, 0xac, 0xe9, 0xdf, 0x15, 0x60, 0x9e, 0x3b, 0x8d, 0xa4, 0x2b, 0x57, 0xf6, 0x89, 0xcf, 0x84, + 0xaa, 0x90, 0x55, 0xe9, 0x2f, 0xd9, 0x39, 0xe9, 0x62, 0x2a, 0x3f, 0x3f, 0xca, 0xdf, 0x64, 0x6c, + 0xcd, 0x93, 0x2e, 0x26, 0x4e, 0xab, 0xfb, 0x9b, 0x28, 0x2a, 0x50, 0xc9, 0x91, 0x1e, 0x2b, 0x1d, + 0x47, 0x34, 0xaf, 0xeb, 0xfa, 0x71, 0x1d, 0xfc, 0xd3, 0x24, 0x57, 0x02, 0x7b, 0x07, 0xcf, 0x1e, + 0xab, 0x8f, 0xf2, 0x59, 0x68, 0x73, 0x2c, 0x58, 0xf1, 0xc4, 0x39, 0x2a, 0x1e, 0x58, 0xa1, 0xf7, + 0xa9, 0xe8, 0x53, 0x08, 0xac, 0xc1, 0xcb, 0xac, 0x4e, 0x2e, 0xfa, 0x39, 0x8f, 0x3a, 0xe6, 0x7d, + 0x29, 0x8c, 0x6e, 0xa3, 0x0a, 0xa4, 0xf1, 0x71, 0x57, 0xd6, 0xb0, 0xad, 0x72, 0xc3, 0x25, 0x0e, + 0xdb, 0xc5, 0x1b, 0xc0, 0x03, 0xb3, 0xf8, 0xb8, 0x4b, 0x88, 0x68, 0x87, 0xcc, 0x5e, 0xae, 0xab, + 0x40, 0x8b, 0x6d, 0x8f, 0x87, 0x17, 0x7e, 0x4f, 0xe1, 0xe2, 0x0a, 0x9e, 0x97, 0xc0, 0x44, 0x88, + 0x3f, 0x10, 0xe0, 0xcd, 0xc8, 0x56, 0x8b, 0x73, 0x22, 0xfb, 0x18, 0x52, 0xb4, 0xf2, 0x89, 0x73, + 0x56, 0x9e, 0x72, 0x89, 0xbf, 0xe3, 0x8e, 0x71, 0x09, 0xb7, 0x4d, 0xa2, 0xd8, 0xaf, 0x60, 0x95, + 0x6d, 0xd6, 0x6d, 0xf0, 0xc4, 0xb9, 0x1b, 0xdc, 0x65, 0xf5, 0x8c, 0x40, 0x5f, 0x31, 0xe3, 0x34, + 0x02, 0xdf, 0x13, 0x60, 0xc1, 0xf3, 0x69, 0x62, 0x76, 0x6f, 0xdf, 0x87, 0xa4, 0x61, 0x1e, 0x9d, + 0x67, 0x89, 0x91, 0xe4, 0x27, 0x53, 0x52, 0xb8, 0x44, 0x71, 0xd6, 0xf7, 0xdf, 0x24, 0x20, 0xf3, + 0xa4, 0x12, 0x67, 0x2d, 0x3f, 0xe6, 0xcb, 0xd7, 0x6c, 0x60, 0x47, 0x75, 0x45, 0xef, 0x7d, 0x4b, + 0x4f, 0x2a, 0x1b, 0xf8, 0xc4, 0xed, 0x8a, 0x84, 0x0b, 0xad, 0x40, 0xc6, 0xd9, 0xb7, 0xb0, 0xbd, + 0x6f, 0xb6, 0xb5, 0xf3, 0xf8, 0x20, 0x3e, 0x57, 0x19, 0xc3, 0x34, 0x95, 0xeb, 0x86, 0x4a, 0x08, + 0x11, 0xa1, 0x12, 0xe4, 0x35, 0x9e, 0x1b, 0x97, 0x38, 0xcf, 0x6b, 0x02, 0xfe, 0xdb, 0x74, 0x71, + 0x46, 0x7c, 0x0e, 0x40, 0xaa, 0x13, 0x67, 0x93, 0xfc, 0x72, 0x12, 0xf2, 0xdb, 0x3d, 0x7b, 0x3f, + 0xe6, 0xde, 0x57, 0x01, 0xe8, 0xf6, 0x6c, 0x8a, 0x0f, 0x8e, 0x0d, 0x5e, 0xe7, 0x31, 0x51, 0x18, + 0x6e, 0xa5, 0x19, 0x5f, 0xf3, 0xd8, 0x40, 0x35, 0x2e, 0x04, 0xcb, 0x7e, 0x28, 0xc7, 0xf5, 0x51, + 0x48, 0xb2, 0x79, 0x6c, 0x6c, 0x62, 0x0f, 0x42, 0x32, 0x49, 0x98, 0x48, 0xfa, 0x18, 0x66, 0xc9, + 0x83, 0xec, 0x98, 0xe7, 0x69, 0xe6, 0x19, 0xc2, 0xd3, 0x34, 0xd1, 0x23, 0xc8, 0x30, 0x6e, 0x32, + 0x35, 0xcd, 0xd0, 0xa9, 0x29, 0xaa, 0x2e, 0x5c, 0x8d, 0x74, 0x52, 0x4a, 0x53, 0x56, 0x32, 0x11, + 0x5d, 0x80, 0xe9, 0x3d, 0xd3, 0x52, 0xdd, 0xcd, 0x58, 0xf6, 0xc0, 0xda, 0x73, 0x3d, 0x95, 0x4e, + 0x17, 0x33, 0xeb, 0xa9, 0x74, 0xa6, 0x08, 0xe2, 0x6f, 0x08, 0x50, 0xf0, 0x1a, 0x22, 0x4e, 0x6b, + 0x5d, 0x09, 0x69, 0xf1, 0xfc, 0x4d, 0x41, 0x14, 0x28, 0xfe, 0x5b, 0xea, 0xae, 0xa8, 0xe6, 0x21, + 0x6d, 0x99, 0x38, 0x7b, 0xca, 0x23, 0x16, 0xa8, 0x93, 0x38, 0x6f, 0xeb, 0xd2, 0x98, 0x9d, 0xfb, + 0x70, 0x41, 0xef, 0x10, 0x3b, 0xae, 0x3b, 0xed, 0x13, 0x8e, 0xa9, 0x1c, 0xec, 0xee, 0xfa, 0x2e, + 0xf8, 0x69, 0x15, 0x37, 0x49, 0xfc, 0xfb, 0x74, 0x75, 0xda, 0xaf, 0x49, 0x9c, 0xaa, 0xae, 0x43, + 0xce, 0x62, 0xa2, 0x89, 0xcf, 0x71, 0x4e, 0x6d, 0xcf, 0x79, 0xac, 0x44, 0xe1, 0xbf, 0x95, 0x80, + 0xc2, 0xf3, 0x1e, 0xb6, 0x4e, 0xbe, 0x4e, 0xea, 0xbe, 0x05, 0x85, 0x23, 0x45, 0x77, 0xe4, 0x3d, + 0xd3, 0x92, 0x7b, 0x5d, 0x4d, 0x71, 0xdc, 0x68, 0x91, 0x1c, 0x21, 0x3f, 0x36, 0xad, 0x1d, 0x4a, + 0x44, 0x18, 0xd0, 0x81, 0x61, 0x1e, 0x19, 0x32, 0x21, 0x53, 0x14, 0x7b, 0x6c, 0xf0, 0x25, 0xe3, + 0xd5, 0x0f, 0xfe, 0xf3, 0xe9, 0xe2, 0xc3, 0x89, 0x62, 0xc0, 0x68, 0xbc, 0x5b, 0xaf, 0xa7, 0x6b, + 0x4b, 0x3b, 0x3b, 0xf5, 0x35, 0xa9, 0x48, 0x45, 0xbe, 0x64, 0x12, 0x9b, 0xc7, 0x86, 0x2d, 0xfe, + 0x83, 0x04, 0x14, 0x7d, 0x1d, 0xc5, 0xd9, 0x90, 0x55, 0xc8, 0xbe, 0xea, 0x61, 0x4b, 0x7f, 0x8d, + 0x66, 0x04, 0xce, 0x48, 0xcc, 0xce, 0x67, 0x30, 0x17, 0xd2, 0x40, 0xf2, 0xcb, 0x69, 0x20, 0x7b, + 0xe4, 0x57, 0x1e, 0xdd, 0x83, 0x79, 0xe7, 0xd8, 0x90, 0x59, 0xf4, 0x1f, 0x8b, 0x18, 0x71, 0x83, + 0x19, 0x0a, 0x0e, 0xd1, 0x07, 0xa1, 0xd3, 0x68, 0x11, 0x5b, 0xfc, 0x7d, 0x01, 0x10, 0x55, 0x54, + 0x9d, 0xad, 0xe9, 0x7f, 0x5d, 0xfa, 0xd3, 0x1d, 0x28, 0xd2, 0x78, 0x4a, 0x59, 0xdf, 0x93, 0x3b, + 0xba, 0x6d, 0xeb, 0x46, 0x8b, 0x77, 0xa8, 0x3c, 0xa5, 0xd7, 0xf7, 0x36, 0x19, 0x55, 0xfc, 0x2b, + 0xb0, 0x10, 0xaa, 0x40, 0x9c, 0x8d, 0x7d, 0x0d, 0xe6, 0xf6, 0xd8, 0x16, 0x2a, 0x15, 0xce, 0x97, + 0x03, 0xb3, 0x94, 0xc6, 0xde, 0x27, 0xfe, 0x8f, 0x04, 0x5c, 0x90, 0xb0, 0x6d, 0xb6, 0x0f, 0x71, + 0xfc, 0x2a, 0xac, 0x01, 0xdf, 0x6b, 0x91, 0x5f, 0x4b, 0x93, 0x19, 0xc6, 0xcc, 0xa6, 0xb9, 0xf0, + 0x9a, 0xfa, 0x8d, 0xd1, 0x3d, 0x76, 0x70, 0x15, 0x9d, 0xaf, 0xc9, 0xa5, 0x42, 0x6b, 0x72, 0x26, + 0x14, 0xf4, 0x96, 0x61, 0x12, 0x9b, 0x66, 0xe3, 0x57, 0x46, 0xaf, 0xe3, 0x22, 0x95, 0xa5, 0x51, + 0x85, 0xac, 0x33, 0x96, 0x06, 0x7e, 0xb5, 0xd5, 0xeb, 0x50, 0xdf, 0x79, 0xf5, 0x12, 0x29, 0xef, + 0xd9, 0xe9, 0x62, 0x3e, 0x94, 0x66, 0x4b, 0x79, 0xdd, 0x7b, 0x26, 0xd2, 0xc5, 0x6f, 0xc3, 0xc5, + 0x3e, 0x65, 0xc7, 0xe9, 0xf1, 0xfc, 0xab, 0x24, 0xbc, 0x11, 0x16, 0x1f, 0x37, 0xfe, 0xf8, 0xba, + 0x37, 0x68, 0x0d, 0x72, 0x1d, 0xdd, 0x78, 0xbd, 0xa5, 0xc5, 0xb9, 0x8e, 0x6e, 0xf8, 0xcb, 0xb8, + 0x11, 0x5d, 0x63, 0xe6, 0x2b, 0xed, 0x1a, 0x0a, 0x94, 0xa3, 0xda, 0x2e, 0xce, 0xfe, 0xf1, 0x2b, + 0x02, 0xcc, 0xc5, 0xbd, 0x66, 0xf6, 0x7a, 0x51, 0x6c, 0x62, 0x13, 0x72, 0x5f, 0xc1, 0x22, 0xdb, + 0x6f, 0x09, 0x80, 0x9a, 0x56, 0xcf, 0x20, 0xa0, 0xf6, 0xa9, 0xd9, 0x8a, 0xb3, 0x9a, 0x17, 0x60, + 0x5a, 0x37, 0x34, 0x7c, 0x4c, 0xab, 0x99, 0x92, 0xd8, 0x43, 0x68, 0xeb, 0x30, 0x39, 0xd1, 0xd6, + 0xa1, 0xf8, 0x19, 0x2c, 0x84, 0x8a, 0x18, 0x67, 0xfd, 0xff, 0x7b, 0x02, 0x16, 0x78, 0x45, 0x62, + 0x5f, 0x5e, 0xfc, 0x26, 0x4c, 0xb7, 0x89, 0xcc, 0x11, 0xed, 0x4c, 0xdf, 0xe9, 0xb6, 0x33, 0xcd, + 0x8c, 0x7e, 0x06, 0xa0, 0x6b, 0xe1, 0x43, 0x99, 0xb1, 0x26, 0x27, 0x62, 0xcd, 0x10, 0x0e, 0x4a, + 0x40, 0xdf, 0x17, 0xa0, 0x40, 0x06, 0x74, 0xd7, 0x32, 0xbb, 0xa6, 0x4d, 0x7c, 0x16, 0x7b, 0x32, + 0x98, 0xf3, 0xfc, 0xec, 0x74, 0x31, 0xb7, 0xa9, 0x1b, 0xdb, 0x9c, 0xb1, 0xd9, 0x98, 0x38, 0x40, + 0xdf, 0x3d, 0xa6, 0xb0, 0x54, 0x69, 0x9b, 0xea, 0x81, 0xbf, 0x19, 0x46, 0x2c, 0x8b, 0x27, 0xce, + 0x16, 0xff, 0xa3, 0x00, 0x17, 0xbe, 0xb2, 0xb5, 0xdc, 0xbf, 0x08, 0x65, 0x8b, 0x2f, 0xa0, 0x48, + 0x7f, 0xd4, 0x8d, 0x3d, 0x33, 0xce, 0x55, 0xf5, 0x5f, 0x15, 0x60, 0x3e, 0x20, 0x38, 0x4e, 0x07, + 0xe7, 0xb5, 0xf4, 0x24, 0xfe, 0x1c, 0x71, 0x79, 0x82, 0xa3, 0x24, 0xce, 0x31, 0xf8, 0x2f, 0x12, + 0x70, 0xa9, 0xc2, 0xf6, 0xa0, 0xdd, 0xa0, 0x8c, 0x38, 0x7b, 0x46, 0x09, 0x66, 0x0f, 0xb1, 0x65, + 0xeb, 0x26, 0x9b, 0x7e, 0x73, 0x92, 0xfb, 0x88, 0xca, 0x90, 0xb6, 0x0d, 0xa5, 0x6b, 0xef, 0x9b, + 0xee, 0xe6, 0x9b, 0xf7, 0xec, 0x05, 0x90, 0x4c, 0xbf, 0x7e, 0x00, 0xc9, 0xcc, 0xe8, 0x00, 0x92, + 0xd9, 0x2f, 0x11, 0x40, 0xc2, 0x77, 0xba, 0xfe, 0x9d, 0x00, 0x97, 0x07, 0x34, 0x17, 0x67, 0x6f, + 0xf9, 0x2e, 0x64, 0x55, 0x2e, 0x98, 0x18, 0x6c, 0xb6, 0x8d, 0x57, 0x27, 0xd9, 0x5e, 0x13, 0xb7, + 0x9c, 0x9d, 0x2e, 0x82, 0x5b, 0xd4, 0xfa, 0x1a, 0x57, 0x0e, 0xf9, 0xad, 0x89, 0xbf, 0x98, 0x83, + 0x42, 0xf5, 0x98, 0x2d, 0x61, 0x37, 0x98, 0x9b, 0x80, 0x1e, 0x43, 0xba, 0x6b, 0x99, 0x87, 0xba, + 0x5b, 0x8d, 0x7c, 0x28, 0x7a, 0xc0, 0xad, 0x46, 0x1f, 0xd7, 0x36, 0xe7, 0x90, 0x3c, 0x5e, 0xd4, + 0x84, 0xcc, 0x53, 0x53, 0x55, 0xda, 0x8f, 0xf5, 0xb6, 0xdb, 0xf3, 0xdf, 0x1b, 0x2f, 0x68, 0xc9, + 0xe3, 0xd9, 0x56, 0x9c, 0x7d, 0xb7, 0x11, 0x3c, 0x22, 0xaa, 0x43, 0xba, 0xe6, 0x38, 0x5d, 0x92, + 0xc8, 0x6d, 0xc7, 0xed, 0x09, 0x84, 0x12, 0x16, 0x37, 0x84, 0xd5, 0x65, 0x47, 0x4d, 0x98, 0x7f, + 0x42, 0x0f, 0x64, 0x55, 0xda, 0x66, 0x4f, 0xab, 0x98, 0xc6, 0x9e, 0xde, 0xe2, 0x76, 0xfb, 0xd6, + 0x04, 0x32, 0x9f, 0x54, 0x1a, 0xd2, 0xa0, 0x00, 0xb4, 0x02, 0xe9, 0xc6, 0x43, 0x2e, 0x8c, 0xf9, + 0x75, 0x37, 0x27, 0x10, 0xd6, 0x78, 0x28, 0x79, 0x6c, 0x68, 0x1d, 0xb2, 0x2b, 0x9f, 0xf7, 0x2c, + 0xcc, 0xa5, 0xcc, 0x0c, 0x0d, 0x5d, 0xe8, 0x97, 0x42, 0xb9, 0xa4, 0x20, 0x33, 0x6a, 0x40, 0xfe, + 0xa5, 0x69, 0x1d, 0xb4, 0x4d, 0xc5, 0xad, 0xe1, 0x2c, 0x15, 0xf7, 0x8d, 0x09, 0xc4, 0xb9, 0x8c, + 0x52, 0x9f, 0x08, 0xf4, 0x6d, 0x28, 0x90, 0xc6, 0x68, 0x2a, 0xbb, 0x6d, 0xb7, 0x90, 0x69, 0x2a, + 0xf5, 0x9d, 0x09, 0xa4, 0x7a, 0x9c, 0xee, 0x1e, 0x4a, 0x9f, 0xa8, 0xb2, 0x04, 0xb9, 0x50, 0x27, + 0x40, 0x08, 0x52, 0x5d, 0xd2, 0xde, 0x02, 0x0d, 0x2e, 0xa2, 0xbf, 0xd1, 0xbb, 0x30, 0x6b, 0x98, + 0x1a, 0x76, 0x47, 0x48, 0x6e, 0xf5, 0xc2, 0xd9, 0xe9, 0xe2, 0xcc, 0x96, 0xa9, 0x31, 0x8f, 0x86, + 0xff, 0x92, 0x66, 0x48, 0xa6, 0xba, 0x56, 0xbe, 0x0a, 0x29, 0xd2, 0xee, 0xc4, 0x30, 0xed, 0x2a, + 0x36, 0xde, 0xb1, 0x74, 0x2e, 0xcd, 0x7d, 0x2c, 0xff, 0xe3, 0x04, 0x24, 0x1a, 0x0f, 0x89, 0xcf, + 0xbe, 0xdb, 0x53, 0x0f, 0xb0, 0xc3, 0xd3, 0xf9, 0x13, 0xf5, 0xe5, 0x2d, 0xbc, 0xa7, 0x33, 0xd7, + 0x2a, 0x23, 0xf1, 0x27, 0xf4, 0x36, 0x80, 0xa2, 0xaa, 0xd8, 0xb6, 0x65, 0xf7, 0xa0, 0x5e, 0x46, + 0xca, 0x30, 0xca, 0x06, 0x3e, 0x21, 0x6c, 0x36, 0x56, 0x2d, 0xec, 0xb8, 0x91, 0x51, 0xec, 0x89, + 0xb0, 0x39, 0xb8, 0xd3, 0x95, 0x1d, 0xf3, 0x00, 0x1b, 0xb4, 0x9f, 0x64, 0x88, 0xa9, 0xe9, 0x74, + 0x9b, 0x84, 0x40, 0xac, 0x24, 0x36, 0x34, 0xdf, 0xa4, 0x65, 0x24, 0xef, 0x99, 0x88, 0xb4, 0x70, + 0x4b, 0xe7, 0x27, 0xce, 0x32, 0x12, 0x7f, 0x22, 0x5a, 0x52, 0x7a, 0xce, 0x3e, 0x6d, 0x89, 0x8c, + 0x44, 0x7f, 0xa3, 0x5b, 0x50, 0x60, 0xc1, 0x94, 0x32, 0x36, 0x54, 0x99, 0x1a, 0xd7, 0x0c, 0x4d, + 0xce, 0x31, 0x72, 0xd5, 0x50, 0x89, 0x29, 0x45, 0x0f, 0x81, 0x13, 0xe4, 0x83, 0x8e, 0x4d, 0x74, + 0x0a, 0x24, 0xd7, 0x6a, 0xe1, 0xec, 0x74, 0x31, 0xdb, 0xa0, 0x09, 0x1b, 0x9b, 0x8d, 0xfa, 0x9a, + 0x94, 0x65, 0xb9, 0x36, 0x3a, 0x76, 0x5d, 0x2b, 0xff, 0x9a, 0x00, 0xc9, 0x27, 0x95, 0xc6, 0xb9, + 0x55, 0xe6, 0x16, 0x34, 0x19, 0x28, 0xe8, 0x6d, 0x28, 0xec, 0xea, 0xed, 0xb6, 0x6e, 0xb4, 0x88, + 0x17, 0xf5, 0x5d, 0xac, 0xba, 0x0a, 0xcb, 0x73, 0xf2, 0x36, 0xa3, 0xa2, 0xab, 0x90, 0x55, 0x2d, + 0xac, 0x61, 0xc3, 0xd1, 0x95, 0xb6, 0xcd, 0x35, 0x17, 0x24, 0x95, 0x7f, 0x41, 0x80, 0x69, 0x3a, + 0x02, 0xd0, 0x5b, 0x90, 0x51, 0x4d, 0xc3, 0x51, 0x74, 0x83, 0x9b, 0xb2, 0x8c, 0xe4, 0x13, 0x86, + 0x16, 0xef, 0x1a, 0xcc, 0x29, 0xaa, 0x6a, 0xf6, 0x0c, 0x47, 0x36, 0x94, 0x0e, 0xe6, 0xc5, 0xcc, + 0x72, 0xda, 0x96, 0xd2, 0xc1, 0x68, 0x11, 0xdc, 0x47, 0xef, 0xfc, 0x65, 0x46, 0x02, 0x4e, 0xda, + 0xc0, 0x27, 0xe5, 0x3f, 0x10, 0x20, 0xed, 0x8e, 0x19, 0x52, 0x8c, 0x16, 0x36, 0x58, 0xc4, 0xb7, + 0x5b, 0x0c, 0x8f, 0xd0, 0x3f, 0x55, 0x66, 0xfc, 0xa9, 0xf2, 0x02, 0x4c, 0x3b, 0x64, 0x58, 0xf0, + 0x12, 0xb0, 0x07, 0xba, 0x9e, 0xdd, 0x56, 0x5a, 0x6c, 0x39, 0x2f, 0x23, 0xb1, 0x07, 0x52, 0x19, + 0x1e, 0x93, 0xcb, 0x34, 0xc2, 0x9f, 0x48, 0x49, 0x59, 0xe4, 0xe8, 0x2e, 0x6e, 0xe9, 0x06, 0xed, + 0x4b, 0x49, 0x09, 0x28, 0x69, 0x95, 0x50, 0xd0, 0x9b, 0x90, 0x61, 0x19, 0xb0, 0xa1, 0xd1, 0x0e, + 0x95, 0x94, 0xd2, 0x94, 0x50, 0x35, 0xb4, 0x32, 0x86, 0x8c, 0x37, 0x38, 0x49, 0xb3, 0xf5, 0x6c, + 0x4f, 0x91, 0xf4, 0x37, 0x7a, 0x0f, 0x2e, 0xbc, 0xea, 0x29, 0x6d, 0x7d, 0x8f, 0xae, 0xd4, 0xd1, + 0x90, 0x78, 0xaa, 0x33, 0x56, 0x13, 0xe4, 0xa5, 0x51, 0x09, 0x54, 0x75, 0xee, 0x58, 0x4e, 0xfa, + 0x63, 0x59, 0xfc, 0x5d, 0x01, 0xe6, 0x59, 0x58, 0x10, 0x8b, 0x66, 0x8d, 0xcf, 0x0f, 0xf9, 0x08, + 0x32, 0x9a, 0xe2, 0x28, 0xec, 0x44, 0x69, 0x62, 0xe4, 0x89, 0x52, 0xef, 0x84, 0x83, 0xe2, 0x28, + 0xf4, 0x54, 0x29, 0x82, 0x14, 0xf9, 0xcd, 0x0e, 0xdf, 0x4a, 0xf4, 0xb7, 0xf8, 0x29, 0xa0, 0x60, + 0x41, 0xe3, 0xf4, 0xc8, 0xee, 0xc2, 0x45, 0xa2, 0xeb, 0xaa, 0xa1, 0x5a, 0x27, 0x5d, 0x47, 0x37, + 0x8d, 0x67, 0xf4, 0xaf, 0x8d, 0x8a, 0x81, 0x8d, 0x2d, 0xba, 0x9f, 0x25, 0xfe, 0xde, 0x0c, 0xe4, + 0xaa, 0xc7, 0x5d, 0xd3, 0x8a, 0x75, 0x19, 0x6c, 0x15, 0x66, 0xf9, 0x4a, 0xc1, 0x88, 0x8d, 0xe5, + 0x3e, 0x63, 0xee, 0xee, 0xd9, 0x72, 0x46, 0xb4, 0x0a, 0xc0, 0x22, 0x4e, 0x69, 0x54, 0x51, 0xf2, + 0x1c, 0x5b, 0x6d, 0x94, 0x8d, 0x9e, 0xae, 0xd8, 0x82, 0x6c, 0xe7, 0x50, 0x55, 0xe5, 0x3d, 0xbd, + 0xed, 0xf0, 0xc0, 0xbd, 0xe8, 0x18, 0xf3, 0xcd, 0x17, 0x95, 0xca, 0x63, 0x9a, 0x89, 0xc5, 0xd0, + 0xf9, 0xcf, 0x12, 0x10, 0x09, 0xec, 0x37, 0x7a, 0x07, 0xf8, 0x49, 0x1f, 0xd9, 0x76, 0x0f, 0xf5, + 0xad, 0xe6, 0xce, 0x4e, 0x17, 0x33, 0x12, 0xa5, 0x36, 0x1a, 0x4d, 0x29, 0xc3, 0x32, 0x34, 0x6c, + 0x07, 0x5d, 0x87, 0x9c, 0xd9, 0xd1, 0x1d, 0xd9, 0x75, 0x92, 0xb8, 0x47, 0x39, 0x47, 0x88, 0xae, + 0x13, 0x75, 0x9e, 0x03, 0x20, 0xb3, 0x13, 0x1f, 0x00, 0x41, 0x7f, 0x5d, 0x80, 0x4b, 0x5c, 0x91, + 0xf2, 0x2e, 0x0d, 0x92, 0x57, 0xda, 0xba, 0x73, 0x22, 0x1f, 0x1c, 0x96, 0xd2, 0xd4, 0x6f, 0xfd, + 0xe9, 0xc8, 0x06, 0x09, 0xf4, 0x83, 0x25, 0xb7, 0x59, 0x4e, 0x9e, 0x72, 0xe6, 0x8d, 0xc3, 0xaa, + 0xe1, 0x58, 0x27, 0xab, 0x97, 0xcf, 0x4e, 0x17, 0x17, 0x06, 0x53, 0x5f, 0x48, 0x0b, 0xf6, 0x20, + 0x0b, 0xaa, 0x01, 0x60, 0xaf, 0x1f, 0xd2, 0x19, 0x23, 0xda, 0xff, 0x88, 0xec, 0xb0, 0x52, 0x80, + 0x17, 0xdd, 0x81, 0x22, 0x3f, 0x70, 0xb3, 0xa7, 0xb7, 0xb1, 0x6c, 0xeb, 0x9f, 0x63, 0x3a, 0xb7, + 0x24, 0xa5, 0x3c, 0xa3, 0x13, 0x11, 0x0d, 0xfd, 0x73, 0x5c, 0xfe, 0x2e, 0x94, 0x86, 0x95, 0x3e, + 0x38, 0x04, 0x32, 0x6c, 0x4b, 0xf7, 0xc3, 0xf0, 0x7a, 0xce, 0x04, 0x5d, 0x95, 0xaf, 0xe9, 0x7c, + 0x94, 0xf8, 0x50, 0x10, 0xff, 0x61, 0x02, 0x72, 0xab, 0xbd, 0xf6, 0xc1, 0xb3, 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, 0x44, 0xce, 0x0e, 0x01, 0xd0, 0x3a, 0x04, 0x4e, 0x86, 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, 0x1c, 0xc9, 0x78, + 0x22, 0xd3, 0x29, 0xc4, 0x5d, 0xe0, 0xbd, 0x1f, 0x51, 0xad, 0x50, 0xb9, 0x97, 0xa8, 0x7e, 0x2a, + 0x94, 0x87, 0xfe, 0x94, 0xb2, 0xd8, 0xa7, 0x94, 0x3f, 0x81, 0x62, 0x7f, 0x86, 0xa0, 0x2e, 0x53, + 0x4c, 0x97, 0x17, 0x82, 0xba, 0x4c, 0x06, 0xf4, 0xb4, 0x9e, 0x4a, 0xa7, 0x8a, 0xd3, 0xe2, 0x9f, + 0x26, 0x21, 0xef, 0x76, 0xb3, 0x38, 0x81, 0xce, 0x2a, 0x4c, 0x93, 0x4e, 0xe1, 0x06, 0x89, 0xdc, + 0x1a, 0xd1, 0xbb, 0x79, 0xd8, 0x39, 0xe9, 0x2c, 0x2e, 0x48, 0xa6, 0xac, 0x71, 0x18, 0x9c, 0xf2, + 0x2f, 0x24, 0x20, 0x45, 0xb1, 0xc5, 0x7d, 0x48, 0xd1, 0x89, 0x42, 0x98, 0x64, 0xa2, 0xa0, 0x59, + 0xbd, 0xe9, 0x2c, 0x11, 0x70, 0x4d, 0x89, 0xcf, 0xb7, 0xaf, 0xbc, 0x7f, 0xff, 0x01, 0x35, 0x36, + 0x73, 0x12, 0x7f, 0x42, 0xab, 0x34, 0x6e, 0xc9, 0xb4, 0x1c, 0xac, 0x71, 0x9f, 0xfe, 0xea, 0xb8, + 0xf6, 0x75, 0x27, 0x25, 0x97, 0x0f, 0xbd, 0x01, 0x49, 0x62, 0xc5, 0x66, 0x59, 0x94, 0xc3, 0xd9, + 0xe9, 0x62, 0x92, 0xd8, 0x2f, 0x42, 0x43, 0xcb, 0x90, 0x0d, 0x9b, 0x0c, 0xe2, 0xc1, 0x51, 0xc3, + 0x18, 0x18, 0xee, 0xd0, 0xf6, 0x86, 0x16, 0xc3, 0xb3, 0xbc, 0x8d, 0xff, 0x2c, 0x05, 0xb9, 0x7a, + 0x27, 0xee, 0x29, 0x65, 0x25, 0xdc, 0xc2, 0x51, 0x40, 0x28, 0xf4, 0xd2, 0x88, 0x06, 0x0e, 0xcd, + 0xe0, 0xc9, 0xf3, 0xcd, 0xe0, 0x75, 0xe2, 0x29, 0xf3, 0x5b, 0x21, 0x92, 0x43, 0x30, 0x4f, 0xf8, + 0xfd, 0xd4, 0x4f, 0x91, 0x08, 0x8f, 0x7f, 0x10, 0x83, 0x46, 0xaa, 0x7c, 0x42, 0x1d, 0x72, 0xd6, + 0xcb, 0x66, 0x26, 0xef, 0x65, 0xb3, 0xd8, 0xd0, 0xe8, 0xa4, 0x16, 0xb6, 0xa8, 0xb3, 0xaf, 0x6f, + 0x51, 0xcb, 0x0e, 0xef, 0xac, 0x1f, 0x41, 0x52, 0xd3, 0xdd, 0xc6, 0x99, 0x7c, 0xaa, 0x26, 0x4c, + 0x63, 0x7a, 0x6d, 0x2a, 0xd8, 0x6b, 0x59, 0x2f, 0x29, 0xd7, 0x01, 0x7c, 0xdd, 0xa0, 0xab, 0x30, + 0x63, 0xb6, 0x35, 0xf7, 0x24, 0x4a, 0x6e, 0x35, 0x73, 0x76, 0xba, 0x38, 0xfd, 0xac, 0xad, 0xd5, + 0xd7, 0xa4, 0x69, 0xb3, 0xad, 0xd5, 0x35, 0x7a, 0x25, 0x07, 0x3e, 0x92, 0xbd, 0x30, 0xb5, 0x39, + 0x69, 0xd6, 0xc0, 0x47, 0x6b, 0xd8, 0x56, 0x79, 0x87, 0xfb, 0x4d, 0x01, 0xf2, 0xae, 0xee, 0xe3, + 0x35, 0x2a, 0x69, 0xbd, 0xc3, 0x07, 0x59, 0xf2, 0x7c, 0x83, 0xcc, 0xe5, 0xe3, 0xa7, 0x7e, 0x7f, + 0x49, 0xe0, 0x81, 0xc7, 0x0d, 0x55, 0x71, 0x88, 0x53, 0x11, 0xe3, 0xc0, 0xb8, 0x0b, 0x45, 0x4b, + 0x31, 0x34, 0xb3, 0xa3, 0x7f, 0x8e, 0xd9, 0x42, 0xa8, 0xcd, 0xb7, 0x3d, 0x0b, 0x1e, 0x9d, 0xae, + 0xfa, 0xd9, 0xe2, 0x1f, 0x24, 0x78, 0x90, 0xb2, 0x57, 0x8c, 0x38, 0xd5, 0xf5, 0x1d, 0x98, 0xef, + 0xbf, 0x1f, 0xc5, 0x1d, 0xad, 0xef, 0x46, 0xc8, 0x8b, 0x2a, 0x08, 0x0b, 0x36, 0x74, 0x23, 0xdf, + 0xfb, 0xee, 0x4a, 0xb1, 0x51, 0x05, 0xb2, 0xc1, 0x6b, 0x57, 0x92, 0x13, 0x5f, 0xbb, 0x02, 0x96, + 0x77, 0xd9, 0x4a, 0xf9, 0x67, 0x61, 0x9a, 0x26, 0xbf, 0x86, 0x89, 0xe6, 0xad, 0xf9, 0x27, 0x09, + 0xb8, 0x41, 0x4b, 0xff, 0x02, 0x5b, 0xfa, 0xde, 0xc9, 0xb6, 0x65, 0x3a, 0x58, 0x75, 0xb0, 0xe6, + 0xaf, 0x9c, 0xc7, 0x6a, 0xf7, 0x32, 0x5d, 0xf7, 0x05, 0xe7, 0x0a, 0x38, 0xf3, 0xb8, 0xd0, 0x06, + 0x14, 0x78, 0x68, 0x81, 0xd2, 0xd6, 0x0f, 0xb1, 0xac, 0x38, 0xe7, 0x99, 0xdd, 0x72, 0x8c, 0x77, + 0x85, 0xb0, 0xae, 0x38, 0x48, 0x83, 0x0c, 0x17, 0xa6, 0x6b, 0xfc, 0xb6, 0xa0, 0x27, 0x5f, 0x6e, + 0x41, 0x31, 0xcd, 0xe2, 0x1b, 0xea, 0x6b, 0x52, 0x9a, 0x49, 0xae, 0x6b, 0xe2, 0x7f, 0x12, 0xe0, + 0xe6, 0x18, 0x15, 0xc7, 0xd9, 0x75, 0xcb, 0x90, 0x3e, 0x24, 0x2f, 0xd2, 0xb9, 0x8e, 0xd3, 0x92, + 0xf7, 0x8c, 0x36, 0x21, 0xb7, 0xa7, 0xe8, 0x6d, 0xbf, 0x4b, 0x0f, 0x8f, 0x4f, 0x8c, 0x0e, 0x95, + 0x9d, 0x63, 0xec, 0xac, 0x0f, 0x8b, 0xbf, 0x9e, 0x80, 0xf9, 0x15, 0x4d, 0x6b, 0x34, 0xb8, 0x0d, + 0x8c, 0xaf, 0xa7, 0xb8, 0x20, 0x33, 0xe1, 0x83, 0x4c, 0xf4, 0x2e, 0x20, 0x4d, 0xb7, 0xd9, 0xad, + 0x24, 0xf6, 0xbe, 0xa2, 0x99, 0x47, 0x7e, 0x5c, 0xc6, 0xbc, 0x9b, 0xd2, 0x70, 0x13, 0x50, 0x03, + 0x28, 0xda, 0x91, 0x6d, 0x47, 0xf1, 0xf6, 0x9d, 0x6e, 0x4e, 0x74, 0xde, 0x8b, 0xc1, 0x20, 0xef, + 0x51, 0xca, 0x10, 0x39, 0xf4, 0x27, 0xf1, 0xdb, 0x75, 0x52, 0x75, 0x47, 0x56, 0x6c, 0xf7, 0x70, + 0x0f, 0xbb, 0x0f, 0x25, 0xcf, 0xe8, 0x2b, 0x36, 0x3b, 0xb3, 0xc3, 0x4e, 0x23, 0xf8, 0xaa, 0x89, + 0x13, 0x12, 0xff, 0x3d, 0x01, 0xf2, 0x12, 0xde, 0xb3, 0xb0, 0x1d, 0xeb, 0xa2, 0xc0, 0x63, 0x98, + 0xb3, 0x98, 0x54, 0x79, 0xcf, 0x32, 0x3b, 0xe7, 0x19, 0x57, 0x59, 0xce, 0xf8, 0xd8, 0x32, 0x3b, + 0xdc, 0xb0, 0xbc, 0x80, 0x82, 0x57, 0xc6, 0x38, 0x2b, 0xff, 0xbb, 0xf4, 0x2c, 0x33, 0x13, 0x1c, + 0x77, 0x80, 0x44, 0xbc, 0x1a, 0xa0, 0x1b, 0x55, 0xc1, 0x82, 0xc6, 0xa9, 0x86, 0xff, 0x26, 0x40, + 0xbe, 0xd1, 0xdb, 0x65, 0xd7, 0x5e, 0xc5, 0xa7, 0x81, 0x2a, 0x64, 0xda, 0x78, 0xcf, 0x91, 0x5f, + 0x2b, 0x8e, 0x3e, 0x4d, 0x58, 0xe9, 0x29, 0x82, 0x27, 0x00, 0x16, 0x3d, 0xff, 0x46, 0xe5, 0x24, + 0xcf, 0x29, 0x27, 0x43, 0x79, 0x09, 0x59, 0xfc, 0xb3, 0x04, 0x14, 0xbc, 0x6a, 0xc6, 0x69, 0x25, + 0x5f, 0x86, 0xac, 0x43, 0xf2, 0x3c, 0xd6, 0x61, 0x9e, 0xc7, 0x84, 0x44, 0x5b, 0x88, 0x25, 0x58, + 0xa0, 0x8e, 0x8b, 0xac, 0x74, 0xbb, 0x6d, 0xdd, 0x85, 0xbb, 0xd4, 0xfe, 0xa4, 0xa4, 0x79, 0x9a, + 0xb4, 0xc2, 0x52, 0x28, 0xd0, 0x45, 0xbf, 0x2c, 0xc0, 0xdc, 0x9e, 0x85, 0xf1, 0xe7, 0x58, 0xa6, + 0xd0, 0x6b, 0xb2, 0xa0, 0x97, 0x35, 0x52, 0x86, 0x2f, 0xbd, 0x29, 0x9e, 0x65, 0x2f, 0x6e, 0x90, + 0xf7, 0xf2, 0x4e, 0xfb, 0x12, 0xe6, 0x69, 0xa3, 0xc4, 0x7d, 0xf0, 0x57, 0xfc, 0x3b, 0x09, 0x40, + 0x41, 0xc9, 0x5f, 0x5d, 0x63, 0x26, 0xe2, 0x6b, 0xcc, 0x77, 0x00, 0xb1, 0xb0, 0x49, 0x5b, 0xee, + 0x62, 0x4b, 0xb6, 0xb1, 0x6a, 0xf2, 0xbb, 0x9b, 0x04, 0xa9, 0xc8, 0x53, 0xb6, 0xb1, 0xd5, 0xa0, + 0x74, 0xf4, 0x08, 0xc0, 0x77, 0xe9, 0xf8, 0x8c, 0x33, 0xd2, 0xa3, 0x93, 0x32, 0x9e, 0x2f, 0x27, + 0x7e, 0x4f, 0x80, 0xfc, 0xa6, 0xde, 0xb2, 0x94, 0x58, 0x2f, 0x25, 0x42, 0x1f, 0x85, 0x97, 0xd1, + 0xb3, 0x0f, 0xca, 0x51, 0x21, 0x3e, 0x2c, 0x87, 0x0b, 0xce, 0x38, 0x03, 0x31, 0xe1, 0x5e, 0x89, + 0xe2, 0xb4, 0x5d, 0xff, 0xbe, 0x0c, 0x73, 0xbc, 0xdc, 0x3b, 0x86, 0x6e, 0x1a, 0xe8, 0x3e, 0x24, + 0x5b, 0x7c, 0x9b, 0x24, 0x1b, 0xb9, 0xa4, 0xe9, 0xdf, 0xcf, 0x57, 0x9b, 0x92, 0x48, 0x5e, 0xc2, + 0xd2, 0xed, 0x39, 0x11, 0x4e, 0xa4, 0x1f, 0xf5, 0x1e, 0x64, 0xe9, 0xf6, 0x1c, 0xd4, 0x80, 0x82, + 0xea, 0xdf, 0x33, 0x26, 0x13, 0xf6, 0xe4, 0x50, 0xc0, 0x19, 0x79, 0xe3, 0x5b, 0x6d, 0x4a, 0xca, + 0xab, 0xa1, 0x04, 0x54, 0x09, 0x5e, 0x6f, 0x95, 0x1a, 0x08, 0xa9, 0xf3, 0x0f, 0x53, 0x87, 0xaf, + 0xd6, 0xaa, 0x4d, 0x05, 0x6e, 0xc1, 0x42, 0x1f, 0xc1, 0x8c, 0x46, 0x2f, 0x52, 0xe2, 0x83, 0x3f, + 0xaa, 0xa1, 0x43, 0xf7, 0x55, 0xd5, 0xa6, 0x24, 0xce, 0x81, 0xd6, 0x61, 0x8e, 0xfd, 0x62, 0x2e, + 0x1d, 0x47, 0xe1, 0x37, 0x87, 0x4b, 0x08, 0x4c, 0x9a, 0xb5, 0x29, 0x29, 0xab, 0xf9, 0x54, 0xf4, + 0x4d, 0x48, 0xd9, 0xaa, 0xe2, 0xe2, 0xf0, 0x2b, 0x43, 0xee, 0x2f, 0xf1, 0x99, 0x69, 0x6e, 0xf4, + 0x88, 0xdd, 0xb8, 0xe9, 0x1c, 0xbb, 0x4b, 0xa2, 0x51, 0xc5, 0x0f, 0x9d, 0x93, 0x27, 0xc5, 0xc7, + 0x94, 0x80, 0x9e, 0x40, 0x56, 0x21, 0xbe, 0xb1, 0x4c, 0x4f, 0x9a, 0xd2, 0x35, 0xd0, 0xe8, 0x40, + 0x84, 0x81, 0x93, 0xc1, 0x35, 0x7a, 0x18, 0xdf, 0x25, 0xfa, 0x82, 0x3a, 0xd8, 0x6a, 0xe1, 0x52, + 0x76, 0xb4, 0xa0, 0x60, 0x20, 0x9d, 0x27, 0x88, 0x12, 0x89, 0x8f, 0xec, 0x1d, 0xf3, 0xa6, 0x95, + 0x9a, 0x1b, 0xba, 0xf5, 0x1d, 0x71, 0x46, 0xaa, 0x36, 0x25, 0xcd, 0xed, 0x07, 0xc8, 0x68, 0x09, + 0x12, 0x2d, 0xb5, 0x94, 0x1b, 0x6a, 0x0c, 0xbc, 0x73, 0x40, 0xb5, 0x29, 0x29, 0xd1, 0x52, 0xd1, + 0x27, 0x90, 0x66, 0x87, 0x3a, 0x8e, 0x8d, 0x52, 0x7e, 0xe8, 0x18, 0x0b, 0x1f, 0x8d, 0xa9, 0x4d, + 0x49, 0xf4, 0x1c, 0x09, 0x79, 0xdf, 0x36, 0xe4, 0x2d, 0x16, 0x89, 0xe8, 0xc6, 0x0d, 0x17, 0x87, + 0x86, 0x03, 0x44, 0x85, 0x0e, 0xd7, 0x28, 0x4a, 0x0a, 0xd0, 0xd1, 0x77, 0xe0, 0x42, 0x58, 0x22, + 0xef, 0x69, 0xf3, 0x43, 0xb7, 0xb6, 0x87, 0x86, 0xb1, 0xd6, 0xa6, 0x24, 0x64, 0x0d, 0x24, 0xa2, + 0x0f, 0x60, 0x9a, 0xb5, 0x1a, 0xa2, 0x22, 0xa3, 0x22, 0x5c, 0xfa, 0x1a, 0x8c, 0xe5, 0x27, 0x9d, + 0xdf, 0xe1, 0xe1, 0x78, 0x72, 0xdb, 0x6c, 0x95, 0x16, 0x86, 0x76, 0xfe, 0xc1, 0xc0, 0x42, 0xd2, + 0xf9, 0x1d, 0x9f, 0x4a, 0xda, 0xdd, 0x62, 0x29, 0x3c, 0x04, 0xeb, 0xc2, 0xd0, 0x76, 0x8f, 0x88, + 0xd2, 0xab, 0xd1, 0x83, 0x12, 0x3e, 0x99, 0x14, 0xcd, 0x62, 0x57, 0xee, 0xc8, 0x74, 0x4c, 0x5d, + 0x1c, 0x5a, 0xb4, 0xc1, 0x3b, 0x89, 0x6a, 0xd4, 0x9f, 0xf4, 0xa8, 0xe8, 0x05, 0x14, 0xf9, 0xc5, + 0x18, 0xfe, 0xfe, 0xcb, 0x25, 0x2a, 0xef, 0x6e, 0xa4, 0xe9, 0x8a, 0x8a, 0x5f, 0xaa, 0x4d, 0x49, + 0x05, 0x35, 0x9c, 0x82, 0x3e, 0x85, 0x79, 0x2a, 0x4f, 0x56, 0xfd, 0xbb, 0x4c, 0x4a, 0xa5, 0x81, + 0x9b, 0x31, 0x86, 0x5f, 0x7b, 0xe2, 0x4a, 0x2e, 0xaa, 0x7d, 0x49, 0xa4, 0x1b, 0xeb, 0x86, 0xee, + 0x50, 0x2b, 0x5b, 0x1e, 0xda, 0x8d, 0xc3, 0xf7, 0x32, 0x92, 0x6e, 0xac, 0x33, 0x0a, 0xe9, 0xc6, + 0x0e, 0x8f, 0xcf, 0xe3, 0xcd, 0xf1, 0xd6, 0xd0, 0x6e, 0x1c, 0x15, 0xc8, 0x47, 0xba, 0xb1, 0x13, + 0xa4, 0x93, 0x6e, 0xcc, 0x0c, 0x44, 0x9f, 0xdc, 0xb7, 0x87, 0x76, 0xe3, 0xa1, 0x27, 0xbe, 0x49, + 0x37, 0x56, 0x06, 0x12, 0xd1, 0x1a, 0x00, 0x73, 0xfd, 0xe8, 0xfc, 0x7f, 0x65, 0xe8, 0x64, 0xd0, + 0x1f, 0xa7, 0x47, 0x26, 0x83, 0xb6, 0x4b, 0x23, 0x86, 0x8c, 0x02, 0x4b, 0x99, 0x6e, 0x37, 0x97, + 0x16, 0x87, 0x1a, 0xb2, 0x81, 0xad, 0x61, 0x62, 0xc8, 0x8e, 0x3c, 0x22, 0x99, 0x55, 0xd8, 0xfa, + 0x78, 0xe9, 0xea, 0x70, 0xb3, 0x1c, 0xdc, 0x26, 0xa3, 0x66, 0x99, 0x12, 0xd0, 0x0a, 0x64, 0x88, + 0x7b, 0x73, 0x42, 0xcd, 0xd0, 0xb5, 0xa1, 0xfe, 0x7b, 0xdf, 0x49, 0xa0, 0xda, 0x94, 0x94, 0x7e, + 0xc5, 0x49, 0xe4, 0xf5, 0x6c, 0xe5, 0xb0, 0x24, 0x0e, 0x7d, 0x7d, 0x68, 0x95, 0x99, 0xbc, 0x9e, + 0x71, 0x20, 0x15, 0x2e, 0xb2, 0xb6, 0xe2, 0x87, 0xaf, 0x2d, 0x7e, 0x5e, 0xb8, 0x74, 0x9d, 0x8a, + 0x1a, 0xba, 0x04, 0x17, 0x79, 0x26, 0xbc, 0x36, 0x25, 0x2d, 0x28, 0x83, 0xa9, 0x64, 0xc0, 0xf3, + 0xa9, 0x87, 0x2d, 0xdc, 0x95, 0x6e, 0x0c, 0x1d, 0xf0, 0x11, 0xeb, 0x9d, 0x64, 0xc0, 0x2b, 0x01, + 0x32, 0x9b, 0x80, 0x34, 0xd9, 0xb6, 0x59, 0x70, 0xc2, 0xcd, 0x11, 0x13, 0x50, 0xdf, 0x8a, 0x09, + 0x9b, 0x80, 0xb4, 0x06, 0xe3, 0x24, 0x82, 0xd4, 0x36, 0x56, 0x2c, 0x6e, 0x66, 0x6f, 0x0d, 0x15, + 0x34, 0x70, 0x05, 0x22, 0x11, 0xa4, 0x7a, 0x44, 0xe2, 0xf0, 0x58, 0xee, 0x8d, 0x3a, 0xdc, 0x37, + 0xbe, 0x3d, 0xd4, 0xe1, 0x89, 0xbc, 0xf2, 0x87, 0x38, 0x3c, 0x56, 0x28, 0x01, 0xfd, 0x0c, 0xcc, + 0x72, 0xa8, 0x5b, 0xba, 0x33, 0xc2, 0x05, 0x0c, 0xae, 0x4e, 0x90, 0x71, 0xcd, 0x79, 0x98, 0x95, + 0x65, 0x10, 0x9b, 0x55, 0xef, 0xee, 0x08, 0x2b, 0x3b, 0x80, 0xf2, 0x99, 0x95, 0xf5, 0xc9, 0xc4, + 0xca, 0xb2, 0x7e, 0xca, 0xe7, 0xba, 0x7b, 0x43, 0xad, 0xec, 0xe0, 0x39, 0x23, 0x62, 0x65, 0x5f, + 0xf9, 0x54, 0x52, 0x33, 0x9b, 0x41, 0xcd, 0xd2, 0x37, 0x86, 0xd6, 0x2c, 0x8c, 0xb9, 0x49, 0xcd, + 0x38, 0x0f, 0x69, 0x36, 0xe6, 0xfd, 0x33, 0x4d, 0xbf, 0x33, 0xfc, 0x86, 0x83, 0x7e, 0x7c, 0x55, + 0x73, 0x17, 0x75, 0x99, 0x86, 0x3d, 0x43, 0x65, 0xf1, 0x13, 0xde, 0x5c, 0x53, 0xef, 0x8e, 0x36, + 0x54, 0x51, 0xc7, 0xd6, 0x3d, 0x43, 0x15, 0x4a, 0xa4, 0x45, 0x65, 0x87, 0xfa, 0xe8, 0xf8, 0x5e, + 0x1a, 0x71, 0x19, 0x43, 0xdf, 0xd1, 0x4a, 0x5a, 0x54, 0x8f, 0xe8, 0x0f, 0xa1, 0x1e, 0xbb, 0x29, + 0xa4, 0xb4, 0x3c, 0x7a, 0x08, 0x85, 0xef, 0x2a, 0xf1, 0x86, 0x10, 0x27, 0x7b, 0x73, 0xa6, 0xeb, + 0x61, 0xbc, 0x37, 0x7a, 0xce, 0xec, 0x77, 0x2d, 0xd8, 0x9c, 0xc9, 0x7d, 0x8a, 0xbf, 0x26, 0xc0, + 0x55, 0x56, 0x36, 0xba, 0xfa, 0x79, 0x22, 0x7b, 0x6b, 0xc8, 0x81, 0x03, 0x26, 0xf7, 0xe9, 0x0b, + 0x3e, 0x18, 0x56, 0xdc, 0x31, 0x6b, 0xe2, 0xb5, 0x29, 0xe9, 0x6d, 0x65, 0x54, 0x3e, 0xd2, 0xa5, + 0x3a, 0x0c, 0x41, 0x95, 0x1e, 0x0c, 0xed, 0x52, 0x61, 0xd4, 0x47, 0xba, 0x14, 0xe7, 0x59, 0x9d, + 0xe5, 0x3b, 0xd0, 0xde, 0x81, 0xdc, 0x42, 0xb1, 0xb8, 0x9e, 0x4a, 0x5f, 0x2e, 0x96, 0xd6, 0x53, + 0xe9, 0x37, 0x8a, 0xe5, 0xf5, 0x54, 0xfa, 0xcd, 0xe2, 0x5b, 0xe2, 0x3f, 0x2a, 0x43, 0xce, 0x05, + 0x5d, 0x0c, 0x50, 0x3d, 0x08, 0x02, 0xaa, 0x2b, 0xc3, 0x00, 0x15, 0x87, 0x69, 0x1c, 0x51, 0x3d, + 0x08, 0x22, 0xaa, 0x2b, 0xc3, 0x10, 0x95, 0xcf, 0x43, 0x20, 0x55, 0x73, 0x18, 0xa4, 0xba, 0x3b, + 0x01, 0xa4, 0xf2, 0x44, 0xf5, 0x63, 0xaa, 0xb5, 0x41, 0x4c, 0x75, 0x63, 0x34, 0xa6, 0xf2, 0x44, + 0x05, 0x40, 0xd5, 0xa3, 0x3e, 0x50, 0x75, 0x6d, 0x04, 0xa8, 0xf2, 0xf8, 0x5d, 0x54, 0xb5, 0x11, + 0x89, 0xaa, 0x6e, 0x8d, 0x43, 0x55, 0x9e, 0x9c, 0x10, 0xac, 0x7a, 0x3f, 0x04, 0xab, 0x16, 0x87, + 0xc2, 0x2a, 0x8f, 0x9b, 0xe1, 0xaa, 0x8f, 0xfb, 0x71, 0xd5, 0xb5, 0x11, 0xb8, 0xca, 0xaf, 0x01, + 0x07, 0x56, 0xb5, 0x28, 0x60, 0x75, 0x73, 0x0c, 0xb0, 0xf2, 0xa4, 0x04, 0x91, 0x55, 0x2d, 0x0a, + 0x59, 0xdd, 0x1c, 0x83, 0xac, 0xfa, 0x24, 0x31, 0x68, 0xb5, 0x15, 0x0d, 0xad, 0x6e, 0x8f, 0x85, + 0x56, 0x9e, 0xb4, 0x30, 0xb6, 0x5a, 0x0e, 0x60, 0xab, 0xb7, 0x87, 0x60, 0x2b, 0x8f, 0x95, 0x80, + 0xab, 0x6f, 0x0d, 0x80, 0x2b, 0x71, 0x14, 0xb8, 0xf2, 0x78, 0x3d, 0x74, 0xf5, 0x7c, 0x08, 0xba, + 0xba, 0x33, 0x1e, 0x5d, 0x79, 0xc2, 0xfa, 0xe0, 0x95, 0x32, 0x12, 0x5e, 0xbd, 0x3b, 0x21, 0xbc, + 0xf2, 0xa4, 0x47, 0xe1, 0xab, 0x0f, 0xc3, 0xf8, 0xea, 0xea, 0x70, 0x7c, 0xe5, 0x89, 0xe1, 0x00, + 0x6b, 0x23, 0x12, 0x60, 0xdd, 0x1a, 0x07, 0xb0, 0xfc, 0x71, 0x10, 0x44, 0x58, 0x5b, 0xd1, 0x08, + 0xeb, 0xf6, 0x58, 0x84, 0xe5, 0x37, 0x7f, 0x08, 0x62, 0x6d, 0x44, 0x42, 0xac, 0x5b, 0xe3, 0x20, + 0x96, 0x5f, 0xb8, 0x20, 0xc6, 0x7a, 0x39, 0x14, 0x63, 0xdd, 0x9b, 0x04, 0x63, 0x79, 0x42, 0x07, + 0x40, 0xd6, 0x67, 0xc3, 0x41, 0xd6, 0x37, 0xce, 0x71, 0xb7, 0x64, 0x24, 0xca, 0xfa, 0xd6, 0x00, + 0xca, 0x12, 0x47, 0xa1, 0x2c, 0xbf, 0x3f, 0xbb, 0x30, 0x4b, 0x19, 0x09, 0x8a, 0xde, 0x9d, 0x10, + 0x14, 0xf9, 0x9d, 0x2f, 0x02, 0x15, 0x55, 0x23, 0x50, 0xd1, 0x8d, 0xd1, 0xa8, 0xc8, 0x37, 0xe7, + 0x3e, 0x2c, 0xaa, 0x45, 0xc1, 0xa2, 0x9b, 0x63, 0x60, 0x91, 0x6f, 0x85, 0x02, 0xb8, 0xe8, 0x51, + 0x1f, 0x2e, 0xba, 0x36, 0x36, 0xc0, 0x2a, 0x00, 0x8c, 0x56, 0x07, 0x81, 0xd1, 0xf5, 0x91, 0xc0, + 0xc8, 0x93, 0xe0, 0x23, 0xa3, 0x47, 0x7d, 0xc8, 0xe8, 0xda, 0x08, 0x64, 0xe4, 0x17, 0x80, 0x43, + 0x23, 0x6d, 0x34, 0x34, 0x5a, 0x9a, 0x14, 0x1a, 0x79, 0x82, 0x23, 0xb1, 0xd1, 0x56, 0x34, 0x36, + 0xba, 0x3d, 0x61, 0xec, 0xc3, 0x00, 0x38, 0xaa, 0x45, 0x81, 0xa3, 0x9b, 0x63, 0xc0, 0x51, 0x70, + 0x0e, 0xf1, 0xd0, 0x51, 0x2d, 0x0a, 0x1d, 0xdd, 0x1c, 0x83, 0x8e, 0x7c, 0x49, 0x01, 0x78, 0xd4, + 0x1c, 0x06, 0x8f, 0xee, 0x4e, 0x00, 0x8f, 0x7c, 0xe7, 0xa5, 0x0f, 0x1f, 0x7d, 0xd2, 0x8f, 0x8f, + 0xc4, 0x51, 0xf8, 0xc8, 0x1f, 0x91, 0x2e, 0x40, 0xda, 0x8a, 0x06, 0x48, 0xb7, 0xc7, 0x02, 0xa4, + 0xa0, 0x91, 0x0c, 0x20, 0xa4, 0x8d, 0x48, 0x84, 0x74, 0x6b, 0x1c, 0x42, 0xf2, 0x8d, 0x64, 0x10, + 0x22, 0x7d, 0xd2, 0x0f, 0x91, 0xc4, 0x51, 0x10, 0xc9, 0xaf, 0x9c, 0x8b, 0x91, 0x6a, 0x51, 0x18, + 0xe9, 0xe6, 0x18, 0x8c, 0xe4, 0x37, 0x5e, 0x00, 0x24, 0x29, 0x23, 0x41, 0xd2, 0xbb, 0x13, 0x82, + 0xa4, 0x3e, 0xc3, 0x15, 0x46, 0x49, 0xb5, 0x28, 0x94, 0x74, 0x73, 0x0c, 0x4a, 0x0a, 0x14, 0xd6, + 0x87, 0x49, 0x5b, 0xd1, 0x30, 0xe9, 0xf6, 0x58, 0x98, 0xd4, 0x37, 0x9a, 0x5c, 0x9c, 0xb4, 0x11, + 0x89, 0x93, 0x6e, 0x8d, 0xc3, 0x49, 0x7d, 0x13, 0x1f, 0x77, 0x0e, 0x7e, 0x71, 0x72, 0xa0, 0xf4, + 0xe1, 0xf9, 0x81, 0x92, 0xf7, 0xce, 0x31, 0x48, 0xe9, 0x93, 0x7e, 0xa4, 0x24, 0x8e, 0x42, 0x4a, + 0x7e, 0xcf, 0x3a, 0x1f, 0x54, 0x5a, 0x4f, 0xa5, 0xdf, 0x2a, 0xbe, 0x2d, 0xfe, 0x8d, 0x59, 0x98, + 0xa9, 0x79, 0x21, 0x49, 0x7e, 0x2d, 0x85, 0xd7, 0xb9, 0x03, 0x0b, 0xad, 0x91, 0x11, 0x4f, 0xed, + 0xe6, 0xf8, 0x6b, 0x0d, 0x07, 0x2f, 0xde, 0xe3, 0xac, 0xaf, 0x71, 0x18, 0x1d, 0xbd, 0x0f, 0xb9, + 0x9e, 0x8d, 0x2d, 0xb9, 0x6b, 0xe9, 0xa6, 0xa5, 0x3b, 0xec, 0xc0, 0x8d, 0xb0, 0x5a, 0xfc, 0xe2, + 0x74, 0x71, 0x6e, 0xc7, 0xc6, 0xd6, 0x36, 0xa7, 0x4b, 0x73, 0xbd, 0xc0, 0x93, 0xfb, 0x09, 0xb2, + 0xe9, 0xc9, 0x3f, 0x41, 0xf6, 0x1c, 0x8a, 0x16, 0x56, 0xb4, 0x90, 0x07, 0xc3, 0xee, 0x9b, 0x8a, + 0xee, 0x73, 0xf4, 0x40, 0x9c, 0x9b, 0x93, 0xde, 0x3b, 0x55, 0xb0, 0xc2, 0x44, 0x74, 0x1f, 0x2e, + 0x76, 0x94, 0x63, 0x1a, 0xbe, 0x2a, 0xbb, 0x4e, 0x21, 0x0d, 0x49, 0x65, 0x5f, 0xf7, 0x42, 0x1d, + 0xe5, 0x98, 0x7e, 0xcf, 0x8c, 0x25, 0xd1, 0x8f, 0x91, 0xdc, 0x84, 0xbc, 0xa6, 0xdb, 0x8e, 0x6e, + 0xa8, 0x0e, 0xbf, 0x67, 0x98, 0xdd, 0xd1, 0x9b, 0x73, 0xa9, 0xec, 0x32, 0xe1, 0x7b, 0x30, 0xcf, + 0xcf, 0x35, 0x04, 0x36, 0x63, 0xf9, 0x5d, 0xbd, 0x2c, 0xc1, 0xdb, 0x7f, 0x45, 0x15, 0x28, 0xb4, + 0x14, 0x07, 0x1f, 0x29, 0x27, 0xb2, 0x7b, 0x6a, 0x2e, 0x4b, 0xaf, 0xdf, 0x7c, 0xf3, 0xec, 0x74, + 0x31, 0xf7, 0x84, 0x25, 0x0d, 0x1c, 0x9e, 0xcb, 0xb5, 0x02, 0x09, 0x1a, 0xba, 0x0d, 0x05, 0xc5, + 0x3e, 0x31, 0x54, 0xaa, 0x1e, 0x6c, 0xd8, 0x3d, 0x9b, 0x42, 0x92, 0xb4, 0x94, 0xa7, 0xe4, 0x8a, + 0x4b, 0x45, 0xd7, 0x60, 0x8e, 0x07, 0xfd, 0xb3, 0x0f, 0x24, 0x15, 0x68, 0x55, 0xf9, 0x97, 0x37, + 0xd8, 0x37, 0x92, 0x1e, 0x41, 0x99, 0x7f, 0x47, 0xe0, 0x48, 0xb1, 0x34, 0x99, 0x6a, 0xdd, 0xef, + 0x9f, 0x45, 0x2a, 0xf6, 0x32, 0xfb, 0x6e, 0x00, 0xc9, 0x40, 0x54, 0xed, 0x8f, 0xa1, 0x2d, 0x98, + 0x57, 0xdb, 0xba, 0x87, 0x20, 0x58, 0xcd, 0xe7, 0x87, 0x8e, 0xa6, 0x0a, 0xcd, 0xeb, 0x6f, 0x46, + 0x17, 0xd4, 0x30, 0x01, 0x35, 0x80, 0xde, 0xec, 0x23, 0x77, 0xcd, 0xb6, 0xae, 0x9e, 0x50, 0xf0, + 0x10, 0xbe, 0x0f, 0x7d, 0xe4, 0x57, 0x09, 0x5e, 0x2a, 0xba, 0xb3, 0x4d, 0x39, 0x25, 0x38, 0xf2, + 0x7e, 0xb3, 0xbb, 0x8c, 0xd7, 0x53, 0xe9, 0xb9, 0x62, 0x6e, 0x3d, 0x95, 0xce, 0x17, 0x0b, 0xe2, + 0xdf, 0x14, 0xa0, 0xd0, 0x57, 0x16, 0x54, 0x83, 0x8b, 0x9a, 0x37, 0x54, 0x64, 0x7e, 0x2a, 0x4c, + 0x37, 0x0d, 0x7e, 0xc5, 0xfb, 0xc2, 0x17, 0xa7, 0x8b, 0x05, 0x9a, 0xfb, 0x89, 0x97, 0x24, 0x5d, + 0xf0, 0x39, 0x7c, 0x2a, 0xfa, 0x10, 0xf2, 0xcc, 0xfd, 0xf4, 0xbe, 0xf9, 0x47, 0xc3, 0xf9, 0x57, + 0xe7, 0xbf, 0x38, 0x5d, 0xcc, 0x51, 0x9f, 0xd3, 0xbd, 0x82, 0x59, 0xca, 0xb5, 0x83, 0x8f, 0xe2, + 0xaf, 0x09, 0x30, 0x17, 0x3a, 0x79, 0xf5, 0xa8, 0x6f, 0xf3, 0xfb, 0x8d, 0x68, 0xdc, 0x3a, 0x2c, + 0xf6, 0x31, 0xcd, 0xfb, 0xb9, 0x1b, 0x48, 0xba, 0x38, 0x1c, 0xf7, 0xd0, 0x55, 0x1c, 0x37, 0xaa, + 0xc6, 0x65, 0xfb, 0x28, 0xf5, 0xfd, 0x1f, 0x2c, 0x4e, 0x89, 0x7f, 0x9e, 0x82, 0x5c, 0xf8, 0x9c, + 0x55, 0xbd, 0xaf, 0x5c, 0x51, 0xf3, 0x4a, 0x88, 0x63, 0x69, 0xc4, 0x15, 0x95, 0x19, 0xff, 0x5b, + 0x0a, 0xac, 0x98, 0x57, 0x47, 0x6c, 0xf1, 0x07, 0xcb, 0xe9, 0x33, 0x96, 0xff, 0x43, 0xd2, 0xb3, + 0xaf, 0x4b, 0x30, 0x4d, 0xaf, 0x4c, 0xe2, 0x45, 0x2b, 0xf5, 0xf7, 0x1e, 0xe2, 0x6b, 0x93, 0x74, + 0x89, 0x65, 0x23, 0xf6, 0xb8, 0xf9, 0x5a, 0x77, 0x12, 0xfa, 0xc3, 0xe0, 0xfc, 0x9f, 0x58, 0xec, + 0xb1, 0x3b, 0x29, 0xff, 0x3f, 0x46, 0xde, 0x90, 0xf7, 0xa1, 0x9f, 0x87, 0x82, 0x6a, 0xb6, 0xdb, + 0x6c, 0xae, 0x65, 0x16, 0x6d, 0xf0, 0x96, 0x1a, 0x5a, 0x04, 0xfe, 0x55, 0xcd, 0x25, 0xef, 0xeb, + 0x9a, 0x4b, 0x12, 0xff, 0xba, 0x66, 0x20, 0xac, 0x37, 0xef, 0x09, 0x63, 0x86, 0xb0, 0x2f, 0xc2, + 0x78, 0xf6, 0x75, 0x22, 0x8c, 0x59, 0x54, 0x3a, 0xef, 0x79, 0x7f, 0x24, 0xf0, 0x10, 0x9e, 0xa7, + 0xa6, 0x79, 0xd0, 0xf3, 0x22, 0x83, 0xcb, 0xc1, 0x1b, 0x26, 0xd3, 0x5f, 0x9c, 0x2e, 0xa6, 0x24, + 0xef, 0x8a, 0xc9, 0xa8, 0x99, 0x23, 0xf1, 0xe5, 0x66, 0x8e, 0x6b, 0x30, 0xd7, 0xb5, 0xf0, 0x1e, + 0x76, 0xd4, 0x7d, 0xd9, 0xe8, 0x75, 0xf8, 0x01, 0xa2, 0xac, 0x4b, 0xdb, 0xea, 0x75, 0xd0, 0x5d, + 0x28, 0x7a, 0x59, 0x38, 0xc6, 0x77, 0xaf, 0x20, 0x73, 0xe9, 0x7c, 0x45, 0x40, 0xfc, 0xdf, 0x02, + 0x2c, 0x84, 0xea, 0xc4, 0xc7, 0xd4, 0x3a, 0x64, 0x7d, 0x73, 0x62, 0x97, 0x84, 0x73, 0xc6, 0xc9, + 0x06, 0x99, 0x91, 0x0c, 0x97, 0xdc, 0xd7, 0xd2, 0xef, 0x0f, 0xf8, 0x62, 0x13, 0xe7, 0x14, 0x7b, + 0xd1, 0x97, 0xb3, 0x16, 0x78, 0x81, 0x37, 0xc8, 0x92, 0x13, 0x0d, 0x32, 0xf1, 0x37, 0x05, 0x28, + 0xd2, 0x17, 0x3c, 0xc6, 0x58, 0x8b, 0xc5, 0xba, 0xb9, 0xf1, 0xe7, 0x89, 0xc9, 0x8f, 0x08, 0x85, + 0xbe, 0x99, 0x92, 0x0c, 0x7f, 0x33, 0x45, 0xfc, 0x81, 0x00, 0x79, 0xaf, 0x84, 0xec, 0x3b, 0x85, + 0x23, 0x2e, 0x32, 0x7d, 0xbd, 0x6f, 0xf1, 0xb9, 0x17, 0xae, 0x4c, 0xf4, 0xe9, 0xc4, 0xe0, 0x85, + 0x2b, 0xec, 0x1b, 0x72, 0x7f, 0xdb, 0xed, 0x39, 0xa4, 0x88, 0x15, 0xff, 0x62, 0x8d, 0xd7, 0x38, + 0x2d, 0x25, 0xd1, 0x4f, 0xbc, 0x9a, 0xed, 0x43, 0x76, 0x47, 0xce, 0x44, 0x66, 0x0f, 0xf1, 0xc0, + 0x35, 0xe0, 0xab, 0x81, 0x5a, 0xb3, 0x41, 0x3f, 0xfe, 0xca, 0x7e, 0xdb, 0xe2, 0xe3, 0x80, 0x02, + 0x69, 0xe3, 0x13, 0x2d, 0x4d, 0x64, 0x8a, 0x5d, 0x2d, 0xb1, 0xbe, 0xf2, 0x87, 0xc1, 0x96, 0xa8, + 0x1e, 0x12, 0x14, 0xf8, 0x10, 0x92, 0x87, 0x4a, 0x7b, 0x54, 0x04, 0x58, 0xa8, 0xe5, 0x24, 0x92, + 0x1b, 0x3d, 0x0e, 0xdd, 0x47, 0x92, 0x18, 0x8e, 0x58, 0x06, 0x55, 0x1a, 0xba, 0xb7, 0xe4, 0x83, + 0x70, 0x5f, 0x1f, 0xf9, 0xfa, 0x60, 0xa7, 0xff, 0x28, 0xf5, 0xc3, 0x1f, 0x2c, 0x0a, 0xe2, 0xc7, + 0x80, 0x24, 0x6c, 0x63, 0xe7, 0x79, 0xcf, 0xb4, 0xfc, 0xbb, 0x5d, 0x6e, 0xf5, 0x7d, 0x44, 0x66, + 0x7a, 0x35, 0x7b, 0x16, 0x75, 0xfd, 0xd3, 0x45, 0x58, 0x08, 0x71, 0x33, 0x63, 0x21, 0x7e, 0x00, + 0x6f, 0x3c, 0x31, 0x6d, 0x5b, 0xef, 0x12, 0xe8, 0x4b, 0x47, 0x25, 0x99, 0x5a, 0x3c, 0xf3, 0x98, + 0xee, 0xd2, 0x45, 0x10, 0x83, 0x99, 0x91, 0x8c, 0xe4, 0x3d, 0x8b, 0xbf, 0x27, 0xc0, 0xe5, 0x41, + 0x4e, 0xa6, 0xe5, 0xa8, 0xc3, 0x9d, 0xb3, 0xaa, 0xe9, 0xdf, 0x05, 0x38, 0xbe, 0xb7, 0xba, 0xd9, + 0x89, 0x8b, 0xca, 0xdf, 0x29, 0x77, 0x14, 0x6a, 0x3e, 0xf8, 0x41, 0xf3, 0x3c, 0x27, 0x6f, 0x32, + 0xaa, 0x6f, 0x49, 0x52, 0x93, 0x59, 0x92, 0x26, 0x14, 0xd6, 0x4d, 0xdd, 0x20, 0x9e, 0xb0, 0x5b, + 0xdf, 0x15, 0xc8, 0xef, 0xea, 0x86, 0x62, 0x9d, 0xc8, 0x6e, 0xe0, 0xa1, 0x30, 0x2e, 0xf0, 0x50, + 0xca, 0x31, 0x0e, 0xfe, 0x28, 0xfe, 0x48, 0x80, 0xa2, 0x2f, 0x96, 0x5b, 0xe4, 0x77, 0x00, 0xd4, + 0x76, 0xcf, 0x76, 0xb0, 0xe5, 0xb6, 0xd2, 0x1c, 0x0b, 0xd4, 0xaf, 0x30, 0x6a, 0x7d, 0x4d, 0xca, + 0xf0, 0x0c, 0x75, 0x0d, 0x5d, 0x0f, 0xdf, 0x83, 0x31, 0xbd, 0x0a, 0x67, 0x03, 0xb7, 0x5f, 0x90, + 0x66, 0xb7, 0x1d, 0xd3, 0xf2, 0x30, 0x17, 0x6f, 0x76, 0xf7, 0xda, 0x7d, 0x7a, 0x7c, 0x9b, 0xe4, + 0x5b, 0x81, 0x3c, 0x71, 0x17, 0x0e, 0xb1, 0x57, 0xa5, 0xd4, 0xf8, 0x2a, 0x31, 0x0e, 0xb7, 0x4a, + 0xbf, 0x43, 0x9c, 0x5c, 0xd6, 0x1a, 0x5e, 0x0b, 0x8f, 0xb0, 0x68, 0x3f, 0x15, 0xbc, 0xbe, 0x72, + 0xb2, 0x0b, 0x3e, 0xa9, 0x27, 0xf3, 0x2d, 0x48, 0xbb, 0x1f, 0xcf, 0xe6, 0x03, 0xe4, 0x8d, 0x25, + 0xf6, 0x75, 0xed, 0x25, 0xf7, 0xeb, 0xda, 0x4b, 0x6b, 0x3c, 0x03, 0x33, 0xe3, 0xdf, 0xff, 0xaf, + 0x8b, 0x82, 0xe4, 0x31, 0xdd, 0x6b, 0x90, 0x1e, 0x3e, 0x30, 0x0b, 0xa3, 0x3c, 0x40, 0xe0, 0x7b, + 0x43, 0xfc, 0x7b, 0xca, 0x2b, 0x6b, 0xf2, 0xce, 0x56, 0xe5, 0xd9, 0xe6, 0x66, 0xbd, 0xd9, 0xac, + 0xae, 0x15, 0x05, 0x54, 0x84, 0xb9, 0xd0, 0xd7, 0x8a, 0x12, 0xec, 0x0b, 0xcb, 0xf7, 0x7e, 0x0a, + 0xc0, 0xff, 0xf0, 0x19, 0x91, 0xb5, 0x51, 0xfd, 0x54, 0x7e, 0xb1, 0xf2, 0x74, 0xa7, 0xda, 0x28, + 0x4e, 0x21, 0x04, 0xf9, 0xd5, 0x95, 0x66, 0xa5, 0x26, 0x4b, 0xd5, 0xc6, 0xf6, 0xb3, 0xad, 0x46, + 0xd5, 0xfd, 0x32, 0xf3, 0xbd, 0x35, 0x98, 0x0b, 0xde, 0x6c, 0x84, 0x16, 0xa0, 0x50, 0xa9, 0x55, + 0x2b, 0x1b, 0xf2, 0x8b, 0xfa, 0x8a, 0xfc, 0x7c, 0xa7, 0xba, 0x53, 0x2d, 0x4e, 0xd1, 0xa2, 0x51, + 0xe2, 0xe3, 0x9d, 0xa7, 0x4f, 0x8b, 0x02, 0x2a, 0x40, 0x96, 0x3d, 0xd3, 0x2f, 0x1b, 0x15, 0x13, + 0xf7, 0x36, 0x21, 0x1b, 0xb8, 0x02, 0x99, 0xbc, 0x6e, 0x7b, 0xa7, 0x51, 0x93, 0x9b, 0xf5, 0xcd, + 0x6a, 0xa3, 0xb9, 0xb2, 0xb9, 0xcd, 0x64, 0x50, 0xda, 0xca, 0xea, 0x33, 0xa9, 0x59, 0x14, 0xbc, + 0xe7, 0xe6, 0xb3, 0x9d, 0x4a, 0xcd, 0xad, 0x86, 0x98, 0x4a, 0x27, 0x8b, 0xc9, 0x7b, 0x7f, 0x55, + 0x80, 0xcb, 0x43, 0x6e, 0xf9, 0x41, 0x59, 0x98, 0xdd, 0x31, 0xe8, 0x7d, 0xb0, 0xc5, 0x29, 0x94, + 0x0b, 0x5c, 0xf4, 0x53, 0x14, 0x50, 0x9a, 0x5d, 0xb5, 0x52, 0x4c, 0xa0, 0x19, 0x48, 0x34, 0x1e, + 0x16, 0x93, 0xa4, 0xa4, 0x81, 0x7b, 0x72, 0x8a, 0x29, 0x94, 0xe1, 0x37, 0x74, 0x14, 0xa7, 0xd1, + 0x9c, 0x7f, 0x51, 0x46, 0x71, 0x86, 0x88, 0xf2, 0x2e, 0x9c, 0x28, 0xce, 0xde, 0xbb, 0x06, 0x81, + 0x43, 0xfd, 0x08, 0x60, 0xe6, 0xa9, 0xe2, 0x60, 0xdb, 0x29, 0x4e, 0xa1, 0x59, 0x48, 0xae, 0xb4, + 0xdb, 0x45, 0xe1, 0xc1, 0xbf, 0x4c, 0x41, 0xda, 0xfd, 0x82, 0x0f, 0x7a, 0x0a, 0xd3, 0x6c, 0xe9, + 0x7b, 0x71, 0x38, 0x32, 0xa0, 0x83, 0xb7, 0x7c, 0x75, 0x1c, 0x74, 0x10, 0xa7, 0xd0, 0x5f, 0x86, + 0x6c, 0xc0, 0x63, 0x42, 0x43, 0x97, 0xef, 0x42, 0x5e, 0x62, 0xf9, 0xd6, 0xb8, 0x6c, 0x9e, 0xfc, + 0x97, 0x90, 0xf1, 0x2c, 0x38, 0xba, 0x3e, 0xca, 0xbe, 0xbb, 0xb2, 0x47, 0x4f, 0x02, 0x64, 0xac, + 0x89, 0x53, 0xef, 0x09, 0xc8, 0x02, 0x34, 0x68, 0x6c, 0x51, 0x54, 0x40, 0xc5, 0x50, 0x6b, 0x5e, + 0xbe, 0x37, 0x51, 0x6e, 0xff, 0x9d, 0x44, 0x59, 0xfe, 0x8c, 0x11, 0xad, 0xac, 0x81, 0xf9, 0x28, + 0x5a, 0x59, 0x11, 0x13, 0xcf, 0x14, 0x7a, 0x0e, 0x29, 0x62, 0x29, 0x51, 0x94, 0x0f, 0xd9, 0x67, + 0x99, 0xcb, 0xd7, 0x47, 0xe6, 0x71, 0x45, 0xae, 0xde, 0xfd, 0xe1, 0x9f, 0x5e, 0x99, 0xfa, 0xe1, + 0xd9, 0x15, 0xe1, 0x47, 0x67, 0x57, 0x84, 0x3f, 0x3e, 0xbb, 0x22, 0xfc, 0xc9, 0xd9, 0x15, 0xe1, + 0x7b, 0x3f, 0xbe, 0x32, 0xf5, 0xa3, 0x1f, 0x5f, 0x99, 0xfa, 0xe3, 0x1f, 0x5f, 0x99, 0xfa, 0x6c, + 0x96, 0x73, 0xef, 0xce, 0x50, 0xa3, 0xf2, 0xf0, 0xff, 0x05, 0x00, 0x00, 0xff, 0xff, 0x1e, 0x03, + 0x54, 0xaa, 0xed, 0x80, 0x00, 0x00, } diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index 987bd8ffa84e..8eedfe591430 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -838,6 +838,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]; } @@ -1143,7 +1146,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() @@ -1647,7 +1651,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 @@ -2025,7 +2030,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..a98c6df2cc1d 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. @@ -63,7 +65,7 @@ var flagStringsInverted = func() map[string]TimestampFlag { return m }() -// String implements the fmt.Formatter interface. +// String implements the fmt.Stringer interface. func (t Timestamp) String() string { // The following code was originally written as // fmt.Sprintf("%d.%09d,%d", t.WallTime/1e9, t.WallTime%1e9, t.Logical). @@ -363,6 +365,92 @@ func (t LegacyTimestamp) Less(s LegacyTimestamp) bool { return t.ToTimestamp().Less(s.ToTimestamp()) } +// String implements the fmt.Stringer 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.Stringer 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; } From ae2edd29d7a7dbfb4bb09bfe9826ac69f85203aa Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Tue, 22 Dec 2020 17:17:30 -0500 Subject: [PATCH 2/7] hlc: don't mutate WallTime or Logical fields directly Instead, use methods that will ensure proper interactions with the upcoming Synthetic field. --- pkg/kv/kvserver/closedts/provider/provider.go | 2 +- pkg/kv/kvserver/tscache/interval_skl.go | 3 +-- pkg/kv/kvserver/tscache/tree_impl.go | 3 +-- pkg/sql/catalog/lease/lease.go | 3 +-- pkg/util/hlc/timestamp.go | 2 ++ pkg/util/hlc/timestamp.pb.go | 16 ++++++++++++---- pkg/util/hlc/timestamp.proto | 8 ++++++++ 7 files changed, 26 insertions(+), 11 deletions(-) diff --git a/pkg/kv/kvserver/closedts/provider/provider.go b/pkg/kv/kvserver/closedts/provider/provider.go index 33fb0b4243d8..4d6125c2d0e3 100644 --- a/pkg/kv/kvserver/closedts/provider/provider.go +++ b/pkg/kv/kvserver/closedts/provider/provider.go @@ -149,7 +149,7 @@ func (p *Provider) runCloser(ctx context.Context) { } next, liveAtEpoch, err := p.cfg.Clock(p.cfg.NodeID) - next.WallTime -= int64(targetDuration) + next = next.Add(-int64(targetDuration), 0) if err != nil { if everBeenLive && p.everyClockLog.ShouldLog() { log.Warningf(ctx, "unable to move closed timestamp forward: %+v", err) diff --git a/pkg/kv/kvserver/tscache/interval_skl.go b/pkg/kv/kvserver/tscache/interval_skl.go index 53abefdc7eb3..88d5217c4bbb 100644 --- a/pkg/kv/kvserver/tscache/interval_skl.go +++ b/pkg/kv/kvserver/tscache/interval_skl.go @@ -441,8 +441,7 @@ func (s *intervalSkl) rotatePages(filledPage *sklPage) { // window. minTSToRetain := hlc.MaxTimestamp if s.clock != nil { - minTSToRetain = s.clock.Now() - minTSToRetain.WallTime -= s.minRet.Nanoseconds() + minTSToRetain = s.clock.Now().Add(-s.minRet.Nanoseconds(), 0) } // Iterate over the pages in reverse, evicting pages that are no longer diff --git a/pkg/kv/kvserver/tscache/tree_impl.go b/pkg/kv/kvserver/tscache/tree_impl.go index 807ff3638672..cbc05ef01c00 100644 --- a/pkg/kv/kvserver/tscache/tree_impl.go +++ b/pkg/kv/kvserver/tscache/tree_impl.go @@ -496,8 +496,7 @@ func (tc *treeImpl) shouldEvict(size int, key, value interface{}) bool { return true } // Compute the edge of the cache window. - edge := tc.latest - edge.WallTime -= MinRetentionWindow.Nanoseconds() + edge := tc.latest.Add(-MinRetentionWindow.Nanoseconds(), 0) // We evict and update the low water mark if the proposed evictee's // timestamp is <= than the edge of the window. if ce.ts.LessEq(edge) { diff --git a/pkg/sql/catalog/lease/lease.go b/pkg/sql/catalog/lease/lease.go index 96c6120d08f1..ef654e5a11a2 100644 --- a/pkg/sql/catalog/lease/lease.go +++ b/pkg/sql/catalog/lease/lease.go @@ -199,8 +199,7 @@ func (s storage) acquire( if err := txn.SetUserPriority(roachpb.MaxUserPriority); err != nil { return err } - expiration := txn.ReadTimestamp() - expiration.WallTime += int64(s.jitteredLeaseDuration()) + expiration := txn.ReadTimestamp().Add(int64(s.jitteredLeaseDuration()), 0) if expiration.LessEq(minExpiration) { // In the rare circumstances where expiration <= minExpiration // use an expiration based on the minExpiration to guarantee diff --git a/pkg/util/hlc/timestamp.go b/pkg/util/hlc/timestamp.go index a98c6df2cc1d..15cb6788d0e3 100644 --- a/pkg/util/hlc/timestamp.go +++ b/pkg/util/hlc/timestamp.go @@ -217,6 +217,8 @@ func (t Timestamp) IsFlagSet(f TimestampFlag) bool { // Add returns a timestamp with the WallTime and Logical components increased. // wallTime is expressed in nanos. +// +// TODO(nvanbenschoten): consider an AddNanos method that takes a time.Duration. func (t Timestamp) Add(wallTime int64, logical int32) Timestamp { return Timestamp{ WallTime: t.WallTime + wallTime, diff --git a/pkg/util/hlc/timestamp.pb.go b/pkg/util/hlc/timestamp.pb.go index e727fcd03317..65050bbff5b5 100644 --- a/pkg/util/hlc/timestamp.pb.go +++ b/pkg/util/hlc/timestamp.pb.go @@ -54,17 +54,25 @@ func (x TimestampFlag) String() string { return proto.EnumName(TimestampFlag_name, int32(x)) } func (TimestampFlag) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_timestamp_3d06174efd2169e7, []int{0} + return fileDescriptor_timestamp_2a42725d81a17263, []int{0} } // Timestamp represents a state of the hybrid logical clock. type Timestamp struct { // Holds a wall time, typically a unix epoch time expressed in // nanoseconds. + // + // It is not safe to mutate this field directly. Instead, use one of the + // methods on Timestamp, which ensure that the synthetic flag is updated + // appropriately. WallTime int64 `protobuf:"varint,1,opt,name=wall_time,json=wallTime,proto3" json:"wall_time,omitempty"` // The logical component captures causality for events whose wall times // are equal. It is effectively bounded by (maximum clock skew)/(minimal // ns between events) and nearly impossible to overflow. + // + // It is not safe to mutate this field directly. Instead, use one of the + // methods on Timestamp, which ensure that the synthetic flag is updated + // appropriately. Logical int32 `protobuf:"varint,2,opt,name=logical,proto3" json:"logical,omitempty"` // A collection of bit flags that provide details about the timestamp // and its meaning. The data type is a uint32, but the number of flags @@ -88,7 +96,7 @@ type Timestamp struct { func (m *Timestamp) Reset() { *m = Timestamp{} } func (*Timestamp) ProtoMessage() {} func (*Timestamp) Descriptor() ([]byte, []int) { - return fileDescriptor_timestamp_3d06174efd2169e7, []int{0} + return fileDescriptor_timestamp_2a42725d81a17263, []int{0} } func (m *Timestamp) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -521,10 +529,10 @@ var ( ) func init() { - proto.RegisterFile("util/hlc/timestamp.proto", fileDescriptor_timestamp_3d06174efd2169e7) + proto.RegisterFile("util/hlc/timestamp.proto", fileDescriptor_timestamp_2a42725d81a17263) } -var fileDescriptor_timestamp_3d06174efd2169e7 = []byte{ +var fileDescriptor_timestamp_2a42725d81a17263 = []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 c1947b4c2ed8..e3711d78c635 100644 --- a/pkg/util/hlc/timestamp.proto +++ b/pkg/util/hlc/timestamp.proto @@ -23,10 +23,18 @@ message Timestamp { // Holds a wall time, typically a unix epoch time expressed in // nanoseconds. + // + // It is not safe to mutate this field directly. Instead, use one of the + // methods on Timestamp, which ensure that the synthetic flag is updated + // appropriately. int64 wall_time = 1; // The logical component captures causality for events whose wall times // are equal. It is effectively bounded by (maximum clock skew)/(minimal // ns between events) and nearly impossible to overflow. + // + // It is not safe to mutate this field directly. Instead, use one of the + // methods on Timestamp, which ensure that the synthetic flag is updated + // appropriately. int32 logical = 2; // A collection of bit flags that provide details about the timestamp // and its meaning. The data type is a uint32, but the number of flags From 2b07912e1d2f950de70143d375843ba2f2424ab9 Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Mon, 23 Nov 2020 15:02:45 -0500 Subject: [PATCH 3/7] storage: use hlc.ParseTimestamp in TestMVCCHistories Adapted from #57077. Also switch over TestIntentInterleavingIter and TestIntentDemuxWriter. No need to re-write the parsing logic again. We'll also want this to use flags in the next commit. This causes a large diff because all nanos are now considered seconds. This doesn't actually change any behavior in the tests themselves. --- pkg/storage/intent_interleaving_iter_test.go | 41 +- pkg/storage/intent_reader_writer_test.go | 12 +- pkg/storage/mvcc_history_test.go | 16 +- pkg/storage/testdata/intent_demux_writer | 272 +++++------ .../testdata/intent_interleaving_iter/basic | 336 +++++++------- .../intent_interleaving_iter/error_race_off | 18 +- .../testdata/mvcc_histories/clear_range | 38 +- .../testdata/mvcc_histories/conditional_put | 52 +-- ...onditional_put_with_txn_disallow_separated | 46 +- .../conditional_put_write_too_old | 28 +- .../testdata/mvcc_histories/delete_range | 172 +++---- pkg/storage/testdata/mvcc_histories/deletes | 4 +- .../mvcc_histories/get_negative_timestamp | 8 +- .../mvcc_histories/idempotent_transactions | 86 ++-- .../testdata/mvcc_histories/ignored_seq_nums | 430 +++++++++--------- .../mvcc_histories/ignored_seq_nums_commit | 16 +- .../mvcc_histories/ignored_seq_nums_cput | 98 ++-- pkg/storage/testdata/mvcc_histories/increment | 26 +- .../intent_history_disallow_separated | 38 +- ...tent_with_write_tracing_disallow_separated | 70 +-- pkg/storage/testdata/mvcc_histories/max_keys | 156 +++---- pkg/storage/testdata/mvcc_histories/merges | 8 +- .../no_read_after_abort_disallow_separated | 6 +- .../mvcc_histories/put_after_rollback | 54 +-- .../put_new_epoch_lower_sequence | 18 +- .../put_new_epoch_lower_timestamp | 16 +- .../testdata/mvcc_histories/put_out_of_order | 22 +- .../read_after_write_disallow_separated | 54 +-- .../mvcc_histories/read_fail_on_more_recent | 52 +-- .../testdata/mvcc_histories/target_bytes | 168 +++---- .../mvcc_histories/uncertainty_interval | 122 ++--- .../mvcc_histories/update_existing_key | 4 +- .../update_existing_key_diff_txn | 8 +- .../mvcc_histories/update_existing_key_in_txn | 6 +- .../update_existing_key_old_version | 16 +- .../testdata/mvcc_histories/write_too_old | 12 +- .../mvcc_histories/write_with_sequence | 36 +- pkg/util/hlc/timestamp.go | 13 +- pkg/util/hlc/timestamp_test.go | 51 ++- 39 files changed, 1340 insertions(+), 1289 deletions(-) diff --git a/pkg/storage/intent_interleaving_iter_test.go b/pkg/storage/intent_interleaving_iter_test.go index 4c6b10b74679..fc268a78bba0 100644 --- a/pkg/storage/intent_interleaving_iter_test.go +++ b/pkg/storage/intent_interleaving_iter_test.go @@ -54,9 +54,13 @@ func makePrintableKey(k MVCCKey) MVCCKey { func scanSeekKey(t *testing.T, td *datadriven.TestData) MVCCKey { key := MVCCKey{Key: scanRoachKey(t, td, "k")} if td.HasArg("ts") { - var ts int - td.ScanArgs(t, "ts", &ts) - key.Timestamp.WallTime = int64(ts) + var tsS string + td.ScanArgs(t, "ts", &tsS) + ts, err := hlc.ParseTimestamp(tsS) + if err != nil { + t.Fatalf("%v", err) + } + key.Timestamp = ts } return key } @@ -139,13 +143,13 @@ func checkAndOutputIter(iter MVCCIterator, b *strings.Builder) { if uuid.Hi != 0 { hiStr = fmt.Sprintf("%d,", uuid.Hi) } - fmt.Fprintf(b, "output: meta k=%s ts=%d txn=%s%d\n", - string(k1.Key), meta.Timestamp.WallTime, hiStr, uuid.Lo) + fmt.Fprintf(b, "output: meta k=%s ts=%s txn=%s%d\n", + string(k1.Key), meta.Timestamp, hiStr, uuid.Lo) } return } - fmt.Fprintf(b, "output: value k=%s ts=%d v=%s\n", - string(k1.Key), k1.Timestamp.WallTime, string(v1)) + fmt.Fprintf(b, "output: value k=%s ts=%s v=%s\n", + string(k1.Key), k1.Timestamp, string(v1)) } // TestIntentInterleavingIter is a datadriven test consisting of two commands: @@ -223,9 +227,14 @@ func TestIntentInterleavingIter(t *testing.T) { var meta enginepb.MVCCMetadata var txnUUID uuid.UUID if locksSection || d.HasArg("ts") { - var ts, txn int - d.ScanArgs(t, "ts", &ts) - meta.Timestamp.WallTime = int64(ts) + var tsS string + d.ScanArgs(t, "ts", &tsS) + ts, err := hlc.ParseTimestamp(tsS) + if err != nil { + t.Fatalf("%v", err) + } + meta.Timestamp = ts.ToLegacyTimestamp() + var txn int d.ScanArgs(t, "txn", &txn) txnUUID = uuid.FromUint128(uint128.FromInts(0, uint64(txn))) meta.Txn = &enginepb.TxnMeta{ID: txnUUID} @@ -252,11 +261,15 @@ func TestIntentInterleavingIter(t *testing.T) { t.Fatalf("%s: value in locks section", d.Pos) } key := scanRoachKey(t, d, "k") - var ts int - d.ScanArgs(t, "ts", &ts) + var tsS string + d.ScanArgs(t, "ts", &tsS) + ts, err := hlc.ParseTimestamp(tsS) + if err != nil { + t.Fatalf("%v", err) + } var value string d.ScanArgs(t, "v", &value) - mvccKey := MVCCKey{Key: key, Timestamp: hlc.Timestamp{WallTime: int64(ts)}} + mvccKey := MVCCKey{Key: key, Timestamp: ts} if err := batch.PutMVCC(mvccKey, []byte(value)); err != nil { return err.Error() } @@ -442,7 +455,7 @@ func generateIterOps(rng *rand.Rand, mvcckv []MVCCKeyValue) []string { fwdDirection = false } if useTimestamp { - op = fmt.Sprintf("%s k=%s ts=%d", op, string(seekKey.Key), seekKey.Timestamp.WallTime) + op = fmt.Sprintf("%s k=%s ts=%s", op, string(seekKey.Key), seekKey.Timestamp) } else { op = fmt.Sprintf("%s k=%s", op, string(seekKey.Key)) } diff --git a/pkg/storage/intent_reader_writer_test.go b/pkg/storage/intent_reader_writer_test.go index 895224fef236..96847f7ab2be 100644 --- a/pkg/storage/intent_reader_writer_test.go +++ b/pkg/storage/intent_reader_writer_test.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" + "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/uint128" @@ -217,9 +218,14 @@ func TestIntentDemuxWriter(t *testing.T) { key := scanRoachKey(t, d, "k") // We don't bother populating most fields in the proto. var meta enginepb.MVCCMetadata - var ts, txn int - d.ScanArgs(t, "ts", &ts) - meta.Timestamp.WallTime = int64(ts) + var tsS string + d.ScanArgs(t, "ts", &tsS) + ts, err := hlc.ParseTimestamp(tsS) + if err != nil { + t.Fatalf("%v", err) + } + meta.Timestamp = ts.ToLegacyTimestamp() + var txn int d.ScanArgs(t, "txn", &txn) txnUUID := uuid.FromUint128(uint128.FromInts(0, uint64(txn))) meta.Txn = &enginepb.TxnMeta{ID: txnUUID} diff --git a/pkg/storage/mvcc_history_test.go b/pkg/storage/mvcc_history_test.go index ffd434b56ab1..b956ac5125e5 100644 --- a/pkg/storage/mvcc_history_test.go +++ b/pkg/storage/mvcc_history_test.go @@ -880,24 +880,10 @@ func (e *evalCtx) getTsWithName(txn *roachpb.Transaction, name string) hlc.Times } var tsS string e.scanArg(name, &tsS) - parts := strings.Split(tsS, ",") - - // Find the wall time part. - tsW, err := strconv.ParseInt(parts[0], 10, 64) + ts, err := hlc.ParseTimestamp(tsS) if err != nil { e.Fatalf("%v", err) } - ts.WallTime = tsW - - // Find the logical part, if there is one. - var tsL int64 - if len(parts) > 1 { - tsL, err = strconv.ParseInt(parts[1], 10, 32) - if err != nil { - e.Fatalf("%v", err) - } - } - ts.Logical = int32(tsL) return ts } diff --git a/pkg/storage/testdata/intent_demux_writer b/pkg/storage/testdata/intent_demux_writer index 5c8378554042..4ddd89e14bb4 100644 --- a/pkg/storage/testdata/intent_demux_writer +++ b/pkg/storage/testdata/intent_demux_writer @@ -9,61 +9,61 @@ new-writer enable-separated=false put-intent k=a ts=50 txn=1 preceding=none txn-did-not-update-meta=true ---- === Calls === -PutUnversioned(a, meta{ts: 0.000000050,0, txn: 1}) +PutUnversioned(a, meta{ts: 50.000000000,0, txn: 1}) === Storage contents === -k: "a"/0,0, v: meta{ts: 0.000000050,0, txn: 1} +k: "a"/0,0, v: meta{ts: 50.000000000,0, txn: 1} put-intent k=b ts=50 txn=2 preceding=interleaved txn-did-not-update-meta=true ---- === Calls === -PutUnversioned(b, meta{ts: 0.000000050,0, txn: 2}) +PutUnversioned(b, meta{ts: 50.000000000,0, txn: 2}) === Storage contents === -k: "a"/0,0, v: meta{ts: 0.000000050,0, txn: 1} -k: "b"/0,0, v: meta{ts: 0.000000050,0, txn: 2} +k: "a"/0,0, v: meta{ts: 50.000000000,0, txn: 1} +k: "b"/0,0, v: meta{ts: 50.000000000,0, txn: 2} put-intent k=c ts=50 txn=3 preceding=separated txn-did-not-update-meta=true ---- === Calls === SingleClearEngineKey(LT{k: c, strength: Exclusive, uuid:3}) -PutUnversioned(c, meta{ts: 0.000000050,0, txn: 3}) +PutUnversioned(c, meta{ts: 50.000000000,0, txn: 3}) === Storage contents === -k: "a"/0,0, v: meta{ts: 0.000000050,0, txn: 1} -k: "b"/0,0, v: meta{ts: 0.000000050,0, txn: 2} -k: "c"/0,0, v: meta{ts: 0.000000050,0, txn: 3} +k: "a"/0,0, v: meta{ts: 50.000000000,0, txn: 1} +k: "b"/0,0, v: meta{ts: 50.000000000,0, txn: 2} +k: "c"/0,0, v: meta{ts: 50.000000000,0, txn: 3} put-intent k=d ts=50 txn=4 preceding=none txn-did-not-update-meta=false ---- === Calls === -PutUnversioned(d, meta{ts: 0.000000050,0, txn: 4}) +PutUnversioned(d, meta{ts: 50.000000000,0, txn: 4}) === Storage contents === -k: "a"/0,0, v: meta{ts: 0.000000050,0, txn: 1} -k: "b"/0,0, v: meta{ts: 0.000000050,0, txn: 2} -k: "c"/0,0, v: meta{ts: 0.000000050,0, txn: 3} -k: "d"/0,0, v: meta{ts: 0.000000050,0, txn: 4} +k: "a"/0,0, v: meta{ts: 50.000000000,0, txn: 1} +k: "b"/0,0, v: meta{ts: 50.000000000,0, txn: 2} +k: "c"/0,0, v: meta{ts: 50.000000000,0, txn: 3} +k: "d"/0,0, v: meta{ts: 50.000000000,0, txn: 4} put-intent k=e ts=50 txn=5 preceding=interleaved txn-did-not-update-meta=false ---- === Calls === -PutUnversioned(e, meta{ts: 0.000000050,0, txn: 5}) +PutUnversioned(e, meta{ts: 50.000000000,0, txn: 5}) === Storage contents === -k: "a"/0,0, v: meta{ts: 0.000000050,0, txn: 1} -k: "b"/0,0, v: meta{ts: 0.000000050,0, txn: 2} -k: "c"/0,0, v: meta{ts: 0.000000050,0, txn: 3} -k: "d"/0,0, v: meta{ts: 0.000000050,0, txn: 4} -k: "e"/0,0, v: meta{ts: 0.000000050,0, txn: 5} +k: "a"/0,0, v: meta{ts: 50.000000000,0, txn: 1} +k: "b"/0,0, v: meta{ts: 50.000000000,0, txn: 2} +k: "c"/0,0, v: meta{ts: 50.000000000,0, txn: 3} +k: "d"/0,0, v: meta{ts: 50.000000000,0, txn: 4} +k: "e"/0,0, v: meta{ts: 50.000000000,0, txn: 5} put-intent k=f ts=50 txn=6 preceding=separated txn-did-not-update-meta=false ---- === Calls === ClearEngineKey(LT{k: f, strength: Exclusive, uuid:6}) -PutUnversioned(f, meta{ts: 0.000000050,0, txn: 6}) +PutUnversioned(f, meta{ts: 50.000000000,0, txn: 6}) === Storage contents === -k: "a"/0,0, v: meta{ts: 0.000000050,0, txn: 1} -k: "b"/0,0, v: meta{ts: 0.000000050,0, txn: 2} -k: "c"/0,0, v: meta{ts: 0.000000050,0, txn: 3} -k: "d"/0,0, v: meta{ts: 0.000000050,0, txn: 4} -k: "e"/0,0, v: meta{ts: 0.000000050,0, txn: 5} -k: "f"/0,0, v: meta{ts: 0.000000050,0, txn: 6} +k: "a"/0,0, v: meta{ts: 50.000000000,0, txn: 1} +k: "b"/0,0, v: meta{ts: 50.000000000,0, txn: 2} +k: "c"/0,0, v: meta{ts: 50.000000000,0, txn: 3} +k: "d"/0,0, v: meta{ts: 50.000000000,0, txn: 4} +k: "e"/0,0, v: meta{ts: 50.000000000,0, txn: 5} +k: "f"/0,0, v: meta{ts: 50.000000000,0, txn: 6} # clear an intent clear-intent k=f txn=6 preceding=interleaved txn-did-not-update-meta=false @@ -71,11 +71,11 @@ clear-intent k=f txn=6 preceding=interleaved txn-did-not-update-meta=false === Calls === ClearUnversioned(f) === Storage contents === -k: "a"/0,0, v: meta{ts: 0.000000050,0, txn: 1} -k: "b"/0,0, v: meta{ts: 0.000000050,0, txn: 2} -k: "c"/0,0, v: meta{ts: 0.000000050,0, txn: 3} -k: "d"/0,0, v: meta{ts: 0.000000050,0, txn: 4} -k: "e"/0,0, v: meta{ts: 0.000000050,0, txn: 5} +k: "a"/0,0, v: meta{ts: 50.000000000,0, txn: 1} +k: "b"/0,0, v: meta{ts: 50.000000000,0, txn: 2} +k: "c"/0,0, v: meta{ts: 50.000000000,0, txn: 3} +k: "d"/0,0, v: meta{ts: 50.000000000,0, txn: 4} +k: "e"/0,0, v: meta{ts: 50.000000000,0, txn: 5} ##### Intents written are separated. ##### new-writer enable-separated=true @@ -87,83 +87,83 @@ clear-intent k=e txn=5 preceding=interleaved txn-did-not-update-meta=false === Calls === ClearUnversioned(e) === Storage contents === -k: "a"/0,0, v: meta{ts: 0.000000050,0, txn: 1} -k: "b"/0,0, v: meta{ts: 0.000000050,0, txn: 2} -k: "c"/0,0, v: meta{ts: 0.000000050,0, txn: 3} -k: "d"/0,0, v: meta{ts: 0.000000050,0, txn: 4} +k: "a"/0,0, v: meta{ts: 50.000000000,0, txn: 1} +k: "b"/0,0, v: meta{ts: 50.000000000,0, txn: 2} +k: "c"/0,0, v: meta{ts: 50.000000000,0, txn: 3} +k: "d"/0,0, v: meta{ts: 50.000000000,0, txn: 4} # Overwrite an existing interleaved intent. put-intent k=a ts=50 txn=1 preceding=interleaved txn-did-not-update-meta=true ---- === Calls === ClearUnversioned(a) -PutEngineKey(LT{k: a, strength: Exclusive, uuid:1}, meta{ts: 0.000000050,0, txn: 1}) +PutEngineKey(LT{k: a, strength: Exclusive, uuid:1}, meta{ts: 50.000000000,0, txn: 1}) === Storage contents === -k: LT{k: a, strength: Exclusive, uuid:1}, v: meta{ts: 0.000000050,0, txn: 1} -k: "b"/0,0, v: meta{ts: 0.000000050,0, txn: 2} -k: "c"/0,0, v: meta{ts: 0.000000050,0, txn: 3} -k: "d"/0,0, v: meta{ts: 0.000000050,0, txn: 4} +k: LT{k: a, strength: Exclusive, uuid:1}, v: meta{ts: 50.000000000,0, txn: 1} +k: "b"/0,0, v: meta{ts: 50.000000000,0, txn: 2} +k: "c"/0,0, v: meta{ts: 50.000000000,0, txn: 3} +k: "d"/0,0, v: meta{ts: 50.000000000,0, txn: 4} # Overwrite an existing interleaved intent. put-intent k=d ts=50 txn=4 preceding=interleaved txn-did-not-update-meta=false ---- === Calls === ClearUnversioned(d) -PutEngineKey(LT{k: d, strength: Exclusive, uuid:4}, meta{ts: 0.000000050,0, txn: 4}) +PutEngineKey(LT{k: d, strength: Exclusive, uuid:4}, meta{ts: 50.000000000,0, txn: 4}) === Storage contents === -k: LT{k: a, strength: Exclusive, uuid:1}, v: meta{ts: 0.000000050,0, txn: 1} -k: LT{k: d, strength: Exclusive, uuid:4}, v: meta{ts: 0.000000050,0, txn: 4} -k: "b"/0,0, v: meta{ts: 0.000000050,0, txn: 2} -k: "c"/0,0, v: meta{ts: 0.000000050,0, txn: 3} +k: LT{k: a, strength: Exclusive, uuid:1}, v: meta{ts: 50.000000000,0, txn: 1} +k: LT{k: d, strength: Exclusive, uuid:4}, v: meta{ts: 50.000000000,0, txn: 4} +k: "b"/0,0, v: meta{ts: 50.000000000,0, txn: 2} +k: "c"/0,0, v: meta{ts: 50.000000000,0, txn: 3} # Write some new intents with txn-did-not-update-meta={true,false} put-intent k=e ts=50 txn=5 preceding=none txn-did-not-update-meta=false ---- === Calls === -PutEngineKey(LT{k: e, strength: Exclusive, uuid:5}, meta{ts: 0.000000050,0, txn: 5}) +PutEngineKey(LT{k: e, strength: Exclusive, uuid:5}, meta{ts: 50.000000000,0, txn: 5}) === Storage contents === -k: LT{k: a, strength: Exclusive, uuid:1}, v: meta{ts: 0.000000050,0, txn: 1} -k: LT{k: d, strength: Exclusive, uuid:4}, v: meta{ts: 0.000000050,0, txn: 4} -k: LT{k: e, strength: Exclusive, uuid:5}, v: meta{ts: 0.000000050,0, txn: 5} -k: "b"/0,0, v: meta{ts: 0.000000050,0, txn: 2} -k: "c"/0,0, v: meta{ts: 0.000000050,0, txn: 3} +k: LT{k: a, strength: Exclusive, uuid:1}, v: meta{ts: 50.000000000,0, txn: 1} +k: LT{k: d, strength: Exclusive, uuid:4}, v: meta{ts: 50.000000000,0, txn: 4} +k: LT{k: e, strength: Exclusive, uuid:5}, v: meta{ts: 50.000000000,0, txn: 5} +k: "b"/0,0, v: meta{ts: 50.000000000,0, txn: 2} +k: "c"/0,0, v: meta{ts: 50.000000000,0, txn: 3} put-intent k=f ts=50 txn=6 preceding=none txn-did-not-update-meta=true ---- === Calls === -PutEngineKey(LT{k: f, strength: Exclusive, uuid:6}, meta{ts: 0.000000050,0, txn: 6}) +PutEngineKey(LT{k: f, strength: Exclusive, uuid:6}, meta{ts: 50.000000000,0, txn: 6}) === Storage contents === -k: LT{k: a, strength: Exclusive, uuid:1}, v: meta{ts: 0.000000050,0, txn: 1} -k: LT{k: d, strength: Exclusive, uuid:4}, v: meta{ts: 0.000000050,0, txn: 4} -k: LT{k: e, strength: Exclusive, uuid:5}, v: meta{ts: 0.000000050,0, txn: 5} -k: LT{k: f, strength: Exclusive, uuid:6}, v: meta{ts: 0.000000050,0, txn: 6} -k: "b"/0,0, v: meta{ts: 0.000000050,0, txn: 2} -k: "c"/0,0, v: meta{ts: 0.000000050,0, txn: 3} +k: LT{k: a, strength: Exclusive, uuid:1}, v: meta{ts: 50.000000000,0, txn: 1} +k: LT{k: d, strength: Exclusive, uuid:4}, v: meta{ts: 50.000000000,0, txn: 4} +k: LT{k: e, strength: Exclusive, uuid:5}, v: meta{ts: 50.000000000,0, txn: 5} +k: LT{k: f, strength: Exclusive, uuid:6}, v: meta{ts: 50.000000000,0, txn: 6} +k: "b"/0,0, v: meta{ts: 50.000000000,0, txn: 2} +k: "c"/0,0, v: meta{ts: 50.000000000,0, txn: 3} # Overwrite an existing separated intent put-intent k=e ts=60 txn=5 preceding=separated txn-did-not-update-meta=false ---- === Calls === -PutEngineKey(LT{k: e, strength: Exclusive, uuid:5}, meta{ts: 0.000000060,0, txn: 5}) +PutEngineKey(LT{k: e, strength: Exclusive, uuid:5}, meta{ts: 60.000000000,0, txn: 5}) === Storage contents === -k: LT{k: a, strength: Exclusive, uuid:1}, v: meta{ts: 0.000000050,0, txn: 1} -k: LT{k: d, strength: Exclusive, uuid:4}, v: meta{ts: 0.000000050,0, txn: 4} -k: LT{k: e, strength: Exclusive, uuid:5}, v: meta{ts: 0.000000060,0, txn: 5} -k: LT{k: f, strength: Exclusive, uuid:6}, v: meta{ts: 0.000000050,0, txn: 6} -k: "b"/0,0, v: meta{ts: 0.000000050,0, txn: 2} -k: "c"/0,0, v: meta{ts: 0.000000050,0, txn: 3} +k: LT{k: a, strength: Exclusive, uuid:1}, v: meta{ts: 50.000000000,0, txn: 1} +k: LT{k: d, strength: Exclusive, uuid:4}, v: meta{ts: 50.000000000,0, txn: 4} +k: LT{k: e, strength: Exclusive, uuid:5}, v: meta{ts: 60.000000000,0, txn: 5} +k: LT{k: f, strength: Exclusive, uuid:6}, v: meta{ts: 50.000000000,0, txn: 6} +k: "b"/0,0, v: meta{ts: 50.000000000,0, txn: 2} +k: "c"/0,0, v: meta{ts: 50.000000000,0, txn: 3} put-intent k=f ts=60 txn=6 preceding=separated txn-did-not-update-meta=true ---- === Calls === -PutEngineKey(LT{k: f, strength: Exclusive, uuid:6}, meta{ts: 0.000000060,0, txn: 6}) +PutEngineKey(LT{k: f, strength: Exclusive, uuid:6}, meta{ts: 60.000000000,0, txn: 6}) === Storage contents === -k: LT{k: a, strength: Exclusive, uuid:1}, v: meta{ts: 0.000000050,0, txn: 1} -k: LT{k: d, strength: Exclusive, uuid:4}, v: meta{ts: 0.000000050,0, txn: 4} -k: LT{k: e, strength: Exclusive, uuid:5}, v: meta{ts: 0.000000060,0, txn: 5} -k: LT{k: f, strength: Exclusive, uuid:6}, v: meta{ts: 0.000000060,0, txn: 6} -k: "b"/0,0, v: meta{ts: 0.000000050,0, txn: 2} -k: "c"/0,0, v: meta{ts: 0.000000050,0, txn: 3} +k: LT{k: a, strength: Exclusive, uuid:1}, v: meta{ts: 50.000000000,0, txn: 1} +k: LT{k: d, strength: Exclusive, uuid:4}, v: meta{ts: 50.000000000,0, txn: 4} +k: LT{k: e, strength: Exclusive, uuid:5}, v: meta{ts: 60.000000000,0, txn: 5} +k: LT{k: f, strength: Exclusive, uuid:6}, v: meta{ts: 60.000000000,0, txn: 6} +k: "b"/0,0, v: meta{ts: 50.000000000,0, txn: 2} +k: "c"/0,0, v: meta{ts: 50.000000000,0, txn: 3} # Clear intents that have been updated. clear-intent k=f txn=6 preceding=separated txn-did-not-update-meta=false @@ -171,80 +171,80 @@ clear-intent k=f txn=6 preceding=separated txn-did-not-update-meta=false === Calls === ClearEngineKey(LT{k: f, strength: Exclusive, uuid:6}) === Storage contents === -k: LT{k: a, strength: Exclusive, uuid:1}, v: meta{ts: 0.000000050,0, txn: 1} -k: LT{k: d, strength: Exclusive, uuid:4}, v: meta{ts: 0.000000050,0, txn: 4} -k: LT{k: e, strength: Exclusive, uuid:5}, v: meta{ts: 0.000000060,0, txn: 5} -k: "b"/0,0, v: meta{ts: 0.000000050,0, txn: 2} -k: "c"/0,0, v: meta{ts: 0.000000050,0, txn: 3} +k: LT{k: a, strength: Exclusive, uuid:1}, v: meta{ts: 50.000000000,0, txn: 1} +k: LT{k: d, strength: Exclusive, uuid:4}, v: meta{ts: 50.000000000,0, txn: 4} +k: LT{k: e, strength: Exclusive, uuid:5}, v: meta{ts: 60.000000000,0, txn: 5} +k: "b"/0,0, v: meta{ts: 50.000000000,0, txn: 2} +k: "c"/0,0, v: meta{ts: 50.000000000,0, txn: 3} clear-intent k=e txn=5 preceding=separated txn-did-not-update-meta=false ---- === Calls === ClearEngineKey(LT{k: e, strength: Exclusive, uuid:5}) === Storage contents === -k: LT{k: a, strength: Exclusive, uuid:1}, v: meta{ts: 0.000000050,0, txn: 1} -k: LT{k: d, strength: Exclusive, uuid:4}, v: meta{ts: 0.000000050,0, txn: 4} -k: "b"/0,0, v: meta{ts: 0.000000050,0, txn: 2} -k: "c"/0,0, v: meta{ts: 0.000000050,0, txn: 3} +k: LT{k: a, strength: Exclusive, uuid:1}, v: meta{ts: 50.000000000,0, txn: 1} +k: LT{k: d, strength: Exclusive, uuid:4}, v: meta{ts: 50.000000000,0, txn: 4} +k: "b"/0,0, v: meta{ts: 50.000000000,0, txn: 2} +k: "c"/0,0, v: meta{ts: 50.000000000,0, txn: 3} # Put and clear an intent that has not been updated. put-intent k=e ts=60 txn=10 preceding=none txn-did-not-update-meta=true ---- === Calls === -PutEngineKey(LT{k: e, strength: Exclusive, uuid:10}, meta{ts: 0.000000060,0, txn: 10}) +PutEngineKey(LT{k: e, strength: Exclusive, uuid:10}, meta{ts: 60.000000000,0, txn: 10}) === Storage contents === -k: LT{k: a, strength: Exclusive, uuid:1}, v: meta{ts: 0.000000050,0, txn: 1} -k: LT{k: d, strength: Exclusive, uuid:4}, v: meta{ts: 0.000000050,0, txn: 4} -k: LT{k: e, strength: Exclusive, uuid:10}, v: meta{ts: 0.000000060,0, txn: 10} -k: "b"/0,0, v: meta{ts: 0.000000050,0, txn: 2} -k: "c"/0,0, v: meta{ts: 0.000000050,0, txn: 3} +k: LT{k: a, strength: Exclusive, uuid:1}, v: meta{ts: 50.000000000,0, txn: 1} +k: LT{k: d, strength: Exclusive, uuid:4}, v: meta{ts: 50.000000000,0, txn: 4} +k: LT{k: e, strength: Exclusive, uuid:10}, v: meta{ts: 60.000000000,0, txn: 10} +k: "b"/0,0, v: meta{ts: 50.000000000,0, txn: 2} +k: "c"/0,0, v: meta{ts: 50.000000000,0, txn: 3} clear-intent k=e txn=10 preceding=separated txn-did-not-update-meta=true ---- === Calls === SingleClearEngineKey(LT{k: e, strength: Exclusive, uuid:10}) === Storage contents === -k: LT{k: a, strength: Exclusive, uuid:1}, v: meta{ts: 0.000000050,0, txn: 1} -k: LT{k: d, strength: Exclusive, uuid:4}, v: meta{ts: 0.000000050,0, txn: 4} -k: "b"/0,0, v: meta{ts: 0.000000050,0, txn: 2} -k: "c"/0,0, v: meta{ts: 0.000000050,0, txn: 3} +k: LT{k: a, strength: Exclusive, uuid:1}, v: meta{ts: 50.000000000,0, txn: 1} +k: LT{k: d, strength: Exclusive, uuid:4}, v: meta{ts: 50.000000000,0, txn: 4} +k: "b"/0,0, v: meta{ts: 50.000000000,0, txn: 2} +k: "c"/0,0, v: meta{ts: 50.000000000,0, txn: 3} # Put new intents at e, f, g. put-intent k=e ts=60 txn=15 preceding=none txn-did-not-update-meta=true ---- === Calls === -PutEngineKey(LT{k: e, strength: Exclusive, uuid:15}, meta{ts: 0.000000060,0, txn: 15}) +PutEngineKey(LT{k: e, strength: Exclusive, uuid:15}, meta{ts: 60.000000000,0, txn: 15}) === Storage contents === -k: LT{k: a, strength: Exclusive, uuid:1}, v: meta{ts: 0.000000050,0, txn: 1} -k: LT{k: d, strength: Exclusive, uuid:4}, v: meta{ts: 0.000000050,0, txn: 4} -k: LT{k: e, strength: Exclusive, uuid:15}, v: meta{ts: 0.000000060,0, txn: 15} -k: "b"/0,0, v: meta{ts: 0.000000050,0, txn: 2} -k: "c"/0,0, v: meta{ts: 0.000000050,0, txn: 3} +k: LT{k: a, strength: Exclusive, uuid:1}, v: meta{ts: 50.000000000,0, txn: 1} +k: LT{k: d, strength: Exclusive, uuid:4}, v: meta{ts: 50.000000000,0, txn: 4} +k: LT{k: e, strength: Exclusive, uuid:15}, v: meta{ts: 60.000000000,0, txn: 15} +k: "b"/0,0, v: meta{ts: 50.000000000,0, txn: 2} +k: "c"/0,0, v: meta{ts: 50.000000000,0, txn: 3} put-intent k=f ts=60 txn=15 preceding=none txn-did-not-update-meta=true ---- === Calls === -PutEngineKey(LT{k: f, strength: Exclusive, uuid:15}, meta{ts: 0.000000060,0, txn: 15}) +PutEngineKey(LT{k: f, strength: Exclusive, uuid:15}, meta{ts: 60.000000000,0, txn: 15}) === Storage contents === -k: LT{k: a, strength: Exclusive, uuid:1}, v: meta{ts: 0.000000050,0, txn: 1} -k: LT{k: d, strength: Exclusive, uuid:4}, v: meta{ts: 0.000000050,0, txn: 4} -k: LT{k: e, strength: Exclusive, uuid:15}, v: meta{ts: 0.000000060,0, txn: 15} -k: LT{k: f, strength: Exclusive, uuid:15}, v: meta{ts: 0.000000060,0, txn: 15} -k: "b"/0,0, v: meta{ts: 0.000000050,0, txn: 2} -k: "c"/0,0, v: meta{ts: 0.000000050,0, txn: 3} +k: LT{k: a, strength: Exclusive, uuid:1}, v: meta{ts: 50.000000000,0, txn: 1} +k: LT{k: d, strength: Exclusive, uuid:4}, v: meta{ts: 50.000000000,0, txn: 4} +k: LT{k: e, strength: Exclusive, uuid:15}, v: meta{ts: 60.000000000,0, txn: 15} +k: LT{k: f, strength: Exclusive, uuid:15}, v: meta{ts: 60.000000000,0, txn: 15} +k: "b"/0,0, v: meta{ts: 50.000000000,0, txn: 2} +k: "c"/0,0, v: meta{ts: 50.000000000,0, txn: 3} put-intent k=g ts=60 txn=15 preceding=none txn-did-not-update-meta=true ---- === Calls === -PutEngineKey(LT{k: g, strength: Exclusive, uuid:15}, meta{ts: 0.000000060,0, txn: 15}) +PutEngineKey(LT{k: g, strength: Exclusive, uuid:15}, meta{ts: 60.000000000,0, txn: 15}) === Storage contents === -k: LT{k: a, strength: Exclusive, uuid:1}, v: meta{ts: 0.000000050,0, txn: 1} -k: LT{k: d, strength: Exclusive, uuid:4}, v: meta{ts: 0.000000050,0, txn: 4} -k: LT{k: e, strength: Exclusive, uuid:15}, v: meta{ts: 0.000000060,0, txn: 15} -k: LT{k: f, strength: Exclusive, uuid:15}, v: meta{ts: 0.000000060,0, txn: 15} -k: LT{k: g, strength: Exclusive, uuid:15}, v: meta{ts: 0.000000060,0, txn: 15} -k: "b"/0,0, v: meta{ts: 0.000000050,0, txn: 2} -k: "c"/0,0, v: meta{ts: 0.000000050,0, txn: 3} +k: LT{k: a, strength: Exclusive, uuid:1}, v: meta{ts: 50.000000000,0, txn: 1} +k: LT{k: d, strength: Exclusive, uuid:4}, v: meta{ts: 50.000000000,0, txn: 4} +k: LT{k: e, strength: Exclusive, uuid:15}, v: meta{ts: 60.000000000,0, txn: 15} +k: LT{k: f, strength: Exclusive, uuid:15}, v: meta{ts: 60.000000000,0, txn: 15} +k: LT{k: g, strength: Exclusive, uuid:15}, v: meta{ts: 60.000000000,0, txn: 15} +k: "b"/0,0, v: meta{ts: 50.000000000,0, txn: 2} +k: "c"/0,0, v: meta{ts: 50.000000000,0, txn: 3} # Clear range of intents that will clear c and d. clear-range start=c end=da @@ -253,11 +253,11 @@ clear-range start=c end=da ClearRawRange(c, da) ClearRawRange(LT{c}, LT{da}) === Storage contents === -k: LT{k: a, strength: Exclusive, uuid:1}, v: meta{ts: 0.000000050,0, txn: 1} -k: LT{k: e, strength: Exclusive, uuid:15}, v: meta{ts: 0.000000060,0, txn: 15} -k: LT{k: f, strength: Exclusive, uuid:15}, v: meta{ts: 0.000000060,0, txn: 15} -k: LT{k: g, strength: Exclusive, uuid:15}, v: meta{ts: 0.000000060,0, txn: 15} -k: "b"/0,0, v: meta{ts: 0.000000050,0, txn: 2} +k: LT{k: a, strength: Exclusive, uuid:1}, v: meta{ts: 50.000000000,0, txn: 1} +k: LT{k: e, strength: Exclusive, uuid:15}, v: meta{ts: 60.000000000,0, txn: 15} +k: LT{k: f, strength: Exclusive, uuid:15}, v: meta{ts: 60.000000000,0, txn: 15} +k: LT{k: g, strength: Exclusive, uuid:15}, v: meta{ts: 60.000000000,0, txn: 15} +k: "b"/0,0, v: meta{ts: 50.000000000,0, txn: 2} ##### Intents written are interleaved. ##### @@ -269,26 +269,26 @@ put-intent k=e ts=60 txn=15 preceding=separated txn-did-not-update-meta=true ---- === Calls === SingleClearEngineKey(LT{k: e, strength: Exclusive, uuid:15}) -PutUnversioned(e, meta{ts: 0.000000060,0, txn: 15}) +PutUnversioned(e, meta{ts: 60.000000000,0, txn: 15}) === Storage contents === -k: LT{k: a, strength: Exclusive, uuid:1}, v: meta{ts: 0.000000050,0, txn: 1} -k: LT{k: f, strength: Exclusive, uuid:15}, v: meta{ts: 0.000000060,0, txn: 15} -k: LT{k: g, strength: Exclusive, uuid:15}, v: meta{ts: 0.000000060,0, txn: 15} -k: "b"/0,0, v: meta{ts: 0.000000050,0, txn: 2} -k: "e"/0,0, v: meta{ts: 0.000000060,0, txn: 15} +k: LT{k: a, strength: Exclusive, uuid:1}, v: meta{ts: 50.000000000,0, txn: 1} +k: LT{k: f, strength: Exclusive, uuid:15}, v: meta{ts: 60.000000000,0, txn: 15} +k: LT{k: g, strength: Exclusive, uuid:15}, v: meta{ts: 60.000000000,0, txn: 15} +k: "b"/0,0, v: meta{ts: 50.000000000,0, txn: 2} +k: "e"/0,0, v: meta{ts: 60.000000000,0, txn: 15} # Overwrite an existing separated intent that was updated. put-intent k=a ts=60 txn=1 preceding=separated txn-did-not-update-meta=false ---- === Calls === ClearEngineKey(LT{k: a, strength: Exclusive, uuid:1}) -PutUnversioned(a, meta{ts: 0.000000060,0, txn: 1}) +PutUnversioned(a, meta{ts: 60.000000000,0, txn: 1}) === Storage contents === -k: LT{k: f, strength: Exclusive, uuid:15}, v: meta{ts: 0.000000060,0, txn: 15} -k: LT{k: g, strength: Exclusive, uuid:15}, v: meta{ts: 0.000000060,0, txn: 15} -k: "a"/0,0, v: meta{ts: 0.000000060,0, txn: 1} -k: "b"/0,0, v: meta{ts: 0.000000050,0, txn: 2} -k: "e"/0,0, v: meta{ts: 0.000000060,0, txn: 15} +k: LT{k: f, strength: Exclusive, uuid:15}, v: meta{ts: 60.000000000,0, txn: 15} +k: LT{k: g, strength: Exclusive, uuid:15}, v: meta{ts: 60.000000000,0, txn: 15} +k: "a"/0,0, v: meta{ts: 60.000000000,0, txn: 1} +k: "b"/0,0, v: meta{ts: 50.000000000,0, txn: 2} +k: "e"/0,0, v: meta{ts: 60.000000000,0, txn: 15} # Clear separated intent. clear-intent k=g ts=60 txn=15 preceding=separated txn-did-not-update-meta=false @@ -296,10 +296,10 @@ clear-intent k=g ts=60 txn=15 preceding=separated txn-did-not-update-meta=false === Calls === ClearEngineKey(LT{k: g, strength: Exclusive, uuid:15}) === Storage contents === -k: LT{k: f, strength: Exclusive, uuid:15}, v: meta{ts: 0.000000060,0, txn: 15} -k: "a"/0,0, v: meta{ts: 0.000000060,0, txn: 1} -k: "b"/0,0, v: meta{ts: 0.000000050,0, txn: 2} -k: "e"/0,0, v: meta{ts: 0.000000060,0, txn: 15} +k: LT{k: f, strength: Exclusive, uuid:15}, v: meta{ts: 60.000000000,0, txn: 15} +k: "a"/0,0, v: meta{ts: 60.000000000,0, txn: 1} +k: "b"/0,0, v: meta{ts: 50.000000000,0, txn: 2} +k: "e"/0,0, v: meta{ts: 60.000000000,0, txn: 15} # Clear range of intents that will clear e, f. clear-range start=e end=fa @@ -308,5 +308,5 @@ clear-range start=e end=fa ClearRawRange(e, fa) ClearRawRange(LT{e}, LT{fa}) === Storage contents === -k: "a"/0,0, v: meta{ts: 0.000000060,0, txn: 1} -k: "b"/0,0, v: meta{ts: 0.000000050,0, txn: 2} +k: "a"/0,0, v: meta{ts: 60.000000000,0, txn: 1} +k: "b"/0,0, v: meta{ts: 50.000000000,0, txn: 2} diff --git a/pkg/storage/testdata/intent_interleaving_iter/basic b/pkg/storage/testdata/intent_interleaving_iter/basic index f81274ea90b0..c2cf1d659acd 100644 --- a/pkg/storage/testdata/intent_interleaving_iter/basic +++ b/pkg/storage/testdata/intent_interleaving_iter/basic @@ -44,36 +44,36 @@ prev prev next ---- -seek-ge "a"/0,0: output: meta k=a ts=20 txn=1 -next: output: value k=a ts=20 v=a20 -next: output: value k=a ts=10 v=a10 -next: output: meta k=b ts=30 txn=2 -next: output: value k=b ts=30 v=b30 +seek-ge "a"/0,0: output: meta k=a ts=20.000000000,0 txn=1 +next: output: value k=a ts=20.000000000,0 v=a20 +next: output: value k=a ts=10.000000000,0 v=a10 +next: output: meta k=b ts=30.000000000,0 txn=2 +next: output: value k=b ts=30.000000000,0 v=b30 next: output: meta k=c -next: output: value k=d ts=25 v=d25 +next: output: value k=d ts=25.000000000,0 v=d25 next: output: . -prev: output: value k=d ts=25 v=d25 +prev: output: value k=d ts=25.000000000,0 v=d25 prev: output: meta k=c -prev: output: value k=b ts=30 v=b30 -prev: output: meta k=b ts=30 txn=2 -prev: output: value k=a ts=10 v=a10 -prev: output: value k=a ts=20 v=a20 -prev: output: meta k=a ts=20 txn=1 +prev: output: value k=b ts=30.000000000,0 v=b30 +prev: output: meta k=b ts=30.000000000,0 txn=2 +prev: output: value k=a ts=10.000000000,0 v=a10 +prev: output: value k=a ts=20.000000000,0 v=a20 +prev: output: meta k=a ts=20.000000000,0 txn=1 prev: output: . set-upper c -seek-ge "b"/0,0: output: meta k=b ts=30 txn=2 -next: output: value k=b ts=30 v=b30 +seek-ge "b"/0,0: output: meta k=b ts=30.000000000,0 txn=2 +next: output: value k=b ts=30.000000000,0 v=b30 next: output: . -prev: output: value k=b ts=30 v=b30 -prev: output: meta k=b ts=30 txn=2 -prev: output: value k=a ts=10 v=a10 -seek-lt "b"/0,0: output: value k=a ts=10 v=a10 -next: output: meta k=b ts=30 txn=2 -prev: output: value k=a ts=10 v=a10 -prev: output: value k=a ts=20 v=a20 -prev: output: meta k=a ts=20 txn=1 +prev: output: value k=b ts=30.000000000,0 v=b30 +prev: output: meta k=b ts=30.000000000,0 txn=2 +prev: output: value k=a ts=10.000000000,0 v=a10 +seek-lt "b"/0,0: output: value k=a ts=10.000000000,0 v=a10 +next: output: meta k=b ts=30.000000000,0 txn=2 +prev: output: value k=a ts=10.000000000,0 v=a10 +prev: output: value k=a ts=20.000000000,0 v=a20 +prev: output: meta k=a ts=20.000000000,0 txn=1 prev: output: . -next: output: meta k=a ts=20 txn=1 +next: output: meta k=a ts=20.000000000,0 txn=1 # Some callers iterate forward without explicitly setting an upper bound. # Confirm that the underlying iterator over the lock table key space does not @@ -84,10 +84,10 @@ next next next ---- -seek-ge "a"/0,0: output: meta k=a ts=20 txn=1 -next: output: value k=a ts=20 v=a20 -next: output: value k=a ts=10 v=a10 -next: output: meta k=b ts=30 txn=2 +seek-ge "a"/0,0: output: meta k=a ts=20.000000000,0 txn=1 +next: output: value k=a ts=20.000000000,0 v=a20 +next: output: value k=a ts=10.000000000,0 v=a10 +next: output: meta k=b ts=30.000000000,0 txn=2 # More forward and reverse iteration. This confirms that the underlying # MVCCIterator does not iterate into the lock table key space despite no lower @@ -104,16 +104,16 @@ prev prev next ---- -seek-ge "a"/0,0: output: meta k=a ts=20 txn=1 -next: output: value k=a ts=20 v=a20 -next: output: value k=a ts=10 v=a10 +seek-ge "a"/0,0: output: meta k=a ts=20.000000000,0 txn=1 +next: output: value k=a ts=20.000000000,0 v=a20 +next: output: value k=a ts=10.000000000,0 v=a10 next: output: . -prev: output: value k=a ts=10 v=a10 -prev: output: value k=a ts=20 v=a20 -prev: output: meta k=a ts=20 txn=1 +prev: output: value k=a ts=10.000000000,0 v=a10 +prev: output: value k=a ts=20.000000000,0 v=a20 +prev: output: meta k=a ts=20.000000000,0 txn=1 prev: output: . prev: output: . -next: output: meta k=a ts=20 txn=1 +next: output: meta k=a ts=20.000000000,0 txn=1 # Prefix iteration. iter prefix=true @@ -129,12 +129,12 @@ seek-ge k=f seek-ge k=c next ---- -seek-ge "b"/0,0: output: meta k=b ts=30 txn=2 -next: output: value k=b ts=30 v=b30 +seek-ge "b"/0,0: output: meta k=b ts=30.000000000,0 txn=2 +next: output: value k=b ts=30.000000000,0 v=b30 next: output: . -seek-ge "a"/0,0: output: meta k=a ts=20 txn=1 -next: output: value k=a ts=20 v=a20 -next: output: value k=a ts=10 v=a10 +seek-ge "a"/0,0: output: meta k=a ts=20.000000000,0 txn=1 +next: output: value k=a ts=20.000000000,0 v=a20 +next: output: value k=a ts=10.000000000,0 v=a10 next: output: . prev: output: err: pebble: unsupported reverse prefix iteration seek-ge "f"/0,0: output: . @@ -152,13 +152,13 @@ seek-ge k=a next next-key ---- -seek-ge "d"/0,0: output: value k=d ts=25 v=d25 +seek-ge "d"/0,0: output: value k=d ts=25.000000000,0 v=d25 next-key: output: . -seek-ge "a"/0,0: output: meta k=a ts=20 txn=1 -next-key: output: meta k=b ts=30 txn=2 -seek-ge "a"/0,0: output: meta k=a ts=20 txn=1 -next: output: value k=a ts=20 v=a20 -next-key: output: meta k=b ts=30 txn=2 +seek-ge "a"/0,0: output: meta k=a ts=20.000000000,0 txn=1 +next-key: output: meta k=b ts=30.000000000,0 txn=2 +seek-ge "a"/0,0: output: meta k=a ts=20.000000000,0 txn=1 +next: output: value k=a ts=20.000000000,0 v=a20 +next-key: output: meta k=b ts=30.000000000,0 txn=2 # Seek to particular timestamp. iter lower=a upper=f @@ -193,36 +193,36 @@ prev prev next ---- -seek-ge "a"/0.000000010,0: output: value k=a ts=10 v=a10 -next: output: meta k=b ts=30 txn=2 -next: output: value k=b ts=30 v=b30 -seek-ge "a"/0.000000025,0: output: value k=a ts=20 v=a20 -next: output: value k=a ts=10 v=a10 -next: output: meta k=b ts=30 txn=2 -seek-lt "a"/0.000000001,0: output: value k=a ts=10 v=a10 -prev: output: value k=a ts=20 v=a20 -prev: output: meta k=a ts=20 txn=1 +seek-ge "a"/10.000000000,0: output: value k=a ts=10.000000000,0 v=a10 +next: output: meta k=b ts=30.000000000,0 txn=2 +next: output: value k=b ts=30.000000000,0 v=b30 +seek-ge "a"/25.000000000,0: output: value k=a ts=20.000000000,0 v=a20 +next: output: value k=a ts=10.000000000,0 v=a10 +next: output: meta k=b ts=30.000000000,0 txn=2 +seek-lt "a"/1.000000000,0: output: value k=a ts=10.000000000,0 v=a10 +prev: output: value k=a ts=20.000000000,0 v=a20 +prev: output: meta k=a ts=20.000000000,0 txn=1 prev: output: . -next: output: meta k=a ts=20 txn=1 -next: output: value k=a ts=20 v=a20 -prev: output: meta k=a ts=20 txn=1 -next: output: value k=a ts=20 v=a20 -seek-lt "a"/0.000000015,0: output: value k=a ts=20 v=a20 -prev: output: meta k=a ts=20 txn=1 +next: output: meta k=a ts=20.000000000,0 txn=1 +next: output: value k=a ts=20.000000000,0 v=a20 +prev: output: meta k=a ts=20.000000000,0 txn=1 +next: output: value k=a ts=20.000000000,0 v=a20 +seek-lt "a"/15.000000000,0: output: value k=a ts=20.000000000,0 v=a20 +prev: output: meta k=a ts=20.000000000,0 txn=1 prev: output: . -next: output: meta k=a ts=20 txn=1 -seek-lt "a"/0.000000025,0: output: meta k=a ts=20 txn=1 +next: output: meta k=a ts=20.000000000,0 txn=1 +seek-lt "a"/25.000000000,0: output: meta k=a ts=20.000000000,0 txn=1 prev: output: . -next: output: meta k=a ts=20 txn=1 -seek-ge "a"/0.000000005,0: output: meta k=b ts=30 txn=2 -next: output: value k=b ts=30 v=b30 +next: output: meta k=a ts=20.000000000,0 txn=1 +seek-ge "a"/5.000000000,0: output: meta k=b ts=30.000000000,0 txn=2 +next: output: value k=b ts=30.000000000,0 v=b30 next: output: meta k=c -prev: output: value k=b ts=30 v=b30 -seek-lt "b"/0.000000040,0: output: meta k=b ts=30 txn=2 -prev: output: value k=a ts=10 v=a10 -prev: output: value k=a ts=20 v=a20 -prev: output: meta k=a ts=20 txn=1 -next: output: value k=a ts=20 v=a20 +prev: output: value k=b ts=30.000000000,0 v=b30 +seek-lt "b"/40.000000000,0: output: meta k=b ts=30.000000000,0 txn=2 +prev: output: value k=a ts=10.000000000,0 v=a10 +prev: output: value k=a ts=20.000000000,0 v=a20 +prev: output: meta k=a ts=20.000000000,0 txn=1 +next: output: value k=a ts=20.000000000,0 v=a20 # Seek to particular timestamp and prefix iteration. What we will # see after the prefix is exhausted is undefined. @@ -235,13 +235,13 @@ next seek-ge k=a ts=5 seek-lt k=a ts=1 ---- -seek-ge "a"/0.000000025,0: output: value k=a ts=20 v=a20 -next: output: value k=a ts=10 v=a10 +seek-ge "a"/25.000000000,0: output: value k=a ts=20.000000000,0 v=a20 +next: output: value k=a ts=10.000000000,0 v=a10 next: output: . -seek-ge "a"/0.000000015,0: output: value k=a ts=10 v=a10 +seek-ge "a"/15.000000000,0: output: value k=a ts=10.000000000,0 v=a10 next: output: . -seek-ge "a"/0.000000005,0: output: . -seek-lt "a"/0.000000001,0: output: err: prefix iteration is not permitted with SeekLT +seek-ge "a"/5.000000000,0: output: . +seek-lt "a"/1.000000000,0: output: err: prefix iteration is not permitted with SeekLT # Exercise NextKey iter lower=a upper=f @@ -260,19 +260,19 @@ next-key next-key next-key ---- -seek-ge "a"/0,0: output: meta k=a ts=20 txn=1 -next-key: output: meta k=b ts=30 txn=2 -prev: output: value k=a ts=10 v=a10 +seek-ge "a"/0,0: output: meta k=a ts=20.000000000,0 txn=1 +next-key: output: meta k=b ts=30.000000000,0 txn=2 +prev: output: value k=a ts=10.000000000,0 v=a10 next-key: output: err: NextKey cannot be used to switch iteration direction next: output: err: NextKey cannot be used to switch iteration direction -seek-ge "b"/0,0: output: meta k=b ts=30 txn=2 +seek-ge "b"/0,0: output: meta k=b ts=30.000000000,0 txn=2 next-key: output: meta k=c -prev: output: value k=b ts=30 v=b30 -prev: output: meta k=b ts=30 txn=2 -prev: output: value k=a ts=10 v=a10 -next: output: meta k=b ts=30 txn=2 +prev: output: value k=b ts=30.000000000,0 v=b30 +prev: output: meta k=b ts=30.000000000,0 txn=2 +prev: output: value k=a ts=10.000000000,0 v=a10 +next: output: meta k=b ts=30.000000000,0 txn=2 next-key: output: meta k=c -next-key: output: value k=d ts=25 v=d25 +next-key: output: value k=d ts=25.000000000,0 v=d25 next-key: output: . @@ -322,32 +322,32 @@ prev prev prev ---- -seek-ge "a"/0,0: output: meta k=a ts=10 txn=1 -next: output: value k=a ts=10 v=a10 -next: output: meta k=b ts=20 txn=2 -prev: output: value k=a ts=10 v=a10 -prev: output: meta k=a ts=10 txn=1 +seek-ge "a"/0,0: output: meta k=a ts=10.000000000,0 txn=1 +next: output: value k=a ts=10.000000000,0 v=a10 +next: output: meta k=b ts=20.000000000,0 txn=2 +prev: output: value k=a ts=10.000000000,0 v=a10 +prev: output: meta k=a ts=10.000000000,0 txn=1 prev: output: . -next: output: meta k=a ts=10 txn=1 -next: output: value k=a ts=10 v=a10 -next: output: meta k=b ts=20 txn=2 -next: output: value k=b ts=20 v=b20 -next: output: meta k=c ts=30 txn=3 -next: output: value k=c ts=30 v=c30 -next: output: meta k=d ts=40 txn=4 -next: output: value k=d ts=40 v=d40 -next: output: meta k=e ts=50 txn=5 +next: output: meta k=a ts=10.000000000,0 txn=1 +next: output: value k=a ts=10.000000000,0 v=a10 +next: output: meta k=b ts=20.000000000,0 txn=2 +next: output: value k=b ts=20.000000000,0 v=b20 +next: output: meta k=c ts=30.000000000,0 txn=3 +next: output: value k=c ts=30.000000000,0 v=c30 +next: output: meta k=d ts=40.000000000,0 txn=4 +next: output: value k=d ts=40.000000000,0 v=d40 +next: output: meta k=e ts=50.000000000,0 txn=5 next-key: output: . -prev: output: value k=e ts=50 v=e50 -prev: output: meta k=e ts=50 txn=5 -prev: output: value k=d ts=40 v=d40 -prev: output: meta k=d ts=40 txn=4 -prev: output: value k=c ts=30 v=c30 -prev: output: meta k=c ts=30 txn=3 -prev: output: value k=b ts=20 v=b20 -prev: output: meta k=b ts=20 txn=2 -prev: output: value k=a ts=10 v=a10 -prev: output: meta k=a ts=10 txn=1 +prev: output: value k=e ts=50.000000000,0 v=e50 +prev: output: meta k=e ts=50.000000000,0 txn=5 +prev: output: value k=d ts=40.000000000,0 v=d40 +prev: output: meta k=d ts=40.000000000,0 txn=4 +prev: output: value k=c ts=30.000000000,0 v=c30 +prev: output: meta k=c ts=30.000000000,0 txn=3 +prev: output: value k=b ts=20.000000000,0 v=b20 +prev: output: meta k=b ts=20.000000000,0 txn=2 +prev: output: value k=a ts=10.000000000,0 v=a10 +prev: output: meta k=a ts=10.000000000,0 txn=1 prev: output: . # Error case: Multiple separated intents with no provisional values @@ -369,15 +369,15 @@ prev seek-lt k=e prev ---- -seek-ge "a"/0,0: output: meta k=b ts=20 txn=2 +seek-ge "a"/0,0: output: meta k=b ts=20.000000000,0 txn=2 next: output: err: intentIter at intent, but iter not at provisional value -seek-lt "e"/0,0: output: meta k=d ts=40 txn=4 +seek-lt "e"/0,0: output: meta k=d ts=40.000000000,0 txn=4 next: output: err: intent has no provisional value -seek-ge "d"/0,0: output: meta k=d ts=40 txn=4 +seek-ge "d"/0,0: output: meta k=d ts=40.000000000,0 txn=4 next-key: output: err: intentIter at intent, but iter not at provisional value -seek-ge "d"/0,0: output: meta k=d ts=40 txn=4 +seek-ge "d"/0,0: output: meta k=d ts=40.000000000,0 txn=4 prev: output: err: iter not at provisional value, cmp: -1 -seek-lt "e"/0,0: output: meta k=d ts=40 txn=4 +seek-lt "e"/0,0: output: meta k=d ts=40.000000000,0 txn=4 prev: output: err: reverse iteration discovered intent without provisional value # Local range keys. This exercises local keys having separated locks. @@ -408,19 +408,19 @@ prev prev prev ---- -seek-ge "La"/0,0: output: meta k=La ts=10 txn=1 -next: output: value k=La ts=10 v=a10 -next: output: meta k=Lb ts=20 txn=2 -next: output: value k=Lb ts=20 v=b20 -next: output: meta k=Lc ts=30 txn=4 -next: output: value k=Lc ts=30 v=c30 +seek-ge "La"/0,0: output: meta k=La ts=10.000000000,0 txn=1 +next: output: value k=La ts=10.000000000,0 v=a10 +next: output: meta k=Lb ts=20.000000000,0 txn=2 +next: output: value k=Lb ts=20.000000000,0 v=b20 +next: output: meta k=Lc ts=30.000000000,0 txn=4 +next: output: value k=Lc ts=30.000000000,0 v=c30 next: output: . -prev: output: value k=Lc ts=30 v=c30 -prev: output: meta k=Lc ts=30 txn=4 -prev: output: value k=Lb ts=20 v=b20 -prev: output: meta k=Lb ts=20 txn=2 -prev: output: value k=La ts=10 v=a10 -prev: output: meta k=La ts=10 txn=1 +prev: output: value k=Lc ts=30.000000000,0 v=c30 +prev: output: meta k=Lc ts=30.000000000,0 txn=4 +prev: output: value k=Lb ts=20.000000000,0 v=b20 +prev: output: meta k=Lb ts=20.000000000,0 txn=2 +prev: output: value k=La ts=10.000000000,0 v=a10 +prev: output: meta k=La ts=10.000000000,0 txn=1 prev: output: . # Confirm that the lock table iterator does not step out of the lock table @@ -435,14 +435,14 @@ next next next ---- -seek-ge "La"/0,0: output: meta k=La ts=10 txn=1 -next: output: value k=La ts=10 v=a10 -prev: output: meta k=La ts=10 txn=1 +seek-ge "La"/0,0: output: meta k=La ts=10.000000000,0 txn=1 +next: output: value k=La ts=10.000000000,0 v=a10 +prev: output: meta k=La ts=10.000000000,0 txn=1 prev: output: . prev: output: . -next: output: meta k=La ts=10 txn=1 -next: output: value k=La ts=10 v=a10 -next: output: meta k=Lb ts=20 txn=2 +next: output: meta k=La ts=10.000000000,0 txn=1 +next: output: value k=La ts=10.000000000,0 v=a10 +next: output: meta k=Lb ts=20.000000000,0 txn=2 # Confirm that the underlying MVCCIterator does not iterate into the lock # table key space despite no upper bound. @@ -464,22 +464,22 @@ next next prev ---- -seek-lt "Ld"/0,0: output: value k=Lc ts=30 v=c30 -prev: output: meta k=Lc ts=30 txn=4 -prev: output: value k=Lb ts=20 v=b20 -prev: output: meta k=Lb ts=20 txn=2 -prev: output: value k=La ts=10 v=a10 -prev: output: meta k=La ts=10 txn=1 +seek-lt "Ld"/0,0: output: value k=Lc ts=30.000000000,0 v=c30 +prev: output: meta k=Lc ts=30.000000000,0 txn=4 +prev: output: value k=Lb ts=20.000000000,0 v=b20 +prev: output: meta k=Lb ts=20.000000000,0 txn=2 +prev: output: value k=La ts=10.000000000,0 v=a10 +prev: output: meta k=La ts=10.000000000,0 txn=1 prev: output: . -next: output: meta k=La ts=10 txn=1 -next: output: value k=La ts=10 v=a10 -next: output: meta k=Lb ts=20 txn=2 -next: output: value k=Lb ts=20 v=b20 -next: output: meta k=Lc ts=30 txn=4 -next: output: value k=Lc ts=30 v=c30 +next: output: meta k=La ts=10.000000000,0 txn=1 +next: output: value k=La ts=10.000000000,0 v=a10 +next: output: meta k=Lb ts=20.000000000,0 txn=2 +next: output: value k=Lb ts=20.000000000,0 v=b20 +next: output: meta k=Lc ts=30.000000000,0 txn=4 +next: output: value k=Lc ts=30.000000000,0 v=c30 next: output: . next: output: . -prev: output: value k=Lc ts=30 v=c30 +prev: output: value k=Lc ts=30.000000000,0 v=c30 iter prefix=true seek-ge k=Lb @@ -494,17 +494,17 @@ seek-ge k=Lb ts=25 next seek-ge k=Lc ts=25 ---- -seek-ge "Lb"/0,0: output: meta k=Lb ts=20 txn=2 -next: output: value k=Lb ts=20 v=b20 +seek-ge "Lb"/0,0: output: meta k=Lb ts=20.000000000,0 txn=2 +next: output: value k=Lb ts=20.000000000,0 v=b20 next: output: . -seek-ge "Lc"/0,0: output: meta k=Lc ts=30 txn=4 -next: output: value k=Lc ts=30 v=c30 +seek-ge "Lc"/0,0: output: meta k=Lc ts=30.000000000,0 txn=4 +next: output: value k=Lc ts=30.000000000,0 v=c30 next: output: . -seek-ge "La"/0.000000025,0: output: value k=La ts=10 v=a10 +seek-ge "La"/25.000000000,0: output: value k=La ts=10.000000000,0 v=a10 next: output: . -seek-ge "Lb"/0.000000025,0: output: value k=Lb ts=20 v=b20 +seek-ge "Lb"/25.000000000,0: output: value k=Lb ts=20.000000000,0 v=b20 next: output: . -seek-ge "Lc"/0.000000025,0: output: . +seek-ge "Lc"/25.000000000,0: output: . # Keys with \x00 byte. To exercise the slow-path in UnsafeRawMVCCKey. The keys # that are length 8, 16 will exercise the slow-path. The len(key) < 8 does @@ -545,21 +545,21 @@ prev prev prev ---- -seek-ge "a"/0,0: output: meta k=abcdefg\0 ts=20 txn=1 -next: output: value k=abcdefg\0 ts=20 v=a -next: output: meta k=b\0c\0d ts=20 txn=1 -next: output: value k=b\0c\0d ts=20 v=b1 -next: output: meta k=bcdefgh\0 ts=20 txn=1 -next: output: value k=bcdefgh\0 ts=20 v=b2 -next: output: meta k=cdefghijklmnopq\0 ts=20 txn=1 -next: output: value k=cdefghijklmnopq\0 ts=20 v=c +seek-ge "a"/0,0: output: meta k=abcdefg\0 ts=20.000000000,0 txn=1 +next: output: value k=abcdefg\0 ts=20.000000000,0 v=a +next: output: meta k=b\0c\0d ts=20.000000000,0 txn=1 +next: output: value k=b\0c\0d ts=20.000000000,0 v=b1 +next: output: meta k=bcdefgh\0 ts=20.000000000,0 txn=1 +next: output: value k=bcdefgh\0 ts=20.000000000,0 v=b2 +next: output: meta k=cdefghijklmnopq\0 ts=20.000000000,0 txn=1 +next: output: value k=cdefghijklmnopq\0 ts=20.000000000,0 v=c next: output: . -prev: output: value k=cdefghijklmnopq\0 ts=20 v=c -prev: output: meta k=cdefghijklmnopq\0 ts=20 txn=1 -prev: output: value k=bcdefgh\0 ts=20 v=b2 -prev: output: meta k=bcdefgh\0 ts=20 txn=1 -prev: output: value k=b\0c\0d ts=20 v=b1 -prev: output: meta k=b\0c\0d ts=20 txn=1 -prev: output: value k=abcdefg\0 ts=20 v=a -prev: output: meta k=abcdefg\0 ts=20 txn=1 +prev: output: value k=cdefghijklmnopq\0 ts=20.000000000,0 v=c +prev: output: meta k=cdefghijklmnopq\0 ts=20.000000000,0 txn=1 +prev: output: value k=bcdefgh\0 ts=20.000000000,0 v=b2 +prev: output: meta k=bcdefgh\0 ts=20.000000000,0 txn=1 +prev: output: value k=b\0c\0d ts=20.000000000,0 v=b1 +prev: output: meta k=b\0c\0d ts=20.000000000,0 txn=1 +prev: output: value k=abcdefg\0 ts=20.000000000,0 v=a +prev: output: meta k=abcdefg\0 ts=20.000000000,0 txn=1 prev: output: . diff --git a/pkg/storage/testdata/intent_interleaving_iter/error_race_off b/pkg/storage/testdata/intent_interleaving_iter/error_race_off index 6c3cd72e06fd..67a7aa7e4799 100644 --- a/pkg/storage/testdata/intent_interleaving_iter/error_race_off +++ b/pkg/storage/testdata/intent_interleaving_iter/error_race_off @@ -33,14 +33,14 @@ prev prev prev ---- -seek-ge "a"/0,0: output: meta k=a ts=10 txn=1 -next: output: value k=a ts=10 v=a10 -next: output: meta k=b ts=20 txn=4 -next: output: value k=b ts=20 v=b20 -next: output: meta k=b ts=20 txn=2 +seek-ge "a"/0,0: output: meta k=a ts=10.000000000,0 txn=1 +next: output: value k=a ts=10.000000000,0 v=a10 +next: output: meta k=b ts=20.000000000,0 txn=4 +next: output: value k=b ts=20.000000000,0 v=b20 +next: output: meta k=b ts=20.000000000,0 txn=2 next: output: err: intentIter at intent, but iter not at provisional value -seek-lt "d"/0,0: output: value k=c ts=30 v=c30 -prev: output: meta k=c ts=30 txn=4 -prev: output: value k=b ts=20 v=b20 -prev: output: meta k=b ts=20 txn=2 +seek-lt "d"/0,0: output: value k=c ts=30.000000000,0 v=c30 +prev: output: meta k=c ts=30.000000000,0 txn=4 +prev: output: value k=b ts=20.000000000,0 v=b20 +prev: output: meta k=b ts=20.000000000,0 txn=2 prev: output: err: intentIter should not be after iter diff --git a/pkg/storage/testdata/mvcc_histories/clear_range b/pkg/storage/testdata/mvcc_histories/clear_range index 0361c68debc9..1710246708a0 100644 --- a/pkg/storage/testdata/mvcc_histories/clear_range +++ b/pkg/storage/testdata/mvcc_histories/clear_range @@ -11,51 +11,51 @@ with t=A v=abc resolve put k=c ---- >> at end: -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000044,0 min=0,0 seq=0} lock=true stat=PENDING rts=0.000000044,0 wto=false max=0,0 -data: "a"/0.000000044,0 -> /BYTES/abc -data: "a/123"/0.000000044,0 -> /BYTES/abc -data: "b"/0.000000044,0 -> /BYTES/abc -data: "b/123"/0.000000044,0 -> /BYTES/abc -data: "c"/0.000000044,0 -> /BYTES/abc +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=44.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=44.000000000,0 wto=false max=0,0 +data: "a"/44.000000000,0 -> /BYTES/abc +data: "a/123"/44.000000000,0 -> /BYTES/abc +data: "b"/44.000000000,0 -> /BYTES/abc +data: "b/123"/44.000000000,0 -> /BYTES/abc +data: "c"/44.000000000,0 -> /BYTES/abc run ok clear_range k=a end=+a ---- >> at end: -data: "a/123"/0.000000044,0 -> /BYTES/abc -data: "b"/0.000000044,0 -> /BYTES/abc -data: "b/123"/0.000000044,0 -> /BYTES/abc -data: "c"/0.000000044,0 -> /BYTES/abc +data: "a/123"/44.000000000,0 -> /BYTES/abc +data: "b"/44.000000000,0 -> /BYTES/abc +data: "b/123"/44.000000000,0 -> /BYTES/abc +data: "c"/44.000000000,0 -> /BYTES/abc run ok clear_range k=a end=-a ---- >> at end: -data: "b"/0.000000044,0 -> /BYTES/abc -data: "b/123"/0.000000044,0 -> /BYTES/abc -data: "c"/0.000000044,0 -> /BYTES/abc +data: "b"/44.000000000,0 -> /BYTES/abc +data: "b/123"/44.000000000,0 -> /BYTES/abc +data: "c"/44.000000000,0 -> /BYTES/abc run ok clear_range k=a end==b ---- >> at end: -data: "b"/0.000000044,0 -> /BYTES/abc -data: "b/123"/0.000000044,0 -> /BYTES/abc -data: "c"/0.000000044,0 -> /BYTES/abc +data: "b"/44.000000000,0 -> /BYTES/abc +data: "b/123"/44.000000000,0 -> /BYTES/abc +data: "c"/44.000000000,0 -> /BYTES/abc run ok clear_range k=a end=+b ---- >> at end: -data: "b/123"/0.000000044,0 -> /BYTES/abc -data: "c"/0.000000044,0 -> /BYTES/abc +data: "b/123"/44.000000000,0 -> /BYTES/abc +data: "c"/44.000000000,0 -> /BYTES/abc run ok clear_range k=a end=-b ---- >> at end: -data: "c"/0.000000044,0 -> /BYTES/abc +data: "c"/44.000000000,0 -> /BYTES/abc run ok clear_range k=a end=-c diff --git a/pkg/storage/testdata/mvcc_histories/conditional_put b/pkg/storage/testdata/mvcc_histories/conditional_put index b2ffabf0e4f6..981f4522e6c9 100644 --- a/pkg/storage/testdata/mvcc_histories/conditional_put +++ b/pkg/storage/testdata/mvcc_histories/conditional_put @@ -20,7 +20,7 @@ run ok cput k=k v=v ts=123,2 ---- >> at end: -data: "k"/0.000000123,2 -> /BYTES/v +data: "k"/123.000000000,2 -> /BYTES/v # Another conditional put expecting value missing will fail, now that value1 is written. @@ -28,8 +28,8 @@ run error cput k=k v=v ts=123,3 ---- >> at end: -data: "k"/0.000000123,2 -> /BYTES/v -error: (*roachpb.ConditionFailedError:) unexpected value: raw_bytes:"\000\000\000\000\003v" timestamp: +data: "k"/123.000000000,2 -> /BYTES/v +error: (*roachpb.ConditionFailedError:) unexpected value: raw_bytes:"\000\000\000\000\003v" timestamp: # Conditional put expecting wrong value2, will fail. @@ -37,8 +37,8 @@ run error cput k=k v=v cond=v2 ts=123,4 ---- >> at end: -data: "k"/0.000000123,2 -> /BYTES/v -error: (*roachpb.ConditionFailedError:) unexpected value: raw_bytes:"\000\000\000\000\003v" timestamp: +data: "k"/123.000000000,2 -> /BYTES/v +error: (*roachpb.ConditionFailedError:) unexpected value: raw_bytes:"\000\000\000\000\003v" timestamp: # Move to an empty value. Will succeed. @@ -46,8 +46,8 @@ run ok cput k=k v= cond=v ts=123,5 ---- >> at end: -data: "k"/0.000000123,5 -> /BYTES/ -data: "k"/0.000000123,2 -> /BYTES/v +data: "k"/123.000000000,5 -> /BYTES/ +data: "k"/123.000000000,2 -> /BYTES/v # Move key2 (which does not exist) to from value1 to value2. # Expect it to fail since it does not exist with value1. @@ -56,8 +56,8 @@ run error cput k=k2 v=v2 cond=v ts=123,6 ---- >> at end: -data: "k"/0.000000123,5 -> /BYTES/ -data: "k"/0.000000123,2 -> /BYTES/v +data: "k"/123.000000000,5 -> /BYTES/ +data: "k"/123.000000000,2 -> /BYTES/v error: (*roachpb.ConditionFailedError:) unexpected value: # Move key2 (which does not yet exist) to from value1 to value2, but @@ -67,9 +67,9 @@ run ok cput k=k2 v=v2 cond=v ts=123,7 allow_missing ---- >> at end: -data: "k"/0.000000123,5 -> /BYTES/ -data: "k"/0.000000123,2 -> /BYTES/v -data: "k2"/0.000000123,7 -> /BYTES/v2 +data: "k"/123.000000000,5 -> /BYTES/ +data: "k"/123.000000000,2 -> /BYTES/v +data: "k2"/123.000000000,7 -> /BYTES/v2 # Try to move key2 (which has value2) from value1 to empty. Expect error. @@ -77,10 +77,10 @@ run error cput k=k2 v= cond=v allow_missing ts=123,8 ---- >> at end: -data: "k"/0.000000123,5 -> /BYTES/ -data: "k"/0.000000123,2 -> /BYTES/v -data: "k2"/0.000000123,7 -> /BYTES/v2 -error: (*roachpb.ConditionFailedError:) unexpected value: raw_bytes:"\000\000\000\000\003v2" timestamp: +data: "k"/123.000000000,5 -> /BYTES/ +data: "k"/123.000000000,2 -> /BYTES/v +data: "k2"/123.000000000,7 -> /BYTES/v2 +error: (*roachpb.ConditionFailedError:) unexpected value: raw_bytes:"\000\000\000\000\003v2" timestamp: # Try to move key2 (which has value2) from value2 to empty. Expect success. @@ -88,10 +88,10 @@ run ok cput k=k2 v= cond=v2 allow_missing ts=123,9 ---- >> at end: -data: "k"/0.000000123,5 -> /BYTES/ -data: "k"/0.000000123,2 -> /BYTES/v -data: "k2"/0.000000123,9 -> /BYTES/ -data: "k2"/0.000000123,7 -> /BYTES/v2 +data: "k"/123.000000000,5 -> /BYTES/ +data: "k"/123.000000000,2 -> /BYTES/v +data: "k2"/123.000000000,9 -> /BYTES/ +data: "k2"/123.000000000,7 -> /BYTES/v2 # Now move to value2 from expected empty value. @@ -99,15 +99,15 @@ run ok cput k=k v=v2 cond= ts=123,10 ---- >> at end: -data: "k"/0.000000123,10 -> /BYTES/v2 -data: "k"/0.000000123,5 -> /BYTES/ -data: "k"/0.000000123,2 -> /BYTES/v -data: "k2"/0.000000123,9 -> /BYTES/ -data: "k2"/0.000000123,7 -> /BYTES/v2 +data: "k"/123.000000000,10 -> /BYTES/v2 +data: "k"/123.000000000,5 -> /BYTES/ +data: "k"/123.000000000,2 -> /BYTES/v +data: "k2"/123.000000000,9 -> /BYTES/ +data: "k2"/123.000000000,7 -> /BYTES/v2 # Verify we get value2 as expected. run ok get k=k ts=123,11 ---- -get: "k" -> /BYTES/v2 @0.000000123,10 +get: "k" -> /BYTES/v2 @123.000000000,10 diff --git a/pkg/storage/testdata/mvcc_histories/conditional_put_with_txn_disallow_separated b/pkg/storage/testdata/mvcc_histories/conditional_put_with_txn_disallow_separated index a618d2ae80ed..6b4d7becabea 100644 --- a/pkg/storage/testdata/mvcc_histories/conditional_put_with_txn_disallow_separated +++ b/pkg/storage/testdata/mvcc_histories/conditional_put_with_txn_disallow_separated @@ -2,7 +2,7 @@ run ok txn_begin t=A ts=123 ---- >> at end: -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000123,0 min=0,0 seq=0} lock=true stat=PENDING rts=0.000000123,0 wto=false max=0,0 +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=123.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=123.000000000,0 wto=false max=0,0 # Write value1. @@ -12,9 +12,9 @@ with t=A cput k=k v=v ---- >> at end: -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000123,0 min=0,0 seq=1} lock=true stat=PENDING rts=0.000000123,0 wto=false max=0,0 -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000123,0 min=0,0 seq=1} ts=0.000000123,0 del=false klen=12 vlen=6 -data: "k"/0.000000123,0 -> /BYTES/v +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=123.000000000,0 min=0,0 seq=1} lock=true stat=PENDING rts=123.000000000,0 wto=false max=0,0 +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=123.000000000,0 min=0,0 seq=1} ts=123.000000000,0 del=false klen=12 vlen=6 +data: "k"/123.000000000,0 -> /BYTES/v # Now, overwrite value1 with value2 from same txn; should see value1 # as pre-existing value. @@ -25,9 +25,9 @@ with t=A cput k=k v=v2 cond=v ---- >> at end: -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000123,0 min=0,0 seq=2} lock=true stat=PENDING rts=0.000000123,0 wto=false max=0,0 -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000123,0 min=0,0 seq=2} ts=0.000000123,0 del=false klen=12 vlen=7 ih={{1 /BYTES/v}} -data: "k"/0.000000123,0 -> /BYTES/v2 +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=123.000000000,0 min=0,0 seq=2} lock=true stat=PENDING rts=123.000000000,0 wto=false max=0,0 +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=123.000000000,0 min=0,0 seq=2} ts=123.000000000,0 del=false klen=12 vlen=7 ih={{1 /BYTES/v}} +data: "k"/123.000000000,0 -> /BYTES/v2 # Writing value3 from a new epoch should see nil again. @@ -38,9 +38,9 @@ with t=A cput k=k v=v3 ---- >> at end: -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=1 ts=0.000000123,0 min=0,0 seq=1} lock=true stat=PENDING rts=0.000000123,0 wto=false max=0,0 -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=0.000000123,0 min=0,0 seq=1} ts=0.000000123,0 del=false klen=12 vlen=7 -data: "k"/0.000000123,0 -> /BYTES/v3 +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=1 ts=123.000000000,0 min=0,0 seq=1} lock=true stat=PENDING rts=123.000000000,0 wto=false max=0,0 +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=123.000000000,0 min=0,0 seq=1} ts=123.000000000,0 del=false klen=12 vlen=7 +data: "k"/123.000000000,0 -> /BYTES/v3 # Commit value3 at a later timestamp. @@ -51,7 +51,7 @@ with t=A txn_remove ---- >> at end: -data: "k"/0.000000124,0 -> /BYTES/v3 +data: "k"/124.000000000,0 -> /BYTES/v3 # Write value4 with an old timestamp without txn...should get a write # too old error. @@ -60,9 +60,9 @@ run error cput k=k v=v4 cond=v3 ts=123 ---- >> at end: -data: "k"/0.000000124,1 -> /BYTES/v4 -data: "k"/0.000000124,0 -> /BYTES/v3 -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write at timestamp 0.000000123,0 too old; wrote at 0.000000124,1 +data: "k"/124.000000000,1 -> /BYTES/v4 +data: "k"/124.000000000,0 -> /BYTES/v3 +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write at timestamp 123.000000000,0 too old; wrote at 124.000000000,1 # Reset for next test @@ -86,10 +86,10 @@ with t=A cput k=c v=cput cond=value ---- >> at end: -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000002,0 min=0,0 seq=1} lock=true stat=PENDING rts=0.000000002,0 wto=false max=0,0 -meta: "c"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000002,0 min=0,0 seq=1} ts=0.000000002,0 del=false klen=12 vlen=9 -data: "c"/0.000000002,0 -> /BYTES/cput -data: "c"/0.000000001,0 -> /BYTES/value +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=1} lock=true stat=PENDING rts=2.000000000,0 wto=false max=0,0 +meta: "c"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=1} ts=2.000000000,0 del=false klen=12 vlen=9 +data: "c"/2.000000000,0 -> /BYTES/cput +data: "c"/1.000000000,0 -> /BYTES/value # Restart and retry cput. It should succeed. @@ -100,11 +100,11 @@ with t=A cput k=c v=cput cond=value ---- >> txn_restart ts=3 t=A -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=1 ts=0.000000003,0 min=0,0 seq=0} lock=true stat=PENDING rts=0.000000003,0 wto=false max=0,0 +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=1 ts=3.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=3.000000000,0 wto=false max=0,0 >> txn_step t=A -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=1 ts=0.000000003,0 min=0,0 seq=1} lock=true stat=PENDING rts=0.000000003,0 wto=false max=0,0 +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=1 ts=3.000000000,0 min=0,0 seq=1} lock=true stat=PENDING rts=3.000000000,0 wto=false max=0,0 >> cput k=c v=cput cond=value t=A called PutIntent("c", _, ExistingIntentInterleaved, TDNUM(false), 00000000-0000-0000-0000-000000000002) -meta: "c"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=0.000000003,0 min=0,0 seq=1} ts=0.000000003,0 del=false klen=12 vlen=9 -data: "c"/0.000000003,0 -> /BYTES/cput -data: "c"/0.000000001,0 -> /BYTES/value +meta: "c"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=9 +data: "c"/3.000000000,0 -> /BYTES/cput +data: "c"/1.000000000,0 -> /BYTES/value diff --git a/pkg/storage/testdata/mvcc_histories/conditional_put_write_too_old b/pkg/storage/testdata/mvcc_histories/conditional_put_write_too_old index 8c51d1c25cc8..6ae2448bf8bc 100644 --- a/pkg/storage/testdata/mvcc_histories/conditional_put_write_too_old +++ b/pkg/storage/testdata/mvcc_histories/conditional_put_write_too_old @@ -8,24 +8,24 @@ run ok put ts=10 k=k v=v1 ---- >> at end: -data: "k"/0.000000010,0 -> /BYTES/v1 +data: "k"/10.000000000,0 -> /BYTES/v1 # Try a non-transactional put @t=1 with expectation of nil; should fail. run error cput ts=1 k=k v=v2 ---- >> at end: -data: "k"/0.000000010,0 -> /BYTES/v1 -error: (*roachpb.ConditionFailedError:) unexpected value: raw_bytes:"\000\000\000\000\003v1" timestamp: +data: "k"/10.000000000,0 -> /BYTES/v1 +error: (*roachpb.ConditionFailedError:) unexpected value: raw_bytes:"\000\000\000\000\003v1" timestamp: # Now do a non-transactional put @t=1 with expectation of value1; will "succeed" @t=10,1 with WriteTooOld. run error cput ts=1 k=k v=v2 cond=v1 ---- >> at end: -data: "k"/0.000000010,1 -> /BYTES/v2 -data: "k"/0.000000010,0 -> /BYTES/v1 -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write at timestamp 0.000000001,0 too old; wrote at 0.000000010,1 +data: "k"/10.000000000,1 -> /BYTES/v2 +data: "k"/10.000000000,0 -> /BYTES/v1 +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write at timestamp 1.000000000,0 too old; wrote at 10.000000000,1 # Try a transactional put @t=1 with expectation of value2; should fail. run error @@ -34,9 +34,9 @@ with t=a cput k=k v=v2 cond=v1 ---- >> at end: -txn: "a" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000001,0 min=0,0 seq=0} lock=true stat=PENDING rts=0.000000001,0 wto=false max=0,0 -data: "k"/0.000000010,1 -> /BYTES/v2 -data: "k"/0.000000010,0 -> /BYTES/v1 +txn: "a" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=1.000000000,0 wto=false max=0,0 +data: "k"/10.000000000,1 -> /BYTES/v2 +data: "k"/10.000000000,0 -> /BYTES/v1 error: (*roachpb.ConditionFailedError:) unexpected value: # Now do a transactional put @t=1 with expectation of nil; will "succeed" @t=10,2 with WriteTooOld. @@ -45,8 +45,8 @@ with t=a cput k=k v=v3 ---- >> at end: -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000010,2 min=0,0 seq=0} ts=0.000000010,2 del=false klen=12 vlen=7 -data: "k"/0.000000010,2 -> /BYTES/v3 -data: "k"/0.000000010,1 -> /BYTES/v2 -data: "k"/0.000000010,0 -> /BYTES/v1 -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write at timestamp 0.000000001,0 too old; wrote at 0.000000010,2 +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=10.000000000,2 min=0,0 seq=0} ts=10.000000000,2 del=false klen=12 vlen=7 +data: "k"/10.000000000,2 -> /BYTES/v3 +data: "k"/10.000000000,1 -> /BYTES/v2 +data: "k"/10.000000000,0 -> /BYTES/v1 +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write at timestamp 1.000000000,0 too old; wrote at 10.000000000,2 diff --git a/pkg/storage/testdata/mvcc_histories/delete_range b/pkg/storage/testdata/mvcc_histories/delete_range index bc00560537f5..2c68b96d5044 100644 --- a/pkg/storage/testdata/mvcc_histories/delete_range +++ b/pkg/storage/testdata/mvcc_histories/delete_range @@ -12,14 +12,14 @@ with ts=44 v=abc put k=d/123 ---- >> at end: -data: "a"/0.000000044,0 -> /BYTES/abc -data: "a/123"/0.000000044,0 -> /BYTES/abc -data: "b"/0.000000044,0 -> /BYTES/abc -data: "b/123"/0.000000044,0 -> /BYTES/abc -data: "c"/0.000000044,0 -> /BYTES/abc -data: "c/123"/0.000000044,0 -> /BYTES/abc -data: "d"/0.000000044,0 -> /BYTES/abc -data: "d/123"/0.000000044,0 -> /BYTES/abc +data: "a"/44.000000000,0 -> /BYTES/abc +data: "a/123"/44.000000000,0 -> /BYTES/abc +data: "b"/44.000000000,0 -> /BYTES/abc +data: "b/123"/44.000000000,0 -> /BYTES/abc +data: "c"/44.000000000,0 -> /BYTES/abc +data: "c/123"/44.000000000,0 -> /BYTES/abc +data: "d"/44.000000000,0 -> /BYTES/abc +data: "d/123"/44.000000000,0 -> /BYTES/abc # A simple non-txn that deletes a range of keys. @@ -30,16 +30,16 @@ del_range k=a end=b ts=45 ---- del_range: "a"-"b" -> deleted 2 key(s) >> at end: -data: "a"/0.000000045,0 -> / -data: "a"/0.000000044,0 -> /BYTES/abc -data: "a/123"/0.000000045,0 -> / -data: "a/123"/0.000000044,0 -> /BYTES/abc -data: "b"/0.000000044,0 -> /BYTES/abc -data: "b/123"/0.000000044,0 -> /BYTES/abc -data: "c"/0.000000044,0 -> /BYTES/abc -data: "c/123"/0.000000044,0 -> /BYTES/abc -data: "d"/0.000000044,0 -> /BYTES/abc -data: "d/123"/0.000000044,0 -> /BYTES/abc +data: "a"/45.000000000,0 -> / +data: "a"/44.000000000,0 -> /BYTES/abc +data: "a/123"/45.000000000,0 -> / +data: "a/123"/44.000000000,0 -> /BYTES/abc +data: "b"/44.000000000,0 -> /BYTES/abc +data: "b/123"/44.000000000,0 -> /BYTES/abc +data: "c"/44.000000000,0 -> /BYTES/abc +data: "c/123"/44.000000000,0 -> /BYTES/abc +data: "d"/44.000000000,0 -> /BYTES/abc +data: "d/123"/44.000000000,0 -> /BYTES/abc # A simple txn that deletes a range of keys. @@ -56,20 +56,20 @@ del_range: "b"-"c" -> deleted 2 key(s) del_range: returned "b" del_range: returned "b/123" >> at end: -data: "a"/0.000000045,0 -> / -data: "a"/0.000000044,0 -> /BYTES/abc -data: "a/123"/0.000000045,0 -> / -data: "a/123"/0.000000044,0 -> /BYTES/abc -meta: "b"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000046,0 min=0,0 seq=0} ts=0.000000046,0 del=true klen=12 vlen=0 -data: "b"/0.000000046,0 -> / -data: "b"/0.000000044,0 -> /BYTES/abc -meta: "b/123"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000046,0 min=0,0 seq=0} ts=0.000000046,0 del=true klen=12 vlen=0 -data: "b/123"/0.000000046,0 -> / -data: "b/123"/0.000000044,0 -> /BYTES/abc -data: "c"/0.000000044,0 -> /BYTES/abc -data: "c/123"/0.000000044,0 -> /BYTES/abc -data: "d"/0.000000044,0 -> /BYTES/abc -data: "d/123"/0.000000044,0 -> /BYTES/abc +data: "a"/45.000000000,0 -> / +data: "a"/44.000000000,0 -> /BYTES/abc +data: "a/123"/45.000000000,0 -> / +data: "a/123"/44.000000000,0 -> /BYTES/abc +meta: "b"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=46.000000000,0 min=0,0 seq=0} ts=46.000000000,0 del=true klen=12 vlen=0 +data: "b"/46.000000000,0 -> / +data: "b"/44.000000000,0 -> /BYTES/abc +meta: "b/123"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=46.000000000,0 min=0,0 seq=0} ts=46.000000000,0 del=true klen=12 vlen=0 +data: "b/123"/46.000000000,0 -> / +data: "b/123"/44.000000000,0 -> /BYTES/abc +data: "c"/44.000000000,0 -> /BYTES/abc +data: "c/123"/44.000000000,0 -> /BYTES/abc +data: "d"/44.000000000,0 -> /BYTES/abc +data: "d/123"/44.000000000,0 -> /BYTES/abc # A limited non-txn that deletes a range of keys. @@ -84,22 +84,22 @@ del_range: returned "c" del_range: returned "c/123" del_range: resume span ["d","z") >> at end: -data: "a"/0.000000045,0 -> / -data: "a"/0.000000044,0 -> /BYTES/abc -data: "a/123"/0.000000045,0 -> / -data: "a/123"/0.000000044,0 -> /BYTES/abc -meta: "b"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000046,0 min=0,0 seq=0} ts=0.000000046,0 del=true klen=12 vlen=0 -data: "b"/0.000000046,0 -> / -data: "b"/0.000000044,0 -> /BYTES/abc -meta: "b/123"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000046,0 min=0,0 seq=0} ts=0.000000046,0 del=true klen=12 vlen=0 -data: "b/123"/0.000000046,0 -> / -data: "b/123"/0.000000044,0 -> /BYTES/abc -data: "c"/0.000000047,0 -> / -data: "c"/0.000000044,0 -> /BYTES/abc -data: "c/123"/0.000000047,0 -> / -data: "c/123"/0.000000044,0 -> /BYTES/abc -data: "d"/0.000000044,0 -> /BYTES/abc -data: "d/123"/0.000000044,0 -> /BYTES/abc +data: "a"/45.000000000,0 -> / +data: "a"/44.000000000,0 -> /BYTES/abc +data: "a/123"/45.000000000,0 -> / +data: "a/123"/44.000000000,0 -> /BYTES/abc +meta: "b"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=46.000000000,0 min=0,0 seq=0} ts=46.000000000,0 del=true klen=12 vlen=0 +data: "b"/46.000000000,0 -> / +data: "b"/44.000000000,0 -> /BYTES/abc +meta: "b/123"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=46.000000000,0 min=0,0 seq=0} ts=46.000000000,0 del=true klen=12 vlen=0 +data: "b/123"/46.000000000,0 -> / +data: "b/123"/44.000000000,0 -> /BYTES/abc +data: "c"/47.000000000,0 -> / +data: "c"/44.000000000,0 -> /BYTES/abc +data: "c/123"/47.000000000,0 -> / +data: "c/123"/44.000000000,0 -> /BYTES/abc +data: "d"/44.000000000,0 -> /BYTES/abc +data: "d/123"/44.000000000,0 -> /BYTES/abc # A txn that performs a delete range at a lower timestamp returns a WriteTooOld error. @@ -111,24 +111,24 @@ with t=A txn_remove ---- >> at end: -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000046,0 min=0,0 seq=0} lock=true stat=PENDING rts=0.000000046,0 wto=false max=0,0 -data: "a"/0.000000045,0 -> / -data: "a"/0.000000044,0 -> /BYTES/abc -data: "a/123"/0.000000045,0 -> / -data: "a/123"/0.000000044,0 -> /BYTES/abc -meta: "b"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000046,0 min=0,0 seq=0} ts=0.000000046,0 del=true klen=12 vlen=0 -data: "b"/0.000000046,0 -> / -data: "b"/0.000000044,0 -> /BYTES/abc -meta: "b/123"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000046,0 min=0,0 seq=0} ts=0.000000046,0 del=true klen=12 vlen=0 -data: "b/123"/0.000000046,0 -> / -data: "b/123"/0.000000044,0 -> /BYTES/abc -data: "c"/0.000000047,0 -> / -data: "c"/0.000000044,0 -> /BYTES/abc -data: "c/123"/0.000000047,0 -> / -data: "c/123"/0.000000044,0 -> /BYTES/abc -data: "d"/0.000000044,0 -> /BYTES/abc -data: "d/123"/0.000000044,0 -> /BYTES/abc -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write at timestamp 0.000000046,0 too old; wrote at 0.000000047,1 +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=46.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=46.000000000,0 wto=false max=0,0 +data: "a"/45.000000000,0 -> / +data: "a"/44.000000000,0 -> /BYTES/abc +data: "a/123"/45.000000000,0 -> / +data: "a/123"/44.000000000,0 -> /BYTES/abc +meta: "b"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=46.000000000,0 min=0,0 seq=0} ts=46.000000000,0 del=true klen=12 vlen=0 +data: "b"/46.000000000,0 -> / +data: "b"/44.000000000,0 -> /BYTES/abc +meta: "b/123"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=46.000000000,0 min=0,0 seq=0} ts=46.000000000,0 del=true klen=12 vlen=0 +data: "b/123"/46.000000000,0 -> / +data: "b/123"/44.000000000,0 -> /BYTES/abc +data: "c"/47.000000000,0 -> / +data: "c"/44.000000000,0 -> /BYTES/abc +data: "c/123"/47.000000000,0 -> / +data: "c/123"/44.000000000,0 -> /BYTES/abc +data: "d"/44.000000000,0 -> /BYTES/abc +data: "d/123"/44.000000000,0 -> /BYTES/abc +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write at timestamp 46.000000000,0 too old; wrote at 47.000000000,1 run ok txn_remove t=A @@ -148,23 +148,23 @@ del_range: "c"-"z" -> deleted 2 key(s) del_range: returned "d" del_range: returned "d/123" >> at end: -data: "a"/0.000000045,0 -> / -data: "a"/0.000000044,0 -> /BYTES/abc -data: "a/123"/0.000000045,0 -> / -data: "a/123"/0.000000044,0 -> /BYTES/abc -meta: "b"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000046,0 min=0,0 seq=0} ts=0.000000046,0 del=true klen=12 vlen=0 -data: "b"/0.000000046,0 -> / -data: "b"/0.000000044,0 -> /BYTES/abc -meta: "b/123"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000046,0 min=0,0 seq=0} ts=0.000000046,0 del=true klen=12 vlen=0 -data: "b/123"/0.000000046,0 -> / -data: "b/123"/0.000000044,0 -> /BYTES/abc -data: "c"/0.000000047,0 -> / -data: "c"/0.000000044,0 -> /BYTES/abc -data: "c/123"/0.000000047,0 -> / -data: "c/123"/0.000000044,0 -> /BYTES/abc -meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000048,0 min=0,0 seq=0} ts=0.000000048,0 del=true klen=12 vlen=0 -data: "d"/0.000000048,0 -> / -data: "d"/0.000000044,0 -> /BYTES/abc -meta: "d/123"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000048,0 min=0,0 seq=0} ts=0.000000048,0 del=true klen=12 vlen=0 -data: "d/123"/0.000000048,0 -> / -data: "d/123"/0.000000044,0 -> /BYTES/abc +data: "a"/45.000000000,0 -> / +data: "a"/44.000000000,0 -> /BYTES/abc +data: "a/123"/45.000000000,0 -> / +data: "a/123"/44.000000000,0 -> /BYTES/abc +meta: "b"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=46.000000000,0 min=0,0 seq=0} ts=46.000000000,0 del=true klen=12 vlen=0 +data: "b"/46.000000000,0 -> / +data: "b"/44.000000000,0 -> /BYTES/abc +meta: "b/123"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=46.000000000,0 min=0,0 seq=0} ts=46.000000000,0 del=true klen=12 vlen=0 +data: "b/123"/46.000000000,0 -> / +data: "b/123"/44.000000000,0 -> /BYTES/abc +data: "c"/47.000000000,0 -> / +data: "c"/44.000000000,0 -> /BYTES/abc +data: "c/123"/47.000000000,0 -> / +data: "c/123"/44.000000000,0 -> /BYTES/abc +meta: "d"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=48.000000000,0 min=0,0 seq=0} ts=48.000000000,0 del=true klen=12 vlen=0 +data: "d"/48.000000000,0 -> / +data: "d"/44.000000000,0 -> /BYTES/abc +meta: "d/123"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=48.000000000,0 min=0,0 seq=0} ts=48.000000000,0 del=true klen=12 vlen=0 +data: "d/123"/48.000000000,0 -> / +data: "d/123"/44.000000000,0 -> /BYTES/abc diff --git a/pkg/storage/testdata/mvcc_histories/deletes b/pkg/storage/testdata/mvcc_histories/deletes index d062c99ba739..a2422ae3bce3 100644 --- a/pkg/storage/testdata/mvcc_histories/deletes +++ b/pkg/storage/testdata/mvcc_histories/deletes @@ -8,7 +8,7 @@ with t=A txn_remove ---- >> at end: -data: "a"/0.000000044,0 -> / +data: "a"/44.000000000,0 -> / # Show the value disappears from gets. @@ -29,5 +29,5 @@ with t=A get k=a tombstones txn_remove ---- -get: "a" -> / @0.000000044,0 +get: "a" -> / @44.000000000,0 >> at end: diff --git a/pkg/storage/testdata/mvcc_histories/get_negative_timestamp b/pkg/storage/testdata/mvcc_histories/get_negative_timestamp index ffcd2ce7744b..409c31179c99 100644 --- a/pkg/storage/testdata/mvcc_histories/get_negative_timestamp +++ b/pkg/storage/testdata/mvcc_histories/get_negative_timestamp @@ -2,18 +2,18 @@ run ok put k=k v=v ts=1 ---- >> at end: -data: "k"/0.000000001,0 -> /BYTES/v +data: "k"/1.000000000,0 -> /BYTES/v run error get k=k ts=-1 ---- get: "k" -> -error: (*withstack.withStack:) cannot write to "k" at timestamp -0.000000001,0 +error: (*withstack.withStack:) cannot write to "k" at timestamp -1.000000000,0 run error put k=k v=v ts=-1 ---- >> at end: -data: "k"/0.000000001,0 -> /BYTES/v -error: (*withstack.withStack:) cannot write to "k" at timestamp -0.000000001,0 +data: "k"/1.000000000,0 -> /BYTES/v +error: (*withstack.withStack:) cannot write to "k" at timestamp -1.000000000,0 diff --git a/pkg/storage/testdata/mvcc_histories/idempotent_transactions b/pkg/storage/testdata/mvcc_histories/idempotent_transactions index 5ee096a74d5d..f58812b87f80 100644 --- a/pkg/storage/testdata/mvcc_histories/idempotent_transactions +++ b/pkg/storage/testdata/mvcc_histories/idempotent_transactions @@ -7,9 +7,9 @@ with t=a k=a put v=first ---- >> at end: -txn: "a" meta={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=0} lock=true stat=PENDING rts=0.000000011,0 wto=false max=0,0 -meta: "a"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=0} ts=0.000000011,0 del=false klen=12 vlen=10 -data: "a"/0.000000011,0 -> /BYTES/first +txn: "a" meta={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=11.000000000,0 wto=false max=0,0 +meta: "a"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=0} ts=11.000000000,0 del=false klen=12 vlen=10 +data: "a"/11.000000000,0 -> /BYTES/first # Lay down an intent without increasing the sequence but with a different value. # Expect an error. @@ -18,8 +18,8 @@ with t=a k=a put v=second ---- >> at end: -meta: "a"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=0} ts=0.000000011,0 del=false klen=12 vlen=10 -data: "a"/0.000000011,0 -> /BYTES/first +meta: "a"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=0} ts=11.000000000,0 del=false klen=12 vlen=10 +data: "a"/11.000000000,0 -> /BYTES/first error: (*withstack.withStack:) transaction 00000000-0000-0000-0000-000000000001 with sequence 0 has a different value [0 0 0 0 3 115 101 99 111 110 100] after recomputing from what was written: [0 0 0 0 3 102 105 114 115 116] run ok @@ -34,11 +34,11 @@ with t=a k=a txn_step check_intent ---- -meta: "a" -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=1} ts=0.000000011,0 del=false klen=12 vlen=11 ih={{0 /BYTES/first}} +meta: "a" -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=1} ts=11.000000000,0 del=false klen=12 vlen=11 ih={{0 /BYTES/first}} >> at end: -txn: "a" meta={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=1} lock=true stat=PENDING rts=0.000000011,0 wto=false max=0,0 -meta: "a"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=1} ts=0.000000011,0 del=false klen=12 vlen=11 ih={{0 /BYTES/first}} -data: "a"/0.000000011,0 -> /BYTES/second +txn: "a" meta={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=1} lock=true stat=PENDING rts=11.000000000,0 wto=false max=0,0 +meta: "a"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=1} ts=11.000000000,0 del=false klen=12 vlen=11 ih={{0 /BYTES/first}} +data: "a"/11.000000000,0 -> /BYTES/second run error with t=a k=a @@ -47,9 +47,9 @@ with t=a k=a put v=second ---- >> at end: -txn: "a" meta={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=-1} lock=true stat=PENDING rts=0.000000011,0 wto=false max=0,0 -meta: "a"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=1} ts=0.000000011,0 del=false klen=12 vlen=11 ih={{0 /BYTES/first}} -data: "a"/0.000000011,0 -> /BYTES/second +txn: "a" meta={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=-1} lock=true stat=PENDING rts=11.000000000,0 wto=false max=0,0 +meta: "a"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=1} ts=11.000000000,0 del=false klen=12 vlen=11 ih={{0 /BYTES/first}} +data: "a"/11.000000000,0 -> /BYTES/second error: (*withstack.withStack:) transaction 00000000-0000-0000-0000-000000000001 with sequence 1 missing an intent with lower sequence -1 run ok @@ -67,11 +67,11 @@ inc: current value = 1 inc: current value = 1 inc: current value = 1 >> at end: -txn: "a" meta={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=2} lock=true stat=PENDING rts=0.000000011,0 wto=false max=0,0 -meta: "a"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=1} ts=0.000000011,0 del=false klen=12 vlen=11 ih={{0 /BYTES/first}} -data: "a"/0.000000011,0 -> /BYTES/second -meta: "i"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=2} ts=0.000000011,0 del=false klen=12 vlen=6 -data: "i"/0.000000011,0 -> /INT/1 +txn: "a" meta={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=2} lock=true stat=PENDING rts=11.000000000,0 wto=false max=0,0 +meta: "a"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=1} ts=11.000000000,0 del=false klen=12 vlen=11 ih={{0 /BYTES/first}} +data: "a"/11.000000000,0 -> /BYTES/second +meta: "i"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=2} ts=11.000000000,0 del=false klen=12 vlen=6 +data: "i"/11.000000000,0 -> /INT/1 run ok with t=a k=i @@ -94,11 +94,11 @@ inc: current value = 1 inc: current value = 1 inc: current value = 1 >> at end: -txn: "a" meta={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=2} lock=true stat=PENDING rts=0.000000011,0 wto=false max=0,0 -meta: "a"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=1} ts=0.000000011,0 del=false klen=12 vlen=11 ih={{0 /BYTES/first}} -data: "a"/0.000000011,0 -> /BYTES/second -meta: "i"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=3} ts=0.000000011,0 del=false klen=12 vlen=6 ih={{2 /INT/1}} -data: "i"/0.000000011,0 -> /INT/2 +txn: "a" meta={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=2} lock=true stat=PENDING rts=11.000000000,0 wto=false max=0,0 +meta: "a"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=1} ts=11.000000000,0 del=false klen=12 vlen=11 ih={{0 /BYTES/first}} +data: "a"/11.000000000,0 -> /BYTES/second +meta: "i"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=3} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{2 /INT/1}} +data: "i"/11.000000000,0 -> /INT/2 # Write a key outside of the transaction. run ok @@ -106,11 +106,11 @@ increment k=i2 ts=10 ---- inc: current value = 1 >> at end: -meta: "a"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=1} ts=0.000000011,0 del=false klen=12 vlen=11 ih={{0 /BYTES/first}} -data: "a"/0.000000011,0 -> /BYTES/second -meta: "i"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=3} ts=0.000000011,0 del=false klen=12 vlen=6 ih={{2 /INT/1}} -data: "i"/0.000000011,0 -> /INT/2 -data: "i2"/0.000000010,0 -> /INT/1 +meta: "a"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=1} ts=11.000000000,0 del=false klen=12 vlen=11 ih={{0 /BYTES/first}} +data: "a"/11.000000000,0 -> /BYTES/second +meta: "i"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=3} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{2 /INT/1}} +data: "i"/11.000000000,0 -> /INT/2 +data: "i2"/10.000000000,0 -> /INT/1 run ok with t=a k=i2 @@ -127,14 +127,14 @@ inc: current value = 2 inc: current value = 2 inc: current value = 2 >> at end: -txn: "a" meta={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=4} lock=true stat=PENDING rts=0.000000011,0 wto=false max=0,0 -meta: "a"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=1} ts=0.000000011,0 del=false klen=12 vlen=11 ih={{0 /BYTES/first}} -data: "a"/0.000000011,0 -> /BYTES/second -meta: "i"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=3} ts=0.000000011,0 del=false klen=12 vlen=6 ih={{2 /INT/1}} -data: "i"/0.000000011,0 -> /INT/2 -meta: "i2"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=4} ts=0.000000011,0 del=false klen=12 vlen=6 -data: "i2"/0.000000011,0 -> /INT/2 -data: "i2"/0.000000010,0 -> /INT/1 +txn: "a" meta={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=4} lock=true stat=PENDING rts=11.000000000,0 wto=false max=0,0 +meta: "a"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=1} ts=11.000000000,0 del=false klen=12 vlen=11 ih={{0 /BYTES/first}} +data: "a"/11.000000000,0 -> /BYTES/second +meta: "i"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=3} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{2 /INT/1}} +data: "i"/11.000000000,0 -> /INT/2 +meta: "i2"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=4} ts=11.000000000,0 del=false klen=12 vlen=6 +data: "i2"/11.000000000,0 -> /INT/2 +data: "i2"/10.000000000,0 -> /INT/1 run ok with t=a k=i2 @@ -157,11 +157,11 @@ inc: current value = 2 inc: current value = 2 inc: current value = 2 >> at end: -txn: "a" meta={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=4} lock=true stat=PENDING rts=0.000000011,0 wto=false max=0,0 -meta: "a"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=1} ts=0.000000011,0 del=false klen=12 vlen=11 ih={{0 /BYTES/first}} -data: "a"/0.000000011,0 -> /BYTES/second -meta: "i"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=3} ts=0.000000011,0 del=false klen=12 vlen=6 ih={{2 /INT/1}} -data: "i"/0.000000011,0 -> /INT/2 -meta: "i2"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=5} ts=0.000000011,0 del=false klen=12 vlen=6 ih={{4 /INT/2}} -data: "i2"/0.000000011,0 -> /INT/3 -data: "i2"/0.000000010,0 -> /INT/1 +txn: "a" meta={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=4} lock=true stat=PENDING rts=11.000000000,0 wto=false max=0,0 +meta: "a"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=1} ts=11.000000000,0 del=false klen=12 vlen=11 ih={{0 /BYTES/first}} +data: "a"/11.000000000,0 -> /BYTES/second +meta: "i"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=3} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{2 /INT/1}} +data: "i"/11.000000000,0 -> /INT/2 +meta: "i2"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=5} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{4 /INT/2}} +data: "i2"/11.000000000,0 -> /INT/3 +data: "i2"/10.000000000,0 -> /INT/1 diff --git a/pkg/storage/testdata/mvcc_histories/ignored_seq_nums b/pkg/storage/testdata/mvcc_histories/ignored_seq_nums index 952b20abe0fe..b57f0702e282 100644 --- a/pkg/storage/testdata/mvcc_histories/ignored_seq_nums +++ b/pkg/storage/testdata/mvcc_histories/ignored_seq_nums @@ -16,15 +16,15 @@ with t=A txn_step seq=40 ---- >> at end: -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=40} lock=true stat=PENDING rts=0.000000011,0 wto=false max=0,0 -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=30} ts=0.000000011,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} -data: "k"/0.000000011,0 -> /BYTES/c -meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=10} ts=0.000000011,0 del=false klen=12 vlen=7 -data: "k/10"/0.000000011,0 -> /BYTES/10 -meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=20} ts=0.000000011,0 del=false klen=12 vlen=7 -data: "k/20"/0.000000011,0 -> /BYTES/20 -meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=30} ts=0.000000011,0 del=false klen=12 vlen=7 -data: "k/30"/0.000000011,0 -> /BYTES/30 +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=40} lock=true stat=PENDING rts=11.000000000,0 wto=false max=0,0 +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +data: "k"/11.000000000,0 -> /BYTES/c +meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 +data: "k/10"/11.000000000,0 -> /BYTES/10 +meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 +data: "k/20"/11.000000000,0 -> /BYTES/20 +meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 +data: "k/30"/11.000000000,0 -> /BYTES/30 # Mask a single write. # The third write should be hidden now. @@ -39,14 +39,14 @@ with t=A txn_ignore_seqs seqs=(29-30) get k=k ---- -scan: "k" -> /BYTES/b @0.000000011,0 -scan: "k/10" -> /BYTES/10 @0.000000011,0 -scan: "k/20" -> /BYTES/20 @0.000000011,0 -get: "k" -> /BYTES/b @0.000000011,0 -get: "k" -> /BYTES/b @0.000000011,0 -get: "k" -> /BYTES/b @0.000000011,0 +scan: "k" -> /BYTES/b @11.000000000,0 +scan: "k/10" -> /BYTES/10 @11.000000000,0 +scan: "k/20" -> /BYTES/20 @11.000000000,0 +get: "k" -> /BYTES/b @11.000000000,0 +get: "k" -> /BYTES/b @11.000000000,0 +get: "k" -> /BYTES/b @11.000000000,0 >> at end: -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=40} lock=true stat=PENDING rts=0.000000011,0 wto=false max=0,0 isn=1 +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=40} lock=true stat=PENDING rts=11.000000000,0 wto=false max=0,0 isn=1 # Mask a write in the middle. @@ -57,12 +57,12 @@ with t=A scan k=k end=-k get k=k ---- -scan: "k" -> /BYTES/c @0.000000011,0 -scan: "k/10" -> /BYTES/10 @0.000000011,0 -scan: "k/30" -> /BYTES/30 @0.000000011,0 -get: "k" -> /BYTES/c @0.000000011,0 +scan: "k" -> /BYTES/c @11.000000000,0 +scan: "k/10" -> /BYTES/10 @11.000000000,0 +scan: "k/30" -> /BYTES/30 @11.000000000,0 +get: "k" -> /BYTES/c @11.000000000,0 >> at end: -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=40} lock=true stat=PENDING rts=0.000000011,0 wto=false max=0,0 isn=1 +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=40} lock=true stat=PENDING rts=11.000000000,0 wto=false max=0,0 isn=1 # Mask all the writes. @@ -76,7 +76,7 @@ with t=A scan: "k"-"l" -> get: "k" -> >> at end: -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=40} lock=true stat=PENDING rts=0.000000011,0 wto=false max=0,0 isn=1 +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=40} lock=true stat=PENDING rts=11.000000000,0 wto=false max=0,0 isn=1 # Disjoint masks. @@ -87,11 +87,11 @@ with t=A scan k=k end=-k get k=k ---- -scan: "k" -> /BYTES/b @0.000000011,0 -scan: "k/20" -> /BYTES/20 @0.000000011,0 -get: "k" -> /BYTES/b @0.000000011,0 +scan: "k" -> /BYTES/b @11.000000000,0 +scan: "k/20" -> /BYTES/20 @11.000000000,0 +get: "k" -> /BYTES/b @11.000000000,0 >> at end: -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=40} lock=true stat=PENDING rts=0.000000011,0 wto=false max=0,0 isn=2 +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=40} lock=true stat=PENDING rts=11.000000000,0 wto=false max=0,0 isn=2 # A historical read before the ignored range should retrieve the # historical value visible at that point. @@ -103,11 +103,11 @@ with t=A scan k=k end=-k get k=k ---- -scan: "k" -> /BYTES/a @0.000000011,0 -scan: "k/10" -> /BYTES/10 @0.000000011,0 -get: "k" -> /BYTES/a @0.000000011,0 +scan: "k" -> /BYTES/a @11.000000000,0 +scan: "k/10" -> /BYTES/10 @11.000000000,0 +get: "k" -> /BYTES/a @11.000000000,0 >> at end: -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=12} lock=true stat=PENDING rts=0.000000011,0 wto=false max=0,0 isn=1 +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=12} lock=true stat=PENDING rts=11.000000000,0 wto=false max=0,0 isn=1 # A historical read with an ignored range before it should hide # the historical value hidden at that point. @@ -119,11 +119,11 @@ with t=A scan k=k end=-k get k=k ---- -scan: "k" -> /BYTES/b @0.000000011,0 -scan: "k/20" -> /BYTES/20 @0.000000011,0 -get: "k" -> /BYTES/b @0.000000011,0 +scan: "k" -> /BYTES/b @11.000000000,0 +scan: "k/20" -> /BYTES/20 @11.000000000,0 +get: "k" -> /BYTES/b @11.000000000,0 >> at end: -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=22} lock=true stat=PENDING rts=0.000000011,0 wto=false max=0,0 isn=1 +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=22} lock=true stat=PENDING rts=11.000000000,0 wto=false max=0,0 isn=1 # A historical read with an ignored range that overlaps should hide # the historical value hidden at that point. @@ -135,12 +135,12 @@ with t=A scan k=k end=-k get k=k ---- -scan: "k" -> /BYTES/b @0.000000011,0 -scan: "k/10" -> /BYTES/10 @0.000000011,0 -scan: "k/20" -> /BYTES/20 @0.000000011,0 -get: "k" -> /BYTES/b @0.000000011,0 +scan: "k" -> /BYTES/b @11.000000000,0 +scan: "k/10" -> /BYTES/10 @11.000000000,0 +scan: "k/20" -> /BYTES/20 @11.000000000,0 +get: "k" -> /BYTES/b @11.000000000,0 >> at end: -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=32} lock=true stat=PENDING rts=0.000000011,0 wto=false max=0,0 isn=1 +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=32} lock=true stat=PENDING rts=11.000000000,0 wto=false max=0,0 isn=1 # Do an intent push by advancing the transaction timestamp, while also having # a range of ignored seqnums. This should permanently delete the value at seqnum @@ -158,21 +158,21 @@ with t=A check_intent k=k get k=k ---- -meta: "k" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=30} ts=0.000000011,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} -get: "k" -> /BYTES/b @0.000000011,0 -get: "k" -> /BYTES/b @0.000000011,0 -meta: "k" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000014,0 min=0,0 seq=20} ts=0.000000014,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} -get: "k" -> /BYTES/b @0.000000014,0 +meta: "k" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +get: "k" -> /BYTES/b @11.000000000,0 +get: "k" -> /BYTES/b @11.000000000,0 +meta: "k" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=20} ts=14.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} +get: "k" -> /BYTES/b @14.000000000,0 >> at end: -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000014,0 min=0,0 seq=32} lock=true stat=PENDING rts=0.000000011,0 wto=false max=0,0 isn=1 -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000014,0 min=0,0 seq=20} ts=0.000000014,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} -data: "k"/0.000000014,0 -> /BYTES/b -meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=10} ts=0.000000011,0 del=false klen=12 vlen=7 -data: "k/10"/0.000000011,0 -> /BYTES/10 -meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=20} ts=0.000000011,0 del=false klen=12 vlen=7 -data: "k/20"/0.000000011,0 -> /BYTES/20 -meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=30} ts=0.000000011,0 del=false klen=12 vlen=7 -data: "k/30"/0.000000011,0 -> /BYTES/30 +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=32} lock=true stat=PENDING rts=11.000000000,0 wto=false max=0,0 isn=1 +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=20} ts=14.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} +data: "k"/14.000000000,0 -> /BYTES/b +meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 +data: "k/10"/11.000000000,0 -> /BYTES/10 +meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 +data: "k/20"/11.000000000,0 -> /BYTES/20 +meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 +data: "k/30"/11.000000000,0 -> /BYTES/30 # Ensure that the deleted value doens't surface. Instead, if we ignore the # now-newest seq, we get the write before it. @@ -184,12 +184,12 @@ with t=A scan k=k end=-k get k=k ---- -scan: "k" -> /BYTES/a @0.000000014,0 -scan: "k/10" -> /BYTES/10 @0.000000011,0 -scan: "k/30" -> /BYTES/30 @0.000000011,0 -get: "k" -> /BYTES/a @0.000000014,0 +scan: "k" -> /BYTES/a @14.000000000,0 +scan: "k/10" -> /BYTES/10 @11.000000000,0 +scan: "k/30" -> /BYTES/30 @11.000000000,0 +get: "k" -> /BYTES/a @14.000000000,0 >> at end: -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000014,0 min=0,0 seq=40} lock=true stat=PENDING rts=0.000000011,0 wto=false max=0,0 isn=1 +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=40} lock=true stat=PENDING rts=11.000000000,0 wto=false max=0,0 isn=1 run ok with t=A @@ -201,17 +201,17 @@ with t=A scan k=k end=-k get k=k ---- -scan: "k" -> /BYTES/b @0.000000014,0 -scan: "k/10" -> /BYTES/10 @0.000000011,0 -scan: "k/20" -> /BYTES/20 @0.000000011,0 -scan: "k/30" -> /BYTES/30 @0.000000011,0 -get: "k" -> /BYTES/b @0.000000014,0 -scan: "k" -> /BYTES/b @0.000000014,0 -scan: "k/10" -> /BYTES/10 @0.000000011,0 -scan: "k/20" -> /BYTES/20 @0.000000011,0 -get: "k" -> /BYTES/b @0.000000014,0 +scan: "k" -> /BYTES/b @14.000000000,0 +scan: "k/10" -> /BYTES/10 @11.000000000,0 +scan: "k/20" -> /BYTES/20 @11.000000000,0 +scan: "k/30" -> /BYTES/30 @11.000000000,0 +get: "k" -> /BYTES/b @14.000000000,0 +scan: "k" -> /BYTES/b @14.000000000,0 +scan: "k/10" -> /BYTES/10 @11.000000000,0 +scan: "k/20" -> /BYTES/20 @11.000000000,0 +get: "k" -> /BYTES/b @14.000000000,0 >> at end: -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000014,0 min=0,0 seq=25} lock=true stat=PENDING rts=0.000000011,0 wto=false max=0,0 isn=1 +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=25} lock=true stat=PENDING rts=11.000000000,0 wto=false max=0,0 isn=1 # Call mvccResolveWriteIntent with status=COMMITTED. This should fold the # intent while leaving the value unmodified. @@ -225,17 +225,17 @@ with t=B txn_begin ts=20 get k=k ---- -meta: "k" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000014,0 min=0,0 seq=20} ts=0.000000014,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} -get: "k" -> /BYTES/b @0.000000014,0 +meta: "k" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=14.000000000,0 min=0,0 seq=20} ts=14.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} +get: "k" -> /BYTES/b @14.000000000,0 >> at end: -txn: "B" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000020,0 min=0,0 seq=0} lock=true stat=PENDING rts=0.000000020,0 wto=false max=0,0 -data: "k"/0.000000014,0 -> /BYTES/b -meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=10} ts=0.000000011,0 del=false klen=12 vlen=7 -data: "k/10"/0.000000011,0 -> /BYTES/10 -meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=20} ts=0.000000011,0 del=false klen=12 vlen=7 -data: "k/20"/0.000000011,0 -> /BYTES/20 -meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=30} ts=0.000000011,0 del=false klen=12 vlen=7 -data: "k/30"/0.000000011,0 -> /BYTES/30 +txn: "B" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=20.000000000,0 wto=false max=0,0 +data: "k"/14.000000000,0 -> /BYTES/b +meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 +data: "k/10"/11.000000000,0 -> /BYTES/10 +meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 +data: "k/20"/11.000000000,0 -> /BYTES/20 +meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 +data: "k/30"/11.000000000,0 -> /BYTES/30 # Put a couple values at different sequence numbers in this new txn. @@ -251,19 +251,19 @@ with t=B check_intent k=l get k=l ---- -meta: "l" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000020,0 min=0,0 seq=30} ts=0.000000020,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} -get: "l" -> /BYTES/c @0.000000020,0 +meta: "l" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=30} ts=20.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +get: "l" -> /BYTES/c @20.000000000,0 >> at end: -txn: "B" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000020,0 min=0,0 seq=30} lock=true stat=PENDING rts=0.000000020,0 wto=false max=0,0 -data: "k"/0.000000014,0 -> /BYTES/b -meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=10} ts=0.000000011,0 del=false klen=12 vlen=7 -data: "k/10"/0.000000011,0 -> /BYTES/10 -meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=20} ts=0.000000011,0 del=false klen=12 vlen=7 -data: "k/20"/0.000000011,0 -> /BYTES/20 -meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=30} ts=0.000000011,0 del=false klen=12 vlen=7 -data: "k/30"/0.000000011,0 -> /BYTES/30 -meta: "l"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000020,0 min=0,0 seq=30} ts=0.000000020,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} -data: "l"/0.000000020,0 -> /BYTES/c +txn: "B" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=30} lock=true stat=PENDING rts=20.000000000,0 wto=false max=0,0 +data: "k"/14.000000000,0 -> /BYTES/b +meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 +data: "k/10"/11.000000000,0 -> /BYTES/10 +meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 +data: "k/20"/11.000000000,0 -> /BYTES/20 +meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 +data: "k/30"/11.000000000,0 -> /BYTES/30 +meta: "l"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=30} ts=20.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +data: "l"/20.000000000,0 -> /BYTES/c # Blow up the entire intent history, then do a commit. No value or intent should @@ -278,17 +278,17 @@ with t=B resolve_intent k=l status=COMMITTED check_intent k=l ---- -meta: "l" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000020,0 min=0,0 seq=30} ts=0.000000020,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +meta: "l" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=30} ts=20.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} get: "l" -> >> at end: -txn: "B" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000020,0 min=0,0 seq=35} lock=true stat=PENDING rts=0.000000020,0 wto=false max=0,0 isn=1 -data: "k"/0.000000014,0 -> /BYTES/b -meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=10} ts=0.000000011,0 del=false klen=12 vlen=7 -data: "k/10"/0.000000011,0 -> /BYTES/10 -meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=20} ts=0.000000011,0 del=false klen=12 vlen=7 -data: "k/20"/0.000000011,0 -> /BYTES/20 -meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=30} ts=0.000000011,0 del=false klen=12 vlen=7 -data: "k/30"/0.000000011,0 -> /BYTES/30 +txn: "B" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=35} lock=true stat=PENDING rts=20.000000000,0 wto=false max=0,0 isn=1 +data: "k"/14.000000000,0 -> /BYTES/b +meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 +data: "k/10"/11.000000000,0 -> /BYTES/10 +meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 +data: "k/20"/11.000000000,0 -> /BYTES/20 +meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 +data: "k/30"/11.000000000,0 -> /BYTES/30 error: (*withstack.withStack:) meta: "l" -> expected intent, found none @@ -299,7 +299,7 @@ with t=C ---- get: "l" -> >> at end: -txn: "C" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000030,0 min=0,0 seq=0} lock=true stat=PENDING rts=0.000000030,0 wto=false max=0,0 +txn: "C" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=30.000000000,0 wto=false max=0,0 # Put some values, then ignore all except the first, then do a commit. The @@ -316,19 +316,19 @@ with t=C check_intent k=m get k=m ---- -meta: "m" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000030,0 min=0,0 seq=30} ts=0.000000030,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} -get: "m" -> /BYTES/c @0.000000030,0 +meta: "m" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=30} ts=30.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +get: "m" -> /BYTES/c @30.000000000,0 >> at end: -txn: "C" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000030,0 min=0,0 seq=30} lock=true stat=PENDING rts=0.000000030,0 wto=false max=0,0 -data: "k"/0.000000014,0 -> /BYTES/b -meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=10} ts=0.000000011,0 del=false klen=12 vlen=7 -data: "k/10"/0.000000011,0 -> /BYTES/10 -meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=20} ts=0.000000011,0 del=false klen=12 vlen=7 -data: "k/20"/0.000000011,0 -> /BYTES/20 -meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=30} ts=0.000000011,0 del=false klen=12 vlen=7 -data: "k/30"/0.000000011,0 -> /BYTES/30 -meta: "m"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000030,0 min=0,0 seq=30} ts=0.000000030,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} -data: "m"/0.000000030,0 -> /BYTES/c +txn: "C" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=30} lock=true stat=PENDING rts=30.000000000,0 wto=false max=0,0 +data: "k"/14.000000000,0 -> /BYTES/b +meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 +data: "k/10"/11.000000000,0 -> /BYTES/10 +meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 +data: "k/20"/11.000000000,0 -> /BYTES/20 +meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 +data: "k/30"/11.000000000,0 -> /BYTES/30 +meta: "m"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=30} ts=30.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +data: "m"/30.000000000,0 -> /BYTES/c run ok @@ -338,18 +338,18 @@ with t=C get k=m resolve_intent k=m status=COMMITTED ---- -meta: "m" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000030,0 min=0,0 seq=30} ts=0.000000030,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} -get: "m" -> /BYTES/a @0.000000030,0 +meta: "m" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=30} ts=30.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +get: "m" -> /BYTES/a @30.000000000,0 >> at end: -txn: "C" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000030,0 min=0,0 seq=30} lock=true stat=PENDING rts=0.000000030,0 wto=false max=0,0 isn=1 -data: "k"/0.000000014,0 -> /BYTES/b -meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=10} ts=0.000000011,0 del=false klen=12 vlen=7 -data: "k/10"/0.000000011,0 -> /BYTES/10 -meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=20} ts=0.000000011,0 del=false klen=12 vlen=7 -data: "k/20"/0.000000011,0 -> /BYTES/20 -meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=30} ts=0.000000011,0 del=false klen=12 vlen=7 -data: "k/30"/0.000000011,0 -> /BYTES/30 -data: "m"/0.000000030,0 -> /BYTES/a +txn: "C" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=30.000000000,0 min=0,0 seq=30} lock=true stat=PENDING rts=30.000000000,0 wto=false max=0,0 isn=1 +data: "k"/14.000000000,0 -> /BYTES/b +meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 +data: "k/10"/11.000000000,0 -> /BYTES/10 +meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 +data: "k/20"/11.000000000,0 -> /BYTES/20 +meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 +data: "k/30"/11.000000000,0 -> /BYTES/30 +data: "m"/30.000000000,0 -> /BYTES/a run ok @@ -365,21 +365,21 @@ with t=D check_intent k=n get k=n ---- -get: "m" -> /BYTES/a @0.000000030,0 -meta: "n" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000040,0 min=0,0 seq=30} ts=0.000000040,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} -get: "n" -> /BYTES/c @0.000000040,0 +get: "m" -> /BYTES/a @30.000000000,0 +meta: "n" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=40.000000000,0 min=0,0 seq=30} ts=40.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +get: "n" -> /BYTES/c @40.000000000,0 >> at end: -txn: "D" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000040,0 min=0,0 seq=30} lock=true stat=PENDING rts=0.000000040,0 wto=false max=0,0 -data: "k"/0.000000014,0 -> /BYTES/b -meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=10} ts=0.000000011,0 del=false klen=12 vlen=7 -data: "k/10"/0.000000011,0 -> /BYTES/10 -meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=20} ts=0.000000011,0 del=false klen=12 vlen=7 -data: "k/20"/0.000000011,0 -> /BYTES/20 -meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=30} ts=0.000000011,0 del=false klen=12 vlen=7 -data: "k/30"/0.000000011,0 -> /BYTES/30 -data: "m"/0.000000030,0 -> /BYTES/a -meta: "n"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000040,0 min=0,0 seq=30} ts=0.000000040,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} -data: "n"/0.000000040,0 -> /BYTES/c +txn: "D" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=40.000000000,0 min=0,0 seq=30} lock=true stat=PENDING rts=40.000000000,0 wto=false max=0,0 +data: "k"/14.000000000,0 -> /BYTES/b +meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 +data: "k/10"/11.000000000,0 -> /BYTES/10 +meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 +data: "k/20"/11.000000000,0 -> /BYTES/20 +meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 +data: "k/30"/11.000000000,0 -> /BYTES/30 +data: "m"/30.000000000,0 -> /BYTES/a +meta: "n"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=40.000000000,0 min=0,0 seq=30} ts=40.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +data: "n"/40.000000000,0 -> /BYTES/c # Do a push. The intent should remain, unmodified, save for the timestamp. @@ -392,22 +392,22 @@ with t=D check_intent k=n get k=n ---- -meta: "n" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000040,0 min=0,0 seq=30} ts=0.000000040,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} -get: "n" -> /BYTES/c @0.000000040,0 -meta: "n" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000045,0 min=0,0 seq=30} ts=0.000000045,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} -get: "n" -> /BYTES/c @0.000000045,0 +meta: "n" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=40.000000000,0 min=0,0 seq=30} ts=40.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +get: "n" -> /BYTES/c @40.000000000,0 +meta: "n" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=45.000000000,0 min=0,0 seq=30} ts=45.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +get: "n" -> /BYTES/c @45.000000000,0 >> at end: -txn: "D" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000045,0 min=0,0 seq=30} lock=true stat=PENDING rts=0.000000040,0 wto=false max=0,0 -data: "k"/0.000000014,0 -> /BYTES/b -meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=10} ts=0.000000011,0 del=false klen=12 vlen=7 -data: "k/10"/0.000000011,0 -> /BYTES/10 -meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=20} ts=0.000000011,0 del=false klen=12 vlen=7 -data: "k/20"/0.000000011,0 -> /BYTES/20 -meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=30} ts=0.000000011,0 del=false klen=12 vlen=7 -data: "k/30"/0.000000011,0 -> /BYTES/30 -data: "m"/0.000000030,0 -> /BYTES/a -meta: "n"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000045,0 min=0,0 seq=30} ts=0.000000045,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} -data: "n"/0.000000045,0 -> /BYTES/c +txn: "D" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=45.000000000,0 min=0,0 seq=30} lock=true stat=PENDING rts=40.000000000,0 wto=false max=0,0 +data: "k"/14.000000000,0 -> /BYTES/b +meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 +data: "k/10"/11.000000000,0 -> /BYTES/10 +meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 +data: "k/20"/11.000000000,0 -> /BYTES/20 +meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 +data: "k/30"/11.000000000,0 -> /BYTES/30 +data: "m"/30.000000000,0 -> /BYTES/a +meta: "n"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=45.000000000,0 min=0,0 seq=30} ts=45.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +data: "n"/45.000000000,0 -> /BYTES/c # Ignore sequence numbers other than the current one, then commit. The value # shouldn't change. @@ -421,20 +421,20 @@ with t=E txn_begin ts=50 get k=n ---- -meta: "n" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000045,0 min=0,0 seq=30} ts=0.000000045,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} -get: "n" -> /BYTES/c @0.000000045,0 -get: "n" -> /BYTES/c @0.000000045,0 +meta: "n" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=45.000000000,0 min=0,0 seq=30} ts=45.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +get: "n" -> /BYTES/c @45.000000000,0 +get: "n" -> /BYTES/c @45.000000000,0 >> at end: -txn: "E" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000050,0 min=0,0 seq=0} lock=true stat=PENDING rts=0.000000050,0 wto=false max=0,0 -data: "k"/0.000000014,0 -> /BYTES/b -meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=10} ts=0.000000011,0 del=false klen=12 vlen=7 -data: "k/10"/0.000000011,0 -> /BYTES/10 -meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=20} ts=0.000000011,0 del=false klen=12 vlen=7 -data: "k/20"/0.000000011,0 -> /BYTES/20 -meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=30} ts=0.000000011,0 del=false klen=12 vlen=7 -data: "k/30"/0.000000011,0 -> /BYTES/30 -data: "m"/0.000000030,0 -> /BYTES/a -data: "n"/0.000000045,0 -> /BYTES/c +txn: "E" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=50.000000000,0 wto=false max=0,0 +data: "k"/14.000000000,0 -> /BYTES/b +meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 +data: "k/10"/11.000000000,0 -> /BYTES/10 +meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 +data: "k/20"/11.000000000,0 -> /BYTES/20 +meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 +data: "k/30"/11.000000000,0 -> /BYTES/30 +data: "m"/30.000000000,0 -> /BYTES/a +data: "n"/45.000000000,0 -> /BYTES/c # Write a couple values at different sequence numbers on this key, then ignore # them all, then do a resolve_intent while the txn is pending. The intent should @@ -454,23 +454,23 @@ with t=E put k=n v=c put k=o v=c ---- -get: "n" -> /BYTES/c @0.000000045,0 +get: "n" -> /BYTES/c @45.000000000,0 get: "o" -> >> at end: -txn: "E" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000050,0 min=0,0 seq=30} lock=true stat=PENDING rts=0.000000050,0 wto=false max=0,0 -data: "k"/0.000000014,0 -> /BYTES/b -meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=10} ts=0.000000011,0 del=false klen=12 vlen=7 -data: "k/10"/0.000000011,0 -> /BYTES/10 -meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=20} ts=0.000000011,0 del=false klen=12 vlen=7 -data: "k/20"/0.000000011,0 -> /BYTES/20 -meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=30} ts=0.000000011,0 del=false klen=12 vlen=7 -data: "k/30"/0.000000011,0 -> /BYTES/30 -data: "m"/0.000000030,0 -> /BYTES/a -meta: "n"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000050,0 min=0,0 seq=30} ts=0.000000050,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} -data: "n"/0.000000050,0 -> /BYTES/c -data: "n"/0.000000045,0 -> /BYTES/c -meta: "o"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000050,0 min=0,0 seq=30} ts=0.000000050,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} -data: "o"/0.000000050,0 -> /BYTES/c +txn: "E" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=30} lock=true stat=PENDING rts=50.000000000,0 wto=false max=0,0 +data: "k"/14.000000000,0 -> /BYTES/b +meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 +data: "k/10"/11.000000000,0 -> /BYTES/10 +meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 +data: "k/20"/11.000000000,0 -> /BYTES/20 +meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 +data: "k/30"/11.000000000,0 -> /BYTES/30 +data: "m"/30.000000000,0 -> /BYTES/a +meta: "n"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=30} ts=50.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +data: "n"/50.000000000,0 -> /BYTES/c +data: "n"/45.000000000,0 -> /BYTES/c +meta: "o"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=30} ts=50.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +data: "o"/50.000000000,0 -> /BYTES/c run ok @@ -481,26 +481,26 @@ with t=E resolve_intent k=n status=PENDING resolve_intent k=o status=PENDING ---- -get: "n" -> /BYTES/c @0.000000045,0 +get: "n" -> /BYTES/c @45.000000000,0 get: "o" -> >> at end: -txn: "E" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000050,0 min=0,0 seq=30} lock=true stat=PENDING rts=0.000000050,0 wto=false max=0,0 isn=1 -data: "k"/0.000000014,0 -> /BYTES/b -meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=10} ts=0.000000011,0 del=false klen=12 vlen=7 -data: "k/10"/0.000000011,0 -> /BYTES/10 -meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=20} ts=0.000000011,0 del=false klen=12 vlen=7 -data: "k/20"/0.000000011,0 -> /BYTES/20 -meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=30} ts=0.000000011,0 del=false klen=12 vlen=7 -data: "k/30"/0.000000011,0 -> /BYTES/30 -data: "m"/0.000000030,0 -> /BYTES/a -data: "n"/0.000000045,0 -> /BYTES/c +txn: "E" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=30} lock=true stat=PENDING rts=50.000000000,0 wto=false max=0,0 isn=1 +data: "k"/14.000000000,0 -> /BYTES/b +meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 +data: "k/10"/11.000000000,0 -> /BYTES/10 +meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 +data: "k/20"/11.000000000,0 -> /BYTES/20 +meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 +data: "k/30"/11.000000000,0 -> /BYTES/30 +data: "m"/30.000000000,0 -> /BYTES/a +data: "n"/45.000000000,0 -> /BYTES/c run ok with t=E get k=n get k=o ---- -get: "n" -> /BYTES/c @0.000000045,0 +get: "n" -> /BYTES/c @45.000000000,0 get: "o" -> # No intent should exist on either value @@ -530,20 +530,20 @@ with t=E put k=o v=c check_intent k=o ---- -meta: "o" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000050,0 min=0,0 seq=30} ts=0.000000050,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +meta: "o" -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=30} ts=50.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} >> at end: -txn: "E" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000050,0 min=0,0 seq=30} lock=true stat=PENDING rts=0.000000050,0 wto=false max=0,0 isn=1 -data: "k"/0.000000014,0 -> /BYTES/b -meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=10} ts=0.000000011,0 del=false klen=12 vlen=7 -data: "k/10"/0.000000011,0 -> /BYTES/10 -meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=20} ts=0.000000011,0 del=false klen=12 vlen=7 -data: "k/20"/0.000000011,0 -> /BYTES/20 -meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=30} ts=0.000000011,0 del=false klen=12 vlen=7 -data: "k/30"/0.000000011,0 -> /BYTES/30 -data: "m"/0.000000030,0 -> /BYTES/a -data: "n"/0.000000045,0 -> /BYTES/c -meta: "o"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000050,0 min=0,0 seq=30} ts=0.000000050,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} -data: "o"/0.000000050,0 -> /BYTES/c +txn: "E" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=30} lock=true stat=PENDING rts=50.000000000,0 wto=false max=0,0 isn=1 +data: "k"/14.000000000,0 -> /BYTES/b +meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 +data: "k/10"/11.000000000,0 -> /BYTES/10 +meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 +data: "k/20"/11.000000000,0 -> /BYTES/20 +meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 +data: "k/30"/11.000000000,0 -> /BYTES/30 +data: "m"/30.000000000,0 -> /BYTES/a +data: "n"/45.000000000,0 -> /BYTES/c +meta: "o"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=50.000000000,0 min=0,0 seq=30} ts=50.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +data: "o"/50.000000000,0 -> /BYTES/c run ok @@ -555,16 +555,16 @@ with t=E ---- get: "o" -> >> at end: -txn: "E" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000055,0 min=0,0 seq=30} lock=true stat=PENDING rts=0.000000050,0 wto=false max=0,0 isn=1 -data: "k"/0.000000014,0 -> /BYTES/b -meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=10} ts=0.000000011,0 del=false klen=12 vlen=7 -data: "k/10"/0.000000011,0 -> /BYTES/10 -meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=20} ts=0.000000011,0 del=false klen=12 vlen=7 -data: "k/20"/0.000000011,0 -> /BYTES/20 -meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=30} ts=0.000000011,0 del=false klen=12 vlen=7 -data: "k/30"/0.000000011,0 -> /BYTES/30 -data: "m"/0.000000030,0 -> /BYTES/a -data: "n"/0.000000045,0 -> /BYTES/c +txn: "E" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=55.000000000,0 min=0,0 seq=30} lock=true stat=PENDING rts=50.000000000,0 wto=false max=0,0 isn=1 +data: "k"/14.000000000,0 -> /BYTES/b +meta: "k/10"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=7 +data: "k/10"/11.000000000,0 -> /BYTES/10 +meta: "k/20"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=7 +data: "k/20"/11.000000000,0 -> /BYTES/20 +meta: "k/30"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=7 +data: "k/30"/11.000000000,0 -> /BYTES/30 +data: "m"/30.000000000,0 -> /BYTES/a +data: "n"/45.000000000,0 -> /BYTES/c run error diff --git a/pkg/storage/testdata/mvcc_histories/ignored_seq_nums_commit b/pkg/storage/testdata/mvcc_histories/ignored_seq_nums_commit index 9a0812615e63..c877f6211b68 100644 --- a/pkg/storage/testdata/mvcc_histories/ignored_seq_nums_commit +++ b/pkg/storage/testdata/mvcc_histories/ignored_seq_nums_commit @@ -23,10 +23,10 @@ with t=A resolve_intent k=k/30 ---- >> at end: -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=40} lock=true stat=PENDING rts=0.000000011,0 wto=false max=0,0 isn=1 -data: "k"/0.000000011,0 -> /BYTES/b -data: "k/10"/0.000000011,0 -> /BYTES/10 -data: "k/20"/0.000000011,0 -> /BYTES/20 +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=40} lock=true stat=PENDING rts=11.000000000,0 wto=false max=0,0 isn=1 +data: "k"/11.000000000,0 -> /BYTES/b +data: "k/10"/11.000000000,0 -> /BYTES/10 +data: "k/20"/11.000000000,0 -> /BYTES/20 run ok scan k=k end=-k @@ -64,10 +64,10 @@ with t=A resolve_intent k=k/30 ---- >> at end: -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=40} lock=true stat=PENDING rts=0.000000011,0 wto=false max=0,0 isn=1 -data: "k"/0.000000011,0 -> /BYTES/c -data: "k/10"/0.000000011,0 -> /BYTES/10 -data: "k/30"/0.000000011,0 -> /BYTES/30 +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=40} lock=true stat=PENDING rts=11.000000000,0 wto=false max=0,0 isn=1 +data: "k"/11.000000000,0 -> /BYTES/c +data: "k/10"/11.000000000,0 -> /BYTES/10 +data: "k/30"/11.000000000,0 -> /BYTES/30 run ok scan k=k end=-k diff --git a/pkg/storage/testdata/mvcc_histories/ignored_seq_nums_cput b/pkg/storage/testdata/mvcc_histories/ignored_seq_nums_cput index 78a879dddd91..9591603c96cb 100644 --- a/pkg/storage/testdata/mvcc_histories/ignored_seq_nums_cput +++ b/pkg/storage/testdata/mvcc_histories/ignored_seq_nums_cput @@ -18,10 +18,10 @@ with t=A txn_step seq=20 ---- >> at end: -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=20} lock=true stat=PENDING rts=0.000000011,0 wto=false max=0,0 isn=1 -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=10} ts=0.000000011,0 del=false klen=12 vlen=6 -data: "k"/0.000000011,0 -> /BYTES/a -data: "k"/0.000000001,0 -> /BYTES/first +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} lock=true stat=PENDING rts=11.000000000,0 wto=false max=0,0 isn=1 +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=6 +data: "k"/11.000000000,0 -> /BYTES/a +data: "k"/1.000000000,0 -> /BYTES/first # Condition must fail to find the last write a. @@ -29,10 +29,10 @@ run error cput t=A k=k cond=a v=c ---- >> at end: -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=10} ts=0.000000011,0 del=false klen=12 vlen=6 -data: "k"/0.000000011,0 -> /BYTES/a -data: "k"/0.000000001,0 -> /BYTES/first -error: (*roachpb.ConditionFailedError:) unexpected value: raw_bytes:"\000\000\000\000\003first" timestamp: +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=10} ts=11.000000000,0 del=false klen=12 vlen=6 +data: "k"/11.000000000,0 -> /BYTES/a +data: "k"/1.000000000,0 -> /BYTES/first +error: (*roachpb.ConditionFailedError:) unexpected value: raw_bytes:"\000\000\000\000\003first" timestamp: # Condition succeeds to find the original value. @@ -40,9 +40,9 @@ run ok cput t=A k=k cond=first v=b ---- >> at end: -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=20} ts=0.000000011,0 del=false klen=12 vlen=6 -data: "k"/0.000000011,0 -> /BYTES/b -data: "k"/0.000000001,0 -> /BYTES/first +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=6 +data: "k"/11.000000000,0 -> /BYTES/b +data: "k"/1.000000000,0 -> /BYTES/first run ok clear_range k=k end=-k @@ -64,10 +64,10 @@ with t=B txn_step seq=30 ---- >> at end: -txn: "B" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=30} lock=true stat=PENDING rts=0.000000011,0 wto=false max=0,0 isn=1 -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=20} ts=0.000000011,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} -data: "k"/0.000000011,0 -> /BYTES/b -data: "k"/0.000000001,0 -> /BYTES/first +txn: "B" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} lock=true stat=PENDING rts=11.000000000,0 wto=false max=0,0 isn=1 +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} +data: "k"/11.000000000,0 -> /BYTES/b +data: "k"/1.000000000,0 -> /BYTES/first # Condition must fail to find the last write b. @@ -75,9 +75,9 @@ run error cput t=B k=k cond=b v=c ---- >> at end: -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=20} ts=0.000000011,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} -data: "k"/0.000000011,0 -> /BYTES/b -data: "k"/0.000000001,0 -> /BYTES/first +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} +data: "k"/11.000000000,0 -> /BYTES/b +data: "k"/1.000000000,0 -> /BYTES/first error: (*roachpb.ConditionFailedError:) unexpected value: raw_bytes:"\000\000\000\000\003a" timestamp:<> # However it succeeds to find the write before that. @@ -86,9 +86,9 @@ run ok cput t=B k=k cond=a v=c ---- >> at end: -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=30} ts=0.000000011,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} -data: "k"/0.000000011,0 -> /BYTES/c -data: "k"/0.000000001,0 -> /BYTES/first +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} +data: "k"/11.000000000,0 -> /BYTES/c +data: "k"/1.000000000,0 -> /BYTES/first run ok clear_range k=k end=-k @@ -112,10 +112,10 @@ with t=C txn_step seq=40 ---- >> at end: -txn: "C" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=40} lock=true stat=PENDING rts=0.000000011,0 wto=false max=0,0 isn=1 -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=30} ts=0.000000011,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} -data: "k"/0.000000011,0 -> /BYTES/c -data: "k"/0.000000001,0 -> /BYTES/first +txn: "C" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=40} lock=true stat=PENDING rts=11.000000000,0 wto=false max=0,0 isn=1 +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +data: "k"/11.000000000,0 -> /BYTES/c +data: "k"/1.000000000,0 -> /BYTES/first # Condition must fail to find the last write b and c. @@ -123,18 +123,18 @@ run error cput t=C k=k cond=c v=d ---- >> at end: -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=30} ts=0.000000011,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} -data: "k"/0.000000011,0 -> /BYTES/c -data: "k"/0.000000001,0 -> /BYTES/first +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +data: "k"/11.000000000,0 -> /BYTES/c +data: "k"/1.000000000,0 -> /BYTES/first error: (*roachpb.ConditionFailedError:) unexpected value: raw_bytes:"\000\000\000\000\003a" timestamp:<> run error cput t=C k=k cond=b v=d ---- >> at end: -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=30} ts=0.000000011,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} -data: "k"/0.000000011,0 -> /BYTES/c -data: "k"/0.000000001,0 -> /BYTES/first +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +data: "k"/11.000000000,0 -> /BYTES/c +data: "k"/1.000000000,0 -> /BYTES/first error: (*roachpb.ConditionFailedError:) unexpected value: raw_bytes:"\000\000\000\000\003a" timestamp:<> # However it succeeds to find the write before that. @@ -143,9 +143,9 @@ run ok cput t=C k=k cond=a v=c ---- >> at end: -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=40} ts=0.000000011,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} -data: "k"/0.000000011,0 -> /BYTES/c -data: "k"/0.000000001,0 -> /BYTES/first +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=40} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +data: "k"/11.000000000,0 -> /BYTES/c +data: "k"/1.000000000,0 -> /BYTES/first run ok clear_range k=k end=-k @@ -168,10 +168,10 @@ with t=D txn_step seq=30 ---- >> at end: -txn: "D" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=30} lock=true stat=PENDING rts=0.000000011,0 wto=false max=0,0 isn=1 -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=20} ts=0.000000011,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} -data: "k"/0.000000011,0 -> /BYTES/b -data: "k"/0.000000001,0 -> /BYTES/first +txn: "D" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} lock=true stat=PENDING rts=11.000000000,0 wto=false max=0,0 isn=1 +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} +data: "k"/11.000000000,0 -> /BYTES/b +data: "k"/1.000000000,0 -> /BYTES/first # Condition must fail to find the last writes a and b. @@ -179,19 +179,19 @@ run error cput t=D k=k cond=a v=c ---- >> at end: -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=20} ts=0.000000011,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} -data: "k"/0.000000011,0 -> /BYTES/b -data: "k"/0.000000001,0 -> /BYTES/first -error: (*roachpb.ConditionFailedError:) unexpected value: raw_bytes:"\000\000\000\000\003first" timestamp: +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} +data: "k"/11.000000000,0 -> /BYTES/b +data: "k"/1.000000000,0 -> /BYTES/first +error: (*roachpb.ConditionFailedError:) unexpected value: raw_bytes:"\000\000\000\000\003first" timestamp: run error cput t=D k=k cond=b v=c ---- >> at end: -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=20} ts=0.000000011,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} -data: "k"/0.000000011,0 -> /BYTES/b -data: "k"/0.000000001,0 -> /BYTES/first -error: (*roachpb.ConditionFailedError:) unexpected value: raw_bytes:"\000\000\000\000\003first" timestamp: +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} +data: "k"/11.000000000,0 -> /BYTES/b +data: "k"/1.000000000,0 -> /BYTES/first +error: (*roachpb.ConditionFailedError:) unexpected value: raw_bytes:"\000\000\000\000\003first" timestamp: # However it succeeds to find the write before that. @@ -199,6 +199,6 @@ run ok cput t=D k=k cond=first v=c ---- >> at end: -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=30} ts=0.000000011,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} -data: "k"/0.000000011,0 -> /BYTES/c -data: "k"/0.000000001,0 -> /BYTES/first +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} +data: "k"/11.000000000,0 -> /BYTES/c +data: "k"/1.000000000,0 -> /BYTES/first diff --git a/pkg/storage/testdata/mvcc_histories/increment b/pkg/storage/testdata/mvcc_histories/increment index 95d60f4103d1..eb9b665d3b46 100644 --- a/pkg/storage/testdata/mvcc_histories/increment +++ b/pkg/storage/testdata/mvcc_histories/increment @@ -51,8 +51,8 @@ inc: current value = 2 >> at end: meta: "k"/0,0 -> txn={id=00000000 key="k" pri=0.00000000 epo=0 ts=0,1 min=0,0 seq=2} ts=0,1 del=false klen=12 vlen=6 ih={{1 /INT/1}} data: "k"/0,1 -> /INT/2 -data: "r"/0.000000003,0 -> /INT/2 -data: "r"/0.000000001,0 -> /INT/1 +data: "r"/3.000000000,0 -> /INT/2 +data: "r"/1.000000000,0 -> /INT/1 run error increment k=r ts=2 @@ -60,10 +60,10 @@ increment k=r ts=2 >> at end: meta: "k"/0,0 -> txn={id=00000000 key="k" pri=0.00000000 epo=0 ts=0,1 min=0,0 seq=2} ts=0,1 del=false klen=12 vlen=6 ih={{1 /INT/1}} data: "k"/0,1 -> /INT/2 -data: "r"/0.000000003,1 -> /INT/3 -data: "r"/0.000000003,0 -> /INT/2 -data: "r"/0.000000001,0 -> /INT/1 -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write at timestamp 0.000000002,0 too old; wrote at 0.000000003,1 +data: "r"/3.000000000,1 -> /INT/3 +data: "r"/3.000000000,0 -> /INT/2 +data: "r"/1.000000000,0 -> /INT/1 +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write at timestamp 2.000000000,0 too old; wrote at 3.000000000,1 # Ditto with transactional. run error @@ -72,12 +72,12 @@ with t=r increment k=r ---- >> at end: -txn: "r" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000002,0 min=0,0 seq=0} lock=true stat=PENDING rts=0.000000002,0 wto=false max=0,0 +txn: "r" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=2.000000000,0 wto=false max=0,0 meta: "k"/0,0 -> txn={id=00000000 key="k" pri=0.00000000 epo=0 ts=0,1 min=0,0 seq=2} ts=0,1 del=false klen=12 vlen=6 ih={{1 /INT/1}} data: "k"/0,1 -> /INT/2 -meta: "r"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000003,2 min=0,0 seq=0} ts=0.000000003,2 del=false klen=12 vlen=6 -data: "r"/0.000000003,2 -> /INT/2 -data: "r"/0.000000003,1 -> /INT/3 -data: "r"/0.000000003,0 -> /INT/2 -data: "r"/0.000000001,0 -> /INT/1 -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write at timestamp 0.000000002,0 too old; wrote at 0.000000003,2 +meta: "r"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,2 min=0,0 seq=0} ts=3.000000000,2 del=false klen=12 vlen=6 +data: "r"/3.000000000,2 -> /INT/2 +data: "r"/3.000000000,1 -> /INT/3 +data: "r"/3.000000000,0 -> /INT/2 +data: "r"/1.000000000,0 -> /INT/1 +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write at timestamp 2.000000000,0 too old; wrote at 3.000000000,2 diff --git a/pkg/storage/testdata/mvcc_histories/intent_history_disallow_separated b/pkg/storage/testdata/mvcc_histories/intent_history_disallow_separated index 773bd24da9c5..b7dec394f246 100644 --- a/pkg/storage/testdata/mvcc_histories/intent_history_disallow_separated +++ b/pkg/storage/testdata/mvcc_histories/intent_history_disallow_separated @@ -7,7 +7,7 @@ with t=A txn_remove ---- >> at end: -data: "a"/0.000000001,0 -> /BYTES/default +data: "a"/1.000000000,0 -> /BYTES/default ## See how the intent history evolves throughout the test. @@ -25,34 +25,34 @@ with t=A resolve_intent ---- >> txn_begin ts=2 t=A -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000002,0 min=0,0 seq=0} lock=true stat=PENDING rts=0.000000002,0 wto=false max=0,0 +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=2.000000000,0 wto=false max=0,0 >> put v=first k=a t=A called PutIntent("a", _, NoExistingIntent, TDNUM(true), 00000000-0000-0000-0000-000000000002) -meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000002,0 min=0,0 seq=0} ts=0.000000002,0 del=false klen=12 vlen=10 -data: "a"/0.000000002,0 -> /BYTES/first -data: "a"/0.000000001,0 -> /BYTES/default +meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=0} ts=2.000000000,0 del=false klen=12 vlen=10 +data: "a"/2.000000000,0 -> /BYTES/first +data: "a"/1.000000000,0 -> /BYTES/default >> txn_step k=a t=A -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000002,0 min=0,0 seq=1} lock=true stat=PENDING rts=0.000000002,0 wto=false max=0,0 +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=1} lock=true stat=PENDING rts=2.000000000,0 wto=false max=0,0 >> put v=second k=a t=A called PutIntent("a", _, ExistingIntentInterleaved, TDNUM(false), 00000000-0000-0000-0000-000000000002) -meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000002,0 min=0,0 seq=1} ts=0.000000002,0 del=false klen=12 vlen=11 ih={{0 /BYTES/first}} -data: "a"/0.000000002,0 -> /BYTES/second -data: "a"/0.000000001,0 -> /BYTES/default +meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=1} ts=2.000000000,0 del=false klen=12 vlen=11 ih={{0 /BYTES/first}} +data: "a"/2.000000000,0 -> /BYTES/second +data: "a"/1.000000000,0 -> /BYTES/default >> txn_step n=2 k=a t=A -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000002,0 min=0,0 seq=3} lock=true stat=PENDING rts=0.000000002,0 wto=false max=0,0 +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=3} lock=true stat=PENDING rts=2.000000000,0 wto=false max=0,0 >> del k=a t=A called PutIntent("a", _, ExistingIntentInterleaved, TDNUM(false), 00000000-0000-0000-0000-000000000002) -meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000002,0 min=0,0 seq=3} ts=0.000000002,0 del=true klen=12 vlen=0 ih={{0 /BYTES/first}{1 /BYTES/second}} -data: "a"/0.000000002,0 -> / -data: "a"/0.000000001,0 -> /BYTES/default +meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=3} ts=2.000000000,0 del=true klen=12 vlen=0 ih={{0 /BYTES/first}{1 /BYTES/second}} +data: "a"/2.000000000,0 -> / +data: "a"/1.000000000,0 -> /BYTES/default >> txn_step n=6 k=a t=A -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000002,0 min=0,0 seq=9} lock=true stat=PENDING rts=0.000000002,0 wto=false max=0,0 +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=9} lock=true stat=PENDING rts=2.000000000,0 wto=false max=0,0 >> put v=first k=a t=A called PutIntent("a", _, ExistingIntentInterleaved, TDNUM(false), 00000000-0000-0000-0000-000000000002) -meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000002,0 min=0,0 seq=9} ts=0.000000002,0 del=false klen=12 vlen=10 ih={{0 /BYTES/first}{1 /BYTES/second}{3 /}} -data: "a"/0.000000002,0 -> /BYTES/first -data: "a"/0.000000001,0 -> /BYTES/default +meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=9} ts=2.000000000,0 del=false klen=12 vlen=10 ih={{0 /BYTES/first}{1 /BYTES/second}{3 /}} +data: "a"/2.000000000,0 -> /BYTES/first +data: "a"/1.000000000,0 -> /BYTES/default >> resolve_intent k=a t=A called ClearIntent("a", ExistingIntentInterleaved, TDNUM(false), 00000000-0000-0000-0000-000000000002) -data: "a"/0.000000002,0 -> /BYTES/first -data: "a"/0.000000001,0 -> /BYTES/default +data: "a"/2.000000000,0 -> /BYTES/first +data: "a"/1.000000000,0 -> /BYTES/default diff --git a/pkg/storage/testdata/mvcc_histories/intent_with_write_tracing_disallow_separated b/pkg/storage/testdata/mvcc_histories/intent_with_write_tracing_disallow_separated index 98ab3c21281a..e501bdb1403c 100644 --- a/pkg/storage/testdata/mvcc_histories/intent_with_write_tracing_disallow_separated +++ b/pkg/storage/testdata/mvcc_histories/intent_with_write_tracing_disallow_separated @@ -4,11 +4,11 @@ with t=A put k=k1 v=v1 ---- >> txn_begin ts=2 t=A -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000002,0 min=0,0 seq=0} lock=true stat=PENDING rts=0.000000002,0 wto=false max=0,0 +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=2.000000000,0 wto=false max=0,0 >> put k=k1 v=v1 t=A called PutIntent("k1", _, NoExistingIntent, TDNUM(true), 00000000-0000-0000-0000-000000000001) -meta: "k1"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000002,0 min=0,0 seq=0} ts=0.000000002,0 del=false klen=12 vlen=7 -data: "k1"/0.000000002,0 -> /BYTES/v1 +meta: "k1"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,0 min=0,0 seq=0} ts=2.000000000,0 del=false klen=12 vlen=7 +data: "k1"/2.000000000,0 -> /BYTES/v1 run trace ok with t=A @@ -18,29 +18,29 @@ with t=A put k=k2 v=v2 ---- >> txn_advance ts=3 t=A -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000003,0 min=0,0 seq=0} lock=true stat=PENDING rts=0.000000002,0 wto=false max=0,0 +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=2.000000000,0 wto=false max=0,0 >> txn_step t=A -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000003,0 min=0,0 seq=1} lock=true stat=PENDING rts=0.000000002,0 wto=false max=0,0 +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} lock=true stat=PENDING rts=2.000000000,0 wto=false max=0,0 >> put k=k1 v=v1 t=A called PutIntent("k1", _, ExistingIntentInterleaved, TDNUM(false), 00000000-0000-0000-0000-000000000001) -meta: "k1"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000003,0 min=0,0 seq=1} ts=0.000000003,0 del=false klen=12 vlen=7 ih={{0 /BYTES/v1}} -data: "k1"/0.000000003,0 -> /BYTES/v1 +meta: "k1"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=7 ih={{0 /BYTES/v1}} +data: "k1"/3.000000000,0 -> /BYTES/v1 >> put k=k2 v=v2 t=A called PutIntent("k2", _, NoExistingIntent, TDNUM(true), 00000000-0000-0000-0000-000000000001) -meta: "k1"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000003,0 min=0,0 seq=1} ts=0.000000003,0 del=false klen=12 vlen=7 ih={{0 /BYTES/v1}} -data: "k1"/0.000000003,0 -> /BYTES/v1 -meta: "k2"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000003,0 min=0,0 seq=1} ts=0.000000003,0 del=false klen=12 vlen=7 -data: "k2"/0.000000003,0 -> /BYTES/v2 +meta: "k1"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=7 ih={{0 /BYTES/v1}} +data: "k1"/3.000000000,0 -> /BYTES/v1 +meta: "k2"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=7 +data: "k2"/3.000000000,0 -> /BYTES/v2 run trace ok put k=k3 v=v3 ts=1 ---- >> put k=k3 v=v3 ts=1 -meta: "k1"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000003,0 min=0,0 seq=1} ts=0.000000003,0 del=false klen=12 vlen=7 ih={{0 /BYTES/v1}} -data: "k1"/0.000000003,0 -> /BYTES/v1 -meta: "k2"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000003,0 min=0,0 seq=1} ts=0.000000003,0 del=false klen=12 vlen=7 -data: "k2"/0.000000003,0 -> /BYTES/v2 -data: "k3"/0.000000001,0 -> /BYTES/v3 +meta: "k1"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=7 ih={{0 /BYTES/v1}} +data: "k1"/3.000000000,0 -> /BYTES/v1 +meta: "k2"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=7 +data: "k2"/3.000000000,0 -> /BYTES/v2 +data: "k3"/1.000000000,0 -> /BYTES/v3 run trace ok with t=A @@ -48,13 +48,13 @@ with t=A ---- >> put k=k3 v=v33 t=A called PutIntent("k3", _, NoExistingIntent, TDNUM(true), 00000000-0000-0000-0000-000000000001) -meta: "k1"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000003,0 min=0,0 seq=1} ts=0.000000003,0 del=false klen=12 vlen=7 ih={{0 /BYTES/v1}} -data: "k1"/0.000000003,0 -> /BYTES/v1 -meta: "k2"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000003,0 min=0,0 seq=1} ts=0.000000003,0 del=false klen=12 vlen=7 -data: "k2"/0.000000003,0 -> /BYTES/v2 -meta: "k3"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000003,0 min=0,0 seq=1} ts=0.000000003,0 del=false klen=12 vlen=8 -data: "k3"/0.000000003,0 -> /BYTES/v33 -data: "k3"/0.000000001,0 -> /BYTES/v3 +meta: "k1"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=7 ih={{0 /BYTES/v1}} +data: "k1"/3.000000000,0 -> /BYTES/v1 +meta: "k2"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=7 +data: "k2"/3.000000000,0 -> /BYTES/v2 +meta: "k3"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=8 +data: "k3"/3.000000000,0 -> /BYTES/v33 +data: "k3"/1.000000000,0 -> /BYTES/v3 # transactionDidNotUpdateMeta (TDNUM) is false below for k2 and k3 since # disallowSeparatedIntents=true causes mvcc.go to always set it to false to maintain @@ -68,20 +68,20 @@ with t=A ---- >> resolve_intent k=k1 t=A called ClearIntent("k1", ExistingIntentInterleaved, TDNUM(false), 00000000-0000-0000-0000-000000000001) -data: "k1"/0.000000003,0 -> /BYTES/v1 -meta: "k2"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000003,0 min=0,0 seq=1} ts=0.000000003,0 del=false klen=12 vlen=7 -data: "k2"/0.000000003,0 -> /BYTES/v2 -meta: "k3"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000003,0 min=0,0 seq=1} ts=0.000000003,0 del=false klen=12 vlen=8 -data: "k3"/0.000000003,0 -> /BYTES/v33 -data: "k3"/0.000000001,0 -> /BYTES/v3 +data: "k1"/3.000000000,0 -> /BYTES/v1 +meta: "k2"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=7 +data: "k2"/3.000000000,0 -> /BYTES/v2 +meta: "k3"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=8 +data: "k3"/3.000000000,0 -> /BYTES/v33 +data: "k3"/1.000000000,0 -> /BYTES/v3 >> resolve_intent k=k2 status=ABORTED t=A called ClearIntent("k2", ExistingIntentInterleaved, TDNUM(false), 00000000-0000-0000-0000-000000000001) -data: "k1"/0.000000003,0 -> /BYTES/v1 -meta: "k3"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000003,0 min=0,0 seq=1} ts=0.000000003,0 del=false klen=12 vlen=8 -data: "k3"/0.000000003,0 -> /BYTES/v33 -data: "k3"/0.000000001,0 -> /BYTES/v3 +data: "k1"/3.000000000,0 -> /BYTES/v1 +meta: "k3"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=3.000000000,0 min=0,0 seq=1} ts=3.000000000,0 del=false klen=12 vlen=8 +data: "k3"/3.000000000,0 -> /BYTES/v33 +data: "k3"/1.000000000,0 -> /BYTES/v3 >> resolve_intent k=k3 status=ABORTED t=A called ClearIntent("k3", ExistingIntentInterleaved, TDNUM(false), 00000000-0000-0000-0000-000000000001) -data: "k1"/0.000000003,0 -> /BYTES/v1 -data: "k3"/0.000000001,0 -> /BYTES/v3 +data: "k1"/3.000000000,0 -> /BYTES/v1 +data: "k3"/1.000000000,0 -> /BYTES/v3 >> txn_remove t=A diff --git a/pkg/storage/testdata/mvcc_histories/max_keys b/pkg/storage/testdata/mvcc_histories/max_keys index dbc019cf2cc9..358302e770c3 100644 --- a/pkg/storage/testdata/mvcc_histories/max_keys +++ b/pkg/storage/testdata/mvcc_histories/max_keys @@ -10,11 +10,11 @@ with ts=1,0 del k=aa ts=2,0 ---- >> at end: -data: "a"/0.000000001,0 -> /BYTES/val-a -data: "aa"/0.000000002,0 -> / -data: "aa"/0.000000001,0 -> /BYTES/val-aa -data: "c"/0.000000001,0 -> /BYTES/val-c -data: "e"/0.000000001,0 -> /BYTES/val-e +data: "a"/1.000000000,0 -> /BYTES/val-a +data: "aa"/2.000000000,0 -> / +data: "aa"/1.000000000,0 -> /BYTES/val-aa +data: "c"/1.000000000,0 -> /BYTES/val-c +data: "e"/1.000000000,0 -> /BYTES/val-e # Limit 1 works. run ok @@ -22,9 +22,9 @@ with ts=300,0 k=a end=z max=1 scan scan reverse=true ---- -scan: "a" -> /BYTES/val-a @0.000000001,0 +scan: "a" -> /BYTES/val-a @1.000000000,0 scan: resume span ["aa","z") -scan: "e" -> /BYTES/val-e @0.000000001,0 +scan: "e" -> /BYTES/val-e @1.000000000,0 scan: resume span ["a","c\x00") # Limit -1 works: nothing is returned, go straight to resume span. We use this @@ -45,8 +45,8 @@ run ok with ts=300,0 k=a end=z max=2 tombstones=true scan ---- -scan: "a" -> /BYTES/val-a @0.000000001,0 -scan: "aa" -> / @0.000000002,0 +scan: "a" -> /BYTES/val-a @1.000000000,0 +scan: "aa" -> / @2.000000000,0 scan: resume span ["c","z") # Ditto in reverse. @@ -54,8 +54,8 @@ run ok with ts=300,0 k=a end=d max=2 tombstones=true reverse=true scan ---- -scan: "c" -> /BYTES/val-c @0.000000001,0 -scan: "aa" -> / @0.000000002,0 +scan: "c" -> /BYTES/val-c @1.000000000,0 +scan: "aa" -> / @2.000000000,0 scan: resume span ["a","a\x00") # No limit = zero limit = infinity limit (zero is preferred). @@ -68,24 +68,24 @@ with ts=300,0 k=a end=z scan max=99999 scan reverse=true max=9999 ---- -scan: "a" -> /BYTES/val-a @0.000000001,0 -scan: "c" -> /BYTES/val-c @0.000000001,0 -scan: "e" -> /BYTES/val-e @0.000000001,0 -scan: "e" -> /BYTES/val-e @0.000000001,0 -scan: "c" -> /BYTES/val-c @0.000000001,0 -scan: "a" -> /BYTES/val-a @0.000000001,0 -scan: "a" -> /BYTES/val-a @0.000000001,0 -scan: "c" -> /BYTES/val-c @0.000000001,0 -scan: "e" -> /BYTES/val-e @0.000000001,0 -scan: "e" -> /BYTES/val-e @0.000000001,0 -scan: "c" -> /BYTES/val-c @0.000000001,0 -scan: "a" -> /BYTES/val-a @0.000000001,0 -scan: "a" -> /BYTES/val-a @0.000000001,0 -scan: "c" -> /BYTES/val-c @0.000000001,0 -scan: "e" -> /BYTES/val-e @0.000000001,0 -scan: "e" -> /BYTES/val-e @0.000000001,0 -scan: "c" -> /BYTES/val-c @0.000000001,0 -scan: "a" -> /BYTES/val-a @0.000000001,0 +scan: "a" -> /BYTES/val-a @1.000000000,0 +scan: "c" -> /BYTES/val-c @1.000000000,0 +scan: "e" -> /BYTES/val-e @1.000000000,0 +scan: "e" -> /BYTES/val-e @1.000000000,0 +scan: "c" -> /BYTES/val-c @1.000000000,0 +scan: "a" -> /BYTES/val-a @1.000000000,0 +scan: "a" -> /BYTES/val-a @1.000000000,0 +scan: "c" -> /BYTES/val-c @1.000000000,0 +scan: "e" -> /BYTES/val-e @1.000000000,0 +scan: "e" -> /BYTES/val-e @1.000000000,0 +scan: "c" -> /BYTES/val-c @1.000000000,0 +scan: "a" -> /BYTES/val-a @1.000000000,0 +scan: "a" -> /BYTES/val-a @1.000000000,0 +scan: "c" -> /BYTES/val-c @1.000000000,0 +scan: "e" -> /BYTES/val-e @1.000000000,0 +scan: "e" -> /BYTES/val-e @1.000000000,0 +scan: "c" -> /BYTES/val-c @1.000000000,0 +scan: "a" -> /BYTES/val-a @1.000000000,0 # Regression test for #46652: Test appropriate termination when the MaxKeys-th # key is in the intent history. @@ -112,29 +112,29 @@ with t=A ts=11,0 max=3 scan k=k end=o scan k=k end=o reverse=true ---- -scan: "k" -> /BYTES/b @0.000000011,0 -scan: "l" -> /BYTES/b @0.000000011,0 -scan: "m" -> /BYTES/b @0.000000011,0 +scan: "k" -> /BYTES/b @11.000000000,0 +scan: "l" -> /BYTES/b @11.000000000,0 +scan: "m" -> /BYTES/b @11.000000000,0 scan: resume span ["n","o") -scan: "n" -> /BYTES/b @0.000000011,0 -scan: "m" -> /BYTES/b @0.000000011,0 -scan: "l" -> /BYTES/b @0.000000011,0 +scan: "n" -> /BYTES/b @11.000000000,0 +scan: "m" -> /BYTES/b @11.000000000,0 +scan: "l" -> /BYTES/b @11.000000000,0 scan: resume span ["k","k\x00") >> at end: -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=20} lock=true stat=PENDING rts=0.000000011,0 wto=false max=0,0 -data: "a"/0.000000001,0 -> /BYTES/val-a -data: "aa"/0.000000002,0 -> / -data: "aa"/0.000000001,0 -> /BYTES/val-aa -data: "c"/0.000000001,0 -> /BYTES/val-c -data: "e"/0.000000001,0 -> /BYTES/val-e -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=30} ts=0.000000011,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} -data: "k"/0.000000011,0 -> /BYTES/c -meta: "l"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=30} ts=0.000000011,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} -data: "l"/0.000000011,0 -> /BYTES/c -meta: "m"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=30} ts=0.000000011,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} -data: "m"/0.000000011,0 -> /BYTES/c -meta: "n"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=30} ts=0.000000011,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} -data: "n"/0.000000011,0 -> /BYTES/c +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} lock=true stat=PENDING rts=11.000000000,0 wto=false max=0,0 +data: "a"/1.000000000,0 -> /BYTES/val-a +data: "aa"/2.000000000,0 -> / +data: "aa"/1.000000000,0 -> /BYTES/val-aa +data: "c"/1.000000000,0 -> /BYTES/val-c +data: "e"/1.000000000,0 -> /BYTES/val-e +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +data: "k"/11.000000000,0 -> /BYTES/c +meta: "l"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +data: "l"/11.000000000,0 -> /BYTES/c +meta: "m"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +data: "m"/11.000000000,0 -> /BYTES/c +meta: "n"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +data: "n"/11.000000000,0 -> /BYTES/c run ok with t=A ts=11,0 max=3 @@ -145,16 +145,16 @@ with t=A ts=11,0 max=3 resolve_intent k=n status=COMMITTED ---- >> at end: -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=30} lock=true stat=PENDING rts=0.000000011,0 wto=false max=0,0 -data: "a"/0.000000001,0 -> /BYTES/val-a -data: "aa"/0.000000002,0 -> / -data: "aa"/0.000000001,0 -> /BYTES/val-aa -data: "c"/0.000000001,0 -> /BYTES/val-c -data: "e"/0.000000001,0 -> /BYTES/val-e -data: "k"/0.000000011,0 -> /BYTES/c -data: "l"/0.000000011,0 -> /BYTES/c -data: "m"/0.000000011,0 -> /BYTES/c -data: "n"/0.000000011,0 -> /BYTES/c +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} lock=true stat=PENDING rts=11.000000000,0 wto=false max=0,0 +data: "a"/1.000000000,0 -> /BYTES/val-a +data: "aa"/2.000000000,0 -> / +data: "aa"/1.000000000,0 -> /BYTES/val-aa +data: "c"/1.000000000,0 -> /BYTES/val-c +data: "e"/1.000000000,0 -> /BYTES/val-e +data: "k"/11.000000000,0 -> /BYTES/c +data: "l"/11.000000000,0 -> /BYTES/c +data: "m"/11.000000000,0 -> /BYTES/c +data: "n"/11.000000000,0 -> /BYTES/c # Same case as above, except with a committed value at the key after MaxKeys. @@ -176,24 +176,24 @@ with t=B ts=12,0 max=3 txn_step seq=20 scan k=k end=o ---- -scan: "k" -> /BYTES/b @0.000000012,0 -scan: "l" -> /BYTES/b @0.000000012,0 -scan: "m" -> /BYTES/b @0.000000012,0 +scan: "k" -> /BYTES/b @12.000000000,0 +scan: "l" -> /BYTES/b @12.000000000,0 +scan: "m" -> /BYTES/b @12.000000000,0 scan: resume span ["n","o") >> at end: -txn: "B" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000012,0 min=0,0 seq=20} lock=true stat=PENDING rts=0.000000012,0 wto=false max=0,0 -data: "a"/0.000000001,0 -> /BYTES/val-a -data: "aa"/0.000000002,0 -> / -data: "aa"/0.000000001,0 -> /BYTES/val-aa -data: "c"/0.000000001,0 -> /BYTES/val-c -data: "e"/0.000000001,0 -> /BYTES/val-e -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000012,0 min=0,0 seq=30} ts=0.000000012,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} -data: "k"/0.000000012,0 -> /BYTES/c -data: "k"/0.000000011,0 -> /BYTES/c -meta: "l"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000012,0 min=0,0 seq=30} ts=0.000000012,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} -data: "l"/0.000000012,0 -> /BYTES/c -data: "l"/0.000000011,0 -> /BYTES/c -meta: "m"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000012,0 min=0,0 seq=30} ts=0.000000012,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} -data: "m"/0.000000012,0 -> /BYTES/c -data: "m"/0.000000011,0 -> /BYTES/c -data: "n"/0.000000011,0 -> /BYTES/c +txn: "B" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=12.000000000,0 min=0,0 seq=20} lock=true stat=PENDING rts=12.000000000,0 wto=false max=0,0 +data: "a"/1.000000000,0 -> /BYTES/val-a +data: "aa"/2.000000000,0 -> / +data: "aa"/1.000000000,0 -> /BYTES/val-aa +data: "c"/1.000000000,0 -> /BYTES/val-c +data: "e"/1.000000000,0 -> /BYTES/val-e +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=12.000000000,0 min=0,0 seq=30} ts=12.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +data: "k"/12.000000000,0 -> /BYTES/c +data: "k"/11.000000000,0 -> /BYTES/c +meta: "l"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=12.000000000,0 min=0,0 seq=30} ts=12.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +data: "l"/12.000000000,0 -> /BYTES/c +data: "l"/11.000000000,0 -> /BYTES/c +meta: "m"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=12.000000000,0 min=0,0 seq=30} ts=12.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +data: "m"/12.000000000,0 -> /BYTES/c +data: "m"/11.000000000,0 -> /BYTES/c +data: "n"/11.000000000,0 -> /BYTES/c diff --git a/pkg/storage/testdata/mvcc_histories/merges b/pkg/storage/testdata/mvcc_histories/merges index b712bb4db25a..2b1149b76c89 100644 --- a/pkg/storage/testdata/mvcc_histories/merges +++ b/pkg/storage/testdata/mvcc_histories/merges @@ -12,7 +12,7 @@ with t=A txn_remove ---- >> at end: -data: "a"/0.000000011,0 -> /BYTES/abc +data: "a"/11.000000000,0 -> /BYTES/abc # Merge appends data in the intent and ignores the regular k/v pairs. @@ -22,10 +22,10 @@ merge k=a v=ghi ts=22 ---- >> merge k=a v=def ts=22 meta: "a"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/def -data: "a"/0.000000011,0 -> /BYTES/abc +data: "a"/11.000000000,0 -> /BYTES/abc >> merge k=a v=ghi ts=22 meta: "a"/0,0 -> txn={} ts=0,0 del=false klen=0 vlen=0 raw=/BYTES/defghi -data: "a"/0.000000011,0 -> /BYTES/abc +data: "a"/11.000000000,0 -> /BYTES/abc # After a merge, only the data in the intent is every used. @@ -36,4 +36,4 @@ with t=A ---- get: "a" -> /BYTES/defghi @0,0 >> at end: -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000033,0 min=0,0 seq=0} lock=true stat=PENDING rts=0.000000033,0 wto=false max=0,0 +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=33.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=33.000000000,0 wto=false max=0,0 diff --git a/pkg/storage/testdata/mvcc_histories/no_read_after_abort_disallow_separated b/pkg/storage/testdata/mvcc_histories/no_read_after_abort_disallow_separated index 18d2f2dc60f8..b8260def36b7 100644 --- a/pkg/storage/testdata/mvcc_histories/no_read_after_abort_disallow_separated +++ b/pkg/storage/testdata/mvcc_histories/no_read_after_abort_disallow_separated @@ -8,11 +8,11 @@ with t=A k=a txn_remove ---- >> txn_begin ts=22 t=A k=a -txn: "A" meta={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000022,0 min=0,0 seq=0} lock=true stat=PENDING rts=0.000000022,0 wto=false max=0,0 +txn: "A" meta={id=00000000 key="a" pri=0.00000000 epo=0 ts=22.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=22.000000000,0 wto=false max=0,0 >> put v=cde t=A k=a called PutIntent("a", _, NoExistingIntent, TDNUM(true), 00000000-0000-0000-0000-000000000001) -meta: "a"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000022,0 min=0,0 seq=0} ts=0.000000022,0 del=false klen=12 vlen=8 -data: "a"/0.000000022,0 -> /BYTES/cde +meta: "a"/0,0 -> txn={id=00000000 key="a" pri=0.00000000 epo=0 ts=22.000000000,0 min=0,0 seq=0} ts=22.000000000,0 del=false klen=12 vlen=8 +data: "a"/22.000000000,0 -> /BYTES/cde >> resolve_intent status=ABORTED t=A k=a called ClearIntent("a", ExistingIntentInterleaved, TDNUM(false), 00000000-0000-0000-0000-000000000001) diff --git a/pkg/storage/testdata/mvcc_histories/put_after_rollback b/pkg/storage/testdata/mvcc_histories/put_after_rollback index c77653f25516..bd32720319f1 100644 --- a/pkg/storage/testdata/mvcc_histories/put_after_rollback +++ b/pkg/storage/testdata/mvcc_histories/put_after_rollback @@ -10,12 +10,12 @@ with t=A k=k2 txn_ignore_seqs seqs=(5-25) get ---- -get: "k2" -> /BYTES/b @0.000000001,0 +get: "k2" -> /BYTES/b @1.000000000,0 get: "k2" -> >> at end: -txn: "A" meta={id=00000000 key="k2" pri=0.00000000 epo=0 ts=0.000000001,0 min=0,0 seq=20} lock=true stat=PENDING rts=0.000000001,0 wto=false max=0,0 isn=1 -meta: "k2"/0,0 -> txn={id=00000000 key="k2" pri=0.00000000 epo=0 ts=0.000000001,0 min=0,0 seq=20} ts=0.000000001,0 del=false klen=12 vlen=6 -data: "k2"/0.000000001,0 -> /BYTES/b +txn: "A" meta={id=00000000 key="k2" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=20} lock=true stat=PENDING rts=1.000000000,0 wto=false max=0,0 isn=1 +meta: "k2"/0,0 -> txn={id=00000000 key="k2" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=20} ts=1.000000000,0 del=false klen=12 vlen=6 +data: "k2"/1.000000000,0 -> /BYTES/b run ok with t=A k=k3 @@ -26,11 +26,11 @@ with t=A k=k3 del ---- >> at end: -txn: "A" meta={id=00000000 key="k2" pri=0.00000000 epo=0 ts=0.000000001,0 min=0,0 seq=40} lock=true stat=PENDING rts=0.000000001,0 wto=false max=0,0 isn=1 -meta: "k2"/0,0 -> txn={id=00000000 key="k2" pri=0.00000000 epo=0 ts=0.000000001,0 min=0,0 seq=20} ts=0.000000001,0 del=false klen=12 vlen=6 -data: "k2"/0.000000001,0 -> /BYTES/b -meta: "k3"/0,0 -> txn={id=00000000 key="k2" pri=0.00000000 epo=0 ts=0.000000001,0 min=0,0 seq=40} ts=0.000000001,0 del=true klen=12 vlen=0 -data: "k3"/0.000000001,0 -> / +txn: "A" meta={id=00000000 key="k2" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=40} lock=true stat=PENDING rts=1.000000000,0 wto=false max=0,0 isn=1 +meta: "k2"/0,0 -> txn={id=00000000 key="k2" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=20} ts=1.000000000,0 del=false klen=12 vlen=6 +data: "k2"/1.000000000,0 -> /BYTES/b +meta: "k3"/0,0 -> txn={id=00000000 key="k2" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=40} ts=1.000000000,0 del=true klen=12 vlen=0 +data: "k3"/1.000000000,0 -> / run ok with t=A k=k4 @@ -43,13 +43,13 @@ with t=A k=k4 cput v=c ---- >> at end: -txn: "A" meta={id=00000000 key="k2" pri=0.00000000 epo=0 ts=0.000000001,0 min=0,0 seq=60} lock=true stat=PENDING rts=0.000000001,0 wto=false max=0,0 isn=1 -meta: "k2"/0,0 -> txn={id=00000000 key="k2" pri=0.00000000 epo=0 ts=0.000000001,0 min=0,0 seq=20} ts=0.000000001,0 del=false klen=12 vlen=6 -data: "k2"/0.000000001,0 -> /BYTES/b -meta: "k3"/0,0 -> txn={id=00000000 key="k2" pri=0.00000000 epo=0 ts=0.000000001,0 min=0,0 seq=40} ts=0.000000001,0 del=true klen=12 vlen=0 -data: "k3"/0.000000001,0 -> / -meta: "k4"/0,0 -> txn={id=00000000 key="k2" pri=0.00000000 epo=0 ts=0.000000001,0 min=0,0 seq=60} ts=0.000000001,0 del=false klen=12 vlen=6 -data: "k4"/0.000000001,0 -> /BYTES/c +txn: "A" meta={id=00000000 key="k2" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=60} lock=true stat=PENDING rts=1.000000000,0 wto=false max=0,0 isn=1 +meta: "k2"/0,0 -> txn={id=00000000 key="k2" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=20} ts=1.000000000,0 del=false klen=12 vlen=6 +data: "k2"/1.000000000,0 -> /BYTES/b +meta: "k3"/0,0 -> txn={id=00000000 key="k2" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=40} ts=1.000000000,0 del=true klen=12 vlen=0 +data: "k3"/1.000000000,0 -> / +meta: "k4"/0,0 -> txn={id=00000000 key="k2" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=60} ts=1.000000000,0 del=false klen=12 vlen=6 +data: "k4"/1.000000000,0 -> /BYTES/c run ok put k=k5 v=foo ts=3 @@ -69,15 +69,15 @@ with t=B k=k5 check_intent resolve_intent status=COMMITTED ---- -meta: "k5" -> txn={id=00000000 key="k5" pri=0.00000000 epo=0 ts=0.000000005,0 min=0,0 seq=30} ts=0.000000005,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} -meta: "k5" -> txn={id=00000000 key="k5" pri=0.00000000 epo=0 ts=0.000000005,0 min=0,0 seq=40} ts=0.000000005,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} +meta: "k5" -> txn={id=00000000 key="k5" pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=30} ts=5.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} +meta: "k5" -> txn={id=00000000 key="k5" pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=40} ts=5.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}} >> at end: -txn: "B" meta={id=00000000 key="k5" pri=0.00000000 epo=0 ts=0.000000005,0 min=0,0 seq=40} lock=true stat=PENDING rts=0.000000005,0 wto=false max=0,0 isn=1 -meta: "k2"/0,0 -> txn={id=00000000 key="k2" pri=0.00000000 epo=0 ts=0.000000001,0 min=0,0 seq=20} ts=0.000000001,0 del=false klen=12 vlen=6 -data: "k2"/0.000000001,0 -> /BYTES/b -meta: "k3"/0,0 -> txn={id=00000000 key="k2" pri=0.00000000 epo=0 ts=0.000000001,0 min=0,0 seq=40} ts=0.000000001,0 del=true klen=12 vlen=0 -data: "k3"/0.000000001,0 -> / -meta: "k4"/0,0 -> txn={id=00000000 key="k2" pri=0.00000000 epo=0 ts=0.000000001,0 min=0,0 seq=60} ts=0.000000001,0 del=false klen=12 vlen=6 -data: "k4"/0.000000001,0 -> /BYTES/c -data: "k5"/0.000000005,0 -> /BYTES/d -data: "k5"/0.000000003,0 -> /BYTES/foo +txn: "B" meta={id=00000000 key="k5" pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=40} lock=true stat=PENDING rts=5.000000000,0 wto=false max=0,0 isn=1 +meta: "k2"/0,0 -> txn={id=00000000 key="k2" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=20} ts=1.000000000,0 del=false klen=12 vlen=6 +data: "k2"/1.000000000,0 -> /BYTES/b +meta: "k3"/0,0 -> txn={id=00000000 key="k2" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=40} ts=1.000000000,0 del=true klen=12 vlen=0 +data: "k3"/1.000000000,0 -> / +meta: "k4"/0,0 -> txn={id=00000000 key="k2" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=60} ts=1.000000000,0 del=false klen=12 vlen=6 +data: "k4"/1.000000000,0 -> /BYTES/c +data: "k5"/5.000000000,0 -> /BYTES/d +data: "k5"/3.000000000,0 -> /BYTES/foo diff --git a/pkg/storage/testdata/mvcc_histories/put_new_epoch_lower_sequence b/pkg/storage/testdata/mvcc_histories/put_new_epoch_lower_sequence index 0f2b7d1e1363..7b0d8e4be50b 100644 --- a/pkg/storage/testdata/mvcc_histories/put_new_epoch_lower_sequence +++ b/pkg/storage/testdata/mvcc_histories/put_new_epoch_lower_sequence @@ -17,11 +17,11 @@ with t=A put k=k v=v2 get k=k ts=3 ---- -get: "k" -> /BYTES/v2 @0.000000001,0 +get: "k" -> /BYTES/v2 @1.000000000,0 >> at end: -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000001,0 min=0,0 seq=5} lock=true stat=PENDING rts=0.000000001,0 wto=false max=0,0 -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000001,0 min=0,0 seq=5} ts=0.000000001,0 del=false klen=12 vlen=7 ih={{4 /BYTES/v}} -data: "k"/0.000000001,0 -> /BYTES/v2 +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=5} lock=true stat=PENDING rts=1.000000000,0 wto=false max=0,0 +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=5} ts=1.000000000,0 del=false klen=12 vlen=7 ih={{4 /BYTES/v}} +data: "k"/1.000000000,0 -> /BYTES/v2 run ok with t=A @@ -29,7 +29,7 @@ with t=A txn_step n=4 ---- >> at end: -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=1 ts=0.000000001,0 min=0,0 seq=4} lock=true stat=PENDING rts=0.000000001,0 wto=false max=0,0 +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=1 ts=1.000000000,0 min=0,0 seq=4} lock=true stat=PENDING rts=1.000000000,0 wto=false max=0,0 # We're operating at a higher epoch but a lower seqnum. @@ -42,10 +42,10 @@ with t=A k=k put v=v3 check_intent exists ---- -meta: "k" -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=0.000000001,0 min=0,0 seq=4} ts=0.000000001,0 del=false klen=12 vlen=7 +meta: "k" -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=1.000000000,0 min=0,0 seq=4} ts=1.000000000,0 del=false klen=12 vlen=7 >> at end: -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=0.000000001,0 min=0,0 seq=4} ts=0.000000001,0 del=false klen=12 vlen=7 -data: "k"/0.000000001,0 -> /BYTES/v3 +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=1.000000000,0 min=0,0 seq=4} ts=1.000000000,0 del=false klen=12 vlen=7 +data: "k"/1.000000000,0 -> /BYTES/v3 # We're expecting v3 here. @@ -53,4 +53,4 @@ run ok with t=A get k=k ---- -get: "k" -> /BYTES/v3 @0.000000001,0 +get: "k" -> /BYTES/v3 @1.000000000,0 diff --git a/pkg/storage/testdata/mvcc_histories/put_new_epoch_lower_timestamp b/pkg/storage/testdata/mvcc_histories/put_new_epoch_lower_timestamp index f2c8d88cdd94..99e294d28a7c 100644 --- a/pkg/storage/testdata/mvcc_histories/put_new_epoch_lower_timestamp +++ b/pkg/storage/testdata/mvcc_histories/put_new_epoch_lower_timestamp @@ -17,11 +17,11 @@ with t=A put k=k v=v get k=k ts=3 ---- -get: "k" -> /BYTES/v @0.000000005,0 +get: "k" -> /BYTES/v @5.000000000,0 >> at end: -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000005,0 min=0,0 seq=4} lock=true stat=PENDING rts=0.000000001,0 wto=false max=0,0 -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000005,0 min=0,0 seq=4} ts=0.000000005,0 del=false klen=12 vlen=6 -data: "k"/0.000000005,0 -> /BYTES/v +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=4} lock=true stat=PENDING rts=1.000000000,0 wto=false max=0,0 +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=4} ts=5.000000000,0 del=false klen=12 vlen=6 +data: "k"/5.000000000,0 -> /BYTES/v run ok with t=A @@ -29,7 +29,7 @@ with t=A txn_restart ---- >> at end: -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=1 ts=0.000000003,0 min=0,0 seq=0} lock=true stat=PENDING rts=0.000000003,0 wto=false max=0,0 +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=1 ts=3.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=3.000000000,0 wto=false max=0,0 # We're operating at a higher epoch but a lower timestamp. @@ -40,7 +40,7 @@ with t=A put k=k v=v2 get k=k ---- -get: "k" -> /BYTES/v2 @0.000000005,0 +get: "k" -> /BYTES/v2 @5.000000000,0 >> at end: -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=0.000000005,0 min=0,0 seq=0} ts=0.000000005,0 del=false klen=12 vlen=7 -data: "k"/0.000000005,0 -> /BYTES/v2 +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=1 ts=5.000000000,0 min=0,0 seq=0} ts=5.000000000,0 del=false klen=12 vlen=7 +data: "k"/5.000000000,0 -> /BYTES/v2 diff --git a/pkg/storage/testdata/mvcc_histories/put_out_of_order b/pkg/storage/testdata/mvcc_histories/put_out_of_order index 77944a1a659e..9e905ef325a3 100644 --- a/pkg/storage/testdata/mvcc_histories/put_out_of_order +++ b/pkg/storage/testdata/mvcc_histories/put_out_of_order @@ -10,9 +10,9 @@ with t=A put ts=1 k=k v=v ---- >> at end: -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000002,1 min=0,0 seq=0} lock=true stat=PENDING rts=0.000000001,0 wto=false max=0,0 -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000002,1 min=0,0 seq=0} ts=0.000000002,1 del=false klen=12 vlen=6 -data: "k"/0.000000002,1 -> /BYTES/v +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,1 min=0,0 seq=0} lock=true stat=PENDING rts=1.000000000,0 wto=false max=0,0 +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,1 min=0,0 seq=0} ts=2.000000000,1 del=false klen=12 vlen=6 +data: "k"/2.000000000,1 -> /BYTES/v # Put operation with earlier wall time. Will NOT be ignored. run ok @@ -22,9 +22,9 @@ with t=A put ts=1 k=k v=v2 ---- >> at end: -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000001,0 min=0,0 seq=1} lock=true stat=PENDING rts=0.000000001,0 wto=false max=0,0 -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000002,1 min=0,0 seq=1} ts=0.000000002,1 del=false klen=12 vlen=7 ih={{0 /BYTES/v}} -data: "k"/0.000000002,1 -> /BYTES/v2 +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=1} lock=true stat=PENDING rts=1.000000000,0 wto=false max=0,0 +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,1 min=0,0 seq=1} ts=2.000000000,1 del=false klen=12 vlen=7 ih={{0 /BYTES/v}} +data: "k"/2.000000000,1 -> /BYTES/v2 # We're expecting v2 here. @@ -32,7 +32,7 @@ run ok with t=A get ts=3 k=k ---- -get: "k" -> /BYTES/v2 @0.000000002,1 +get: "k" -> /BYTES/v2 @2.000000000,1 # Another put operation with earlier logical time. Will NOT be ignored. @@ -42,9 +42,9 @@ with t=A put ts=1 k=k v=v2 ---- >> at end: -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000001,0 min=0,0 seq=2} lock=true stat=PENDING rts=0.000000001,0 wto=false max=0,0 -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000002,1 min=0,0 seq=2} ts=0.000000002,1 del=false klen=12 vlen=7 ih={{0 /BYTES/v}{1 /BYTES/v2}} -data: "k"/0.000000002,1 -> /BYTES/v2 +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=2} lock=true stat=PENDING rts=1.000000000,0 wto=false max=0,0 +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=2.000000000,1 min=0,0 seq=2} ts=2.000000000,1 del=false klen=12 vlen=7 ih={{0 /BYTES/v}{1 /BYTES/v2}} +data: "k"/2.000000000,1 -> /BYTES/v2 # We're expecting v2 here. @@ -52,4 +52,4 @@ run ok with t=A get ts=3 k=k ---- -get: "k" -> /BYTES/v2 @0.000000002,1 +get: "k" -> /BYTES/v2 @2.000000000,1 diff --git a/pkg/storage/testdata/mvcc_histories/read_after_write_disallow_separated b/pkg/storage/testdata/mvcc_histories/read_after_write_disallow_separated index ad980c8615ec..cc71fafb4eda 100644 --- a/pkg/storage/testdata/mvcc_histories/read_after_write_disallow_separated +++ b/pkg/storage/testdata/mvcc_histories/read_after_write_disallow_separated @@ -9,16 +9,16 @@ with t=A resolve_intent ---- >> txn_begin ts=11 t=A -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=0} lock=true stat=PENDING rts=0.000000011,0 wto=false max=0,0 +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=11.000000000,0 wto=false max=0,0 >> put v=abc k=a t=A called PutIntent("a", _, NoExistingIntent, TDNUM(true), 00000000-0000-0000-0000-000000000001) -meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=0} ts=0.000000011,0 del=false klen=12 vlen=8 -data: "a"/0.000000011,0 -> /BYTES/abc +meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=0} ts=11.000000000,0 del=false klen=12 vlen=8 +data: "a"/11.000000000,0 -> /BYTES/abc >> get k=a t=A -get: "a" -> /BYTES/abc @0.000000011,0 +get: "a" -> /BYTES/abc @11.000000000,0 >> resolve_intent k=a t=A called ClearIntent("a", ExistingIntentInterleaved, TDNUM(false), 00000000-0000-0000-0000-000000000001) -data: "a"/0.000000011,0 -> /BYTES/abc +data: "a"/11.000000000,0 -> /BYTES/abc run ok with t=A resolve @@ -29,11 +29,11 @@ with t=A resolve txn_remove ---- >> at end: -data: "a"/0.000000011,0 -> /BYTES/abc -data: "a/1"/0.000000011,0 -> /BYTES/eee -data: "b"/0.000000011,0 -> /BYTES/fff -data: "b/2"/0.000000011,0 -> /BYTES/ggg -data: "c"/0.000000011,0 -> /BYTES/hhh +data: "a"/11.000000000,0 -> /BYTES/abc +data: "a/1"/11.000000000,0 -> /BYTES/eee +data: "b"/11.000000000,0 -> /BYTES/fff +data: "b/2"/11.000000000,0 -> /BYTES/ggg +data: "c"/11.000000000,0 -> /BYTES/hhh # Reads previous writes, transactional. @@ -42,9 +42,9 @@ with t=A txn_begin ts=11 get k=a ---- -get: "a" -> /BYTES/abc @0.000000011,0 +get: "a" -> /BYTES/abc @11.000000000,0 >> at end: -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=0} lock=true stat=PENDING rts=0.000000011,0 wto=false max=0,0 +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=11.000000000,0 wto=false max=0,0 run trace ok with t=A @@ -57,25 +57,25 @@ with t=A txn_remove ---- >> scan k=a end==b t=A -scan: "a" -> /BYTES/abc @0.000000011,0 -scan: "a/1" -> /BYTES/eee @0.000000011,0 +scan: "a" -> /BYTES/abc @11.000000000,0 +scan: "a/1" -> /BYTES/eee @11.000000000,0 >> scan k=a end=+a t=A -scan: "a" -> /BYTES/abc @0.000000011,0 +scan: "a" -> /BYTES/abc @11.000000000,0 >> scan k=a end=-a t=A -scan: "a" -> /BYTES/abc @0.000000011,0 -scan: "a/1" -> /BYTES/eee @0.000000011,0 +scan: "a" -> /BYTES/abc @11.000000000,0 +scan: "a/1" -> /BYTES/eee @11.000000000,0 >> scan k=a end=+b t=A -scan: "a" -> /BYTES/abc @0.000000011,0 -scan: "a/1" -> /BYTES/eee @0.000000011,0 -scan: "b" -> /BYTES/fff @0.000000011,0 +scan: "a" -> /BYTES/abc @11.000000000,0 +scan: "a/1" -> /BYTES/eee @11.000000000,0 +scan: "b" -> /BYTES/fff @11.000000000,0 >> scan k=a end==b t=A -scan: "a" -> /BYTES/abc @0.000000011,0 -scan: "a/1" -> /BYTES/eee @0.000000011,0 +scan: "a" -> /BYTES/abc @11.000000000,0 +scan: "a/1" -> /BYTES/eee @11.000000000,0 >> scan k=a end=-b t=A -scan: "a" -> /BYTES/abc @0.000000011,0 -scan: "a/1" -> /BYTES/eee @0.000000011,0 -scan: "b" -> /BYTES/fff @0.000000011,0 -scan: "b/2" -> /BYTES/ggg @0.000000011,0 +scan: "a" -> /BYTES/abc @11.000000000,0 +scan: "a/1" -> /BYTES/eee @11.000000000,0 +scan: "b" -> /BYTES/fff @11.000000000,0 +scan: "b/2" -> /BYTES/ggg @11.000000000,0 >> txn_remove t=A @@ -87,5 +87,5 @@ with t=A k=a txn_remove ---- >> txn_begin ts=1 t=A k=a -txn: "A" meta={id=00000000 key="a" pri=0.00000000 epo=0 ts=0.000000001,0 min=0,0 seq=0} lock=true stat=PENDING rts=0.000000001,0 wto=false max=0,0 +txn: "A" meta={id=00000000 key="a" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=1.000000000,0 wto=false max=0,0 >> txn_remove t=A k=a diff --git a/pkg/storage/testdata/mvcc_histories/read_fail_on_more_recent b/pkg/storage/testdata/mvcc_histories/read_fail_on_more_recent index 03731e6371c9..f3b61741c93a 100644 --- a/pkg/storage/testdata/mvcc_histories/read_fail_on_more_recent +++ b/pkg/storage/testdata/mvcc_histories/read_fail_on_more_recent @@ -6,7 +6,7 @@ run ok put k=k1 v=v ts=10,0 ---- >> at end: -data: "k1"/0.000000010,0 -> /BYTES/v +data: "k1"/10.000000000,0 -> /BYTES/v run ok with t=A @@ -14,10 +14,10 @@ with t=A put k=k2 v=v ---- >> at end: -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000010,0 min=0,0 seq=0} lock=true stat=PENDING rts=0.000000010,0 wto=false max=0,0 -data: "k1"/0.000000010,0 -> /BYTES/v -meta: "k2"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000010,0 min=0,0 seq=0} ts=0.000000010,0 del=false klen=12 vlen=6 -data: "k2"/0.000000010,0 -> /BYTES/v +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=10.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=10.000000000,0 wto=false max=0,0 +data: "k1"/10.000000000,0 -> /BYTES/v +meta: "k2"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=10.000000000,0 min=0,0 seq=0} ts=10.000000000,0 del=false klen=12 vlen=6 +data: "k2"/10.000000000,0 -> /BYTES/v # Test cases: # @@ -37,28 +37,28 @@ run error get k=k1 ts=9,0 failOnMoreRecent ---- get: "k1" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write at timestamp 0.000000009,0 too old; wrote at 0.000000010,1 +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write at timestamp 9.000000000,0 too old; wrote at 10.000000000,1 run ok get k=k1 ts=10,0 ---- -get: "k1" -> /BYTES/v @0.000000010,0 +get: "k1" -> /BYTES/v @10.000000000,0 run error get k=k1 ts=10,0 failOnMoreRecent ---- get: "k1" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write at timestamp 0.000000010,0 too old; wrote at 0.000000010,1 +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write at timestamp 10.000000000,0 too old; wrote at 10.000000000,1 run ok get k=k1 ts=11,0 ---- -get: "k1" -> /BYTES/v @0.000000010,0 +get: "k1" -> /BYTES/v @10.000000000,0 run ok get k=k1 ts=11,0 failOnMoreRecent ---- -get: "k1" -> /BYTES/v @0.000000010,0 +get: "k1" -> /BYTES/v @10.000000000,0 run ok scan k=k1 end=k2 ts=9,0 @@ -69,28 +69,28 @@ run error scan k=k1 end=k2 ts=9,0 failOnMoreRecent ---- scan: "k1"-"k2" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write at timestamp 0.000000009,0 too old; wrote at 0.000000010,1 +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write at timestamp 9.000000000,0 too old; wrote at 10.000000000,1 run ok scan k=k1 end=k2 ts=10,0 ---- -scan: "k1" -> /BYTES/v @0.000000010,0 +scan: "k1" -> /BYTES/v @10.000000000,0 run error scan k=k1 end=k2 ts=10,0 failOnMoreRecent ---- scan: "k1"-"k2" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write at timestamp 0.000000010,0 too old; wrote at 0.000000010,1 +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write at timestamp 10.000000000,0 too old; wrote at 10.000000000,1 run ok scan k=k1 end=k2 ts=11,0 ---- -scan: "k1" -> /BYTES/v @0.000000010,0 +scan: "k1" -> /BYTES/v @10.000000000,0 run ok scan k=k1 end=k2 ts=11,0 failOnMoreRecent ---- -scan: "k1" -> /BYTES/v @0.000000010,0 +scan: "k1" -> /BYTES/v @10.000000000,0 run ok get k=k2 ts=9,0 @@ -180,7 +180,7 @@ run error scan k=k1 end=k3 ts=9,0 failOnMoreRecent ---- scan: "k1"-"k3" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write at timestamp 0.000000009,0 too old; wrote at 0.000000010,1 +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write at timestamp 9.000000000,0 too old; wrote at 10.000000000,1 run error scan k=k1 end=k3 ts=10,0 @@ -192,7 +192,7 @@ run error scan k=k1 end=k3 ts=10,0 failOnMoreRecent ---- scan: "k1"-"k3" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write at timestamp 0.000000010,0 too old; wrote at 0.000000010,1 +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write at timestamp 10.000000000,0 too old; wrote at 10.000000000,1 run error scan k=k1 end=k3 ts=11,0 @@ -230,27 +230,27 @@ put k=b v=v ts=13,0 put k=c v=v ts=12,0 ---- >> at end: -data: "a"/0.000000011,0 -> /BYTES/v -data: "b"/0.000000013,0 -> /BYTES/v -data: "c"/0.000000012,0 -> /BYTES/v -data: "k1"/0.000000010,0 -> /BYTES/v -meta: "k2"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000010,0 min=0,0 seq=0} ts=0.000000010,0 del=false klen=12 vlen=6 -data: "k2"/0.000000010,0 -> /BYTES/v +data: "a"/11.000000000,0 -> /BYTES/v +data: "b"/13.000000000,0 -> /BYTES/v +data: "c"/12.000000000,0 -> /BYTES/v +data: "k1"/10.000000000,0 -> /BYTES/v +meta: "k2"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=10.000000000,0 min=0,0 seq=0} ts=10.000000000,0 del=false klen=12 vlen=6 +data: "k2"/10.000000000,0 -> /BYTES/v run error scan k=a end=b_next ts=9,0 failOnMoreRecent ---- scan: "a"-"b_next" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write at timestamp 0.000000009,0 too old; wrote at 0.000000013,1 +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write at timestamp 9.000000000,0 too old; wrote at 13.000000000,1 run error scan k=a end=c_next ts=9,0 failOnMoreRecent ---- scan: "a"-"c_next" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write at timestamp 0.000000009,0 too old; wrote at 0.000000013,1 +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write at timestamp 9.000000000,0 too old; wrote at 13.000000000,1 run error scan k=b end=c_next ts=9,0 failOnMoreRecent ---- scan: "b"-"c_next" -> -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write at timestamp 0.000000009,0 too old; wrote at 0.000000013,1 +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write at timestamp 9.000000000,0 too old; wrote at 13.000000000,1 diff --git a/pkg/storage/testdata/mvcc_histories/target_bytes b/pkg/storage/testdata/mvcc_histories/target_bytes index 0f593b2544ab..850929db0355 100644 --- a/pkg/storage/testdata/mvcc_histories/target_bytes +++ b/pkg/storage/testdata/mvcc_histories/target_bytes @@ -9,10 +9,10 @@ with ts=1,0 del k=aa ts=250,1 ---- >> at end: -data: "a"/0.000000001,0 -> /BYTES/nevergoingtobeseen -data: "aa"/0.000000250,1 -> / -data: "aa"/0.000000001,0 -> /BYTES/willbetombstoned -data: "e"/0.000000001,0 -> /BYTES/sameasabove +data: "a"/1.000000000,0 -> /BYTES/nevergoingtobeseen +data: "aa"/250.000000000,1 -> / +data: "aa"/1.000000000,0 -> /BYTES/willbetombstoned +data: "e"/1.000000000,0 -> /BYTES/sameasabove run ok with ts=123,45 @@ -21,13 +21,13 @@ with ts=123,45 put k=e v=mnopqr ---- >> at end: -data: "a"/0.000000123,45 -> /BYTES/abcdef -data: "a"/0.000000001,0 -> /BYTES/nevergoingtobeseen -data: "aa"/0.000000250,1 -> / -data: "aa"/0.000000001,0 -> /BYTES/willbetombstoned -data: "c"/0.000000123,45 -> /BYTES/ghijkllkjihg -data: "e"/0.000000123,45 -> /BYTES/mnopqr -data: "e"/0.000000001,0 -> /BYTES/sameasabove +data: "a"/123.000000000,45 -> /BYTES/abcdef +data: "a"/1.000000000,0 -> /BYTES/nevergoingtobeseen +data: "aa"/250.000000000,1 -> / +data: "aa"/1.000000000,0 -> /BYTES/willbetombstoned +data: "c"/123.000000000,45 -> /BYTES/ghijkllkjihg +data: "e"/123.000000000,45 -> /BYTES/mnopqr +data: "e"/1.000000000,0 -> /BYTES/sameasabove # Scans without or with a large target size return all results. @@ -36,37 +36,37 @@ with ts=300,0 k=a end=z scan scan reverse=true ---- -scan: "a" -> /BYTES/abcdef @0.000000123,45 -scan: "c" -> /BYTES/ghijkllkjihg @0.000000123,45 -scan: "e" -> /BYTES/mnopqr @0.000000123,45 -scan: "e" -> /BYTES/mnopqr @0.000000123,45 -scan: "c" -> /BYTES/ghijkllkjihg @0.000000123,45 -scan: "a" -> /BYTES/abcdef @0.000000123,45 +scan: "a" -> /BYTES/abcdef @123.000000000,45 +scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 +scan: "e" -> /BYTES/mnopqr @123.000000000,45 +scan: "e" -> /BYTES/mnopqr @123.000000000,45 +scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 +scan: "a" -> /BYTES/abcdef @123.000000000,45 run ok with ts=300,0 k=a end=z targetbytes=0 scan scan reverse=true ---- -scan: "a" -> /BYTES/abcdef @0.000000123,45 -scan: "c" -> /BYTES/ghijkllkjihg @0.000000123,45 -scan: "e" -> /BYTES/mnopqr @0.000000123,45 -scan: "e" -> /BYTES/mnopqr @0.000000123,45 -scan: "c" -> /BYTES/ghijkllkjihg @0.000000123,45 -scan: "a" -> /BYTES/abcdef @0.000000123,45 +scan: "a" -> /BYTES/abcdef @123.000000000,45 +scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 +scan: "e" -> /BYTES/mnopqr @123.000000000,45 +scan: "e" -> /BYTES/mnopqr @123.000000000,45 +scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 +scan: "a" -> /BYTES/abcdef @123.000000000,45 run ok with ts=300,0 k=a end=z targetbytes=10000000 scan scan reverse=true ---- -scan: "a" -> /BYTES/abcdef @0.000000123,45 -scan: "c" -> /BYTES/ghijkllkjihg @0.000000123,45 -scan: "e" -> /BYTES/mnopqr @0.000000123,45 +scan: "a" -> /BYTES/abcdef @123.000000000,45 +scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 +scan: "e" -> /BYTES/mnopqr @123.000000000,45 scan: 108 bytes (target 10000000) -scan: "e" -> /BYTES/mnopqr @0.000000123,45 -scan: "c" -> /BYTES/ghijkllkjihg @0.000000123,45 -scan: "a" -> /BYTES/abcdef @0.000000123,45 +scan: "e" -> /BYTES/mnopqr @123.000000000,45 +scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 +scan: "a" -> /BYTES/abcdef @123.000000000,45 scan: 108 bytes (target 10000000) # Scans with target size -1 return no results. @@ -96,46 +96,46 @@ run ok scan k=a end=z ts=300,0 targetbytes=1 ---- -scan: "a" -> /BYTES/abcdef @0.000000123,45 +scan: "a" -> /BYTES/abcdef @123.000000000,45 scan: resume span ["aa","z") scan: 34 bytes (target 1) run ok scan k=a end=z ts=300,0 targetbytes=34 ---- -scan: "a" -> /BYTES/abcdef @0.000000123,45 +scan: "a" -> /BYTES/abcdef @123.000000000,45 scan: resume span ["aa","z") scan: 34 bytes (target 34) run ok scan k=a end=z ts=300,0 targetbytes=35 ---- -scan: "a" -> /BYTES/abcdef @0.000000123,45 -scan: "c" -> /BYTES/ghijkllkjihg @0.000000123,45 +scan: "a" -> /BYTES/abcdef @123.000000000,45 +scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 scan: resume span ["e","z") scan: 74 bytes (target 35) run ok scan k=a end=z ts=300,0 targetbytes=74 ---- -scan: "a" -> /BYTES/abcdef @0.000000123,45 -scan: "c" -> /BYTES/ghijkllkjihg @0.000000123,45 +scan: "a" -> /BYTES/abcdef @123.000000000,45 +scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 scan: resume span ["e","z") scan: 74 bytes (target 74) run ok scan k=a end=z ts=300,0 targetbytes=75 ---- -scan: "a" -> /BYTES/abcdef @0.000000123,45 -scan: "c" -> /BYTES/ghijkllkjihg @0.000000123,45 -scan: "e" -> /BYTES/mnopqr @0.000000123,45 +scan: "a" -> /BYTES/abcdef @123.000000000,45 +scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 +scan: "e" -> /BYTES/mnopqr @123.000000000,45 scan: 108 bytes (target 75) # Works just the same when not starting on an existing key. run ok scan k=b end=z ts=300 targetbytes=1 ---- -scan: "c" -> /BYTES/ghijkllkjihg @0.000000123,45 +scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 scan: resume span ["e","z") scan: 40 bytes (target 1) @@ -144,39 +144,39 @@ scan: 40 bytes (target 1) run ok scan k=a end=z ts=300,0 targetbytes=1 reverse=true ---- -scan: "e" -> /BYTES/mnopqr @0.000000123,45 +scan: "e" -> /BYTES/mnopqr @123.000000000,45 scan: resume span ["a","c\x00") scan: 34 bytes (target 1) run ok scan k=a end=z ts=300,0 targetbytes=34 reverse=true ---- -scan: "e" -> /BYTES/mnopqr @0.000000123,45 +scan: "e" -> /BYTES/mnopqr @123.000000000,45 scan: resume span ["a","c\x00") scan: 34 bytes (target 34) run ok scan k=a end=z ts=300,0 targetbytes=35 reverse=true ---- -scan: "e" -> /BYTES/mnopqr @0.000000123,45 -scan: "c" -> /BYTES/ghijkllkjihg @0.000000123,45 +scan: "e" -> /BYTES/mnopqr @123.000000000,45 +scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 scan: resume span ["a","aa\x00") scan: 74 bytes (target 35) run ok scan k=a end=z ts=300,0 targetbytes=74 reverse=true ---- -scan: "e" -> /BYTES/mnopqr @0.000000123,45 -scan: "c" -> /BYTES/ghijkllkjihg @0.000000123,45 +scan: "e" -> /BYTES/mnopqr @123.000000000,45 +scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 scan: resume span ["a","aa\x00") scan: 74 bytes (target 74) run ok scan k=a end=z ts=300,0 targetbytes=75 reverse=true ---- -scan: "e" -> /BYTES/mnopqr @0.000000123,45 -scan: "c" -> /BYTES/ghijkllkjihg @0.000000123,45 -scan: "a" -> /BYTES/abcdef @0.000000123,45 +scan: "e" -> /BYTES/mnopqr @123.000000000,45 +scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 +scan: "a" -> /BYTES/abcdef @123.000000000,45 scan: 108 bytes (target 75) # Scans that return the tombstone (at aa@250,1). The kv pair at a has 34 bytes, @@ -185,32 +185,32 @@ scan: 108 bytes (target 75) run ok scan k=a end=z ts=300,0 targetbytes=34 tombstones=true ---- -scan: "a" -> /BYTES/abcdef @0.000000123,45 +scan: "a" -> /BYTES/abcdef @123.000000000,45 scan: resume span ["aa","z") scan: 34 bytes (target 34) run ok scan k=a end=z ts=300,0 targetbytes=35 tombstones=true ---- -scan: "a" -> /BYTES/abcdef @0.000000123,45 -scan: "aa" -> / @0.000000250,1 +scan: "a" -> /BYTES/abcdef @123.000000000,45 +scan: "aa" -> / @250.000000000,1 scan: resume span ["c","z") scan: 58 bytes (target 35) run ok scan k=a end=z ts=300,0 targetbytes=58 tombstones=true ---- -scan: "a" -> /BYTES/abcdef @0.000000123,45 -scan: "aa" -> / @0.000000250,1 +scan: "a" -> /BYTES/abcdef @123.000000000,45 +scan: "aa" -> / @250.000000000,1 scan: resume span ["c","z") scan: 58 bytes (target 58) run ok scan k=a end=z ts=300,0 targetbytes=59 tombstones=true ---- -scan: "a" -> /BYTES/abcdef @0.000000123,45 -scan: "aa" -> / @0.000000250,1 -scan: "c" -> /BYTES/ghijkllkjihg @0.000000123,45 +scan: "a" -> /BYTES/abcdef @123.000000000,45 +scan: "aa" -> / @250.000000000,1 +scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 scan: resume span ["e","z") scan: 98 bytes (target 59) @@ -219,32 +219,32 @@ scan: 98 bytes (target 59) run ok scan k=a end=d ts=300,0 targetbytes=40 reverse=true tombstones=true ---- -scan: "c" -> /BYTES/ghijkllkjihg @0.000000123,45 +scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 scan: resume span ["a","aa\x00") scan: 40 bytes (target 40) run ok scan k=a end=d ts=300,0 targetbytes=41 reverse=true tombstones=true ---- -scan: "c" -> /BYTES/ghijkllkjihg @0.000000123,45 -scan: "aa" -> / @0.000000250,1 +scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 +scan: "aa" -> / @250.000000000,1 scan: resume span ["a","a\x00") scan: 64 bytes (target 41) run ok scan k=a end=d ts=300,0 targetbytes=64 reverse=true tombstones=true ---- -scan: "c" -> /BYTES/ghijkllkjihg @0.000000123,45 -scan: "aa" -> / @0.000000250,1 +scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 +scan: "aa" -> / @250.000000000,1 scan: resume span ["a","a\x00") scan: 64 bytes (target 64) run ok scan k=a end=d ts=300,0 targetbytes=65 reverse=true tombstones=true ---- -scan: "c" -> /BYTES/ghijkllkjihg @0.000000123,45 -scan: "aa" -> / @0.000000250,1 -scan: "a" -> /BYTES/abcdef @0.000000123,45 +scan: "c" -> /BYTES/ghijkllkjihg @123.000000000,45 +scan: "aa" -> / @250.000000000,1 +scan: "a" -> /BYTES/abcdef @123.000000000,45 scan: 98 bytes (target 65) # Regression test for a bug simiar to #46652: Test appropriate termination when @@ -272,28 +272,28 @@ with t=A ts=11,0 targetbytes=32 scan k=k end=o scan k=k end=o reverse=true ---- -scan: "k" -> /BYTES/b @0.000000011,0 -scan: "l" -> /BYTES/b @0.000000011,0 +scan: "k" -> /BYTES/b @11.000000000,0 +scan: "l" -> /BYTES/b @11.000000000,0 scan: resume span ["m","o") scan: 50 bytes (target 32) -scan: "n" -> /BYTES/b @0.000000011,0 -scan: "m" -> /BYTES/b @0.000000011,0 +scan: "n" -> /BYTES/b @11.000000000,0 +scan: "m" -> /BYTES/b @11.000000000,0 scan: resume span ["k","l\x00") scan: 50 bytes (target 32) >> at end: -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=20} lock=true stat=PENDING rts=0.000000011,0 wto=false max=0,0 -data: "a"/0.000000123,45 -> /BYTES/abcdef -data: "a"/0.000000001,0 -> /BYTES/nevergoingtobeseen -data: "aa"/0.000000250,1 -> / -data: "aa"/0.000000001,0 -> /BYTES/willbetombstoned -data: "c"/0.000000123,45 -> /BYTES/ghijkllkjihg -data: "e"/0.000000123,45 -> /BYTES/mnopqr -data: "e"/0.000000001,0 -> /BYTES/sameasabove -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=30} ts=0.000000011,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} -data: "k"/0.000000011,0 -> /BYTES/c -meta: "l"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=30} ts=0.000000011,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} -data: "l"/0.000000011,0 -> /BYTES/c -meta: "m"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=30} ts=0.000000011,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} -data: "m"/0.000000011,0 -> /BYTES/c -meta: "n"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000011,0 min=0,0 seq=30} ts=0.000000011,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} -data: "n"/0.000000011,0 -> /BYTES/c +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=20} lock=true stat=PENDING rts=11.000000000,0 wto=false max=0,0 +data: "a"/123.000000000,45 -> /BYTES/abcdef +data: "a"/1.000000000,0 -> /BYTES/nevergoingtobeseen +data: "aa"/250.000000000,1 -> / +data: "aa"/1.000000000,0 -> /BYTES/willbetombstoned +data: "c"/123.000000000,45 -> /BYTES/ghijkllkjihg +data: "e"/123.000000000,45 -> /BYTES/mnopqr +data: "e"/1.000000000,0 -> /BYTES/sameasabove +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +data: "k"/11.000000000,0 -> /BYTES/c +meta: "l"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +data: "l"/11.000000000,0 -> /BYTES/c +meta: "m"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +data: "m"/11.000000000,0 -> /BYTES/c +meta: "n"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=11.000000000,0 min=0,0 seq=30} ts=11.000000000,0 del=false klen=12 vlen=6 ih={{10 /BYTES/a}{20 /BYTES/b}} +data: "n"/11.000000000,0 -> /BYTES/c diff --git a/pkg/storage/testdata/mvcc_histories/uncertainty_interval b/pkg/storage/testdata/mvcc_histories/uncertainty_interval index 12743a2a0bd5..7ecacf277873 100644 --- a/pkg/storage/testdata/mvcc_histories/uncertainty_interval +++ b/pkg/storage/testdata/mvcc_histories/uncertainty_interval @@ -12,8 +12,8 @@ with k=k1 put v=v2 ts=20,0 ---- >> at end: -data: "k1"/0.000000020,0 -> /BYTES/v2 -data: "k1"/0.000000010,0 -> /BYTES/v +data: "k1"/20.000000000,0 -> /BYTES/v2 +data: "k1"/10.000000000,0 -> /BYTES/v run ok with k=k2 @@ -22,12 +22,12 @@ with k=k2 put t=A v=v4 ---- >> at end: -txn: "A" meta={id=00000000 key="k2" pri=0.00000000 epo=0 ts=0.000000020,0 min=0,0 seq=0} lock=true stat=PENDING rts=0.000000020,0 wto=false max=0,0 -data: "k1"/0.000000020,0 -> /BYTES/v2 -data: "k1"/0.000000010,0 -> /BYTES/v -meta: "k2"/0,0 -> txn={id=00000000 key="k2" pri=0.00000000 epo=0 ts=0.000000020,0 min=0,0 seq=0} ts=0.000000020,0 del=false klen=12 vlen=7 -data: "k2"/0.000000020,0 -> /BYTES/v4 -data: "k2"/0.000000010,0 -> /BYTES/v3 +txn: "A" meta={id=00000000 key="k2" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=20.000000000,0 wto=false max=0,0 +data: "k1"/20.000000000,0 -> /BYTES/v2 +data: "k1"/10.000000000,0 -> /BYTES/v +meta: "k2"/0,0 -> txn={id=00000000 key="k2" pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} ts=20.000000000,0 del=false klen=12 vlen=7 +data: "k2"/20.000000000,0 -> /BYTES/v4 +data: "k2"/10.000000000,0 -> /BYTES/v3 # Test cases: # @@ -43,7 +43,7 @@ run ok txn_begin t=txn1 ts=5,0 maxTs=5,0 ---- >> at end: -txn: "txn1" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000005,0 min=0,0 seq=0} lock=true stat=PENDING rts=0.000000005,0 wto=false max=0.000000005,0 +txn: "txn1" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=5.000000000,0 wto=false max=5.000000000,0 run ok get t=txn1 k=k1 @@ -70,81 +70,81 @@ run ok txn_begin t=txn2 ts=5,0 maxTs=10,0 ---- >> at end: -txn: "txn2" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000005,0 min=0,0 seq=0} lock=true stat=PENDING rts=0.000000005,0 wto=false max=0.000000010,0 +txn: "txn2" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=5.000000000,0 wto=false max=10.000000000,0 run error get t=txn2 k=k1 ---- get: "k1" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 0.000000005,0 encountered previous write with future timestamp 0.000000010,0 within uncertainty interval `t <= 0.000000010,0`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= 10.000000000,0`; observed timestamps: [] run error scan t=txn2 k=k1 ---- scan: "k1"-"k1\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 0.000000005,0 encountered previous write with future timestamp 0.000000010,0 within uncertainty interval `t <= 0.000000010,0`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= 10.000000000,0`; observed timestamps: [] run error get t=txn2 k=k2 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 0.000000005,0 encountered previous write with future timestamp 0.000000010,0 within uncertainty interval `t <= 0.000000010,0`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= 10.000000000,0`; observed timestamps: [] run error scan t=txn2 k=k2 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 0.000000005,0 encountered previous write with future timestamp 0.000000010,0 within uncertainty interval `t <= 0.000000010,0`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= 10.000000000,0`; observed timestamps: [] run ok txn_begin t=txn3 ts=5,0 maxTs=15,0 ---- >> at end: -txn: "txn3" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000005,0 min=0,0 seq=0} lock=true stat=PENDING rts=0.000000005,0 wto=false max=0.000000015,0 +txn: "txn3" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=5.000000000,0 wto=false max=15.000000000,0 run error get t=txn3 k=k1 ---- get: "k1" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 0.000000005,0 encountered previous write with future timestamp 0.000000010,0 within uncertainty interval `t <= 0.000000015,0`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= 15.000000000,0`; observed timestamps: [] run error scan t=txn3 k=k1 ---- scan: "k1"-"k1\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 0.000000005,0 encountered previous write with future timestamp 0.000000010,0 within uncertainty interval `t <= 0.000000015,0`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= 15.000000000,0`; observed timestamps: [] run error get t=txn3 k=k2 ---- get: "k2" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 0.000000005,0 encountered previous write with future timestamp 0.000000010,0 within uncertainty interval `t <= 0.000000015,0`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= 15.000000000,0`; observed timestamps: [] run error scan t=txn3 k=k2 ---- scan: "k2"-"k2\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 0.000000005,0 encountered previous write with future timestamp 0.000000010,0 within uncertainty interval `t <= 0.000000015,0`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 10.000000000,0 within uncertainty interval `t <= 15.000000000,0`; observed timestamps: [] run ok txn_begin t=txn4 ts=5,0 maxTs=20,0 ---- >> at end: -txn: "txn4" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000005,0 min=0,0 seq=0} lock=true stat=PENDING rts=0.000000005,0 wto=false max=0.000000020,0 +txn: "txn4" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=5.000000000,0 wto=false max=20.000000000,0 run error get t=txn4 k=k1 ---- get: "k1" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 0.000000005,0 encountered previous write with future timestamp 0.000000020,0 within uncertainty interval `t <= 0.000000020,0`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= 20.000000000,0`; observed timestamps: [] run error scan t=txn4 k=k1 ---- scan: "k1"-"k1\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 0.000000005,0 encountered previous write with future timestamp 0.000000020,0 within uncertainty interval `t <= 0.000000020,0`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= 20.000000000,0`; observed timestamps: [] run error get t=txn4 k=k2 @@ -163,19 +163,19 @@ run ok txn_begin t=txn5 ts=5,0 maxTs=25,0 ---- >> at end: -txn: "txn5" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000005,0 min=0,0 seq=0} lock=true stat=PENDING rts=0.000000005,0 wto=false max=0.000000025,0 +txn: "txn5" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=5.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=5.000000000,0 wto=false max=25.000000000,0 run error get t=txn5 k=k1 ---- get: "k1" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 0.000000005,0 encountered previous write with future timestamp 0.000000020,0 within uncertainty interval `t <= 0.000000025,0`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= 25.000000000,0`; observed timestamps: [] run error scan t=txn5 k=k1 ---- scan: "k1"-"k1\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 0.000000005,0 encountered previous write with future timestamp 0.000000020,0 within uncertainty interval `t <= 0.000000025,0`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 5.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= 25.000000000,0`; observed timestamps: [] run error get t=txn5 k=k2 @@ -194,73 +194,73 @@ run ok txn_begin t=txn6 ts=10,0 maxTs=10,0 ---- >> at end: -txn: "txn6" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000010,0 min=0,0 seq=0} lock=true stat=PENDING rts=0.000000010,0 wto=false max=0.000000010,0 +txn: "txn6" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=10.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=10.000000000,0 wto=false max=10.000000000,0 run ok get t=txn6 k=k1 ---- -get: "k1" -> /BYTES/v @0.000000010,0 +get: "k1" -> /BYTES/v @10.000000000,0 run ok scan t=txn6 k=k1 ---- -scan: "k1" -> /BYTES/v @0.000000010,0 +scan: "k1" -> /BYTES/v @10.000000000,0 run ok get t=txn6 k=k2 ---- -get: "k2" -> /BYTES/v3 @0.000000010,0 +get: "k2" -> /BYTES/v3 @10.000000000,0 run ok scan t=txn6 k=k2 ---- -scan: "k2" -> /BYTES/v3 @0.000000010,0 +scan: "k2" -> /BYTES/v3 @10.000000000,0 run ok txn_begin t=txn7 ts=10,0 maxTs=15,0 ---- >> at end: -txn: "txn7" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000010,0 min=0,0 seq=0} lock=true stat=PENDING rts=0.000000010,0 wto=false max=0.000000015,0 +txn: "txn7" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=10.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=10.000000000,0 wto=false max=15.000000000,0 run ok get t=txn7 k=k1 ---- -get: "k1" -> /BYTES/v @0.000000010,0 +get: "k1" -> /BYTES/v @10.000000000,0 run ok scan t=txn7 k=k1 ---- -scan: "k1" -> /BYTES/v @0.000000010,0 +scan: "k1" -> /BYTES/v @10.000000000,0 run ok get t=txn7 k=k2 ---- -get: "k2" -> /BYTES/v3 @0.000000010,0 +get: "k2" -> /BYTES/v3 @10.000000000,0 run ok scan t=txn7 k=k2 ---- -scan: "k2" -> /BYTES/v3 @0.000000010,0 +scan: "k2" -> /BYTES/v3 @10.000000000,0 run ok txn_begin t=txn8 ts=10,0 maxTs=20,0 ---- >> at end: -txn: "txn8" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000010,0 min=0,0 seq=0} lock=true stat=PENDING rts=0.000000010,0 wto=false max=0.000000020,0 +txn: "txn8" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=10.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=10.000000000,0 wto=false max=20.000000000,0 run error get t=txn8 k=k1 ---- get: "k1" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 0.000000010,0 encountered previous write with future timestamp 0.000000020,0 within uncertainty interval `t <= 0.000000020,0`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= 20.000000000,0`; observed timestamps: [] run error scan t=txn8 k=k1 ---- scan: "k1"-"k1\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 0.000000010,0 encountered previous write with future timestamp 0.000000020,0 within uncertainty interval `t <= 0.000000020,0`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= 20.000000000,0`; observed timestamps: [] run error get t=txn8 k=k2 @@ -279,19 +279,19 @@ run ok txn_begin t=txn9 ts=10,0 maxTs=25,0 ---- >> at end: -txn: "txn9" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000010,0 min=0,0 seq=0} lock=true stat=PENDING rts=0.000000010,0 wto=false max=0.000000025,0 +txn: "txn9" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=10.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=10.000000000,0 wto=false max=25.000000000,0 run error get t=txn9 k=k1 ---- get: "k1" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 0.000000010,0 encountered previous write with future timestamp 0.000000020,0 within uncertainty interval `t <= 0.000000025,0`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= 25.000000000,0`; observed timestamps: [] run error scan t=txn9 k=k1 ---- scan: "k1"-"k1\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 0.000000010,0 encountered previous write with future timestamp 0.000000020,0 within uncertainty interval `t <= 0.000000025,0`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 10.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= 25.000000000,0`; observed timestamps: [] run error get t=txn9 k=k2 @@ -310,46 +310,46 @@ run ok txn_begin t=txn10 ts=15,0 maxTs=15,0 ---- >> at end: -txn: "txn10" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000015,0 min=0,0 seq=0} lock=true stat=PENDING rts=0.000000015,0 wto=false max=0.000000015,0 +txn: "txn10" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=15.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=15.000000000,0 wto=false max=15.000000000,0 run ok get t=txn10 k=k1 ---- -get: "k1" -> /BYTES/v @0.000000010,0 +get: "k1" -> /BYTES/v @10.000000000,0 run ok scan t=txn10 k=k1 ---- -scan: "k1" -> /BYTES/v @0.000000010,0 +scan: "k1" -> /BYTES/v @10.000000000,0 run ok get t=txn10 k=k2 ---- -get: "k2" -> /BYTES/v3 @0.000000010,0 +get: "k2" -> /BYTES/v3 @10.000000000,0 run ok scan t=txn10 k=k2 ---- -scan: "k2" -> /BYTES/v3 @0.000000010,0 +scan: "k2" -> /BYTES/v3 @10.000000000,0 run ok txn_begin t=txn11 ts=15,0 maxTs=20,0 ---- >> at end: -txn: "txn11" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000015,0 min=0,0 seq=0} lock=true stat=PENDING rts=0.000000015,0 wto=false max=0.000000020,0 +txn: "txn11" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=15.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=15.000000000,0 wto=false max=20.000000000,0 run error get t=txn11 k=k1 ---- get: "k1" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 0.000000015,0 encountered previous write with future timestamp 0.000000020,0 within uncertainty interval `t <= 0.000000020,0`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= 20.000000000,0`; observed timestamps: [] run error scan t=txn11 k=k1 ---- scan: "k1"-"k1\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 0.000000015,0 encountered previous write with future timestamp 0.000000020,0 within uncertainty interval `t <= 0.000000020,0`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= 20.000000000,0`; observed timestamps: [] run error get t=txn11 k=k2 @@ -368,19 +368,19 @@ run ok txn_begin t=txn12 ts=15,0 maxTs=25,0 ---- >> at end: -txn: "txn12" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000015,0 min=0,0 seq=0} lock=true stat=PENDING rts=0.000000015,0 wto=false max=0.000000025,0 +txn: "txn12" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=15.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=15.000000000,0 wto=false max=25.000000000,0 run error get t=txn12 k=k1 ---- get: "k1" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 0.000000015,0 encountered previous write with future timestamp 0.000000020,0 within uncertainty interval `t <= 0.000000025,0`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= 25.000000000,0`; observed timestamps: [] run error scan t=txn12 k=k1 ---- scan: "k1"-"k1\x00" -> -error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 0.000000015,0 encountered previous write with future timestamp 0.000000020,0 within uncertainty interval `t <= 0.000000025,0`; observed timestamps: [] +error: (*roachpb.ReadWithinUncertaintyIntervalError:) ReadWithinUncertaintyIntervalError: read at time 15.000000000,0 encountered previous write with future timestamp 20.000000000,0 within uncertainty interval `t <= 25.000000000,0`; observed timestamps: [] run error get t=txn12 k=k2 @@ -399,17 +399,17 @@ run ok txn_begin t=txn13 ts=20,0 maxTs=20,0 ---- >> at end: -txn: "txn13" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000020,0 min=0,0 seq=0} lock=true stat=PENDING rts=0.000000020,0 wto=false max=0.000000020,0 +txn: "txn13" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=20.000000000,0 wto=false max=20.000000000,0 run ok get t=txn13 k=k1 ---- -get: "k1" -> /BYTES/v2 @0.000000020,0 +get: "k1" -> /BYTES/v2 @20.000000000,0 run ok scan t=txn13 k=k1 ---- -scan: "k1" -> /BYTES/v2 @0.000000020,0 +scan: "k1" -> /BYTES/v2 @20.000000000,0 run error get t=txn13 k=k2 @@ -428,17 +428,17 @@ run ok txn_begin t=txn14 ts=20,0 maxTs=25,0 ---- >> at end: -txn: "txn14" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000020,0 min=0,0 seq=0} lock=true stat=PENDING rts=0.000000020,0 wto=false max=0.000000025,0 +txn: "txn14" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=20.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=20.000000000,0 wto=false max=25.000000000,0 run ok get t=txn14 k=k1 ---- -get: "k1" -> /BYTES/v2 @0.000000020,0 +get: "k1" -> /BYTES/v2 @20.000000000,0 run ok scan t=txn14 k=k1 ---- -scan: "k1" -> /BYTES/v2 @0.000000020,0 +scan: "k1" -> /BYTES/v2 @20.000000000,0 run error get t=txn14 k=k2 @@ -457,17 +457,17 @@ run ok txn_begin t=txn15 ts=25,0 maxTs=25,0 ---- >> at end: -txn: "txn15" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000025,0 min=0,0 seq=0} lock=true stat=PENDING rts=0.000000025,0 wto=false max=0.000000025,0 +txn: "txn15" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=25.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=25.000000000,0 wto=false max=25.000000000,0 run ok get t=txn15 k=k1 ---- -get: "k1" -> /BYTES/v2 @0.000000020,0 +get: "k1" -> /BYTES/v2 @20.000000000,0 run ok scan t=txn15 k=k1 ---- -scan: "k1" -> /BYTES/v2 @0.000000020,0 +scan: "k1" -> /BYTES/v2 @20.000000000,0 run error get t=txn15 k=k2 diff --git a/pkg/storage/testdata/mvcc_histories/update_existing_key b/pkg/storage/testdata/mvcc_histories/update_existing_key index 45b1e1c7d074..762c596274e5 100644 --- a/pkg/storage/testdata/mvcc_histories/update_existing_key +++ b/pkg/storage/testdata/mvcc_histories/update_existing_key @@ -6,14 +6,14 @@ with k=k ---- get: "k" -> /BYTES/v @0,1 >> at end: -data: "k"/0.000000002,0 -> /BYTES/v2 +data: "k"/2.000000000,0 -> /BYTES/v2 data: "k"/0,1 -> /BYTES/v # Read the latest version. run ok get k=k ts=3 ---- -get: "k" -> /BYTES/v2 @0.000000002,0 +get: "k" -> /BYTES/v2 @2.000000000,0 # Read the old version. run ok diff --git a/pkg/storage/testdata/mvcc_histories/update_existing_key_diff_txn b/pkg/storage/testdata/mvcc_histories/update_existing_key_diff_txn index 6602b8c01929..aa78f38fa51d 100644 --- a/pkg/storage/testdata/mvcc_histories/update_existing_key_diff_txn +++ b/pkg/storage/testdata/mvcc_histories/update_existing_key_diff_txn @@ -10,9 +10,9 @@ with t=B put k=a v=zzz ---- >> at end: -txn: "B" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000044,0 min=0,0 seq=0} lock=true stat=PENDING rts=0.000000044,0 wto=false max=0,0 -meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000033,0 min=0,0 seq=0} ts=0.000000033,0 del=false klen=12 vlen=8 -data: "a"/0.000000033,0 -> /BYTES/xyz +txn: "B" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=44.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=44.000000000,0 wto=false max=0,0 +meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=33.000000000,0 min=0,0 seq=0} ts=33.000000000,0 del=false klen=12 vlen=8 +data: "a"/33.000000000,0 -> /BYTES/xyz error: (*roachpb.WriteIntentError:) conflicting intents on "a" ## It's possible to observe the intent with an inconsistent read. @@ -21,5 +21,5 @@ run ok with t=B get k=a inconsistent ---- -get: "a" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000033,0 min=0,0 seq=0} +get: "a" -> intent {id=00000000 key=/Min pri=0.00000000 epo=0 ts=33.000000000,0 min=0,0 seq=0} get: "a" -> diff --git a/pkg/storage/testdata/mvcc_histories/update_existing_key_in_txn b/pkg/storage/testdata/mvcc_histories/update_existing_key_in_txn index 6a2b03ce7e7f..80c22900ef7b 100644 --- a/pkg/storage/testdata/mvcc_histories/update_existing_key_in_txn +++ b/pkg/storage/testdata/mvcc_histories/update_existing_key_in_txn @@ -18,6 +18,6 @@ with t=A put k=k v=v ---- >> at end: -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000001,0 min=0,0 seq=1} lock=true stat=PENDING rts=0,1 wto=false max=0,0 -meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000001,0 min=0,0 seq=1} ts=0.000000001,0 del=false klen=12 vlen=6 ih={{0 /BYTES/v}} -data: "k"/0.000000001,0 -> /BYTES/v +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=1} lock=true stat=PENDING rts=0,1 wto=false max=0,0 +meta: "k"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=1} ts=1.000000000,0 del=false klen=12 vlen=6 ih={{0 /BYTES/v}} +data: "k"/1.000000000,0 -> /BYTES/v diff --git a/pkg/storage/testdata/mvcc_histories/update_existing_key_old_version b/pkg/storage/testdata/mvcc_histories/update_existing_key_old_version index fda7792361e8..9b5084f5b605 100644 --- a/pkg/storage/testdata/mvcc_histories/update_existing_key_old_version +++ b/pkg/storage/testdata/mvcc_histories/update_existing_key_old_version @@ -2,7 +2,7 @@ run ok put k=k v=v ts=1,1 ---- >> at end: -data: "k"/0.000000001,1 -> /BYTES/v +data: "k"/1.000000000,1 -> /BYTES/v # Earlier wall time. # Note: a WriteTooOld "error" is returned, but really the write completed, @@ -12,9 +12,9 @@ run error put k=k v=v2 ts=0,1 ---- >> at end: -data: "k"/0.000000001,2 -> /BYTES/v2 -data: "k"/0.000000001,1 -> /BYTES/v -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write at timestamp 0,1 too old; wrote at 0.000000001,2 +data: "k"/1.000000000,2 -> /BYTES/v2 +data: "k"/1.000000000,1 -> /BYTES/v +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write at timestamp 0,1 too old; wrote at 1.000000000,2 # Earlier logical time. @@ -22,7 +22,7 @@ run error put k=k v=v2 ts=1,0 ---- >> at end: -data: "k"/0.000000001,3 -> /BYTES/v2 -data: "k"/0.000000001,2 -> /BYTES/v2 -data: "k"/0.000000001,1 -> /BYTES/v -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write at timestamp 0.000000001,0 too old; wrote at 0.000000001,3 +data: "k"/1.000000000,3 -> /BYTES/v2 +data: "k"/1.000000000,2 -> /BYTES/v2 +data: "k"/1.000000000,1 -> /BYTES/v +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write at timestamp 1.000000000,0 too old; wrote at 1.000000000,3 diff --git a/pkg/storage/testdata/mvcc_histories/write_too_old b/pkg/storage/testdata/mvcc_histories/write_too_old index d164b6c813a0..bc9e41a9f133 100644 --- a/pkg/storage/testdata/mvcc_histories/write_too_old +++ b/pkg/storage/testdata/mvcc_histories/write_too_old @@ -10,7 +10,7 @@ with t=A txn_remove ---- >> at end: -data: "a"/0.000000044,0 -> /BYTES/abc +data: "a"/44.000000000,0 -> /BYTES/abc # Try an old write, expect an error. @@ -20,8 +20,8 @@ with t=A del k=a ---- >> at end: -txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000033,0 min=0,0 seq=0} lock=true stat=PENDING rts=0.000000033,0 wto=false max=0,0 -meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=0.000000044,1 min=0,0 seq=0} ts=0.000000044,1 del=true klen=12 vlen=0 -data: "a"/0.000000044,1 -> / -data: "a"/0.000000044,0 -> /BYTES/abc -error: (*roachpb.WriteTooOldError:) WriteTooOldError: write at timestamp 0.000000033,0 too old; wrote at 0.000000044,1 +txn: "A" meta={id=00000000 key=/Min pri=0.00000000 epo=0 ts=33.000000000,0 min=0,0 seq=0} lock=true stat=PENDING rts=33.000000000,0 wto=false max=0,0 +meta: "a"/0,0 -> txn={id=00000000 key=/Min pri=0.00000000 epo=0 ts=44.000000000,1 min=0,0 seq=0} ts=44.000000000,1 del=true klen=12 vlen=0 +data: "a"/44.000000000,1 -> / +data: "a"/44.000000000,0 -> /BYTES/abc +error: (*roachpb.WriteTooOldError:) WriteTooOldError: write at timestamp 33.000000000,0 too old; wrote at 44.000000000,1 diff --git a/pkg/storage/testdata/mvcc_histories/write_with_sequence b/pkg/storage/testdata/mvcc_histories/write_with_sequence index 4e1027a08046..75dc358f99c5 100644 --- a/pkg/storage/testdata/mvcc_histories/write_with_sequence +++ b/pkg/storage/testdata/mvcc_histories/write_with_sequence @@ -17,9 +17,9 @@ with t=t k=k ---- put: batch after write is empty >> at end: -txn: "t" meta={id=00000000 key="k" pri=0.00000000 epo=0 ts=0.000000001,0 min=0,0 seq=1} lock=true stat=PENDING rts=0.000000001,0 wto=false max=0,0 -meta: "k"/0,0 -> txn={id=00000000 key="k" pri=0.00000000 epo=0 ts=0.000000001,0 min=0,0 seq=3} ts=0.000000001,0 del=false klen=12 vlen=7 ih={{2 /BYTES/v1}} -data: "k"/0.000000001,0 -> /BYTES/v2 +txn: "t" meta={id=00000000 key="k" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=1} lock=true stat=PENDING rts=1.000000000,0 wto=false max=0,0 +meta: "k"/0,0 -> txn={id=00000000 key="k" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=3} ts=1.000000000,0 del=false klen=12 vlen=7 ih={{2 /BYTES/v1}} +data: "k"/1.000000000,0 -> /BYTES/v2 error: (*withstack.withStack:) transaction 00000000-0000-0000-0000-000000000001 with sequence 3 missing an intent with lower sequence 1 run ok @@ -43,9 +43,9 @@ with t=t k=k ---- put: batch after write is empty >> at end: -txn: "t" meta={id=00000000 key="k" pri=0.00000000 epo=0 ts=0.000000001,0 min=0,0 seq=2} lock=true stat=PENDING rts=0.000000001,0 wto=false max=0,0 -meta: "k"/0,0 -> txn={id=00000000 key="k" pri=0.00000000 epo=0 ts=0.000000001,0 min=0,0 seq=3} ts=0.000000001,0 del=false klen=12 vlen=7 ih={{2 /BYTES/v1}} -data: "k"/0.000000001,0 -> /BYTES/v2 +txn: "t" meta={id=00000000 key="k" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=2} lock=true stat=PENDING rts=1.000000000,0 wto=false max=0,0 +meta: "k"/0,0 -> txn={id=00000000 key="k" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=3} ts=1.000000000,0 del=false klen=12 vlen=7 ih={{2 /BYTES/v1}} +data: "k"/1.000000000,0 -> /BYTES/v2 run ok txn_remove t=t @@ -68,9 +68,9 @@ with t=t k=k ---- put: batch after write is empty >> at end: -txn: "t" meta={id=00000000 key="k" pri=0.00000000 epo=0 ts=0.000000001,0 min=0,0 seq=2} lock=true stat=PENDING rts=0.000000001,0 wto=false max=0,0 -meta: "k"/0,0 -> txn={id=00000000 key="k" pri=0.00000000 epo=0 ts=0.000000001,0 min=0,0 seq=3} ts=0.000000001,0 del=false klen=12 vlen=7 ih={{2 /BYTES/v1}} -data: "k"/0.000000001,0 -> /BYTES/v2 +txn: "t" meta={id=00000000 key="k" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=2} lock=true stat=PENDING rts=1.000000000,0 wto=false max=0,0 +meta: "k"/0,0 -> txn={id=00000000 key="k" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=3} ts=1.000000000,0 del=false klen=12 vlen=7 ih={{2 /BYTES/v1}} +data: "k"/1.000000000,0 -> /BYTES/v2 error: (*withstack.withStack:) transaction 00000000-0000-0000-0000-000000000003 with sequence 2 has a different value [0 0 0 0 3 118 50] after recomputing from what was written: [0 0 0 0 3 118 49] run ok @@ -94,9 +94,9 @@ with t=t k=k ---- put: batch after write is empty >> at end: -txn: "t" meta={id=00000000 key="k" pri=0.00000000 epo=0 ts=0.000000001,0 min=0,0 seq=3} lock=true stat=PENDING rts=0.000000001,0 wto=false max=0,0 -meta: "k"/0,0 -> txn={id=00000000 key="k" pri=0.00000000 epo=0 ts=0.000000001,0 min=0,0 seq=3} ts=0.000000001,0 del=false klen=12 vlen=7 ih={{2 /BYTES/v1}} -data: "k"/0.000000001,0 -> /BYTES/v2 +txn: "t" meta={id=00000000 key="k" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=3} lock=true stat=PENDING rts=1.000000000,0 wto=false max=0,0 +meta: "k"/0,0 -> txn={id=00000000 key="k" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=3} ts=1.000000000,0 del=false klen=12 vlen=7 ih={{2 /BYTES/v1}} +data: "k"/1.000000000,0 -> /BYTES/v2 run ok @@ -119,9 +119,9 @@ with t=t k=k ---- put: batch after write is empty >> at end: -txn: "t" meta={id=00000000 key="k" pri=0.00000000 epo=0 ts=0.000000001,0 min=0,0 seq=3} lock=true stat=PENDING rts=0.000000001,0 wto=false max=0,0 -meta: "k"/0,0 -> txn={id=00000000 key="k" pri=0.00000000 epo=0 ts=0.000000001,0 min=0,0 seq=3} ts=0.000000001,0 del=false klen=12 vlen=7 ih={{2 /BYTES/v1}} -data: "k"/0.000000001,0 -> /BYTES/v2 +txn: "t" meta={id=00000000 key="k" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=3} lock=true stat=PENDING rts=1.000000000,0 wto=false max=0,0 +meta: "k"/0,0 -> txn={id=00000000 key="k" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=3} ts=1.000000000,0 del=false klen=12 vlen=7 ih={{2 /BYTES/v1}} +data: "k"/1.000000000,0 -> /BYTES/v2 error: (*withstack.withStack:) transaction 00000000-0000-0000-0000-000000000005 with sequence 3 has a different value [0 0 0 0 3 118 51] after recomputing from what was written: [0 0 0 0 3 118 50] @@ -147,8 +147,8 @@ with t=t k=k ---- put: batch after write is non-empty >> at end: -txn: "t" meta={id=00000000 key="k" pri=0.00000000 epo=0 ts=0.000000001,0 min=0,0 seq=4} lock=true stat=PENDING rts=0.000000001,0 wto=false max=0,0 -meta: "k"/0,0 -> txn={id=00000000 key="k" pri=0.00000000 epo=0 ts=0.000000001,0 min=0,0 seq=4} ts=0.000000001,0 del=false klen=12 vlen=7 ih={{2 /BYTES/v1}{3 /BYTES/v2}} -data: "k"/0.000000001,0 -> /BYTES/v4 +txn: "t" meta={id=00000000 key="k" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=4} lock=true stat=PENDING rts=1.000000000,0 wto=false max=0,0 +meta: "k"/0,0 -> txn={id=00000000 key="k" pri=0.00000000 epo=0 ts=1.000000000,0 min=0,0 seq=4} ts=1.000000000,0 del=false klen=12 vlen=7 ih={{2 /BYTES/v1}{3 /BYTES/v2}} +data: "k"/1.000000000,0 -> /BYTES/v4 # FIXME(knz): assert batching diff --git a/pkg/util/hlc/timestamp.go b/pkg/util/hlc/timestamp.go index 15cb6788d0e3..f3a85d094fb0 100644 --- a/pkg/util/hlc/timestamp.go +++ b/pkg/util/hlc/timestamp.go @@ -138,7 +138,7 @@ func (Timestamp) SafeValue() {} var ( timestampRegexp = regexp.MustCompile( - `^(?P-)?(?P\d{1,19})(?:\.(?P\d{1,20}))?,(?P-?\d{1,10})(?:\[(?P[\w,]+)\])?$`) + `^(?P-)?(?P\d{1,19})(?:\.(?P\d{1,20}))?(?:,(?P-?\d{1,10}))?(?:\[(?P[\w,]+)\])?$`) signSubexp = 1 secsSubexp = 2 nanosSubexp = 3 @@ -169,14 +169,17 @@ func ParseTimestamp(str string) (_ Timestamp, err error) { return Timestamp{}, err } } - logical, err := strconv.ParseInt(matches[logicalSubexp], 10, 32) - if err != nil { - return Timestamp{}, err - } wallTime := seconds*time.Second.Nanoseconds() + nanos if matches[signSubexp] == "-" { wallTime *= -1 } + var logical int64 + if logicalMatch := matches[logicalSubexp]; logicalMatch != "" { + logical, err = strconv.ParseInt(logicalMatch, 10, 32) + if err != nil { + return Timestamp{}, err + } + } t := Timestamp{ WallTime: wallTime, Logical: int32(logical), diff --git a/pkg/util/hlc/timestamp_test.go b/pkg/util/hlc/timestamp_test.go index 96609e0bf708..f7f795a289cc 100644 --- a/pkg/util/hlc/timestamp_test.go +++ b/pkg/util/hlc/timestamp_test.go @@ -248,7 +248,9 @@ func TestAsOfSystemTime(t *testing.T) { } } -func TestTimestampString(t *testing.T) { +// TestTimestampFormatParseRoundTrip tests the majority of timestamps that +// round-trip through formatting then parsing. +func TestTimestampFormatParseRoundTrip(t *testing.T) { testCases := []struct { ts Timestamp exp string @@ -272,14 +274,55 @@ func TestTimestampString(t *testing.T) { {makeTSWithFlags(1, 123, TimestampFlag_SYNTHETIC), "0.000000001,123[syn]"}, } for _, c := range testCases { - assert.Equal(t, c.exp, c.ts.String()) - parsed, err := ParseTimestamp(c.ts.String()) + str := c.ts.String() + assert.Equal(t, c.exp, str) + + parsed, err := ParseTimestamp(str) assert.NoError(t, err) assert.Equal(t, c.ts, parsed) } } -func TestParseTimestamp(t *testing.T) { +// TestTimestampParseFormatNonRoundTrip tests the minority of timestamps that do +// not round-trip through parsing then formatting. +func TestTimestampParseFormatNonRoundTrip(t *testing.T) { + testCases := []struct { + s string + exp Timestamp + expStr string + }{ + // Logical portion can be omitted. + {"0", makeTS(0, 0), "0,0"}, + // Fractional portion can be omitted. + {"99,0", makeTS(99000000000, 0), "99.000000000,0"}, + // Fractional and logical portion can be omitted. + {"99", makeTS(99000000000, 0), "99.000000000,0"}, + // Negatives can be omitted for portions that are 0. + {"-0", makeTS(0, 0), "0,0"}, + {"-0.0", makeTS(0, 0), "0,0"}, + {"-0.0,-0", makeTS(0, 0), "0,0"}, + {"1,-0", makeTS(1000000000, 0), "1.000000000,0"}, + {"1.1,-0", makeTS(1000000001, 0), "1.000000001,0"}, + {"-0,1", makeTS(0, 1), "0,1"}, + // Other cases. + {"0.000000001", makeTS(1, 0), "0.000000001,0"}, + {"99.000000001", makeTS(99000000001, 0), "99.000000001,0"}, + {"0[syn]", makeTSWithFlags(0, 0, TimestampFlag_SYNTHETIC), "0,0[syn]"}, + {"99[syn]", makeTSWithFlags(99000000000, 0, TimestampFlag_SYNTHETIC), "99.000000000,0[syn]"}, + {"0.000000001[syn]", makeTSWithFlags(1, 0, TimestampFlag_SYNTHETIC), "0.000000001,0[syn]"}, + {"99.000000001[syn]", makeTSWithFlags(99000000001, 0, TimestampFlag_SYNTHETIC), "99.000000001,0[syn]"}, + } + for _, c := range testCases { + parsed, err := ParseTimestamp(c.s) + assert.NoError(t, err) + assert.Equal(t, c.exp, parsed) + + str := parsed.String() + assert.Equal(t, c.expStr, str) + } +} + +func TestTimestampParseError(t *testing.T) { for _, c := range []struct { s string expErr string From 4c3d8637f9e6e647c7380f3c4d62c30f7aca4a01 Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Mon, 28 Dec 2020 16:34:16 -0500 Subject: [PATCH 4/7] kv/concurrency: use hlc.ParseTimestamp in data driven tests No need to re-write the parsing logic again. We'll also want this to use flags in the next commit. This causes a large diff because all nanos are now considered seconds. This doesn't actually change any behavior in the tests themselves. --- .../concurrency/datadriven_util_test.go | 18 +- .../kvserver/concurrency/lock_table_test.go | 17 +- .../acquire_wrong_txn_race | 4 +- .../testdata/concurrency_manager/basic | 8 +- .../clear_abandoned_intents | 60 ++-- .../testdata/concurrency_manager/deadlocks | 84 ++--- .../discover_lock_after_lease_race | 8 +- .../concurrency_manager/discovered_lock | 8 +- .../concurrency_manager/range_state_listener | 48 +-- .../testdata/concurrency_manager/uncertainty | 8 +- .../testdata/concurrency_manager/update | 24 +- .../concurrency_manager/wait_policy_error | 18 +- .../testdata/lock_table/acquire_idempotency | 28 +- .../testdata/lock_table/add_discovered | 12 +- .../lock_table/add_discovered_old_lease | 4 +- .../concurrency/testdata/lock_table/basic | 290 +++++++++--------- .../concurrency/testdata/lock_table/clear | 22 +- .../concurrency/testdata/lock_table/disable | 10 +- .../testdata/lock_table/dup_access | 172 +++++------ .../testdata/lock_table/lock_changes | 22 +- .../testdata/lock_table/lock_dropped | 20 +- .../testdata/lock_table/non_active_waiter | 60 ++-- .../testdata/lock_table/non_txn_write | 70 ++--- .../testdata/lock_table/size_limit_exceeded | 58 ++-- .../concurrency/testdata/lock_table/update | 46 +-- .../concurrency/testdata/lock_table/wait_self | 12 +- 26 files changed, 550 insertions(+), 581 deletions(-) diff --git a/pkg/kv/kvserver/concurrency/datadriven_util_test.go b/pkg/kv/kvserver/concurrency/datadriven_util_test.go index a3d39c7c56ce..cd713b1dea20 100644 --- a/pkg/kv/kvserver/concurrency/datadriven_util_test.go +++ b/pkg/kv/kvserver/concurrency/datadriven_util_test.go @@ -12,7 +12,6 @@ package concurrency_test import ( "strconv" - "strings" "testing" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" @@ -35,27 +34,12 @@ func scanTimestamp(t *testing.T, d *datadriven.TestData) hlc.Timestamp { } func scanTimestampWithName(t *testing.T, d *datadriven.TestData, name string) hlc.Timestamp { - var ts hlc.Timestamp var tsS string d.ScanArgs(t, name, &tsS) - parts := strings.Split(tsS, ",") - - // Find the wall time part. - tsW, err := strconv.ParseInt(parts[0], 10, 64) + ts, err := hlc.ParseTimestamp(tsS) if err != nil { d.Fatalf(t, "%v", err) } - ts.WallTime = tsW - - // Find the logical part, if there is one. - var tsL int64 - if len(parts) > 1 { - tsL, err = strconv.ParseInt(parts[1], 10, 32) - if err != nil { - d.Fatalf(t, "%v", err) - } - } - ts.Logical = int32(tsL) return ts } diff --git a/pkg/kv/kvserver/concurrency/lock_table_test.go b/pkg/kv/kvserver/concurrency/lock_table_test.go index 87757d2efa59..9b7a9127deaa 100644 --- a/pkg/kv/kvserver/concurrency/lock_table_test.go +++ b/pkg/kv/kvserver/concurrency/lock_table_test.go @@ -436,27 +436,12 @@ func nextUUID(counter *uint128.Uint128) uuid.UUID { } func scanTimestamp(t *testing.T, d *datadriven.TestData) hlc.Timestamp { - var ts hlc.Timestamp var tsS string d.ScanArgs(t, "ts", &tsS) - parts := strings.Split(tsS, ",") - - // Find the wall time part. - tsW, err := strconv.ParseInt(parts[0], 10, 64) + ts, err := hlc.ParseTimestamp(tsS) if err != nil { d.Fatalf(t, "%v", err) } - ts.WallTime = tsW - - // Find the logical part, if there is one. - var tsL int64 - if len(parts) > 1 { - tsL, err = strconv.ParseInt(parts[1], 10, 32) - if err != nil { - d.Fatalf(t, "%v", err) - } - } - ts.Logical = int32(tsL) return ts } diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/acquire_wrong_txn_race b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/acquire_wrong_txn_race index 5301b7615da2..a761a63c0545 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/acquire_wrong_txn_race +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/acquire_wrong_txn_race @@ -72,7 +72,7 @@ debug-lock-table ---- global: num=1 lock: "k" - holder: txn: 00000001-0000-0000-0000-000000000000, ts: 0.000000015,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 15.000000000,0, info: unrepl epoch: 0, seqs: [0] local: num=0 on-split @@ -160,7 +160,7 @@ debug-lock-table ---- global: num=1 lock: "k" - holder: txn: 00000003-0000-0000-0000-000000000000, ts: 0.000000025,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000003-0000-0000-0000-000000000000, ts: 25.000000000,0, info: unrepl epoch: 0, seqs: [0] local: num=0 reset diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/basic b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/basic index 63a01c1e82ff..083adfa9d6c9 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/basic +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/basic @@ -58,7 +58,7 @@ debug-lock-table ---- global: num=1 lock: "k" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 0.000000012,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 12.000000000,1, info: unrepl epoch: 0, seqs: [0] local: num=0 finish req=req2 @@ -69,7 +69,7 @@ debug-lock-table ---- global: num=1 lock: "k" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 0.000000012,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 12.000000000,1, info: unrepl epoch: 0, seqs: [0] local: num=0 reset @@ -116,7 +116,7 @@ sequence req=req3 [2] sequence req3: acquiring latches [2] sequence req3: scanning lock table for conflicting locks [2] sequence req3: waiting in lock wait-queues -[2] sequence req3: pushing timestamp of txn 00000002 above 0.000000014,1 +[2] sequence req3: pushing timestamp of txn 00000002 above 14.000000000,1 [2] sequence req3: blocked on select in concurrency_test.(*cluster).PushTransaction on-txn-updated txn=txn2 status=committed @@ -196,7 +196,7 @@ sequence req=req5 [2] sequence req5: acquiring latches [2] sequence req5: scanning lock table for conflicting locks [2] sequence req5: waiting in lock wait-queues -[2] sequence req5: pushing timestamp of txn 00000002 above 0.000000014,1 +[2] sequence req5: pushing timestamp of txn 00000002 above 14.000000000,1 [2] sequence req5: blocked on select in concurrency_test.(*cluster).PushTransaction new-request name=req6 txn=none ts=16,1 diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/clear_abandoned_intents b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/clear_abandoned_intents index 4631542a93f0..cc08def4560d 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/clear_abandoned_intents +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/clear_abandoned_intents @@ -37,25 +37,25 @@ debug-lock-table ---- global: num=10 lock: "a" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 0.000000010,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] lock: "b" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 0.000000010,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] lock: "c" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 0.000000010,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] lock: "d" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 0.000000010,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] lock: "e" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 0.000000010,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] lock: "f" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 0.000000010,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] lock: "g" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 0.000000010,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] lock: "h" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 0.000000010,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] lock: "i" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 0.000000010,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] lock: "j" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 0.000000010,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] local: num=0 sequence req=req1 @@ -64,35 +64,35 @@ sequence req=req1 [3] sequence req1: acquiring latches [3] sequence req1: scanning lock table for conflicting locks [3] sequence req1: waiting in lock wait-queues -[3] sequence req1: pushing timestamp of txn 00000002 above 0.000000010,1 +[3] sequence req1: pushing timestamp of txn 00000002 above 10.000000000,1 [3] sequence req1: blocked on select in concurrency_test.(*cluster).PushTransaction debug-lock-table ---- global: num=10 lock: "a" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 0.000000010,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] waiting readers: req: 1, txn: 00000001-0000-0000-0000-000000000000 distinguished req: 1 lock: "b" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 0.000000010,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] lock: "c" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 0.000000010,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] lock: "d" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 0.000000010,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] lock: "e" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 0.000000010,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] lock: "f" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 0.000000010,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] lock: "g" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 0.000000010,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] lock: "h" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 0.000000010,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] lock: "i" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 0.000000010,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] lock: "j" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 0.000000010,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] local: num=0 # txn1 is the distinguished waiter on key "a". It will push txn2, notice that it @@ -160,7 +160,7 @@ debug-lock-table ---- global: num=1 lock: "a" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 0.000000010,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] queued writers: active: false req: 2, txn: 00000001-0000-0000-0000-000000000000 local: num=0 @@ -191,9 +191,9 @@ debug-lock-table ---- global: num=2 lock: "a" - res: req: 2, txn: 00000001-0000-0000-0000-000000000000, ts: 0.000000010,1, seq: 0 + res: req: 2, txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, seq: 0 lock: "b" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 0.000000010,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] queued writers: active: false req: 2, txn: 00000001-0000-0000-0000-000000000000 local: num=0 @@ -219,11 +219,11 @@ debug-lock-table ---- global: num=3 lock: "a" - res: req: 2, txn: 00000001-0000-0000-0000-000000000000, ts: 0.000000010,1, seq: 0 + res: req: 2, txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, seq: 0 lock: "b" - res: req: 2, txn: 00000001-0000-0000-0000-000000000000, ts: 0.000000010,1, seq: 0 + res: req: 2, txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, seq: 0 lock: "c" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 0.000000010,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] queued writers: active: false req: 2, txn: 00000001-0000-0000-0000-000000000000 local: num=0 @@ -244,11 +244,11 @@ debug-lock-table ---- global: num=3 lock: "a" - res: req: 2, txn: 00000001-0000-0000-0000-000000000000, ts: 0.000000010,1, seq: 0 + res: req: 2, txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, seq: 0 lock: "b" - res: req: 2, txn: 00000001-0000-0000-0000-000000000000, ts: 0.000000010,1, seq: 0 + res: req: 2, txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, seq: 0 lock: "c" - res: req: 2, txn: 00000001-0000-0000-0000-000000000000, ts: 0.000000010,1, seq: 0 + res: req: 2, txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, seq: 0 local: num=0 finish req=req1 diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/deadlocks b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/deadlocks index 32740c5cc87e..3bec6cb7831e 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/deadlocks +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/deadlocks @@ -83,11 +83,11 @@ debug-lock-table ---- global: num=3 lock: "a" - holder: txn: 00000001-0000-0000-0000-000000000000, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "b" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "c" - holder: txn: 00000003-0000-0000-0000-000000000000, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] local: num=0 # -------------------------------- @@ -112,7 +112,7 @@ sequence req=req1r [4] sequence req1r: acquiring latches [4] sequence req1r: scanning lock table for conflicting locks [4] sequence req1r: waiting in lock wait-queues -[4] sequence req1r: pushing timestamp of txn 00000002 above 0.000000010,1 +[4] sequence req1r: pushing timestamp of txn 00000002 above 10.000000000,1 [4] sequence req1r: blocked on select in concurrency_test.(*cluster).PushTransaction sequence req=req2r @@ -121,7 +121,7 @@ sequence req=req2r [5] sequence req2r: acquiring latches [5] sequence req2r: scanning lock table for conflicting locks [5] sequence req2r: waiting in lock wait-queues -[5] sequence req2r: pushing timestamp of txn 00000003 above 0.000000010,1 +[5] sequence req2r: pushing timestamp of txn 00000003 above 10.000000000,1 [5] sequence req2r: blocked on select in concurrency_test.(*cluster).PushTransaction sequence req=req3r @@ -132,7 +132,7 @@ sequence req=req3r [6] sequence req3r: acquiring latches [6] sequence req3r: scanning lock table for conflicting locks [6] sequence req3r: waiting in lock wait-queues -[6] sequence req3r: pushing timestamp of txn 00000001 above 0.000000010,1 +[6] sequence req3r: pushing timestamp of txn 00000001 above 10.000000000,1 [6] sequence req3r: blocked on select in concurrency_test.(*cluster).PushTransaction [6] sequence req3r: dependency cycle detected 00000003->00000001->00000002->00000003 @@ -140,17 +140,17 @@ debug-lock-table ---- global: num=3 lock: "a" - holder: txn: 00000001-0000-0000-0000-000000000000, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] waiting readers: req: 6, txn: 00000003-0000-0000-0000-000000000000 distinguished req: 6 lock: "b" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] waiting readers: req: 4, txn: 00000001-0000-0000-0000-000000000000 distinguished req: 4 lock: "c" - holder: txn: 00000003-0000-0000-0000-000000000000, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] waiting readers: req: 5, txn: 00000002-0000-0000-0000-000000000000 distinguished req: 5 @@ -284,11 +284,11 @@ debug-lock-table ---- global: num=3 lock: "a" - holder: txn: 00000001-0000-0000-0000-000000000000, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "b" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "c" - holder: txn: 00000003-0000-0000-0000-000000000000, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] local: num=0 # -------------------------------- @@ -354,18 +354,18 @@ debug-lock-table ---- global: num=3 lock: "a" - holder: txn: 00000001-0000-0000-0000-000000000000, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 10, txn: 00000004-0000-0000-0000-000000000000 active: true req: 13, txn: 00000003-0000-0000-0000-000000000000 distinguished req: 10 lock: "b" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 11, txn: 00000001-0000-0000-0000-000000000000 distinguished req: 11 lock: "c" - holder: txn: 00000003-0000-0000-0000-000000000000, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 12, txn: 00000002-0000-0000-0000-000000000000 distinguished req: 12 @@ -512,11 +512,11 @@ debug-lock-table ---- global: num=3 lock: "a" - holder: txn: 00000001-0000-0000-0000-000000000000, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "b" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "c" - holder: txn: 00000003-0000-0000-0000-000000000000, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] local: num=0 new-request name=req4w txn=txn4 ts=10,1 @@ -544,11 +544,11 @@ debug-lock-table ---- global: num=3 lock: "a" - holder: txn: 00000001-0000-0000-0000-000000000000, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "b" - res: req: 17, txn: 00000004-0000-0000-0000-000000000000, ts: 0.000000010,1, seq: 0 + res: req: 17, txn: 00000004-0000-0000-0000-000000000000, ts: 10.000000000,1, seq: 0 lock: "c" - holder: txn: 00000003-0000-0000-0000-000000000000, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 17, txn: 00000004-0000-0000-0000-000000000000 distinguished req: 17 @@ -591,17 +591,17 @@ debug-lock-table ---- global: num=3 lock: "a" - holder: txn: 00000001-0000-0000-0000-000000000000, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 19, txn: 00000003-0000-0000-0000-000000000000 distinguished req: 19 lock: "b" - res: req: 17, txn: 00000004-0000-0000-0000-000000000000, ts: 0.000000010,1, seq: 0 + res: req: 17, txn: 00000004-0000-0000-0000-000000000000, ts: 10.000000000,1, seq: 0 queued writers: active: true req: 18, txn: 00000001-0000-0000-0000-000000000000 distinguished req: 18 lock: "c" - holder: txn: 00000003-0000-0000-0000-000000000000, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 17, txn: 00000004-0000-0000-0000-000000000000 distinguished req: 17 @@ -736,11 +736,11 @@ debug-lock-table ---- global: num=3 lock: "a" - holder: txn: 00000001-0000-0000-0000-000000000000, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "b" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "c" - holder: txn: 00000003-0000-0000-0000-000000000000, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] local: num=0 new-request name=req4w txn=txn4 ts=10,1 @@ -768,11 +768,11 @@ debug-lock-table ---- global: num=3 lock: "a" - holder: txn: 00000001-0000-0000-0000-000000000000, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "b" - res: req: 23, txn: 00000004-0000-0000-0000-000000000000, ts: 0.000000010,1, seq: 0 + res: req: 23, txn: 00000004-0000-0000-0000-000000000000, ts: 10.000000000,1, seq: 0 lock: "c" - holder: txn: 00000003-0000-0000-0000-000000000000, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 23, txn: 00000004-0000-0000-0000-000000000000 distinguished req: 23 @@ -815,17 +815,17 @@ debug-lock-table ---- global: num=3 lock: "a" - holder: txn: 00000001-0000-0000-0000-000000000000, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 25, txn: 00000003-0000-0000-0000-000000000000 distinguished req: 25 lock: "b" - res: req: 23, txn: 00000004-0000-0000-0000-000000000000, ts: 0.000000010,1, seq: 0 + res: req: 23, txn: 00000004-0000-0000-0000-000000000000, ts: 10.000000000,1, seq: 0 queued writers: active: true req: 24, txn: 00000001-0000-0000-0000-000000000000 distinguished req: 24 lock: "c" - holder: txn: 00000003-0000-0000-0000-000000000000, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 23, txn: 00000004-0000-0000-0000-000000000000 distinguished req: 23 @@ -967,11 +967,11 @@ debug-lock-table ---- global: num=3 lock: "a" - holder: txn: 00000001-0000-0000-0000-000000000000, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "b" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "c" - holder: txn: 00000003-0000-0000-0000-000000000000, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] local: num=0 new-request name=req4w txn=txn4 ts=10,1 @@ -1023,14 +1023,14 @@ debug-lock-table ---- global: num=3 lock: "a" - res: req: 30, txn: 00000004-0000-0000-0000-000000000000, ts: 0.000000010,1, seq: 0 + res: req: 30, txn: 00000004-0000-0000-0000-000000000000, ts: 10.000000000,1, seq: 0 lock: "b" - res: req: 29, txn: 00000005-0000-0000-0000-000000000000, ts: 0.000000010,1, seq: 0 + res: req: 29, txn: 00000005-0000-0000-0000-000000000000, ts: 10.000000000,1, seq: 0 queued writers: active: true req: 30, txn: 00000004-0000-0000-0000-000000000000 distinguished req: 30 lock: "c" - holder: txn: 00000003-0000-0000-0000-000000000000, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 29, txn: 00000005-0000-0000-0000-000000000000 distinguished req: 29 @@ -1060,17 +1060,17 @@ debug-lock-table ---- global: num=3 lock: "a" - res: req: 30, txn: 00000004-0000-0000-0000-000000000000, ts: 0.000000010,1, seq: 0 + res: req: 30, txn: 00000004-0000-0000-0000-000000000000, ts: 10.000000000,1, seq: 0 queued writers: active: true req: 31, txn: 00000003-0000-0000-0000-000000000000 distinguished req: 31 lock: "b" - res: req: 29, txn: 00000005-0000-0000-0000-000000000000, ts: 0.000000010,1, seq: 0 + res: req: 29, txn: 00000005-0000-0000-0000-000000000000, ts: 10.000000000,1, seq: 0 queued writers: active: true req: 30, txn: 00000004-0000-0000-0000-000000000000 distinguished req: 30 lock: "c" - holder: txn: 00000003-0000-0000-0000-000000000000, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 29, txn: 00000005-0000-0000-0000-000000000000 distinguished req: 29 diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/discover_lock_after_lease_race b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/discover_lock_after_lease_race index 0640fc79872b..79b552e44052 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/discover_lock_after_lease_race +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/discover_lock_after_lease_race @@ -140,7 +140,7 @@ debug-lock-table ---- global: num=1 lock: "k" - holder: txn: 00000003-0000-0000-0000-000000000000, ts: 0.000000010,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] local: num=0 sequence req=req4 @@ -149,7 +149,7 @@ sequence req=req4 [5] sequence req4: acquiring latches [5] sequence req4: scanning lock table for conflicting locks [5] sequence req4: waiting in lock wait-queues -[5] sequence req4: pushing timestamp of txn 00000003 above 0.000000010,0 +[5] sequence req4: pushing timestamp of txn 00000003 above 10.000000000,0 [5] sequence req4: blocked on select in concurrency_test.(*cluster).PushTransaction handle-write-intent-error req=req2 lease-seq=1 @@ -161,7 +161,7 @@ debug-lock-table ---- global: num=1 lock: "k" - holder: txn: 00000003-0000-0000-0000-000000000000, ts: 0.000000010,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] waiting readers: req: 3, txn: 00000004-0000-0000-0000-000000000000 distinguished req: 3 @@ -173,7 +173,7 @@ sequence req=req2 [7] sequence req2: acquiring latches [7] sequence req2: scanning lock table for conflicting locks [7] sequence req2: waiting in lock wait-queues -[7] sequence req2: pushing timestamp of txn 00000003 above 0.000000010,0 +[7] sequence req2: pushing timestamp of txn 00000003 above 10.000000000,0 [7] sequence req2: blocked on select in concurrency_test.(*cluster).PushTransaction on-txn-updated txn=txn3 status=committed diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/discovered_lock b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/discovered_lock index 75e4496961c4..d2505161371d 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/discovered_lock +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/discovered_lock @@ -29,7 +29,7 @@ debug-lock-table ---- global: num=1 lock: "k" - holder: txn: 00000001-0000-0000-0000-000000000000, ts: 0.000000010,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] local: num=0 sequence req=req1 @@ -38,7 +38,7 @@ sequence req=req1 [3] sequence req1: acquiring latches [3] sequence req1: scanning lock table for conflicting locks [3] sequence req1: waiting in lock wait-queues -[3] sequence req1: pushing timestamp of txn 00000001 above 0.000000012,1 +[3] sequence req1: pushing timestamp of txn 00000001 above 12.000000000,1 [3] sequence req1: blocked on select in concurrency_test.(*cluster).PushTransaction on-txn-updated txn=txn1 status=aborted @@ -86,7 +86,7 @@ sequence req=req1 handle-write-intent-error req=req1 lease-seq=2 intent txn=txn1 key=k ---- -[2] handle write intent error req1: pushing timestamp of txn 00000001 above 0.000000012,1 +[2] handle write intent error req1: pushing timestamp of txn 00000001 above 12.000000000,1 [2] handle write intent error req1: blocked on select in concurrency_test.(*cluster).PushTransaction on-txn-updated txn=txn1 status=aborted @@ -203,7 +203,7 @@ sequence req=req1 handle-write-intent-error req=req1 lease-seq=2 intent txn=txn1 key=k ---- -[2] handle write intent error req1: pushing timestamp of txn 00000001 above 0.000000012,1 +[2] handle write intent error req1: pushing timestamp of txn 00000001 above 12.000000000,1 [2] handle write intent error req1: blocked on select in concurrency_test.(*cluster).PushTransaction on-txn-updated txn=txn2 status=aborted diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/range_state_listener b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/range_state_listener index 41878eb3c2b5..78d24ff6b059 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/range_state_listener +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/range_state_listener @@ -79,9 +79,9 @@ debug-lock-table ---- global: num=2 lock: "k" - holder: txn: 00000001-0000-0000-0000-000000000000, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "k2" - holder: txn: 00000001-0000-0000-0000-000000000000, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] local: num=0 # -------------------------------- @@ -100,12 +100,12 @@ debug-lock-table ---- global: num=2 lock: "k" - holder: txn: 00000001-0000-0000-0000-000000000000, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 2, txn: 00000002-0000-0000-0000-000000000000 distinguished req: 2 lock: "k2" - holder: txn: 00000001-0000-0000-0000-000000000000, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] local: num=0 # Replica loses lease. @@ -148,7 +148,7 @@ sequence req=req2 handle-write-intent-error req=req2 lease-seq=2 intent txn=txn1 key=k2 ---- -[5] handle write intent error req2: pushing timestamp of txn 00000001 above 0.000000010,1 +[5] handle write intent error req2: pushing timestamp of txn 00000001 above 10.000000000,1 [5] handle write intent error req2: resolving intent "k2" for txn 00000001 with COMMITTED status [5] handle write intent error req2: handled conflicting intents on "k2", released latches @@ -224,7 +224,7 @@ debug-lock-table ---- global: num=1 lock: "k" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 0.000000010,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] queued writers: active: false req: 3, txn: 00000003-0000-0000-0000-000000000000 local: num=0 @@ -241,7 +241,7 @@ debug-lock-table ---- global: num=1 lock: "k" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 0.000000010,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] queued writers: active: true req: 3, txn: 00000003-0000-0000-0000-000000000000 distinguished req: 3 @@ -273,7 +273,7 @@ debug-lock-table ---- global: num=1 lock: "k" - res: req: 3, txn: 00000003-0000-0000-0000-000000000000, ts: 0.000000010,1, seq: 0 + res: req: 3, txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, seq: 0 local: num=0 on-lock-acquired req=req3 key=k @@ -284,7 +284,7 @@ debug-lock-table ---- global: num=1 lock: "k" - holder: txn: 00000003-0000-0000-0000-000000000000, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] local: num=0 finish req=req3 @@ -346,7 +346,7 @@ debug-lock-table ---- global: num=1 lock: "k" - holder: txn: 00000001-0000-0000-0000-000000000000, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] local: num=0 # -------------------------------- @@ -365,7 +365,7 @@ debug-lock-table ---- global: num=1 lock: "k" - holder: txn: 00000001-0000-0000-0000-000000000000, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 5, txn: 00000002-0000-0000-0000-000000000000 distinguished req: 5 @@ -392,7 +392,7 @@ debug-lock-table ---- global: num=1 lock: "k" - holder: txn: 00000001-0000-0000-0000-000000000000, ts: 0.000000010,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] queued writers: active: false req: 5, txn: 00000002-0000-0000-0000-000000000000 local: num=0 @@ -431,7 +431,7 @@ debug-lock-table ---- global: num=1 lock: "k" - res: req: 5, txn: 00000002-0000-0000-0000-000000000000, ts: 0.000000010,1, seq: 0 + res: req: 5, txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, seq: 0 local: num=0 on-lock-acquired req=req2 key=k @@ -442,7 +442,7 @@ debug-lock-table ---- global: num=1 lock: "k" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] local: num=0 finish req=req2 @@ -512,9 +512,9 @@ debug-lock-table ---- global: num=2 lock: "k" - holder: txn: 00000001-0000-0000-0000-000000000000, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "k2" - holder: txn: 00000001-0000-0000-0000-000000000000, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] local: num=0 # -------------------------------- @@ -533,12 +533,12 @@ debug-lock-table ---- global: num=2 lock: "k" - holder: txn: 00000001-0000-0000-0000-000000000000, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 7, txn: 00000002-0000-0000-0000-000000000000 distinguished req: 7 lock: "k2" - holder: txn: 00000001-0000-0000-0000-000000000000, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] local: num=0 on-merge @@ -580,7 +580,7 @@ sequence req=req2 handle-write-intent-error req=req2 lease-seq=1 intent txn=txn1 key=k2 ---- -[5] handle write intent error req2: pushing timestamp of txn 00000001 above 0.000000010,1 +[5] handle write intent error req2: pushing timestamp of txn 00000001 above 10.000000000,1 [5] handle write intent error req2: resolving intent "k2" for txn 00000001 with COMMITTED status [5] handle write intent error req2: handled conflicting intents on "k2", released latches @@ -664,7 +664,7 @@ debug-lock-table ---- global: num=1 lock: "k" - holder: txn: 00000001-0000-0000-0000-000000000000, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] local: num=0 # -------------------------------- @@ -683,7 +683,7 @@ debug-lock-table ---- global: num=1 lock: "k" - holder: txn: 00000001-0000-0000-0000-000000000000, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 9, txn: 00000002-0000-0000-0000-000000000000 distinguished req: 9 @@ -710,7 +710,7 @@ debug-lock-table ---- global: num=1 lock: "k" - holder: txn: 00000001-0000-0000-0000-000000000000, ts: 0.000000010,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] queued writers: active: false req: 9, txn: 00000002-0000-0000-0000-000000000000 local: num=0 @@ -749,7 +749,7 @@ debug-lock-table ---- global: num=1 lock: "k" - res: req: 9, txn: 00000002-0000-0000-0000-000000000000, ts: 0.000000010,1, seq: 0 + res: req: 9, txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, seq: 0 local: num=0 on-lock-acquired req=req2 key=k @@ -760,7 +760,7 @@ debug-lock-table ---- global: num=1 lock: "k" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] local: num=0 finish req=req2 diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/uncertainty b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/uncertainty index f720f3604567..39edf98ed602 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/uncertainty +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/uncertainty @@ -31,7 +31,7 @@ debug-lock-table ---- global: num=1 lock: "k" - holder: txn: 00000001-0000-0000-0000-000000000000, ts: 0.000000010,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] local: num=0 sequence req=req1 @@ -40,7 +40,7 @@ sequence req=req1 [3] sequence req1: acquiring latches [3] sequence req1: scanning lock table for conflicting locks [3] sequence req1: waiting in lock wait-queues -[3] sequence req1: pushing timestamp of txn 00000001 above 0.000000015,1 +[3] sequence req1: pushing timestamp of txn 00000001 above 15.000000000,1 [3] sequence req1: blocked on select in concurrency_test.(*cluster).PushTransaction on-txn-updated txn=txn1 status=pending ts=15,2 @@ -92,7 +92,7 @@ debug-lock-table ---- global: num=1 lock: "k" - holder: txn: 00000001-0000-0000-0000-000000000000, ts: 0.000000014,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 14.000000000,1, info: repl epoch: 0, seqs: [0] local: num=0 sequence req=req1 @@ -101,7 +101,7 @@ sequence req=req1 [3] sequence req1: acquiring latches [3] sequence req1: scanning lock table for conflicting locks [3] sequence req1: waiting in lock wait-queues -[3] sequence req1: pushing timestamp of txn 00000001 above 0.000000015,1 +[3] sequence req1: pushing timestamp of txn 00000001 above 15.000000000,1 [3] sequence req1: blocked on select in concurrency_test.(*cluster).PushTransaction on-txn-updated txn=txn1 status=pending ts=15,2 diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/update b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/update index 2ee445ae9ee3..75879afad3cd 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/update +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/update @@ -49,14 +49,14 @@ sequence req=req2 [2] sequence req2: acquiring latches [2] sequence req2: scanning lock table for conflicting locks [2] sequence req2: waiting in lock wait-queues -[2] sequence req2: pushing timestamp of txn 00000001 above 0.000000012,1 +[2] sequence req2: pushing timestamp of txn 00000001 above 12.000000000,1 [2] sequence req2: blocked on select in concurrency_test.(*cluster).PushTransaction debug-lock-table ---- global: num=1 lock: "k" - holder: txn: 00000001-0000-0000-0000-000000000000, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] waiting readers: req: 2, txn: 00000002-0000-0000-0000-000000000000 distinguished req: 2 @@ -82,7 +82,7 @@ debug-lock-table ---- global: num=1 lock: "k" - holder: txn: 00000001-0000-0000-0000-000000000000, ts: 0.000000012,2, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 12.000000000,2, info: unrepl epoch: 0, seqs: [0] local: num=0 # Issue another write to the same key for txn1 at its initial @@ -111,7 +111,7 @@ debug-lock-table ---- global: num=1 lock: "k" - holder: txn: 00000001-0000-0000-0000-000000000000, ts: 0.000000012,2, info: unrepl epoch: 0, seqs: [0, 1] + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 12.000000000,2, info: unrepl epoch: 0, seqs: [0, 1] local: num=0 reset namespace @@ -167,14 +167,14 @@ sequence req=req2 [2] sequence req2: acquiring latches [2] sequence req2: scanning lock table for conflicting locks [2] sequence req2: waiting in lock wait-queues -[2] sequence req2: pushing timestamp of txn 00000001 above 0.000000012,1 +[2] sequence req2: pushing timestamp of txn 00000001 above 12.000000000,1 [2] sequence req2: blocked on select in concurrency_test.(*cluster).PushTransaction debug-lock-table ---- global: num=1 lock: "k" - holder: txn: 00000001-0000-0000-0000-000000000000, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] waiting readers: req: 5, txn: 00000002-0000-0000-0000-000000000000 distinguished req: 5 @@ -200,7 +200,7 @@ debug-lock-table ---- global: num=1 lock: "k" - holder: txn: 00000001-0000-0000-0000-000000000000, ts: 0.000000012,2, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 12.000000000,2, info: unrepl epoch: 0, seqs: [0] local: num=0 # The txn restarts at a new timestamp, but below the pushed @@ -233,7 +233,7 @@ debug-lock-table ---- global: num=1 lock: "k" - holder: txn: 00000001-0000-0000-0000-000000000000, ts: 0.000000012,2, info: unrepl epoch: 1, seqs: [0] + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 12.000000000,2, info: unrepl epoch: 1, seqs: [0] local: num=0 reset namespace @@ -294,14 +294,14 @@ sequence req=req2 [2] sequence req2: acquiring latches [2] sequence req2: scanning lock table for conflicting locks [2] sequence req2: waiting in lock wait-queues -[2] sequence req2: pushing timestamp of txn 00000001 above 0.000000012,1 +[2] sequence req2: pushing timestamp of txn 00000001 above 12.000000000,1 [2] sequence req2: blocked on select in concurrency_test.(*cluster).PushTransaction debug-lock-table ---- global: num=1 lock: "k" - holder: txn: 00000001-0000-0000-0000-000000000000, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] waiting readers: req: 8, txn: 00000002-0000-0000-0000-000000000000 distinguished req: 8 @@ -327,7 +327,7 @@ debug-lock-table ---- global: num=1 lock: "k" - holder: txn: 00000001-0000-0000-0000-000000000000, ts: 0.000000012,2, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 12.000000000,2, info: unrepl epoch: 0, seqs: [0] local: num=0 # Issue another write to the same key for txn1 at its initial timestamp, @@ -375,7 +375,7 @@ debug-lock-table ---- global: num=1 lock: "k" - holder: txn: 00000001-0000-0000-0000-000000000000, ts: 0.000000010,1, info: repl epoch: 0, seqs: [1], unrepl epoch: 0, seqs: [0] + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [1], unrepl epoch: 0, seqs: [0] queued writers: active: true req: 9, txn: none distinguished req: 9 diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_policy_error b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_policy_error index 60e9fdec291a..1a174ac1ad1f 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_policy_error +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_policy_error @@ -77,14 +77,14 @@ debug-lock-table ---- global: num=3 lock: "k" - holder: txn: 00000001-0000-0000-0000-000000000000, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "k2" - holder: txn: 00000001-0000-0000-0000-000000000000, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 3, txn: 00000003-0000-0000-0000-000000000000 distinguished req: 3 lock: "k3" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 0.000000011,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 11.000000000,0, info: unrepl epoch: 0, seqs: [0] local: num=0 # ------------------------------------------------------------- @@ -138,9 +138,9 @@ debug-lock-table ---- global: num=2 lock: "k2" - res: req: 3, txn: 00000003-0000-0000-0000-000000000000, ts: 0.000000011,0, seq: 0 + res: req: 3, txn: 00000003-0000-0000-0000-000000000000, ts: 11.000000000,0, seq: 0 lock: "k3" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 0.000000011,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 11.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 3, txn: 00000003-0000-0000-0000-000000000000 distinguished req: 3 @@ -199,14 +199,14 @@ debug-lock-table ---- global: num=3 lock: "k2" - res: req: 3, txn: 00000003-0000-0000-0000-000000000000, ts: 0.000000011,0, seq: 0 + res: req: 3, txn: 00000003-0000-0000-0000-000000000000, ts: 11.000000000,0, seq: 0 lock: "k3" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 0.000000011,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 11.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 3, txn: 00000003-0000-0000-0000-000000000000 distinguished req: 3 lock: "k4" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 0.000000011,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] local: num=0 # ------------------------------------------------------------- @@ -262,7 +262,7 @@ debug-lock-table ---- global: num=1 lock: "k4" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 0.000000011,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] local: num=0 reset diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/acquire_idempotency b/pkg/kv/kvserver/concurrency/testdata/lock_table/acquire_idempotency index 108d442fe744..9d67c43308eb 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/acquire_idempotency +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/acquire_idempotency @@ -25,14 +25,14 @@ acquire r=req1 k=a durability=u ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [1] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [1] local: num=0 dequeue r=req1 ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [1] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [1] local: num=0 new-txn txn=txn1 ts=10,1 epoch=0 seq=2 @@ -49,14 +49,14 @@ acquire r=req2 k=a durability=u ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [1, 2] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [1, 2] local: num=0 dequeue r=req2 ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [1, 2] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [1, 2] local: num=0 new-txn txn=txn1 ts=10,1 epoch=0 seq=4 @@ -73,14 +73,14 @@ acquire r=req3 k=a durability=u ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [1, 2, 4] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [1, 2, 4] local: num=0 dequeue r=req3 ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [1, 2, 4] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [1, 2, 4] local: num=0 # ------------------------------------------------------------- @@ -101,14 +101,14 @@ acquire r=req3 k=a durability=u ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [1, 2, 4] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [1, 2, 4] local: num=0 dequeue r=req3 ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [1, 2, 4] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [1, 2, 4] local: num=0 # ------------------------------------------------------------- @@ -129,14 +129,14 @@ acquire r=req4 k=a durability=u ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [1, 2, 4] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [1, 2, 4] local: num=0 dequeue r=req4 ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [1, 2, 4] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [1, 2, 4] local: num=0 # ------------------------------------------------------------- @@ -159,14 +159,14 @@ acquire r=req5 k=a durability=u ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [1, 2, 3, 4] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [1, 2, 3, 4] local: num=0 dequeue r=req5 ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [1, 2, 3, 4] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [1, 2, 3, 4] local: num=0 # ------------------------------------------------------------- @@ -187,12 +187,12 @@ acquire r=req6 k=a durability=u ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [1, 2, 3, 4, 5] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [1, 2, 3, 4, 5] local: num=0 dequeue r=req6 ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [1, 2, 3, 4, 5] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [1, 2, 3, 4, 5] local: num=0 diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/add_discovered b/pkg/kv/kvserver/concurrency/testdata/lock_table/add_discovered index fed77918440d..f80389d8b475 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/add_discovered +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/add_discovered @@ -42,14 +42,14 @@ acquire r=req1 k=a durability=u ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] local: num=0 dequeue r=req1 ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] local: num=0 new-request r=req2 txn=txn2 ts=10,1 spans=w@a @@ -70,7 +70,7 @@ release txn=txn1 span=a ---- global: num=1 lock: "a" - res: req: 2, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,1, seq: 0 + res: req: 2, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, seq: 0 queued writers: active: true req: 3, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 3 @@ -92,7 +92,7 @@ add-discovered r=req2 k=a txn=txn3 ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000010,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] queued writers: active: false req: 2, txn: 00000000-0000-0000-0000-000000000002 local: num=0 @@ -113,7 +113,7 @@ dequeue r=req3 ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000010,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] queued writers: active: true req: 2, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 2 @@ -123,7 +123,7 @@ release txn=txn3 span=a ---- global: num=1 lock: "a" - res: req: 2, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,1, seq: 0 + res: req: 2, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, seq: 0 local: num=0 guard-state r=req2 diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/add_discovered_old_lease b/pkg/kv/kvserver/concurrency/testdata/lock_table/add_discovered_old_lease index 17ce786408f6..441408872c81 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/add_discovered_old_lease +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/add_discovered_old_lease @@ -31,7 +31,7 @@ add-discovered r=req1 k=b txn=txn2 lease-seq=5 ---- global: num=1 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] local: num=0 add-discovered r=req1 k=c txn=txn2 lease-seq=6 @@ -42,5 +42,5 @@ print ---- global: num=1 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] local: num=0 diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/basic b/pkg/kv/kvserver/concurrency/testdata/lock_table/basic index fe683c871ecf..0fb19d5298fd 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/basic +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/basic @@ -28,25 +28,25 @@ acquire r=req1 k=c durability=u ---- global: num=1 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] local: num=0 acquire r=req1 k=e durability=u ---- global: num=2 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "e" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] local: num=0 dequeue r=req1 ---- global: num=2 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "e" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] local: num=0 # req2 is also for txn1 and will not wait for locks that are held by self. @@ -62,22 +62,22 @@ acquire r=req2 k=b durability=u ---- global: num=3 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,2, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,2, info: unrepl epoch: 0, seqs: [0] lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "e" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] local: num=0 dequeue r=req2 ---- global: num=3 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,2, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,2, info: unrepl epoch: 0, seqs: [0] lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "e" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] local: num=0 # txn1 holds locks on b, c, e. @@ -97,11 +97,11 @@ dequeue r=req3 ---- global: num=3 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,2, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,2, info: unrepl epoch: 0, seqs: [0] lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "e" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] local: num=0 # req4 from txn2 will conflict with locks on b, c since wants to write to [a, d). But does @@ -123,11 +123,11 @@ update txn=txn1 ts=11,1 epoch=1 span=b ---- global: num=3 lock: "b" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000008,12, seq: 0 + res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "e" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] local: num=0 # Still waiting, but on lock c which has a different ts in the TxnMeta. @@ -141,11 +141,11 @@ update txn=txn1 ts=11,1 epoch=1 span=c,e ---- global: num=3 lock: "b" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000008,12, seq: 0 + res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 lock: "c" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000008,12, seq: 0 + res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 lock: "e" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] local: num=0 # No longer waiting since does not conflict with lock on e. @@ -170,32 +170,32 @@ add-discovered r=req4 k=a txn=txn3 ---- global: num=4 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000006,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, info: repl epoch: 0, seqs: [0] queued writers: active: false req: 4, txn: 00000000-0000-0000-0000-000000000002 lock: "b" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000008,12, seq: 0 + res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 lock: "c" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000008,12, seq: 0 + res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 lock: "e" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] local: num=0 add-discovered r=req4 k=f txn=txn3 ---- global: num=5 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000006,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, info: repl epoch: 0, seqs: [0] queued writers: active: false req: 4, txn: 00000000-0000-0000-0000-000000000002 lock: "b" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000008,12, seq: 0 + res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 lock: "c" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000008,12, seq: 0 + res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 lock: "e" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "f" - holder: txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000006,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, info: repl epoch: 0, seqs: [0] local: num=0 # Note that guard state has not changed yet. Discovering these locks means the caller has to @@ -230,18 +230,18 @@ print ---- global: num=5 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000006,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, info: repl epoch: 0, seqs: [0] queued writers: active: true req: 4, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 4 lock: "b" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000008,12, seq: 0 + res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 lock: "c" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000008,12, seq: 0 + res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 lock: "e" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "f" - holder: txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000006,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, info: repl epoch: 0, seqs: [0] local: num=0 # req5 is again from transaction 1. Since it is reading from b, c, and even though txn1 @@ -258,18 +258,18 @@ dequeue r=req5 ---- global: num=5 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000006,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, info: repl epoch: 0, seqs: [0] queued writers: active: true req: 4, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 4 lock: "b" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000008,12, seq: 0 + res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 lock: "c" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000008,12, seq: 0 + res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 lock: "e" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "f" - holder: txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000006,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, info: repl epoch: 0, seqs: [0] local: num=0 # req6 from txn1 conflicts with lock at f, and reservations at b, c. @@ -300,21 +300,21 @@ print ---- global: num=5 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000006,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, info: repl epoch: 0, seqs: [0] queued writers: active: true req: 4, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 4 lock: "b" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000008,12, seq: 0 + res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 queued writers: active: true req: 6, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 6 lock: "c" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000008,12, seq: 0 + res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 lock: "e" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "f" - holder: txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000006,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, info: repl epoch: 0, seqs: [0] local: num=0 guard-state r=req6 @@ -354,24 +354,24 @@ print ---- global: num=5 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000006,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, info: repl epoch: 0, seqs: [0] queued writers: active: true req: 4, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 4 lock: "b" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000008,12, seq: 0 + res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 queued writers: active: true req: 6, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 6 lock: "c" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000008,12, seq: 0 + res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 queued writers: active: true req: 7, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 7 lock: "e" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "f" - holder: txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000006,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, info: repl epoch: 0, seqs: [0] local: num=0 # Release a. req4 waits at f. @@ -379,21 +379,21 @@ release txn=txn3 span=a ---- global: num=5 lock: "a" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000008,12, seq: 0 + res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 lock: "b" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000008,12, seq: 0 + res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 queued writers: active: true req: 6, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 6 lock: "c" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000008,12, seq: 0 + res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 queued writers: active: true req: 7, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 7 lock: "e" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "f" - holder: txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000006,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, info: repl epoch: 0, seqs: [0] local: num=0 guard-state r=req4 @@ -408,21 +408,21 @@ print ---- global: num=5 lock: "a" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000008,12, seq: 0 + res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 lock: "b" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000008,12, seq: 0 + res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 queued writers: active: true req: 6, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 6 lock: "c" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000008,12, seq: 0 + res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 queued writers: active: true req: 7, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 7 lock: "e" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "f" - holder: txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000006,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, info: repl epoch: 0, seqs: [0] waiting readers: req: 4, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 4 @@ -434,19 +434,19 @@ release txn=txn3 span=f ---- global: num=4 lock: "a" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000008,12, seq: 0 + res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 lock: "b" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000008,12, seq: 0 + res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 queued writers: active: true req: 6, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 6 lock: "c" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000008,12, seq: 0 + res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 queued writers: active: true req: 7, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 7 lock: "e" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] local: num=0 guard-state r=req4 @@ -477,38 +477,38 @@ acquire r=req4 k=b durability=r ---- global: num=4 lock: "a" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000008,12, seq: 0 + res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000008,12, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, info: repl epoch: 0, seqs: [0] queued writers: active: true req: 6, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 6 lock: "c" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000008,12, seq: 0 + res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 queued writers: active: true req: 7, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 7 lock: "e" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] local: num=0 acquire r=req4 k=c durability=r ---- global: num=4 lock: "a" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000008,12, seq: 0 + res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000008,12, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, info: repl epoch: 0, seqs: [0] queued writers: active: true req: 6, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 6 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000008,12, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, info: repl epoch: 0, seqs: [0] queued writers: active: true req: 7, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 7 lock: "e" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] local: num=0 guard-state r=req6 @@ -523,36 +523,36 @@ print ---- global: num=4 lock: "a" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000008,12, seq: 0 + res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000008,12, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, info: repl epoch: 0, seqs: [0] queued writers: active: true req: 6, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 6 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000008,12, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, info: repl epoch: 0, seqs: [0] queued writers: active: true req: 7, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 7 lock: "e" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] local: num=0 dequeue r=req4 ---- global: num=3 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000008,12, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, info: repl epoch: 0, seqs: [0] queued writers: active: true req: 6, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 6 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000008,12, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, info: repl epoch: 0, seqs: [0] queued writers: active: true req: 7, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 7 lock: "e" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] local: num=0 # Locks: @@ -574,14 +574,14 @@ release txn=txn2 span=c,f ---- global: num=3 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000008,12, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, info: repl epoch: 0, seqs: [0] queued writers: active: true req: 6, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 6 lock: "c" - res: req: 7, txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000006,0, seq: 0 + res: req: 7, txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, seq: 0 lock: "e" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] local: num=0 guard-state r=req7 @@ -596,14 +596,14 @@ print ---- global: num=3 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000008,12, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, info: repl epoch: 0, seqs: [0] queued writers: active: true req: 6, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 6 lock: "c" - res: req: 7, txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000006,0, seq: 0 + res: req: 7, txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, seq: 0 lock: "e" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] local: num=0 # Now before req7 can scan again, release the lock at b. This will cause req6 to break the @@ -613,11 +613,11 @@ release txn=txn2 span=b ---- global: num=3 lock: "b" - res: req: 6, txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000011,1, seq: 0 + res: req: 6, txn: 00000000-0000-0000-0000-000000000001, ts: 11.000000000,1, seq: 0 lock: "c" - res: req: 7, txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000006,0, seq: 0 + res: req: 7, txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, seq: 0 lock: "e" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] local: num=0 guard-state r=req6 @@ -648,13 +648,13 @@ print ---- global: num=3 lock: "b" - res: req: 6, txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000011,1, seq: 0 + res: req: 6, txn: 00000000-0000-0000-0000-000000000001, ts: 11.000000000,1, seq: 0 lock: "c" - res: req: 6, txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000011,1, seq: 0 + res: req: 6, txn: 00000000-0000-0000-0000-000000000001, ts: 11.000000000,1, seq: 0 queued writers: active: false req: 7, txn: 00000000-0000-0000-0000-000000000003 lock: "e" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] local: num=0 scan r=req7 @@ -674,9 +674,9 @@ dequeue r=req6 ---- global: num=2 lock: "c" - res: req: 7, txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000006,0, seq: 0 + res: req: 7, txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, seq: 0 lock: "e" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] local: num=0 guard-state r=req7 @@ -691,7 +691,7 @@ dequeue r=req7 ---- global: num=1 lock: "e" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] local: num=0 # e is still locked @@ -711,7 +711,7 @@ dequeue r=req8 ---- global: num=1 lock: "e" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] local: num=0 release txn=txn1 span=c,f @@ -738,21 +738,21 @@ acquire r=req9 k=c durability=u ---- global: num=1 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 1, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 1, seqs: [0] local: num=0 dequeue r=req9 ---- global: num=1 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 1, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 1, seqs: [0] local: num=0 print ---- global: num=1 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 1, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 1, seqs: [0] local: num=0 new-request r=req10 txn=txn2 ts=8,12 spans=w@c @@ -792,7 +792,7 @@ print ---- global: num=1 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 1, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 1, seqs: [0] queued writers: active: true req: 10, txn: 00000000-0000-0000-0000-000000000002 active: true req: 11, txn: 00000000-0000-0000-0000-000000000003 @@ -804,7 +804,7 @@ release txn=txn1 span=c ---- global: num=1 lock: "c" - res: req: 10, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000008,12, seq: 0 + res: req: 10, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 queued writers: active: true req: 11, txn: 00000000-0000-0000-0000-000000000003 active: true req: 12, txn: 00000000-0000-0000-0000-000000000002 @@ -831,7 +831,7 @@ print ---- global: num=1 lock: "c" - res: req: 10, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000008,12, seq: 0 + res: req: 10, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 queued writers: active: true req: 11, txn: 00000000-0000-0000-0000-000000000003 active: true req: 12, txn: 00000000-0000-0000-0000-000000000002 @@ -842,7 +842,7 @@ acquire r=req10 k=c durability=u ---- global: num=1 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000008,12, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 11, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 11 @@ -861,7 +861,7 @@ print ---- global: num=1 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000008,12, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 11, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 11 @@ -871,7 +871,7 @@ dequeue r=req10 ---- global: num=1 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000008,12, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 11, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 11 @@ -881,7 +881,7 @@ acquire r=req12 k=c durability=r ---- global: num=1 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000008,12, info: repl epoch: 0, seqs: [0], unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, info: repl epoch: 0, seqs: [0], unrepl epoch: 0, seqs: [0] queued writers: active: true req: 11, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 11 @@ -891,7 +891,7 @@ dequeue r=req12 ---- global: num=1 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000008,12, info: repl epoch: 0, seqs: [0], unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, info: repl epoch: 0, seqs: [0], unrepl epoch: 0, seqs: [0] queued writers: active: true req: 11, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 11 @@ -905,7 +905,7 @@ release txn=txn2 span=b,d ---- global: num=1 lock: "c" - res: req: 11, txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000006,0, seq: 0 + res: req: 11, txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, seq: 0 local: num=0 guard-state r=req11 @@ -916,7 +916,7 @@ print ---- global: num=1 lock: "c" - res: req: 11, txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000006,0, seq: 0 + res: req: 11, txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, seq: 0 local: num=0 dequeue r=req11 @@ -938,7 +938,7 @@ acquire r=req13 k=c durability=u ---- global: num=1 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000008,12, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, info: unrepl epoch: 0, seqs: [0] local: num=0 new-request r=req14 txn=txn1 ts=9,0 spans=w@c @@ -959,14 +959,14 @@ release txn=txn2 span=b,d ---- global: num=1 lock: "c" - res: req: 14, txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000009,0, seq: 0 + res: req: 14, txn: 00000000-0000-0000-0000-000000000001, ts: 9.000000000,0, seq: 0 local: num=0 dequeue r=req15 ---- global: num=1 lock: "c" - res: req: 14, txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000009,0, seq: 0 + res: req: 14, txn: 00000000-0000-0000-0000-000000000001, ts: 9.000000000,0, seq: 0 local: num=0 new-request r=req16 txn=none ts=10,12 spans=r@c @@ -1006,25 +1006,25 @@ acquire r=req17 k=c durability=u ---- global: num=1 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000009,0, info: unrepl epoch: 1, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 9.000000000,0, info: unrepl epoch: 1, seqs: [0] local: num=0 acquire r=req17 k=d durability=u ---- global: num=2 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000009,0, info: unrepl epoch: 1, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 9.000000000,0, info: unrepl epoch: 1, seqs: [0] lock: "d" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000009,0, info: unrepl epoch: 1, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 9.000000000,0, info: unrepl epoch: 1, seqs: [0] local: num=0 dequeue r=req17 ---- global: num=2 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000009,0, info: unrepl epoch: 1, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 9.000000000,0, info: unrepl epoch: 1, seqs: [0] lock: "d" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000009,0, info: unrepl epoch: 1, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 9.000000000,0, info: unrepl epoch: 1, seqs: [0] local: num=0 new-request r=req18 txn=txn2 ts=10,0 spans=w@c+w@d @@ -1045,12 +1045,12 @@ print ---- global: num=2 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000009,0, info: unrepl epoch: 1, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 9.000000000,0, info: unrepl epoch: 1, seqs: [0] queued writers: active: true req: 18, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 18 lock: "d" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000009,0, info: unrepl epoch: 1, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 9.000000000,0, info: unrepl epoch: 1, seqs: [0] queued writers: active: true req: 19, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 19 @@ -1060,9 +1060,9 @@ release txn=txn1 span=c ---- global: num=2 lock: "c" - res: req: 18, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, seq: 0 + res: req: 18, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 lock: "d" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000009,0, info: unrepl epoch: 1, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 9.000000000,0, info: unrepl epoch: 1, seqs: [0] queued writers: active: true req: 19, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 19 @@ -1076,9 +1076,9 @@ print ---- global: num=2 lock: "c" - res: req: 18, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, seq: 0 + res: req: 18, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 lock: "d" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000009,0, info: unrepl epoch: 1, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 9.000000000,0, info: unrepl epoch: 1, seqs: [0] queued writers: active: true req: 18, txn: 00000000-0000-0000-0000-000000000002 active: true req: 19, txn: 00000000-0000-0000-0000-000000000002 @@ -1089,9 +1089,9 @@ release txn=txn1 span=d ---- global: num=2 lock: "c" - res: req: 18, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, seq: 0 + res: req: 18, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 lock: "d" - res: req: 18, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, seq: 0 + res: req: 18, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 queued writers: active: true req: 19, txn: 00000000-0000-0000-0000-000000000002 local: num=0 @@ -1104,9 +1104,9 @@ acquire r=req18 k=d durability=u ---- global: num=2 lock: "c" - res: req: 18, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, seq: 0 + res: req: 18, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 lock: "d" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] local: num=0 guard-state r=req19 @@ -1121,14 +1121,14 @@ dequeue r=req18 ---- global: num=1 lock: "d" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] local: num=0 dequeue r=req19 ---- global: num=1 lock: "d" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] local: num=0 release txn=txn2 span=d @@ -1150,14 +1150,14 @@ acquire r=req20 k=c durability=u ---- global: num=1 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 1, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 1, seqs: [0] local: num=0 dequeue r=req20 ---- global: num=1 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 1, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 1, seqs: [0] local: num=0 new-request r=req21 txn=txn1 ts=10 spans=w@d @@ -1171,18 +1171,18 @@ acquire r=req21 k=d durability=u ---- global: num=2 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 1, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 1, seqs: [0] lock: "d" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 1, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 1, seqs: [0] local: num=0 dequeue r=req21 ---- global: num=2 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 1, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 1, seqs: [0] lock: "d" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 1, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 1, seqs: [0] local: num=0 new-request r=req22 txn=txn2 ts=10 spans=w@c+w@d @@ -1196,12 +1196,12 @@ print ---- global: num=2 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 1, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 1, seqs: [0] queued writers: active: true req: 22, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 22 lock: "d" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 1, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 1, seqs: [0] local: num=0 new-request r=req23 txn=txn3 ts=10 spans=w@d @@ -1215,12 +1215,12 @@ release txn=txn1 span=d ---- global: num=2 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 1, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 1, seqs: [0] queued writers: active: true req: 22, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 22 lock: "d" - res: req: 23, txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000010,0, seq: 0 + res: req: 23, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 local: num=0 guard-state r=req23 @@ -1240,9 +1240,9 @@ release txn=txn1 span=c ---- global: num=2 lock: "c" - res: req: 22, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, seq: 0 + res: req: 22, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 lock: "d" - res: req: 23, txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000010,0, seq: 0 + res: req: 23, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 local: num=0 guard-state r=req22 @@ -1253,9 +1253,9 @@ print ---- global: num=2 lock: "c" - res: req: 22, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, seq: 0 + res: req: 22, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 lock: "d" - res: req: 22, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, seq: 0 + res: req: 22, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 queued writers: active: false req: 23, txn: 00000000-0000-0000-0000-000000000003 local: num=0 @@ -1264,9 +1264,9 @@ acquire r=req23 k=d durability=u ---- global: num=2 lock: "c" - res: req: 22, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, seq: 0 + res: req: 22, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 lock: "d" - holder: txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: active: false req: 22, txn: 00000000-0000-0000-0000-000000000002 local: num=0 @@ -1275,14 +1275,14 @@ dequeue r=req22 ---- global: num=1 lock: "d" - holder: txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] local: num=0 dequeue r=req23 ---- global: num=1 lock: "d" - holder: txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] local: num=0 release txn=txn3 span=d diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/clear b/pkg/kv/kvserver/concurrency/testdata/lock_table/clear index 415ae0c1f243..d73aea9107c9 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/clear +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/clear @@ -29,25 +29,25 @@ acquire r=req1 k=a durability=u ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] local: num=0 acquire r=req1 k=b durability=u ---- global: num=2 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] local: num=0 dequeue r=req1 ---- global: num=2 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] local: num=0 # In its next request, txn1 discovers a lock at c held by txn2. @@ -67,11 +67,11 @@ add-discovered r=req2 k=c txn=txn2 ---- global: num=3 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000008,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,1, info: repl epoch: 0, seqs: [0] local: num=0 # A non-transactional read comes in at a and blocks on the lock. @@ -117,19 +117,19 @@ print ---- global: num=3 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] waiting readers: req: 3, txn: none queued writers: active: true req: 4, txn: none distinguished req: 3 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 5, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 5 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000008,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,1, info: repl epoch: 0, seqs: [0] local: num=0 # Clearing removes all locks and allows all waiting requests to proceed. diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/disable b/pkg/kv/kvserver/concurrency/testdata/lock_table/disable index 659df2f695a5..6aeb71adfe7f 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/disable +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/disable @@ -77,7 +77,7 @@ add-discovered r=req2 k=a txn=txn2 ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] queued writers: active: false req: 2, txn: 00000000-0000-0000-0000-000000000001 local: num=0 @@ -94,7 +94,7 @@ release txn=txn2 span=a ---- global: num=1 lock: "a" - res: req: 2, txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, seq: 0 + res: req: 2, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, seq: 0 local: num=0 guard-state r=req2 @@ -105,14 +105,14 @@ acquire r=req2 k=c durability=u ---- global: num=2 lock: "a" - res: req: 2, txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, seq: 0 + res: req: 2, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, seq: 0 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] local: num=0 dequeue r=req2 ---- global: num=1 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] local: num=0 diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/dup_access b/pkg/kv/kvserver/concurrency/testdata/lock_table/dup_access index dc87f1f644c0..ee690451db9f 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/dup_access +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/dup_access @@ -28,14 +28,14 @@ acquire r=req1 k=a durability=u ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] local: num=0 dequeue r=req1 ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] local: num=0 new-request r=req2 txn=txn2 ts=10 spans=w@a+r@a @@ -53,7 +53,7 @@ print ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 2, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 2 @@ -63,7 +63,7 @@ release txn=txn1 span=a ---- global: num=1 lock: "a" - res: req: 2, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, seq: 0 + res: req: 2, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 local: num=0 guard-state r=req2 @@ -91,38 +91,38 @@ acquire r=req3 k=a durability=u ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] local: num=0 acquire r=req3 k=b durability=u ---- global: num=2 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] local: num=0 acquire r=req3 k=c durability=u ---- global: num=3 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] local: num=0 dequeue r=req3 ---- global: num=3 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] local: num=0 new-request r=req4 txn=txn2 ts=10 spans=w@a+w@b @@ -151,17 +151,17 @@ print ---- global: num=3 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 4, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 4 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 5, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 5 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] local: num=0 # req5 reserves "b" and waits at "c". @@ -170,14 +170,14 @@ release txn=txn1 span=b ---- global: num=3 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 4, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 4 lock: "b" - res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000010,0, seq: 0 + res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] local: num=0 guard-state r=req5 @@ -188,14 +188,14 @@ print ---- global: num=3 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 4, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 4 lock: "b" - res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000010,0, seq: 0 + res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 5, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 5 @@ -207,11 +207,11 @@ release txn=txn1 span=a ---- global: num=3 lock: "a" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, seq: 0 + res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 lock: "b" - res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000010,0, seq: 0 + res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 5, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 5 @@ -225,13 +225,13 @@ print ---- global: num=3 lock: "a" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, seq: 0 + res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 lock: "b" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, seq: 0 + res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 queued writers: active: false req: 5, txn: 00000000-0000-0000-0000-000000000003 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 5, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 5 @@ -243,13 +243,13 @@ release txn=txn1 span=c ---- global: num=3 lock: "a" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, seq: 0 + res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 lock: "b" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, seq: 0 + res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 queued writers: active: false req: 5, txn: 00000000-0000-0000-0000-000000000003 lock: "c" - res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000010,0, seq: 0 + res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 local: num=0 guard-state r=req5 @@ -264,23 +264,23 @@ print ---- global: num=3 lock: "a" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, seq: 0 + res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 lock: "b" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, seq: 0 + res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 queued writers: active: true req: 5, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 5 lock: "c" - res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000010,0, seq: 0 + res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 local: num=0 dequeue r=req4 ---- global: num=2 lock: "b" - res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000010,0, seq: 0 + res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 lock: "c" - res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000010,0, seq: 0 + res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 local: num=0 dequeue r=req5 @@ -311,38 +311,38 @@ acquire r=req6 k=a durability=u ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] local: num=0 acquire r=req6 k=b durability=u ---- global: num=2 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] local: num=0 acquire r=req6 k=c durability=u ---- global: num=3 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] local: num=0 dequeue r=req6 ---- global: num=3 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] local: num=0 new-request r=req7 txn=txn2 ts=10 spans=w@a+w@b @@ -385,18 +385,18 @@ print ---- global: num=3 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 7, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 7 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 8, txn: none active: true req: 9, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 8 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] local: num=0 # req8 waits at "c". @@ -405,14 +405,14 @@ release txn=txn1 span=b ---- global: num=3 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 7, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 7 lock: "b" - res: req: 9, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, seq: 0 + res: req: 9, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] local: num=0 guard-state r=req8 @@ -423,14 +423,14 @@ print ---- global: num=3 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 7, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 7 lock: "b" - res: req: 9, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, seq: 0 + res: req: 9, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 8, txn: none distinguished req: 8 @@ -442,11 +442,11 @@ release txn=txn1 span=a ---- global: num=3 lock: "a" - res: req: 7, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, seq: 0 + res: req: 7, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 lock: "b" - res: req: 9, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, seq: 0 + res: req: 9, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 8, txn: none distinguished req: 8 @@ -460,13 +460,13 @@ print ---- global: num=3 lock: "a" - res: req: 7, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, seq: 0 + res: req: 7, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 lock: "b" - res: req: 7, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, seq: 0 + res: req: 7, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 queued writers: active: false req: 9, txn: 00000000-0000-0000-0000-000000000002 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 8, txn: none distinguished req: 8 @@ -480,11 +480,11 @@ acquire r=req7 k=b durability=u ---- global: num=3 lock: "a" - res: req: 7, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, seq: 0 + res: req: 7, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 8, txn: none distinguished req: 8 @@ -495,9 +495,9 @@ release txn=txn1 span=c ---- global: num=2 lock: "a" - res: req: 7, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, seq: 0 + res: req: 7, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] local: num=0 guard-state r=req8 @@ -508,9 +508,9 @@ print ---- global: num=2 lock: "a" - res: req: 7, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, seq: 0 + res: req: 7, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] waiting readers: req: 8, txn: none distinguished req: 8 @@ -520,7 +520,7 @@ dequeue r=req7 ---- global: num=1 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] waiting readers: req: 8, txn: none distinguished req: 8 @@ -530,7 +530,7 @@ dequeue r=req8 ---- global: num=1 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] local: num=0 release txn=txn2 span=b @@ -555,25 +555,25 @@ acquire r=req10 k=a durability=u ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] local: num=0 acquire r=req10 k=b durability=u ---- global: num=2 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] local: num=0 dequeue r=req10 ---- global: num=2 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] local: num=0 new-request r=req11 txn=txn2 ts=10 spans=w@a+w@b @@ -602,12 +602,12 @@ print ---- global: num=2 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 11, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 11 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 12, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 12 @@ -619,12 +619,12 @@ release txn=txn1 span=b ---- global: num=2 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 11, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 11 lock: "b" - res: req: 12, txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000010,0, seq: 0 + res: req: 12, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 local: num=0 # req11 reserves "a" @@ -632,9 +632,9 @@ release txn=txn1 span=a ---- global: num=2 lock: "a" - res: req: 11, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, seq: 0 + res: req: 11, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 lock: "b" - res: req: 12, txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000010,0, seq: 0 + res: req: 12, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 local: num=0 # req11 breaks the reservation at "b" @@ -646,9 +646,9 @@ print ---- global: num=2 lock: "a" - res: req: 11, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, seq: 0 + res: req: 11, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 lock: "b" - res: req: 11, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, seq: 0 + res: req: 11, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 queued writers: active: false req: 12, txn: 00000000-0000-0000-0000-000000000003 local: num=0 @@ -661,9 +661,9 @@ acquire r=req11 k=b durability=u ---- global: num=2 lock: "a" - res: req: 11, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, seq: 0 + res: req: 11, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: active: false req: 12, txn: 00000000-0000-0000-0000-000000000003 local: num=0 @@ -687,7 +687,7 @@ dequeue r=req11 ---- global: num=1 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 12, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 12 @@ -697,7 +697,7 @@ dequeue r=req12 ---- global: num=1 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] local: num=0 release txn=txn2 span=b diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/lock_changes b/pkg/kv/kvserver/concurrency/testdata/lock_table/lock_changes index 747cb30a9fd9..316971ea888c 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/lock_changes +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/lock_changes @@ -20,14 +20,14 @@ acquire r=req1 k=a durability=u ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [2] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [2] local: num=0 release txn=txn2 span=a,c ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [2] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [2] local: num=0 # --------------------------------------------------------------------------------- @@ -46,7 +46,7 @@ acquire r=req2 k=a durability=u ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [2, 3] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [2, 3] local: num=0 # --------------------------------------------------------------------------------- @@ -69,7 +69,7 @@ acquire r=req2 k=a durability=r ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000008,0, info: repl epoch: 0, seqs: [3], unrepl epoch: 0, seqs: [2, 3] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 8.000000000,0, info: repl epoch: 0, seqs: [3], unrepl epoch: 0, seqs: [2, 3] queued writers: active: true req: 1, txn: none distinguished req: 1 @@ -79,7 +79,7 @@ dequeue r=reqContend ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000008,0, info: repl epoch: 0, seqs: [3], unrepl epoch: 0, seqs: [2, 3] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 8.000000000,0, info: repl epoch: 0, seqs: [3], unrepl epoch: 0, seqs: [2, 3] local: num=0 # --------------------------------------------------------------------------------- @@ -96,7 +96,7 @@ acquire r=req3 k=a durability=u ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000008,0, info: repl epoch: 0, seqs: [3], unrepl epoch: 1, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 8.000000000,0, info: repl epoch: 0, seqs: [3], unrepl epoch: 1, seqs: [0] local: num=0 # --------------------------------------------------------------------------------- @@ -115,7 +115,7 @@ acquire r=req4 k=a durability=u ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000008,0, info: repl epoch: 0, seqs: [3], unrepl epoch: 2, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 8.000000000,0, info: repl epoch: 0, seqs: [3], unrepl epoch: 2, seqs: [0] local: num=0 # --------------------------------------------------------------------------------- @@ -137,7 +137,7 @@ print ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000008,0, info: repl epoch: 0, seqs: [3], unrepl epoch: 2, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 8.000000000,0, info: repl epoch: 0, seqs: [3], unrepl epoch: 2, seqs: [0] waiting readers: req: 2, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 2 @@ -153,7 +153,7 @@ acquire r=req6 k=a durability=r ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: repl epoch: 1, seqs: [1], unrepl epoch: 2, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: repl epoch: 1, seqs: [1], unrepl epoch: 2, seqs: [0] waiting readers: req: 2, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 2 @@ -167,7 +167,7 @@ acquire r=req6 k=a durability=u ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000014,0, info: repl epoch: 1, seqs: [1], unrepl epoch: 1, seqs: [0, 1] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 14.000000000,0, info: repl epoch: 1, seqs: [1], unrepl epoch: 1, seqs: [0, 1] local: num=0 guard-state r=req5 @@ -196,7 +196,7 @@ add-discovered r=req7 k=a txn=txn1 ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000014,0, info: repl epoch: 1, seqs: [1], unrepl epoch: 1, seqs: [0, 1] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 14.000000000,0, info: repl epoch: 1, seqs: [1], unrepl epoch: 1, seqs: [0, 1] waiting readers: req: 3, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 3 diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/lock_dropped b/pkg/kv/kvserver/concurrency/testdata/lock_table/lock_dropped index 8510678957bc..d8d5083f0af5 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/lock_dropped +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/lock_dropped @@ -27,7 +27,7 @@ acquire r=req1 k=a durability=u ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [2] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [2] local: num=0 acquire r=req1 k=a durability=r @@ -47,7 +47,7 @@ acquire r=req1 k=a durability=u ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [2] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [2] local: num=0 scan r=reqContendReader @@ -58,7 +58,7 @@ print ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [2] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [2] waiting readers: req: 1, txn: none distinguished req: 1 @@ -85,7 +85,7 @@ acquire r=req1 k=a durability=u ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [2] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [2] local: num=0 scan r=reqContendReader @@ -100,7 +100,7 @@ print ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [2] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [2] waiting readers: req: 1, txn: none queued writers: @@ -112,7 +112,7 @@ acquire r=req1 k=a durability=r ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: repl epoch: 0, seqs: [2], unrepl epoch: 0, seqs: [2] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: repl epoch: 0, seqs: [2], unrepl epoch: 0, seqs: [2] waiting readers: req: 1, txn: none queued writers: @@ -149,7 +149,7 @@ acquire r=req1 k=a durability=u ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [2] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [2] local: num=0 new-txn txn=txn2 ts=10 epoch=0 seq=0 @@ -162,9 +162,9 @@ acquire r=req2 k=b durability=u ---- global: num=2 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [2] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [2] lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] local: num=0 new-request r=req3 txn=none ts=10 spans=r@a,c @@ -182,7 +182,7 @@ acquire r=req2 k=b durability=r ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [2] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [2] waiting readers: req: 3, txn: none distinguished req: 3 diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/non_active_waiter b/pkg/kv/kvserver/concurrency/testdata/lock_table/non_active_waiter index fd625a2c66ae..744a56fe9e86 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/non_active_waiter +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/non_active_waiter @@ -20,7 +20,7 @@ add-discovered r=req1 k=a txn=txn2 ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] queued writers: active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 local: num=0 @@ -29,24 +29,24 @@ add-discovered r=req1 k=b txn=txn2 ---- global: num=2 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] queued writers: active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] local: num=0 add-discovered r=req1 k=c txn=txn2 ---- global: num=3 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] queued writers: active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] queued writers: active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 local: num=0 @@ -57,13 +57,13 @@ print ---- global: num=3 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] queued writers: active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] queued writers: active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 local: num=0 @@ -81,13 +81,13 @@ print ---- global: num=3 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] queued writers: active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] queued writers: active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 active: true req: 2, txn: 00000000-0000-0000-0000-000000000001 @@ -106,14 +106,14 @@ print ---- global: num=3 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] queued writers: active: true req: 1, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 1 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] queued writers: active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 active: true req: 2, txn: 00000000-0000-0000-0000-000000000001 @@ -125,11 +125,11 @@ release txn=txn2 span=a ---- global: num=3 lock: "a" - res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, seq: 0 + res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] queued writers: active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 active: true req: 2, txn: 00000000-0000-0000-0000-000000000001 @@ -144,11 +144,11 @@ print ---- global: num=3 lock: "a" - res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, seq: 0 + res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] queued writers: active: true req: 1, txn: 00000000-0000-0000-0000-000000000001 active: true req: 2, txn: 00000000-0000-0000-0000-000000000001 @@ -161,11 +161,11 @@ release txn=txn2 span=c ---- global: num=3 lock: "a" - res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, seq: 0 + res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] lock: "c" - res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, seq: 0 + res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 queued writers: active: true req: 2, txn: 00000000-0000-0000-0000-000000000001 local: num=0 @@ -182,14 +182,14 @@ print ---- global: num=3 lock: "a" - res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, seq: 0 + res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] waiting readers: req: 1, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 1 lock: "c" - res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, seq: 0 + res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 queued writers: active: true req: 2, txn: 00000000-0000-0000-0000-000000000001 local: num=0 @@ -200,9 +200,9 @@ release txn=txn2 span=b ---- global: num=2 lock: "a" - res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, seq: 0 + res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 lock: "c" - res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, seq: 0 + res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 queued writers: active: true req: 2, txn: 00000000-0000-0000-0000-000000000001 local: num=0 @@ -215,7 +215,7 @@ dequeue r=req1 ---- global: num=1 lock: "c" - res: req: 2, txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, seq: 0 + res: req: 2, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 local: num=0 guard-state r=req2 diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/non_txn_write b/pkg/kv/kvserver/concurrency/testdata/lock_table/non_txn_write index d61fb9e01b0b..338da397d80c 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/non_txn_write +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/non_txn_write @@ -22,38 +22,38 @@ acquire r=req1 k=a durability=u ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] local: num=0 acquire r=req1 k=b durability=u ---- global: num=2 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] local: num=0 acquire r=req1 k=c durability=u ---- global: num=3 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] local: num=0 dequeue r=req1 ---- global: num=3 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] local: num=0 # Next, two different transactional requests wait at a and b. @@ -93,18 +93,18 @@ print ---- global: num=3 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 2, txn: 00000000-0000-0000-0000-000000000002 active: true req: 4, txn: none distinguished req: 2 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 3, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 3 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 5, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 5 @@ -117,14 +117,14 @@ release txn=txn1 span=a,d ---- global: num=3 lock: "a" - res: req: 2, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, seq: 0 + res: req: 2, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 queued writers: active: true req: 4, txn: none distinguished req: 4 lock: "b" - res: req: 3, txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000010,0, seq: 0 + res: req: 3, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 lock: "c" - res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000010,0, seq: 0 + res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 local: num=0 guard-state r=req2 @@ -156,15 +156,15 @@ print ---- global: num=3 lock: "a" - res: req: 2, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, seq: 0 + res: req: 2, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 queued writers: active: true req: 4, txn: none active: true req: 6, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 4 lock: "b" - res: req: 3, txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000010,0, seq: 0 + res: req: 3, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 lock: "c" - res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000010,0, seq: 0 + res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 local: num=0 # Release the reservation at a. The first waiter is non-transactional so it will not acquire the @@ -175,11 +175,11 @@ dequeue r=req2 ---- global: num=3 lock: "a" - res: req: 6, txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, seq: 0 + res: req: 6, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 lock: "b" - res: req: 3, txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000010,0, seq: 0 + res: req: 3, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 lock: "c" - res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000010,0, seq: 0 + res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 local: num=0 guard-state r=req4 @@ -194,14 +194,14 @@ print ---- global: num=3 lock: "a" - res: req: 6, txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, seq: 0 + res: req: 6, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 lock: "b" - res: req: 3, txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000010,0, seq: 0 + res: req: 3, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 queued writers: active: true req: 4, txn: none distinguished req: 4 lock: "c" - res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000010,0, seq: 0 + res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 local: num=0 # Release the reservation at b. The non-transactional waiter will be done at b, and when it gets @@ -211,9 +211,9 @@ dequeue r=req3 ---- global: num=2 lock: "a" - res: req: 6, txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, seq: 0 + res: req: 6, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 lock: "c" - res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000010,0, seq: 0 + res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 local: num=0 guard-state r=req4 @@ -228,9 +228,9 @@ print ---- global: num=2 lock: "a" - res: req: 6, txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, seq: 0 + res: req: 6, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 lock: "c" - res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000010,0, seq: 0 + res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 local: num=0 # Non-transactional request scans again and proceeds to evaluation and discovers a lock at c @@ -243,9 +243,9 @@ add-discovered r=req4 k=c txn=txn2 ---- global: num=2 lock: "a" - res: req: 6, txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, seq: 0 + res: req: 6, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] queued writers: active: false req: 4, txn: none active: false req: 5, txn: 00000000-0000-0000-0000-000000000003 @@ -273,9 +273,9 @@ release txn=txn2 span=c ---- global: num=2 lock: "a" - res: req: 6, txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, seq: 0 + res: req: 6, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 lock: "c" - res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000010,0, seq: 0 + res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 local: num=0 guard-state r=req4 @@ -292,16 +292,16 @@ dequeue r=req4 ---- global: num=2 lock: "a" - res: req: 6, txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, seq: 0 + res: req: 6, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 lock: "c" - res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000010,0, seq: 0 + res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 local: num=0 dequeue r=req5 ---- global: num=1 lock: "a" - res: req: 6, txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, seq: 0 + res: req: 6, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 local: num=0 dequeue r=req6 diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/size_limit_exceeded b/pkg/kv/kvserver/concurrency/testdata/lock_table/size_limit_exceeded index b94d2ab02826..eae6e6256347 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/size_limit_exceeded +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/size_limit_exceeded @@ -26,16 +26,16 @@ acquire r=req1 k=a durability=u ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] local: num=0 acquire r=req1 k=b durability=u ---- global: num=2 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] local: num=0 # c is first locked as unreplicated and establishes a writer queue @@ -49,11 +49,11 @@ acquire r=req1 k=c durability=u ---- global: num=3 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] local: num=0 new-request r=reqContend txn=none ts=10 spans=w@c @@ -67,11 +67,11 @@ acquire r=req1 k=c durability=r ---- global: num=3 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: repl epoch: 0, seqs: [0], unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0], unrepl epoch: 0, seqs: [0] queued writers: active: true req: 2, txn: none distinguished req: 2 @@ -81,22 +81,22 @@ dequeue r=reqContend ---- global: num=3 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: repl epoch: 0, seqs: [0], unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0], unrepl epoch: 0, seqs: [0] local: num=0 dequeue r=req1 ---- global: num=3 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: repl epoch: 0, seqs: [0], unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0], unrepl epoch: 0, seqs: [0] local: num=0 new-request r=req2 txn=txn2 ts=10 spans=w@a,c @@ -117,28 +117,28 @@ print ---- global: num=3 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 3, txn: 00000000-0000-0000-0000-000000000002 active: true req: 4, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 3 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: repl epoch: 0, seqs: [0], unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0], unrepl epoch: 0, seqs: [0] local: num=0 release txn=txn1 span=a ---- global: num=3 lock: "a" - res: req: 3, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, seq: 0 + res: req: 3, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 queued writers: active: true req: 4, txn: 00000000-0000-0000-0000-000000000002 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: repl epoch: 0, seqs: [0], unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0], unrepl epoch: 0, seqs: [0] local: num=0 guard-state r=req2 @@ -153,16 +153,16 @@ print ---- global: num=3 lock: "a" - res: req: 3, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, seq: 0 + res: req: 3, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 queued writers: active: true req: 4, txn: 00000000-0000-0000-0000-000000000002 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 3, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 3 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: repl epoch: 0, seqs: [0], unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0], unrepl epoch: 0, seqs: [0] local: num=0 new-request r=req4 txn=txn2 ts=10 spans=r@b @@ -201,11 +201,11 @@ add-discovered r=req7 k=d txn=txn1 ---- global: num=4 lock: "a" - res: req: 3, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, seq: 0 + res: req: 3, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 queued writers: active: true req: 4, txn: 00000000-0000-0000-0000-000000000002 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] waiting readers: req: 5, txn: 00000000-0000-0000-0000-000000000002 queued writers: @@ -213,12 +213,12 @@ global: num=4 active: true req: 6, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 3 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: repl epoch: 0, seqs: [0], unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0], unrepl epoch: 0, seqs: [0] queued writers: active: true req: 7, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 7 lock: "d" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] queued writers: active: false req: 8, txn: 00000000-0000-0000-0000-000000000002 local: num=0 @@ -236,7 +236,7 @@ acquire r=req8 k=e durability=u ---- global: num=1 lock: "d" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] queued writers: active: false req: 8, txn: 00000000-0000-0000-0000-000000000002 local: num=0 diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/update b/pkg/kv/kvserver/concurrency/testdata/lock_table/update index c888fcbdec3b..08a838a455c3 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/update +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/update @@ -29,14 +29,14 @@ acquire r=req1 k=a durability=u ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] local: num=0 dequeue r=req1 ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] local: num=0 # ------------------------------------------------------------- @@ -95,7 +95,7 @@ print ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] waiting readers: req: 4, txn: none req: 2, txn: 00000000-0000-0000-0000-000000000002 @@ -113,7 +113,7 @@ update txn=txn1 ts=11,1 epoch=0 span=a ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000011,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 11.000000000,1, info: unrepl epoch: 0, seqs: [0] waiting readers: req: 4, txn: none req: 2, txn: 00000000-0000-0000-0000-000000000002 @@ -133,7 +133,7 @@ update txn=txn1 ts=13,1 epoch=0 span=a ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000013,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 13.000000000,1, info: unrepl epoch: 0, seqs: [0] waiting readers: req: 4, txn: none queued writers: @@ -154,7 +154,7 @@ dequeue r=req2 ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000013,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 13.000000000,1, info: unrepl epoch: 0, seqs: [0] waiting readers: req: 4, txn: none queued writers: @@ -172,7 +172,7 @@ update txn=txn1 ts=10,1 epoch=0 span=a ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000013,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 13.000000000,1, info: unrepl epoch: 0, seqs: [0] waiting readers: req: 4, txn: none queued writers: @@ -189,7 +189,7 @@ update txn=txn1 ts=15,1 epoch=0 span=a ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000015,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 15.000000000,1, info: unrepl epoch: 0, seqs: [0] waiting readers: req: 4, txn: none queued writers: @@ -208,7 +208,7 @@ update txn=txn1 ts=17,1 epoch=0 span=a ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000017,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 17.000000000,1, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 3, txn: 00000000-0000-0000-0000-000000000003 active: true req: 5, txn: none @@ -227,7 +227,7 @@ dequeue r=req4 ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000017,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 17.000000000,1, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 3, txn: 00000000-0000-0000-0000-000000000003 active: true req: 5, txn: none @@ -242,7 +242,7 @@ update txn=txn1 ts=19,1 epoch=0 span=a ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000019,1, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 19.000000000,1, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 3, txn: 00000000-0000-0000-0000-000000000003 active: true req: 5, txn: none @@ -260,7 +260,7 @@ update txn=txn1 ts=19,1 epoch=1 span=a ---- global: num=1 lock: "a" - res: req: 3, txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000014,1, seq: 0 + res: req: 3, txn: 00000000-0000-0000-0000-000000000003, ts: 14.000000000,1, seq: 0 queued writers: active: true req: 5, txn: none distinguished req: 5 @@ -317,7 +317,7 @@ acquire r=req1 k=a durability=u ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 1, seqs: [1] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 1, seqs: [1] local: num=0 new-txn txn=txn1 ts=10 epoch=1 seq=5 @@ -330,7 +330,7 @@ acquire r=req2 k=a durability=u ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 1, seqs: [1, 5] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 1, seqs: [1, 5] local: num=0 new-txn txn=txn1 ts=10 epoch=1 seq=7 @@ -343,7 +343,7 @@ acquire r=req3 k=a durability=u ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 1, seqs: [1, 5, 7] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 1, seqs: [1, 5, 7] local: num=0 new-txn txn=txn1 ts=10 epoch=1 seq=10 @@ -356,7 +356,7 @@ acquire r=req4 k=a durability=u ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 1, seqs: [1, 5, 7, 10] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 1, seqs: [1, 5, 7, 10] local: num=0 # No seqnum change since lock is not held at seqnum 3, 8, 9. @@ -365,7 +365,7 @@ update txn=txn1 ts=10 epoch=1 span=a ignored-seqs=3,8-9 ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 1, seqs: [1, 5, 7, 10] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 1, seqs: [1, 5, 7, 10] local: num=0 # No change since update is using older epoch. @@ -374,21 +374,21 @@ update txn=txn1 ts=10 epoch=0 span=a ignored-seqs=3,5-7 ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 1, seqs: [1, 5, 7, 10] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 1, seqs: [1, 5, 7, 10] local: num=0 update txn=txn1 ts=10 epoch=1 span=a ignored-seqs=3,5-7 ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 1, seqs: [1, 10] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 1, seqs: [1, 10] local: num=0 update txn=txn1 ts=10 epoch=1 span=a ignored-seqs=9-11 ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 1, seqs: [1] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 1, seqs: [1] local: num=0 # No seqnum change since update is using older epoch. But since the update is using @@ -398,7 +398,7 @@ update txn=txn1 ts=15 epoch=0 span=a ignored-seqs=1 ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000015,0, info: unrepl epoch: 1, seqs: [1] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 15.000000000,0, info: unrepl epoch: 1, seqs: [1] local: num=0 # No change, since seqnum 3 is not held. Note that the ts is not updated. @@ -407,7 +407,7 @@ update txn=txn1 ts=10 epoch=1 span=a ignored-seqs=3 ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000015,0, info: unrepl epoch: 1, seqs: [1] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 15.000000000,0, info: unrepl epoch: 1, seqs: [1] local: num=0 # Timestamp is updated again. @@ -415,7 +415,7 @@ update txn=txn1 ts=16 epoch=1 span=a ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000016,0, info: unrepl epoch: 1, seqs: [1] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 16.000000000,0, info: unrepl epoch: 1, seqs: [1] local: num=0 # Seqnum 1 is also ignored, so the lock is released. Note that it does not diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/wait_self b/pkg/kv/kvserver/concurrency/testdata/lock_table/wait_self index f4fadda5b56a..03a76537bbb9 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/wait_self +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/wait_self @@ -37,14 +37,14 @@ acquire r=req1 k=a durability=u ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] local: num=0 dequeue r=req1 ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] local: num=0 scan r=req2 @@ -75,7 +75,7 @@ print ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 2, txn: 00000000-0000-0000-0000-000000000002 active: true req: 3, txn: 00000000-0000-0000-0000-000000000003 @@ -87,7 +87,7 @@ release txn=txn1 span=a ---- global: num=1 lock: "a" - res: req: 2, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, seq: 0 + res: req: 2, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 queued writers: active: true req: 3, txn: 00000000-0000-0000-0000-000000000003 active: true req: 4, txn: 00000000-0000-0000-0000-000000000002 @@ -110,7 +110,7 @@ print ---- global: num=1 lock: "a" - res: req: 2, txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, seq: 0 + res: req: 2, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 queued writers: active: true req: 3, txn: 00000000-0000-0000-0000-000000000003 active: true req: 4, txn: 00000000-0000-0000-0000-000000000002 @@ -135,7 +135,7 @@ acquire r=req2 k=a durability=u ---- global: num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000010,0, info: unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 3, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 3 From cb657b322cf78907f34334e8d110a91110e7ed85 Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Tue, 29 Dec 2020 00:13:59 -0500 Subject: [PATCH 5/7] hlc: replace Timestamp.flags bit field with synthetic boolean This commit replaces Timestamp's (and LegacyTimestamp's) `flags` field with a single boolean field called `synthetic`. This shaves a byte off the encoded message when the field is set and makes synthetic timestamps easier to work with. If we ever decide that we need to go back on this because we need more flags (unlikely), we can without breaking anything. The commit also starts the process of updating tests to handle the fact that `Timestamp.Add` should set the synthetic flag. However, it stops before completing the change because it turned out to be fairly large and not a pre-requisite for non-blocking transactions. --- pkg/ccl/changefeedccl/kvfeed/buffer.go | 4 - pkg/ccl/changefeedccl/sink.go | 2 +- .../kvcoord/dist_sender_server_test.go | 2 +- .../kvclient/kvcoord/txn_coord_sender_test.go | 4 +- .../txn_interceptor_span_refresher_test.go | 2 +- pkg/kv/kvserver/below_raft_protos_test.go | 14 +- pkg/kv/kvserver/client_replica_test.go | 112 ++++++++++- .../kvserver/rangefeed/resolved_timestamp.go | 2 +- .../kvserver/rditer/replica_data_iter_test.go | 4 +- pkg/kv/kvserver/store_test.go | 4 +- pkg/kv/kvserver/tscache/cache_test.go | 2 +- pkg/roachpb/data_test.go | 40 ++-- pkg/roachpb/string_test.go | 4 +- pkg/storage/batch.go | 20 +- pkg/storage/batch_test.go | 2 +- pkg/storage/engine_key.go | 16 +- pkg/storage/engine_key_test.go | 8 +- pkg/storage/enginepb/decode.go | 2 +- pkg/storage/mvcc.go | 10 +- pkg/storage/mvcc_test.go | 4 +- pkg/util/hlc/BUILD.bazel | 1 - pkg/util/hlc/hlc_test.go | 2 +- pkg/util/hlc/legacy_timestamp.pb.go | 83 ++++---- pkg/util/hlc/legacy_timestamp.proto | 20 +- pkg/util/hlc/timestamp.go | 179 +++++++----------- pkg/util/hlc/timestamp.pb.go | 123 +++++------- pkg/util/hlc/timestamp.proto | 57 ++---- pkg/util/hlc/timestamp_test.go | 153 ++++++--------- 28 files changed, 435 insertions(+), 441 deletions(-) diff --git a/pkg/ccl/changefeedccl/kvfeed/buffer.go b/pkg/ccl/changefeedccl/kvfeed/buffer.go index 638569b5b4db..f3510f0d4e0a 100644 --- a/pkg/ccl/changefeedccl/kvfeed/buffer.go +++ b/pkg/ccl/changefeedccl/kvfeed/buffer.go @@ -211,7 +211,6 @@ var memBufferColTypes = []*types.T{ types.Bytes, // span.EndKey types.Int, // ts.WallTime types.Int, // ts.Logical - types.Int, // ts.Flags } // memBuffer is an in-memory buffer for changed KV and Resolved timestamp @@ -267,7 +266,6 @@ func (b *memBuffer) AddKV( tree.DNull, b.allocMu.a.NewDInt(tree.DInt(kv.Value.Timestamp.WallTime)), b.allocMu.a.NewDInt(tree.DInt(kv.Value.Timestamp.Logical)), - b.allocMu.a.NewDInt(tree.DInt(kv.Value.Timestamp.Flags)), } b.allocMu.Unlock() return b.addRow(ctx, row) @@ -286,7 +284,6 @@ func (b *memBuffer) AddResolved( b.allocMu.a.NewDBytes(tree.DBytes(span.EndKey)), b.allocMu.a.NewDInt(tree.DInt(ts.WallTime)), b.allocMu.a.NewDInt(tree.DInt(ts.Logical)), - b.allocMu.a.NewDInt(tree.DInt(ts.Flags)), } b.allocMu.Unlock() return b.addRow(ctx, row) @@ -303,7 +300,6 @@ func (b *memBuffer) Get(ctx context.Context) (Event, error) { ts := hlc.Timestamp{ WallTime: int64(*row[5].(*tree.DInt)), Logical: int32(*row[6].(*tree.DInt)), - Flags: uint32(*row[7].(*tree.DInt)), } if row[2] != tree.DNull { e.prevVal = roachpb.Value{ diff --git a/pkg/ccl/changefeedccl/sink.go b/pkg/ccl/changefeedccl/sink.go index b14c6e5f0a7e..b5904005cda2 100644 --- a/pkg/ccl/changefeedccl/sink.go +++ b/pkg/ccl/changefeedccl/sink.go @@ -822,7 +822,7 @@ func (s *bufferSink) EmitRow( {Datum: tree.DNull}, // resolved span {Datum: s.alloc.NewDString(tree.DString(topic))}, // topic {Datum: s.alloc.NewDBytes(tree.DBytes(key))}, // key - {Datum: s.alloc.NewDBytes(tree.DBytes(value))}, //value + {Datum: s.alloc.NewDBytes(tree.DBytes(value))}, // value }) return nil } diff --git a/pkg/kv/kvclient/kvcoord/dist_sender_server_test.go b/pkg/kv/kvclient/kvcoord/dist_sender_server_test.go index 6a0c893098bb..177ff8e548b1 100644 --- a/pkg/kv/kvclient/kvcoord/dist_sender_server_test.go +++ b/pkg/kv/kvclient/kvcoord/dist_sender_server_test.go @@ -1190,7 +1190,7 @@ func TestMultiRangeScanReverseScanInconsistent(t *testing.T) { t.Fatal(err) } ts[i] = s.Clock().Now() - log.Infof(ctx, "%d: %s %d", i, key, ts[i]) + log.Infof(ctx, "%d: %s %s", i, key, ts[i]) if i == 0 { testutils.SucceedsSoon(t, func() error { // Enforce that when we write the second key, it's written diff --git a/pkg/kv/kvclient/kvcoord/txn_coord_sender_test.go b/pkg/kv/kvclient/kvcoord/txn_coord_sender_test.go index 0eff49ec39fe..7e266cf2cd72 100644 --- a/pkg/kv/kvclient/kvcoord/txn_coord_sender_test.go +++ b/pkg/kv/kvclient/kvcoord/txn_coord_sender_test.go @@ -697,8 +697,8 @@ func TestTxnCoordSenderTxnUpdatedOnError(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() origTS := makeTS(123, 0) - plus10 := origTS.Add(10, 10) - plus20 := origTS.Add(20, 0) + plus10 := origTS.Add(10, 10).WithSynthetic(false) + plus20 := origTS.Add(20, 0).WithSynthetic(false) testCases := []struct { // The test's name. name string 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 f851b1d39ff5..9fdc1acc2da2 100644 --- a/pkg/kv/kvclient/kvcoord/txn_interceptor_span_refresher_test.go +++ b/pkg/kv/kvclient/kvcoord/txn_interceptor_span_refresher_test.go @@ -164,7 +164,7 @@ func TestTxnSpanRefresherRefreshesTransactions(t *testing.T) { return pErr }, expRefresh: true, - expRefreshTS: txn.WriteTimestamp.Add(20, 0), // see UpdateObservedTimestamp + expRefreshTS: txn.WriteTimestamp.Add(20, 0).WithSynthetic(false), // see UpdateObservedTimestamp }, { pErr: func() *roachpb.Error { diff --git a/pkg/kv/kvserver/below_raft_protos_test.go b/pkg/kv/kvserver/below_raft_protos_test.go index f7e29f139e64..53ec62441351 100644 --- a/pkg/kv/kvserver/below_raft_protos_test.go +++ b/pkg/kv/kvserver/below_raft_protos_test.go @@ -63,11 +63,15 @@ var belowRaftGoldenProtos = map[reflect.Type]fixture{ populatedConstructor: func(r *rand.Rand) protoutil.Message { m := enginepb.NewPopulatedMVCCMetadata(r, false) m.Txn = nil // never populated below Raft + m.Timestamp.Synthetic = nil // never populated below Raft + if m.MergeTimestamp != nil { + m.MergeTimestamp.Synthetic = nil // never populated below Raft + } m.TxnDidNotUpdateMeta = nil // never populated below Raft return m }, emptySum: 7551962144604783939, - populatedSum: 11599955036265189084, + populatedSum: 12366000535951165621, }, reflect.TypeOf(&enginepb.RangeAppliedState{}): { populatedConstructor: func(r *rand.Rand) protoutil.Message { @@ -124,10 +128,14 @@ var belowRaftGoldenProtos = map[reflect.Type]fixture{ }, reflect.TypeOf(&enginepb.MVCCMetadataSubsetForMergeSerialization{}): { populatedConstructor: func(r *rand.Rand) protoutil.Message { - return enginepb.NewPopulatedMVCCMetadataSubsetForMergeSerialization(r, false) + m := enginepb.NewPopulatedMVCCMetadataSubsetForMergeSerialization(r, false) + if m.MergeTimestamp != nil { + m.MergeTimestamp.Synthetic = nil // never populated below Raft + } + return m }, emptySum: 14695981039346656037, - populatedSum: 834545685817460463, + populatedSum: 6109178572734990978, }, } diff --git a/pkg/kv/kvserver/client_replica_test.go b/pkg/kv/kvserver/client_replica_test.go index 3a69c7b51c2a..e814b28eb2df 100644 --- a/pkg/kv/kvserver/client_replica_test.go +++ b/pkg/kv/kvserver/client_replica_test.go @@ -62,10 +62,103 @@ import ( "go.etcd.io/etcd/raft/v3/raftpb" ) -// TestRangeCommandClockUpdate verifies that followers update their -// clocks when executing a command, even if the lease holder's clock is far -// in the future. -func TestRangeCommandClockUpdate(t *testing.T) { +// TestReplicaClockUpdates verifies that the leaseholder and followers both +// update their clocks when executing a command to the command's timestamp, as +// long as the request timestamp is from a clock (i.e. is not synthetic). +func TestReplicaClockUpdates(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + run := func(t *testing.T, write bool, synthetic bool) { + const numNodes = 3 + const maxOffset = 100 * time.Millisecond + var manuals []*hlc.ManualClock + var clocks []*hlc.Clock + for i := 0; i < numNodes; i++ { + manuals = append(manuals, hlc.NewManualClock(1)) + clocks = append(clocks, hlc.NewClock(manuals[i].UnixNano, maxOffset)) + } + ctx := context.Background() + cfg := kvserver.TestStoreConfig(nil) + cfg.TestingKnobs.DisableReplicateQueue = true + cfg.Clock = nil + mtc := &multiTestContext{ + storeConfig: &cfg, + clocks: clocks, + // This test was written before the multiTestContext started creating many + // system ranges at startup, and hasn't been update to take that into + // account. + startWithSingleRange: true, + } + defer mtc.Stop() + mtc.Start(t, numNodes) + mtc.replicateRange(1, 1, 2) + + // Pick a timestamp in the future of all nodes by less than the + // MaxOffset. Set the synthetic flag according to the test case. + reqTS := clocks[0].Now().Add(int64(maxOffset/2), 0).WithSynthetic(synthetic) + h := roachpb.Header{Timestamp: reqTS} + + // Execute the command. + var req roachpb.Request + reqKey := roachpb.Key("a") + if write { + req = incrementArgs(reqKey, 5) + } else { + req = getArgs(reqKey) + } + if _, err := kv.SendWrappedWith(ctx, mtc.stores[0].TestSender(), h, req); err != nil { + t.Fatal(err) + } + + // If writing, wait for that command to execute on all the replicas. + // Consensus is asynchronous outside of the majority quorum, and Raft + // application is asynchronous on all nodes. + if write { + testutils.SucceedsSoon(t, func() error { + var values []int64 + for _, eng := range mtc.engines { + val, _, err := storage.MVCCGet(ctx, eng, reqKey, reqTS, storage.MVCCGetOptions{}) + if err != nil { + return err + } + values = append(values, mustGetInt(val)) + } + if !reflect.DeepEqual(values, []int64{5, 5, 5}) { + return errors.Errorf("expected (5, 5, 5), got %v", values) + } + return nil + }) + } + + // Verify that clocks were updated as expected. Check all clocks if we + // issued a write, but only the leaseholder's if we issued a read. In + // theory, we should be able to assert that _only_ the leaseholder's + // clock is updated by a read, but in practice an assertion against + // followers' clocks being updated is very difficult to make without + // being flaky because it's difficult to prevent other channels + // (background work, etc.) from carrying the clock update. + expUpdated := !synthetic + clocksToCheck := clocks + if !write { + clocksToCheck = clocks[:1] + } + for _, c := range clocksToCheck { + require.Equal(t, expUpdated, reqTS.Less(c.Now())) + } + } + + testutils.RunTrueAndFalse(t, "write", func(t *testing.T, write bool) { + testutils.RunTrueAndFalse(t, "synthetic", func(t *testing.T, synthetic bool) { + run(t, write, synthetic) + }) + }) +} + +// TestFollowersDontRejectClockUpdateWithJump verifies that followers update +// their clocks when executing a command, even if the leaseholder's clock is +// far in the future. +func TestFollowersDontRejectClockUpdateWithJump(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -128,9 +221,9 @@ func TestRangeCommandClockUpdate(t *testing.T) { } } -// TestRejectFutureCommand verifies that lease holders reject commands that -// would cause a large time jump. -func TestRejectFutureCommand(t *testing.T) { +// TestLeaseholdersRejectClockUpdateWithJump verifies that leaseholders reject +// commands that would cause a large time jump. +func TestLeaseholdersRejectClockUpdateWithJump(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -151,7 +244,7 @@ func TestRejectFutureCommand(t *testing.T) { const numCmds = 3 clockOffset := clock.MaxOffset() / numCmds for i := int64(1); i <= numCmds; i++ { - ts := ts1.Add(i*clockOffset.Nanoseconds(), 0) + ts := ts1.Add(i*clockOffset.Nanoseconds(), 0).WithSynthetic(false) if _, err := kv.SendWrappedWith(context.Background(), mtc.stores[0].TestSender(), roachpb.Header{Timestamp: ts}, incArgs); err != nil { t.Fatal(err) } @@ -163,7 +256,8 @@ func TestRejectFutureCommand(t *testing.T) { } // Once the accumulated offset reaches MaxOffset, commands will be rejected. - _, pErr := kv.SendWrappedWith(context.Background(), mtc.stores[0].TestSender(), roachpb.Header{Timestamp: ts1.Add(clock.MaxOffset().Nanoseconds()+1, 0)}, incArgs) + tsFuture := ts1.Add(clock.MaxOffset().Nanoseconds()+1, 0).WithSynthetic(false) + _, pErr := kv.SendWrappedWith(context.Background(), mtc.stores[0].TestSender(), roachpb.Header{Timestamp: tsFuture}, incArgs) if !testutils.IsPError(pErr, "remote wall time is too far ahead") { t.Fatalf("unexpected error %v", pErr) } diff --git a/pkg/kv/kvserver/rangefeed/resolved_timestamp.go b/pkg/kv/kvserver/rangefeed/resolved_timestamp.go index 120cab958357..22f22a25a57f 100644 --- a/pkg/kv/kvserver/rangefeed/resolved_timestamp.go +++ b/pkg/kv/kvserver/rangefeed/resolved_timestamp.go @@ -237,7 +237,7 @@ func (rts *resolvedTimestamp) recompute() bool { func (rts *resolvedTimestamp) assertNoChange() { before := rts.resolvedTS changed := rts.recompute() - if changed || (before != rts.resolvedTS) { + if changed || !before.EqOrdering(rts.resolvedTS) { panic(fmt.Sprintf("unexpected resolved timestamp change on recomputation, "+ "was %s, recomputed as %s", before, rts.resolvedTS)) } diff --git a/pkg/kv/kvserver/rditer/replica_data_iter_test.go b/pkg/kv/kvserver/rditer/replica_data_iter_test.go index 806809427fa2..84163a137bb3 100644 --- a/pkg/kv/kvserver/rditer/replica_data_iter_test.go +++ b/pkg/kv/kvserver/rditer/replica_data_iter_test.go @@ -170,7 +170,7 @@ func verifyRDReplicatedOnlyMVCCIter( if key := iter.Key(); !key.Equal(expectedKeys[i]) { k1, ts1 := key.Key, key.Timestamp k2, ts2 := expectedKeys[i].Key, expectedKeys[i].Timestamp - t.Errorf("%d: expected %q(%d); got %q(%d)", i, k2, ts2, k1, ts1) + t.Errorf("%d: expected %q(%s); got %q(%s)", i, k2, ts2, k1, ts1) } if reverse { i-- @@ -220,7 +220,7 @@ func verifyRDEngineIter( if !k.Equal(expectedKeys[i]) { k1, ts1 := k.Key, k.Timestamp k2, ts2 := expectedKeys[i].Key, expectedKeys[i].Timestamp - t.Errorf("%d: expected %q(%d); got %q(%d)", i, k2, ts2, k1, ts1) + t.Errorf("%d: expected %q(%s); got %q(%s)", i, k2, ts2, k1, ts1) } i++ iter.Next() diff --git a/pkg/kv/kvserver/store_test.go b/pkg/kv/kvserver/store_test.go index f0fae176a747..c00932bfac18 100644 --- a/pkg/kv/kvserver/store_test.go +++ b/pkg/kv/kvserver/store_test.go @@ -1340,7 +1340,7 @@ func TestStoreSendUpdateTime(t *testing.T) { defer stopper.Stop(context.Background()) store, _ := createTestStore(t, testStoreOpts{createSystemRanges: true}, stopper) args := getArgs([]byte("a")) - reqTS := store.cfg.Clock.Now().Add(store.cfg.Clock.MaxOffset().Nanoseconds(), 0) + reqTS := store.cfg.Clock.Now().Add(store.cfg.Clock.MaxOffset().Nanoseconds(), 0).WithSynthetic(false) _, pErr := kv.SendWrappedWith(context.Background(), store.TestSender(), roachpb.Header{Timestamp: reqTS}, &args) if pErr != nil { t.Fatal(pErr) @@ -1386,7 +1386,7 @@ func TestStoreSendWithClockOffset(t *testing.T) { store, _ := createTestStore(t, testStoreOpts{createSystemRanges: true}, stopper) args := getArgs([]byte("a")) // Set args timestamp to exceed max offset. - reqTS := store.cfg.Clock.Now().Add(store.cfg.Clock.MaxOffset().Nanoseconds()+1, 0) + reqTS := store.cfg.Clock.Now().Add(store.cfg.Clock.MaxOffset().Nanoseconds()+1, 0).WithSynthetic(false) _, pErr := kv.SendWrappedWith(context.Background(), store.TestSender(), roachpb.Header{Timestamp: reqTS}, &args) if !testutils.IsPError(pErr, "remote wall time is too far ahead") { t.Errorf("unexpected error: %v", pErr) diff --git a/pkg/kv/kvserver/tscache/cache_test.go b/pkg/kv/kvserver/tscache/cache_test.go index 2d7a163b1c87..aed5fa0131a6 100644 --- a/pkg/kv/kvserver/tscache/cache_test.go +++ b/pkg/kv/kvserver/tscache/cache_test.go @@ -545,7 +545,7 @@ func TestTimestampCacheImplsIdentical(t *testing.T) { to = nil } - ts := start.Add(int64(j), 100) + ts := start.Add(int64(j), 100).WithSynthetic(false) if useClock { ts = clock.Now() } diff --git a/pkg/roachpb/data_test.go b/pkg/roachpb/data_test.go index b6ae8d733c2e..3dfffeeab5c8 100644 --- a/pkg/roachpb/data_test.go +++ b/pkg/roachpb/data_test.go @@ -47,20 +47,19 @@ func makeClockTS(walltime int64, logical int32) hlc.ClockTimestamp { } } -func makeClockTSWithFlag(walltime int64, logical int32) hlc.ClockTimestamp { - return hlc.ClockTimestamp{ +func makeTS(walltime int64, logical int32) hlc.Timestamp { + return hlc.Timestamp{ 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 makeClockTSWithFlag(walltime, logical).ToTimestamp() +func makeSynTS(walltime int64, logical int32) hlc.Timestamp { + return hlc.Timestamp{ + WallTime: walltime, + Logical: logical, + Synthetic: true, + } } // TestKeyNext tests that the method for creating lexicographic @@ -474,17 +473,24 @@ var nonZeroTxn = Transaction{ Key: Key("foo"), ID: uuid.MakeV4(), Epoch: 2, - WriteTimestamp: makeTSWithFlag(20, 21), - MinTimestamp: makeTSWithFlag(10, 11), + WriteTimestamp: makeSynTS(20, 21), + MinTimestamp: makeSynTS(10, 11), Priority: 957356782, Sequence: 123, }, - Name: "name", - Status: COMMITTED, - LastHeartbeat: makeTSWithFlag(1, 2), - ReadTimestamp: makeTSWithFlag(20, 22), - MaxTimestamp: makeTSWithFlag(40, 41), - ObservedTimestamps: []ObservedTimestamp{{NodeID: 1, Timestamp: makeClockTSWithFlag(1, 2)}}, + Name: "name", + Status: COMMITTED, + LastHeartbeat: makeSynTS(1, 2), + ReadTimestamp: makeSynTS(20, 22), + MaxTimestamp: makeSynTS(40, 41), + ObservedTimestamps: []ObservedTimestamp{{ + NodeID: 1, + Timestamp: hlc.ClockTimestamp{ + WallTime: 1, + Logical: 2, + Synthetic: true, // normally not set, but needed for zerofields.NoZeroField + }, + }}, WriteTooOld: true, LockSpans: []Span{{Key: []byte("a"), EndKey: []byte("b")}}, InFlightWrites: []SequencedWrite{{Key: []byte("c"), Sequence: 1}}, diff --git a/pkg/roachpb/string_test.go b/pkg/roachpb/string_test.go index 6f7305104279..f48b1dc6ae36 100644 --- a/pkg/roachpb/string_test.go +++ b/pkg/roachpb/string_test.go @@ -43,10 +43,10 @@ func TestTransactionString(t *testing.T) { Status: roachpb.COMMITTED, LastHeartbeat: hlc.Timestamp{WallTime: 10, Logical: 11}, ReadTimestamp: hlc.Timestamp{WallTime: 30, Logical: 31}, - MaxTimestamp: hlc.Timestamp{WallTime: 40, Logical: 41}, + MaxTimestamp: hlc.Timestamp{WallTime: 40, Logical: 41, Synthetic: true}, } expStr := `"name" meta={id=d7aa0f5e key="foo" pri=44.58039917 epo=2 ts=0.000000020,21 min=0.000000010,11 seq=15}` + - ` lock=true stat=COMMITTED rts=0.000000030,31 wto=false max=0.000000040,41` + ` lock=true stat=COMMITTED rts=0.000000030,31 wto=false max=0.000000040,41?` if str := txn.String(); str != expStr { t.Errorf( diff --git a/pkg/storage/batch.go b/pkg/storage/batch.go index 0d75f023315d..59296ff21288 100644 --- a/pkg/storage/batch.go +++ b/pkg/storage/batch.go @@ -77,12 +77,12 @@ const ( // The keys encoded into the batch are MVCC keys: a string key with a timestamp // suffix. MVCC keys are encoded as: // -// [[[]]]<#timestamp-bytes> +// [[[]]]<#timestamp-bytes> // -// The , , and portions of the key are encoded as -// 64-bit, 32-bit, and 8-bit big-endian integers, respectively. A custom RocksDB -// comparator is used to maintain the desired ordering as these keys do not sort -// lexicographically correctly. +// The , , and portions of the key are encoded +// as 64-bit, 32-bit, and 8-bit big-endian integers, respectively. A custom +// RocksDB comparator is used to maintain the desired ordering as these keys do +// not sort lexicographically correctly. // // TODO(bilal): This struct exists mostly as a historic artifact. Transition the // remaining few test uses of this struct over to pebble.Batch, and remove it @@ -145,7 +145,7 @@ func encodeKeyToBuf(buf []byte, key MVCCKey, keyLen int) { timestampSentinelLen = 1 walltimeEncodedLen = 8 logicalEncodedLen = 4 - flagsEncodedLen = 1 + syntheticEncodedLen = 1 ) copy(buf, key.Key) @@ -157,13 +157,13 @@ func encodeKeyToBuf(buf []byte, key MVCCKey, keyLen int) { pos += timestampSentinelLen binary.BigEndian.PutUint64(buf[pos:], uint64(key.Timestamp.WallTime)) pos += walltimeEncodedLen - if key.Timestamp.Logical != 0 || key.Timestamp.Flags != 0 { + if key.Timestamp.Logical != 0 || key.Timestamp.Synthetic { binary.BigEndian.PutUint32(buf[pos:], uint32(key.Timestamp.Logical)) pos += logicalEncodedLen } - if key.Timestamp.Flags != 0 { - buf[pos] = uint8(key.Timestamp.Flags) - pos += flagsEncodedLen + if key.Timestamp.Synthetic { + buf[pos] = 1 + pos += syntheticEncodedLen } } buf[len(buf)-1] = byte(timestampLength) diff --git a/pkg/storage/batch_test.go b/pkg/storage/batch_test.go index 961ae1a1b0b9..2fc359164db4 100644 --- a/pkg/storage/batch_test.go +++ b/pkg/storage/batch_test.go @@ -1207,7 +1207,7 @@ func TestDecodeKey(t *testing.T) { {Key: []byte("foo")}, {Key: []byte("foo"), Timestamp: hlc.Timestamp{WallTime: 1}}, {Key: []byte("foo"), Timestamp: hlc.Timestamp{WallTime: 1, Logical: 1}}, - {Key: []byte("foo"), Timestamp: hlc.Timestamp{WallTime: 1, Logical: 1, Flags: 3}}, + {Key: []byte("foo"), Timestamp: hlc.Timestamp{WallTime: 1, Logical: 1, Synthetic: true}}, } for _, test := range tests { t.Run(test.String(), func(t *testing.T) { diff --git a/pkg/storage/engine_key.go b/pkg/storage/engine_key.go index c5db58045770..b851f6b7180f 100644 --- a/pkg/storage/engine_key.go +++ b/pkg/storage/engine_key.go @@ -39,11 +39,11 @@ type EngineKey struct { } const ( - engineKeyNoVersion = 0 - engineKeyVersionWallTimeLen = 8 - engineKeyVersionWallAndLogicalTimeLen = 12 - engineKeyVersionWallLogicalAndFlagsTimeLen = 13 - engineKeyVersionLockTableLen = 17 + engineKeyNoVersion = 0 + engineKeyVersionWallTimeLen = 8 + engineKeyVersionWallAndLogicalTimeLen = 12 + engineKeyVersionWallLogicalAndSyntheticTimeLen = 13 + engineKeyVersionLockTableLen = 17 ) // Format implements the fmt.Formatter interface @@ -131,7 +131,7 @@ func (k EngineKey) IsMVCCKey() bool { return l == engineKeyNoVersion || l == engineKeyVersionWallTimeLen || l == engineKeyVersionWallAndLogicalTimeLen || - l == engineKeyVersionWallLogicalAndFlagsTimeLen + l == engineKeyVersionWallLogicalAndSyntheticTimeLen } // IsLockTableKey returns true if the key can be decoded as a LockTableKey. @@ -150,10 +150,10 @@ func (k EngineKey) ToMVCCKey() (MVCCKey, error) { case engineKeyVersionWallAndLogicalTimeLen: key.Timestamp.WallTime = int64(binary.BigEndian.Uint64(k.Version[0:8])) key.Timestamp.Logical = int32(binary.BigEndian.Uint32(k.Version[8:12])) - case engineKeyVersionWallLogicalAndFlagsTimeLen: + case engineKeyVersionWallLogicalAndSyntheticTimeLen: key.Timestamp.WallTime = int64(binary.BigEndian.Uint64(k.Version[0:8])) key.Timestamp.Logical = int32(binary.BigEndian.Uint32(k.Version[8:12])) - key.Timestamp.Flags = uint32(k.Version[12]) + key.Timestamp.Synthetic = k.Version[12] != 0 default: return MVCCKey{}, errors.Errorf("version is not an encoded timestamp %x", k.Version) } diff --git a/pkg/storage/engine_key_test.go b/pkg/storage/engine_key_test.go index d6108c664429..3a68a2bfcb5d 100644 --- a/pkg/storage/engine_key_test.go +++ b/pkg/storage/engine_key_test.go @@ -87,14 +87,14 @@ func TestMVCCAndEngineKeyEncodeDecode(t *testing.T) { {key: MVCCKey{Key: roachpb.Key("a")}}, {key: MVCCKey{Key: roachpb.Key("glue"), Timestamp: hlc.Timestamp{WallTime: 89999}}}, {key: MVCCKey{Key: roachpb.Key("foo"), Timestamp: hlc.Timestamp{WallTime: 99, Logical: 45}}}, - {key: MVCCKey{Key: roachpb.Key("flags"), Timestamp: hlc.Timestamp{WallTime: 99, Logical: 45, Flags: 3}}}, + {key: MVCCKey{Key: roachpb.Key("bar"), Timestamp: hlc.Timestamp{WallTime: 99, Logical: 45, Synthetic: true}}}, } for _, test := range testCases { t.Run("", func(t *testing.T) { var encodedTS []byte if !test.key.Timestamp.IsEmpty() { var size int - if test.key.Timestamp.Flags != 0 { + if test.key.Timestamp.Synthetic { size = 13 } else if test.key.Timestamp.Logical != 0 { size = 12 @@ -106,8 +106,8 @@ func TestMVCCAndEngineKeyEncodeDecode(t *testing.T) { if test.key.Timestamp.Logical != 0 { binary.BigEndian.PutUint32(encodedTS[8:], uint32(test.key.Timestamp.Logical)) } - if test.key.Timestamp.Flags != 0 { - encodedTS[12] = uint8(test.key.Timestamp.Flags) + if test.key.Timestamp.Synthetic { + encodedTS[12] = 1 } } eKey := EngineKey{Key: test.key.Key, Version: encodedTS} diff --git a/pkg/storage/enginepb/decode.go b/pkg/storage/enginepb/decode.go index 0e321ecd7960..5857ed34f270 100644 --- a/pkg/storage/enginepb/decode.go +++ b/pkg/storage/enginepb/decode.go @@ -59,7 +59,7 @@ func DecodeKey(encodedKey []byte) (key []byte, timestamp hlc.Timestamp, _ error) case 13: timestamp.WallTime = int64(binary.BigEndian.Uint64(ts[0:8])) timestamp.Logical = int32(binary.BigEndian.Uint32(ts[8:12])) - timestamp.Flags = uint32(ts[12]) + timestamp.Synthetic = ts[12] != 0 default: return nil, timestamp, errors.Errorf( "invalid encoded mvcc key: %x bad timestamp %x", encodedKey, ts) diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go index 67d4f85dea96..ab06410a7618 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -173,18 +173,18 @@ func (k MVCCKey) Len() int { timestampSentinelLen = 1 walltimeEncodedLen = 8 logicalEncodedLen = 4 - flagsEncodedLen = 1 + syntheticEncodedLen = 1 timestampEncodedLengthLen = 1 ) n := len(k.Key) + timestampEncodedLengthLen if !k.Timestamp.IsEmpty() { n += timestampSentinelLen + walltimeEncodedLen - if k.Timestamp.Logical != 0 || k.Timestamp.Flags != 0 { + if k.Timestamp.Logical != 0 || k.Timestamp.Synthetic { n += logicalEncodedLen } - if k.Timestamp.Flags != 0 { - n += flagsEncodedLen + if k.Timestamp.Synthetic { + n += syntheticEncodedLen } } return n @@ -1677,7 +1677,7 @@ func mvccPutInternal( txnMeta = &txn.TxnMeta // If we bumped the WriteTimestamp, we update both the TxnMeta and the // MVCCMetadata.Timestamp. - if txnMeta.WriteTimestamp.Less(writeTimestamp) { + if txnMeta.WriteTimestamp != writeTimestamp { txnMetaCpy := *txnMeta txnMetaCpy.WriteTimestamp.Forward(writeTimestamp) txnMeta = &txnMetaCpy diff --git a/pkg/storage/mvcc_test.go b/pkg/storage/mvcc_test.go index da37942ff1e4..7fa1b626b0a3 100644 --- a/pkg/storage/mvcc_test.go +++ b/pkg/storage/mvcc_test.go @@ -3289,7 +3289,7 @@ func TestMVCCWriteWithDiffTimestampsAndEpochs(t *testing.T) { t.Fatal(err) } - expTS := txne2Commit.WriteTimestamp.Add(0, 1) + expTS := txne2Commit.WriteTimestamp.Next() // Now try writing an earlier value without a txn--should get WriteTooOldError. err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 1}, value4, nil) @@ -3306,7 +3306,7 @@ func TestMVCCWriteWithDiffTimestampsAndEpochs(t *testing.T) { } // Now write an intent with exactly the same timestamp--ties also get WriteTooOldError. err = MVCCPut(ctx, engine, nil, testKey1, txn2.ReadTimestamp, value5, txn2) - intentTS := expTS.Add(0, 1) + intentTS := expTS.Next() if wtoErr := (*roachpb.WriteTooOldError)(nil); !errors.As(err, &wtoErr) { t.Fatal("unexpected success") } else if wtoErr.ActualTimestamp != intentTS { diff --git a/pkg/util/hlc/BUILD.bazel b/pkg/util/hlc/BUILD.bazel index 7bc1ab84e4ce..b7f302cf39a9 100644 --- a/pkg/util/hlc/BUILD.bazel +++ b/pkg/util/hlc/BUILD.bazel @@ -19,7 +19,6 @@ go_library( "//pkg/util/timeutil", "@com_github_cockroachdb_errors//:errors", "@com_github_gogo_protobuf//proto", - "@org_golang_google_protobuf//proto", ], ) diff --git a/pkg/util/hlc/hlc_test.go b/pkg/util/hlc/hlc_test.go index 4c41ac847f2e..bf2c07b01ffe 100644 --- a/pkg/util/hlc/hlc_test.go +++ b/pkg/util/hlc/hlc_test.go @@ -54,7 +54,7 @@ func ExampleNewClock() { } if t.WallTime-s.WallTime > 0 { - log.Fatalf(context.Background(), "HLC timestamp %d deviates from physical clock %d", s, t) + log.Fatalf(context.Background(), "HLC timestamp %s deviates from physical clock %s", s, t) } if s.Logical > 0 { diff --git a/pkg/util/hlc/legacy_timestamp.pb.go b/pkg/util/hlc/legacy_timestamp.pb.go index 560ee8531492..6b5374943e47 100644 --- a/pkg/util/hlc/legacy_timestamp.pb.go +++ b/pkg/util/hlc/legacy_timestamp.pb.go @@ -31,25 +31,23 @@ type LegacyTimestamp struct { // skew)/(minimal ns between events) and nearly impossible to // overflow. Logical int32 `protobuf:"varint,2,opt,name=logical" json:"logical"` - // A collection of bit flags that provide details about the timestamp - // and its meaning. The data type is a uint32, but the number of flags - // is limited to 8 so that the flags can be encoded into a single byte. + // Indicates that the Timestamp did not come from an HLC clock somewhere + // in the system and, therefore, does not have the ability to update a + // peer's HLC clock. If set to true, the "synthetic timestamp" may be + // arbitrarily disconnected from real time. // - // Flags do not affect the sort order of Timestamps. However, they are - // considered when performing structural equality checks (e.g. using the - // == operator). Consider use of the EqOrdering method when testing for - // equality. + // See the commentary on Timestamp.synthetic for more information. // - // The field is nullable so that it is not serialized when no flags are - // set. This ensures that the timestamp encoding does not change across - // nodes that are and are not aware of this field. - Flags *uint32 `protobuf:"varint,3,opt,name=flags" json:"flags,omitempty"` + // The field is nullable so that it is not serialized when set to false. + // This ensures that the timestamp encoding does not change across nodes + // that are and are not aware of this field. + Synthetic *bool `protobuf:"varint,3,opt,name=synthetic" json:"synthetic,omitempty"` } func (m *LegacyTimestamp) Reset() { *m = LegacyTimestamp{} } func (*LegacyTimestamp) ProtoMessage() {} func (*LegacyTimestamp) Descriptor() ([]byte, []int) { - return fileDescriptor_legacy_timestamp_d72283f54eaf58e6, []int{0} + return fileDescriptor_legacy_timestamp_7e009b38ac9b864a, []int{0} } func (m *LegacyTimestamp) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -102,13 +100,13 @@ func (this *LegacyTimestamp) Equal(that interface{}) bool { if this.Logical != that1.Logical { return false } - if this.Flags != nil && that1.Flags != nil { - if *this.Flags != *that1.Flags { + if this.Synthetic != nil && that1.Synthetic != nil { + if *this.Synthetic != *that1.Synthetic { return false } - } else if this.Flags != nil { + } else if this.Synthetic != nil { return false - } else if that1.Flags != nil { + } else if that1.Synthetic != nil { return false } return true @@ -134,10 +132,15 @@ func (m *LegacyTimestamp) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x10 i++ i = encodeVarintLegacyTimestamp(dAtA, i, uint64(m.Logical)) - if m.Flags != nil { + if m.Synthetic != nil { dAtA[i] = 0x18 i++ - i = encodeVarintLegacyTimestamp(dAtA, i, uint64(*m.Flags)) + if *m.Synthetic { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i++ } return i, nil } @@ -162,8 +165,8 @@ func NewPopulatedLegacyTimestamp(r randyLegacyTimestamp, easy bool) *LegacyTimes this.Logical *= -1 } if r.Intn(10) != 0 { - v1 := uint32(r.Uint32()) - this.Flags = &v1 + v1 := bool(bool(r.Intn(2) == 0)) + this.Synthetic = &v1 } if !easy && r.Intn(10) != 0 { } @@ -250,8 +253,8 @@ func (m *LegacyTimestamp) Size() (n int) { _ = l n += 1 + sovLegacyTimestamp(uint64(m.WallTime)) n += 1 + sovLegacyTimestamp(uint64(m.Logical)) - if m.Flags != nil { - n += 1 + sovLegacyTimestamp(uint64(*m.Flags)) + if m.Synthetic != nil { + n += 2 } return n } @@ -338,9 +341,9 @@ func (m *LegacyTimestamp) Unmarshal(dAtA []byte) error { } case 3: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Flags", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Synthetic", wireType) } - var v uint32 + var v int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowLegacyTimestamp @@ -350,12 +353,13 @@ func (m *LegacyTimestamp) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - v |= (uint32(b) & 0x7F) << shift + v |= (int(b) & 0x7F) << shift if b < 0x80 { break } } - m.Flags = &v + b := bool(v != 0) + m.Synthetic = &b default: iNdEx = preIndex skippy, err := skipLegacyTimestamp(dAtA[iNdEx:]) @@ -483,23 +487,24 @@ var ( ) func init() { - proto.RegisterFile("util/hlc/legacy_timestamp.proto", fileDescriptor_legacy_timestamp_d72283f54eaf58e6) + proto.RegisterFile("util/hlc/legacy_timestamp.proto", fileDescriptor_legacy_timestamp_7e009b38ac9b864a) } -var fileDescriptor_legacy_timestamp_d72283f54eaf58e6 = []byte{ - // 221 bytes of a gzipped FileDescriptorProto +var fileDescriptor_legacy_timestamp_7e009b38ac9b864a = []byte{ + // 226 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x92, 0x2f, 0x2d, 0xc9, 0xcc, 0xd1, 0xcf, 0xc8, 0x49, 0xd6, 0xcf, 0x49, 0x4d, 0x4f, 0x4c, 0xae, 0x8c, 0x2f, 0xc9, 0xcc, 0x4d, 0x2d, 0x2e, 0x49, 0xcc, 0x2d, 0xd0, 0x2b, 0x28, 0xca, 0x2f, 0xc9, 0x17, 0x12, 0x4a, 0xce, 0x4f, 0xce, 0x2e, 0xca, 0x4f, 0x4c, 0xce, 0xd0, 0x03, 0x29, 0xd5, 0xcb, 0xc8, 0x49, 0x96, 0x12, 0x49, - 0xcf, 0x4f, 0xcf, 0x07, 0x4b, 0xeb, 0x83, 0x58, 0x10, 0x95, 0x4a, 0x15, 0x5c, 0xfc, 0x3e, 0x60, - 0x33, 0x42, 0x60, 0x46, 0x08, 0x29, 0x72, 0x71, 0x96, 0x27, 0xe6, 0xe4, 0x80, 0x0d, 0x95, 0x60, - 0x54, 0x60, 0xd4, 0x60, 0x76, 0x62, 0x39, 0x71, 0x4f, 0x9e, 0x21, 0x88, 0x03, 0x24, 0x0c, 0x52, - 0x27, 0x24, 0xc7, 0xc5, 0x9e, 0x93, 0x9f, 0x9e, 0x99, 0x9c, 0x98, 0x23, 0xc1, 0xa4, 0xc0, 0xa8, - 0xc1, 0x0a, 0x55, 0x00, 0x13, 0x14, 0x12, 0xe1, 0x62, 0x4d, 0xcb, 0x49, 0x4c, 0x2f, 0x96, 0x60, - 0x56, 0x60, 0xd4, 0xe0, 0x0d, 0x82, 0x70, 0xac, 0x78, 0x66, 0x2c, 0x90, 0x67, 0xd8, 0xb1, 0x40, - 0x9e, 0xf1, 0xc5, 0x02, 0x79, 0x46, 0x27, 0xd5, 0x13, 0x0f, 0xe5, 0x18, 0x4e, 0x3c, 0x92, 0x63, - 0xbc, 0xf0, 0x48, 0x8e, 0xf1, 0xc6, 0x23, 0x39, 0xc6, 0x07, 0x8f, 0xe4, 0x18, 0x27, 0x3c, 0x96, - 0x63, 0xb8, 0xf0, 0x58, 0x8e, 0xe1, 0xc6, 0x63, 0x39, 0x86, 0x28, 0xe6, 0x8c, 0x9c, 0x64, 0x40, - 0x00, 0x00, 0x00, 0xff, 0xff, 0x05, 0x5c, 0x3e, 0x65, 0xec, 0x00, 0x00, 0x00, + 0xcf, 0x4f, 0xcf, 0x07, 0x4b, 0xeb, 0x83, 0x58, 0x10, 0x95, 0x4a, 0x0d, 0x8c, 0x5c, 0xfc, 0x3e, + 0x60, 0x43, 0x42, 0x60, 0x66, 0x08, 0x29, 0x72, 0x71, 0x96, 0x27, 0xe6, 0xe4, 0x80, 0x4d, 0x95, + 0x60, 0x54, 0x60, 0xd4, 0x60, 0x76, 0x62, 0x39, 0x71, 0x4f, 0x9e, 0x21, 0x88, 0x03, 0x24, 0x0c, + 0x52, 0x27, 0x24, 0xc7, 0xc5, 0x9e, 0x93, 0x9f, 0x9e, 0x99, 0x9c, 0x98, 0x23, 0xc1, 0xa4, 0xc0, + 0xa8, 0xc1, 0x0a, 0x55, 0x00, 0x13, 0x14, 0x92, 0xe1, 0xe2, 0x2c, 0xae, 0xcc, 0x2b, 0xc9, 0x48, + 0x2d, 0xc9, 0x4c, 0x96, 0x60, 0x56, 0x60, 0xd4, 0xe0, 0x08, 0x42, 0x08, 0x58, 0xf1, 0xcc, 0x58, + 0x20, 0xcf, 0xb0, 0x63, 0x81, 0x3c, 0xe3, 0x8b, 0x05, 0xf2, 0x8c, 0x4e, 0xaa, 0x27, 0x1e, 0xca, + 0x31, 0x9c, 0x78, 0x24, 0xc7, 0x78, 0xe1, 0x91, 0x1c, 0xe3, 0x8d, 0x47, 0x72, 0x8c, 0x0f, 0x1e, + 0xc9, 0x31, 0x4e, 0x78, 0x2c, 0xc7, 0x70, 0xe1, 0xb1, 0x1c, 0xc3, 0x8d, 0xc7, 0x72, 0x0c, 0x51, + 0xcc, 0x19, 0x39, 0xc9, 0x80, 0x00, 0x00, 0x00, 0xff, 0xff, 0xe9, 0x42, 0x31, 0x94, 0xf5, 0x00, + 0x00, 0x00, } diff --git a/pkg/util/hlc/legacy_timestamp.proto b/pkg/util/hlc/legacy_timestamp.proto index ea90e4182658..588cde406e09 100644 --- a/pkg/util/hlc/legacy_timestamp.proto +++ b/pkg/util/hlc/legacy_timestamp.proto @@ -30,17 +30,15 @@ message LegacyTimestamp { // skew)/(minimal ns between events) and nearly impossible to // overflow. optional int32 logical = 2 [(gogoproto.nullable) = false]; - // A collection of bit flags that provide details about the timestamp - // and its meaning. The data type is a uint32, but the number of flags - // is limited to 8 so that the flags can be encoded into a single byte. + // Indicates that the Timestamp did not come from an HLC clock somewhere + // in the system and, therefore, does not have the ability to update a + // peer's HLC clock. If set to true, the "synthetic timestamp" may be + // arbitrarily disconnected from real time. // - // Flags do not affect the sort order of Timestamps. However, they are - // considered when performing structural equality checks (e.g. using the - // == operator). Consider use of the EqOrdering method when testing for - // equality. + // See the commentary on Timestamp.synthetic for more information. // - // The field is nullable so that it is not serialized when no flags are - // set. This ensures that the timestamp encoding does not change across - // nodes that are and are not aware of this field. - optional uint32 flags = 3; + // The field is nullable so that it is not serialized when set to false. + // This ensures that the timestamp encoding does not change across nodes + // that are and are not aware of this field. + optional bool synthetic = 3; } diff --git a/pkg/util/hlc/timestamp.go b/pkg/util/hlc/timestamp.go index f3a85d094fb0..998201ceda3f 100644 --- a/pkg/util/hlc/timestamp.go +++ b/pkg/util/hlc/timestamp.go @@ -15,13 +15,11 @@ import ( "math" "regexp" "strconv" - "strings" "time" "unsafe" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/errors" - "google.golang.org/protobuf/proto" ) // Timestamp constant values. @@ -54,17 +52,6 @@ func (t Timestamp) LessEq(s Timestamp) bool { return t.WallTime < s.WallTime || (t.WallTime == s.WallTime && t.Logical <= s.Logical) } -var flagStrings = map[TimestampFlag]string{ - TimestampFlag_SYNTHETIC: "syn", -} -var flagStringsInverted = func() map[string]TimestampFlag { - m := make(map[string]TimestampFlag) - for k, v := range flagStrings { - m[v] = k - } - return m -}() - // String implements the fmt.Stringer interface. func (t Timestamp) String() string { // The following code was originally written as @@ -72,7 +59,7 @@ func (t Timestamp) String() string { // The main problem with the original code was that it would put // a negative sign in the middle (after the decimal point) if // the value happened to be negative. - buf := make([]byte, 0, 20) + buf := make([]byte, 0, 21) w := t.WallTime if w == 0 { @@ -114,20 +101,8 @@ func (t Timestamp) String() string { } buf = strconv.AppendInt(buf, int64(t.Logical), 10) - if t.Flags != 0 { - buf = append(buf, '[') - comma := false - for i := 0; i < 8; i++ { - f := TimestampFlag(1 << i) - if t.IsFlagSet(f) { - if comma { - buf = append(buf, ',') - } - comma = true - buf = append(buf, flagStrings[f]...) - } - } - buf = append(buf, ']') + if t.Synthetic { + buf = append(buf, '?') } return *(*string)(unsafe.Pointer(&buf)) @@ -138,12 +113,12 @@ func (Timestamp) SafeValue() {} var ( timestampRegexp = regexp.MustCompile( - `^(?P-)?(?P\d{1,19})(?:\.(?P\d{1,20}))?(?:,(?P-?\d{1,10}))?(?:\[(?P[\w,]+)\])?$`) - signSubexp = 1 - secsSubexp = 2 - nanosSubexp = 3 - logicalSubexp = 4 - flagsSubexp = 5 + `^(?P-)?(?P\d{1,19})(?:\.(?P\d{1,20}))?(?:,(?P-?\d{1,10}))?(?P\?)?$`) + signSubexp = 1 + secsSubexp = 2 + nanosSubexp = 3 + logicalSubexp = 4 + syntheticSubexp = 5 ) // ParseTimestamp attempts to parse the string generated from @@ -180,25 +155,11 @@ func ParseTimestamp(str string) (_ Timestamp, err error) { return Timestamp{}, err } } + synthetic := matches[syntheticSubexp] != "" t := Timestamp{ - WallTime: wallTime, - Logical: int32(logical), - } - if flagsMatch := matches[flagsSubexp]; flagsMatch != "" { - flagStrs := strings.Split(flagsMatch, ",") - for _, flagStr := range flagStrs { - if flagStr == "" { - return Timestamp{}, errors.Errorf("empty flag provided") - } - flagMatch, ok := flagStringsInverted[flagStr] - if !ok { - return Timestamp{}, errors.Errorf("unknown flag %q provided", flagStr) - } - if t.IsFlagSet(flagMatch) { - return Timestamp{}, errors.Errorf("duplicate flag %q provided", flagStr) - } - t = t.SetFlag(flagMatch) - } + WallTime: wallTime, + Logical: int32(logical), + Synthetic: synthetic, } return t, nil } @@ -213,32 +174,32 @@ func (t Timestamp) IsEmpty() bool { return t == Timestamp{} } -// IsFlagSet returns whether the specified flag is set on the timestamp. -func (t Timestamp) IsFlagSet(f TimestampFlag) bool { - return t.Flags&uint32(f) != 0 -} - // Add returns a timestamp with the WallTime and Logical components increased. // wallTime is expressed in nanos. // // TODO(nvanbenschoten): consider an AddNanos method that takes a time.Duration. func (t Timestamp) Add(wallTime int64, logical int32) Timestamp { - return Timestamp{ - WallTime: t.WallTime + wallTime, - Logical: t.Logical + logical, - Flags: t.Flags, + s := Timestamp{ + WallTime: t.WallTime + wallTime, + Logical: t.Logical + logical, + Synthetic: t.Synthetic, } + // TODO(nvanbenschoten): adding to a timestamp should make it synthetic. + // This breaks a number of tests, so make this change in a separate PR. We + // might also want to wait until we've migrated in the Synthetic flag so we + // don't risk setting it when doing so could cause complications in a mixed + // version cluster. + // + // if t.Less(s) { + // // Adding a positive value to a Timestamp adds the Synthetic flag. + // s.Synthetic = true + // } + return s } -// SetFlag returns a timestamp with the specified flag set. -func (t Timestamp) SetFlag(f TimestampFlag) Timestamp { - t.Flags = t.Flags | uint32(f) - return t -} - -// ClearFlag returns a timestamp with the specified flag cleared. -func (t Timestamp) ClearFlag(f TimestampFlag) Timestamp { - t.Flags = t.Flags &^ uint32(f) +// WithSynthetic returns a timestamp with the Synthetic flag set to val. +func (t Timestamp) WithSynthetic(val bool) Timestamp { + t.Synthetic = val return t } @@ -254,14 +215,14 @@ func (t Timestamp) Next() Timestamp { panic("cannot take the next value to a max timestamp") } return Timestamp{ - WallTime: t.WallTime + 1, - Flags: t.Flags, + WallTime: t.WallTime + 1, + Synthetic: t.Synthetic, } } return Timestamp{ - WallTime: t.WallTime, - Logical: t.Logical + 1, - Flags: t.Flags, + WallTime: t.WallTime, + Logical: t.Logical + 1, + Synthetic: t.Synthetic, } } @@ -269,15 +230,15 @@ func (t Timestamp) Next() Timestamp { func (t Timestamp) Prev() Timestamp { if t.Logical > 0 { return Timestamp{ - WallTime: t.WallTime, - Logical: t.Logical - 1, - Flags: t.Flags, + WallTime: t.WallTime, + Logical: t.Logical - 1, + Synthetic: t.Synthetic, } } else if t.WallTime > 0 { return Timestamp{ - WallTime: t.WallTime - 1, - Logical: math.MaxInt32, - Flags: t.Flags, + WallTime: t.WallTime - 1, + Logical: math.MaxInt32, + Synthetic: t.Synthetic, } } panic("cannot take the previous value to a zero timestamp") @@ -289,15 +250,15 @@ func (t Timestamp) Prev() Timestamp { func (t Timestamp) FloorPrev() Timestamp { if t.Logical > 0 { return Timestamp{ - WallTime: t.WallTime, - Logical: t.Logical - 1, - Flags: t.Flags, + WallTime: t.WallTime, + Logical: t.Logical - 1, + Synthetic: t.Synthetic, } } else if t.WallTime > 0 { return Timestamp{ - WallTime: t.WallTime - 1, - Logical: 0, - Flags: t.Flags, + WallTime: t.WallTime - 1, + Logical: 0, + Synthetic: t.Synthetic, } } panic("cannot take the previous value to a zero timestamp") @@ -310,10 +271,8 @@ func (t *Timestamp) Forward(s Timestamp) bool { if t.Less(s) { *t = s return true - } else if t.EqOrdering(s) && onlyLeftSynthetic(*t, s) { - // If the times are equal but t is synthetic while s is not, remove the - // synthtic flag but continue to return false. - *t = t.ClearFlag(TimestampFlag_SYNTHETIC) + } else if t.EqOrdering(s) { + t.Synthetic = bothSynthetic(*t, s) } return false } @@ -321,20 +280,15 @@ func (t *Timestamp) Forward(s Timestamp) bool { // Backward replaces the receiver with the argument, if that moves it backwards // in time. func (t *Timestamp) Backward(s Timestamp) { + syn := bothSynthetic(*t, s) if s.Less(*t) { - // Replace t with s. If s is synthetic while t is not, remove the - // synthtic flag. - if onlyLeftSynthetic(s, *t) { - s = s.ClearFlag(TimestampFlag_SYNTHETIC) - } *t = s - } else if onlyLeftSynthetic(*t, s) { - *t = t.ClearFlag(TimestampFlag_SYNTHETIC) } + t.Synthetic = syn } -func onlyLeftSynthetic(l, r Timestamp) bool { - return l.IsFlagSet(TimestampFlag_SYNTHETIC) && !r.IsFlagSet(TimestampFlag_SYNTHETIC) +func bothSynthetic(l, r Timestamp) bool { + return l.Synthetic && r.Synthetic } // GoTime converts the timestamp to a time.Time. @@ -342,22 +296,24 @@ func (t Timestamp) GoTime() time.Time { return timeutil.Unix(0, t.WallTime) } +var trueBool = true + // ToLegacyTimestamp converts a Timestamp to a LegacyTimestamp. func (t Timestamp) ToLegacyTimestamp() LegacyTimestamp { - var flags *uint32 - if t.Flags != 0 { - flags = proto.Uint32(t.Flags) + var synthetic *bool + if t.Synthetic { + synthetic = &trueBool } - return LegacyTimestamp{WallTime: t.WallTime, Logical: t.Logical, Flags: flags} + return LegacyTimestamp{WallTime: t.WallTime, Logical: t.Logical, Synthetic: synthetic} } // ToTimestamp converts a LegacyTimestamp to a Timestamp. func (t LegacyTimestamp) ToTimestamp() Timestamp { - var flags uint32 - if t.Flags != nil { - flags = *t.Flags + var synthetic bool + if t.Synthetic != nil { + synthetic = *t.Synthetic } - return Timestamp{WallTime: t.WallTime, Logical: t.Logical, Flags: flags} + return Timestamp{WallTime: t.WallTime, Logical: t.Logical, Synthetic: synthetic} } // EqOrdering returns whether the receiver sorts equally to the parameter. @@ -388,7 +344,7 @@ 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) { + if t.Synthetic { return ClockTimestamp{}, false } return ClockTimestamp(t), true @@ -398,12 +354,15 @@ func (t Timestamp) TryToClockTimestamp() (ClockTimestamp, bool) { // 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. + t.Synthetic = false return ClockTimestamp(t) } // ToTimestamp upcasts a ClockTimestamp into a Timestamp. func (t ClockTimestamp) ToTimestamp() Timestamp { + if t.Synthetic { + panic("ClockTimestamp with Synthetic flag set") + } return Timestamp(t) } diff --git a/pkg/util/hlc/timestamp.pb.go b/pkg/util/hlc/timestamp.pb.go index 65050bbff5b5..4c329209e3e4 100644 --- a/pkg/util/hlc/timestamp.pb.go +++ b/pkg/util/hlc/timestamp.pb.go @@ -20,43 +20,6 @@ var _ = math.Inf // proto package needs to be updated. const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package -// TimestampFlag is used to provide extra classification for Timestamps. -type TimestampFlag int32 - -const ( - TimestampFlag_UNKNOWN TimestampFlag = 0 - // A synthetic timestamp is defined as a timestamp that makes no claim - // about the value of clocks in the system. While standard timestamps - // are pulled from HLC clocks and indicate that some node in the system - // has a clock with a reading equal to or above its value, a synthetic - // timestamp makes no such indication. - // - // Synthetic timestamps are central to non-blocking transactions, which - // write at "future timestamps". They are also used to disconnect some - // committed versions from observed timestamps, where they indicate that - // versions were moved from the timestamp at which they were originally - // written. Only synthetic timestamps require observing the full - // uncertainty interval, whereas readings off the leaseholders's clock - // can tighten it for non-synthetic versions. - TimestampFlag_SYNTHETIC TimestampFlag = 1 -) - -var TimestampFlag_name = map[int32]string{ - 0: "UNKNOWN", - 1: "SYNTHETIC", -} -var TimestampFlag_value = map[string]int32{ - "UNKNOWN": 0, - "SYNTHETIC": 1, -} - -func (x TimestampFlag) String() string { - return proto.EnumName(TimestampFlag_name, int32(x)) -} -func (TimestampFlag) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_timestamp_2a42725d81a17263, []int{0} -} - // Timestamp represents a state of the hybrid logical clock. type Timestamp struct { // Holds a wall time, typically a unix epoch time expressed in @@ -74,29 +37,36 @@ type Timestamp struct { // methods on Timestamp, which ensure that the synthetic flag is updated // appropriately. Logical int32 `protobuf:"varint,2,opt,name=logical,proto3" json:"logical,omitempty"` - // A collection of bit flags that provide details about the timestamp - // and its meaning. The data type is a uint32, but the number of flags - // is limited to 8 so that the flags can be encoded into a single byte. - // - // Flags do not affect the sort order of Timestamps. However, they are - // considered when performing structural equality checks (e.g. using the - // == operator). Consider use of the EqOrdering method when testing for - // equality. + // Indicates that the Timestamp did not come from an HLC clock somewhere + // in the system and, therefore, does not have the ability to update a + // peer's HLC clock. If set to true, the "synthetic timestamp" may be + // arbitrarily disconnected from real time. // - // 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. + // The flag serves as the dynamically typed version of a ClockTimestamp + // (but inverted). Only Timestamps with this flag set to false can be + // downcast to a ClockTimestamp successfully (see TryToClockTimestamp). // - // Should look like: - // bool synthetic = 3; + // Synthetic timestamps with this flag set to true are central to + // non-blocking transactions, which write "into the future". Setting the + // flag to true is also used to disconnect some committed MVCC versions + // from observed timestamps by indicating that those versions were moved + // from the timestamp at which they were originally written. Committed + // MVCC versions with synthetic timestamps require observing the full + // uncertainty interval, whereas readings off the leaseholders's clock + // can tighten the uncertainty interval that is applied to MVCC versions + // with clock timestamp. // - Flags uint32 `protobuf:"varint,3,opt,name=flags,proto3" json:"flags,omitempty"` + // This flag does not affect the sort order of Timestamps. However, it + // is considered when performing structural equality checks (e.g. using + // the == operator). Consider use of the EqOrdering method when testing + // for equality. + Synthetic bool `protobuf:"varint,3,opt,name=synthetic,proto3" json:"synthetic,omitempty"` } func (m *Timestamp) Reset() { *m = Timestamp{} } func (*Timestamp) ProtoMessage() {} func (*Timestamp) Descriptor() ([]byte, []int) { - return fileDescriptor_timestamp_2a42725d81a17263, []int{0} + return fileDescriptor_timestamp_018bda073af0c1c7, []int{0} } func (m *Timestamp) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -123,7 +93,6 @@ var xxx_messageInfo_Timestamp proto.InternalMessageInfo func init() { proto.RegisterType((*Timestamp)(nil), "cockroach.util.hlc.Timestamp") - proto.RegisterEnum("cockroach.util.hlc.TimestampFlag", TimestampFlag_name, TimestampFlag_value) } func (this *Timestamp) Equal(that interface{}) bool { if that == nil { @@ -150,7 +119,7 @@ func (this *Timestamp) Equal(that interface{}) bool { if this.Logical != that1.Logical { return false } - if this.Flags != that1.Flags { + if this.Synthetic != that1.Synthetic { return false } return true @@ -180,10 +149,15 @@ func (m *Timestamp) MarshalTo(dAtA []byte) (int, error) { i++ i = encodeVarintTimestamp(dAtA, i, uint64(m.Logical)) } - if m.Flags != 0 { + if m.Synthetic { dAtA[i] = 0x18 i++ - i = encodeVarintTimestamp(dAtA, i, uint64(m.Flags)) + if m.Synthetic { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i++ } return i, nil } @@ -207,7 +181,7 @@ func NewPopulatedTimestamp(r randyTimestamp, easy bool) *Timestamp { if r.Intn(2) == 0 { this.Logical *= -1 } - this.Flags = uint32(r.Uint32()) + this.Synthetic = bool(bool(r.Intn(2) == 0)) if !easy && r.Intn(10) != 0 { } return this @@ -297,8 +271,8 @@ func (m *Timestamp) Size() (n int) { if m.Logical != 0 { n += 1 + sovTimestamp(uint64(m.Logical)) } - if m.Flags != 0 { - n += 1 + sovTimestamp(uint64(m.Flags)) + if m.Synthetic { + n += 2 } return n } @@ -385,9 +359,9 @@ func (m *Timestamp) Unmarshal(dAtA []byte) error { } case 3: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field Flags", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field Synthetic", wireType) } - m.Flags = 0 + var v int for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowTimestamp @@ -397,11 +371,12 @@ func (m *Timestamp) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.Flags |= (uint32(b) & 0x7F) << shift + v |= (int(b) & 0x7F) << shift if b < 0x80 { break } } + m.Synthetic = bool(v != 0) default: iNdEx = preIndex skippy, err := skipTimestamp(dAtA[iNdEx:]) @@ -529,25 +504,23 @@ var ( ) func init() { - proto.RegisterFile("util/hlc/timestamp.proto", fileDescriptor_timestamp_2a42725d81a17263) + proto.RegisterFile("util/hlc/timestamp.proto", fileDescriptor_timestamp_018bda073af0c1c7) } -var fileDescriptor_timestamp_2a42725d81a17263 = []byte{ - // 247 bytes of a gzipped FileDescriptorProto +var fileDescriptor_timestamp_018bda073af0c1c7 = []byte{ + // 213 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, 0x28, 0xca, 0x2f, 0xc9, 0x17, 0x12, 0x4a, 0xce, 0x4f, 0xce, 0x2e, 0xca, 0x4f, 0x4c, 0xce, 0xd0, 0x03, 0xa9, 0xd1, 0xcb, 0xc8, 0x49, 0x96, 0x12, 0x49, 0xcf, 0x4f, 0xcf, 0x07, 0x4b, 0xeb, 0x83, - 0x58, 0x10, 0x95, 0x4a, 0x69, 0x5c, 0x9c, 0x21, 0x30, 0xcd, 0x42, 0xd2, 0x5c, 0x9c, 0xe5, 0x89, + 0x58, 0x10, 0x95, 0x4a, 0x79, 0x5c, 0x9c, 0x21, 0x30, 0xcd, 0x42, 0xd2, 0x5c, 0x9c, 0xe5, 0x89, 0x39, 0x39, 0xf1, 0x20, 0xe3, 0x24, 0x18, 0x15, 0x18, 0x35, 0x98, 0x83, 0x38, 0x40, 0x02, 0x20, 0x15, 0x42, 0x12, 0x5c, 0xec, 0x39, 0xf9, 0xe9, 0x99, 0xc9, 0x89, 0x39, 0x12, 0x4c, 0x0a, 0x8c, - 0x1a, 0xac, 0x41, 0x30, 0xae, 0x90, 0x08, 0x17, 0x6b, 0x5a, 0x4e, 0x62, 0x7a, 0xb1, 0x04, 0xb3, - 0x02, 0xa3, 0x06, 0x6f, 0x10, 0x84, 0x63, 0xc5, 0x33, 0x63, 0x81, 0x3c, 0xc3, 0x8e, 0x05, 0xf2, - 0x8c, 0x2f, 0x16, 0xc8, 0x33, 0x6a, 0x69, 0x73, 0xf1, 0xc2, 0xed, 0x71, 0xcb, 0x49, 0x4c, 0x17, - 0xe2, 0xe6, 0x62, 0x0f, 0xf5, 0xf3, 0xf6, 0xf3, 0x0f, 0xf7, 0x13, 0x60, 0x10, 0xe2, 0xe5, 0xe2, - 0x0c, 0x8e, 0xf4, 0x0b, 0xf1, 0x70, 0x0d, 0xf1, 0x74, 0x16, 0x60, 0x74, 0x52, 0x3d, 0xf1, 0x50, - 0x8e, 0xe1, 0xc4, 0x23, 0x39, 0xc6, 0x0b, 0x8f, 0xe4, 0x18, 0x6f, 0x3c, 0x92, 0x63, 0x7c, 0xf0, - 0x48, 0x8e, 0x71, 0xc2, 0x63, 0x39, 0x86, 0x0b, 0x8f, 0xe5, 0x18, 0x6e, 0x3c, 0x96, 0x63, 0x88, - 0x62, 0xce, 0xc8, 0x49, 0x4e, 0x62, 0x03, 0x7b, 0xc1, 0x18, 0x10, 0x00, 0x00, 0xff, 0xff, 0xf4, - 0x8d, 0x21, 0xb8, 0x08, 0x01, 0x00, 0x00, + 0x1a, 0xac, 0x41, 0x30, 0xae, 0x90, 0x0c, 0x17, 0x67, 0x71, 0x65, 0x5e, 0x49, 0x46, 0x6a, 0x49, + 0x66, 0xb2, 0x04, 0xb3, 0x02, 0xa3, 0x06, 0x47, 0x10, 0x42, 0xc0, 0x8a, 0x67, 0xc6, 0x02, 0x79, + 0x86, 0x1d, 0x0b, 0xe4, 0x19, 0x5f, 0x2c, 0x90, 0x67, 0x74, 0x52, 0x3d, 0xf1, 0x50, 0x8e, 0xe1, + 0xc4, 0x23, 0x39, 0xc6, 0x0b, 0x8f, 0xe4, 0x18, 0x6f, 0x3c, 0x92, 0x63, 0x7c, 0xf0, 0x48, 0x8e, + 0x71, 0xc2, 0x63, 0x39, 0x86, 0x0b, 0x8f, 0xe5, 0x18, 0x6e, 0x3c, 0x96, 0x63, 0x88, 0x62, 0xce, + 0xc8, 0x49, 0x4e, 0x62, 0x03, 0xbb, 0xce, 0x18, 0x10, 0x00, 0x00, 0xff, 0xff, 0xa2, 0x14, 0x0e, + 0x21, 0xe3, 0x00, 0x00, 0x00, } diff --git a/pkg/util/hlc/timestamp.proto b/pkg/util/hlc/timestamp.proto index e3711d78c635..0ee60edcaba4 100644 --- a/pkg/util/hlc/timestamp.proto +++ b/pkg/util/hlc/timestamp.proto @@ -36,43 +36,28 @@ message Timestamp { // methods on Timestamp, which ensure that the synthetic flag is updated // appropriately. int32 logical = 2; - // A collection of bit flags that provide details about the timestamp - // and its meaning. The data type is a uint32, but the number of flags - // is limited to 8 so that the flags can be encoded into a single byte. + // Indicates that the Timestamp did not come from an HLC clock somewhere + // in the system and, therefore, does not have the ability to update a + // peer's HLC clock. If set to true, the "synthetic timestamp" may be + // arbitrarily disconnected from real time. // - // Flags do not affect the sort order of Timestamps. However, they are - // considered when performing structural equality checks (e.g. using the - // == operator). Consider use of the EqOrdering method when testing for - // equality. + // The flag serves as the dynamically typed version of a ClockTimestamp + // (but inverted). Only Timestamps with this flag set to false can be + // downcast to a ClockTimestamp successfully (see TryToClockTimestamp). // - // 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; -} - -// TimestampFlag is used to provide extra classification for Timestamps. -enum TimestampFlag { - UNKNOWN = 0x00; - // A synthetic timestamp is defined as a timestamp that makes no claim - // about the value of clocks in the system. While standard timestamps - // are pulled from HLC clocks and indicate that some node in the system - // has a clock with a reading equal to or above its value, a synthetic - // timestamp makes no such indication. - // - // Synthetic timestamps are central to non-blocking transactions, which - // write at "future timestamps". They are also used to disconnect some - // committed versions from observed timestamps, where they indicate that - // versions were moved from the timestamp at which they were originally - // written. Only synthetic timestamps require observing the full + // Synthetic timestamps with this flag set to true are central to + // non-blocking transactions, which write "into the future". Setting the + // flag to true is also used to disconnect some committed MVCC versions + // from observed timestamps by indicating that those versions were moved + // from the timestamp at which they were originally written. Committed + // MVCC versions with synthetic timestamps require observing the full // uncertainty interval, whereas readings off the leaseholders's clock - // can tighten it for non-synthetic versions. - SYNTHETIC = 0x01; - // ... = 0x02; - // ... = 0x04; - // max = 0x80; + // can tighten the uncertainty interval that is applied to MVCC versions + // with clock timestamp. + // + // This flag does not affect the sort order of Timestamps. However, it + // is considered when performing structural equality checks (e.g. using + // the == operator). Consider use of the EqOrdering method when testing + // for equality. + bool synthetic = 3; } diff --git a/pkg/util/hlc/timestamp_test.go b/pkg/util/hlc/timestamp_test.go index f7f795a289cc..4e4433874f58 100644 --- a/pkg/util/hlc/timestamp_test.go +++ b/pkg/util/hlc/timestamp_test.go @@ -18,18 +18,11 @@ import ( ) func makeTS(walltime int64, logical int32) Timestamp { - return Timestamp{ - WallTime: walltime, - Logical: logical, - } + return Timestamp{WallTime: walltime, Logical: logical} } -func makeTSWithFlags(walltime int64, logical int32, flags TimestampFlag) Timestamp { - return Timestamp{ - WallTime: walltime, - Logical: logical, - Flags: uint32(flags), - } +func makeSynTS(walltime int64, logical int32) Timestamp { + return makeTS(walltime, logical).WithSynthetic(true) } func TestEqOrdering(t *testing.T) { @@ -46,7 +39,7 @@ func TestEqOrdering(t *testing.T) { if a.EqOrdering(b) { t.Errorf("expected %+v != %+v", b, a) } - b = makeTSWithFlags(1, 1, 3) + b = makeSynTS(1, 1) if !a.EqOrdering(b) { t.Errorf("expected %+v == %+v", b, a) } @@ -66,7 +59,7 @@ func TestLess(t *testing.T) { if !b.Less(a) { t.Errorf("expected %+v < %+v", b, a) } - b = makeTSWithFlags(1, 1, 3) + b = makeSynTS(1, 1) if a.Less(b) || b.Less(a) { t.Errorf("expected %+v == %+v", a, b) } @@ -86,34 +79,23 @@ func TestLessEq(t *testing.T) { if !b.LessEq(a) || a.LessEq(b) { t.Errorf("expected %+v < %+v", b, a) } - b = makeTSWithFlags(1, 1, 3) + b = makeSynTS(1, 1) if !a.LessEq(b) || !b.LessEq(a) { t.Errorf("expected %+v == %+v", a, b) } } func TestIsEmpty(t *testing.T) { - a := Timestamp{} + a := makeTS(0, 0) assert.True(t, a.IsEmpty()) a = makeTS(1, 0) assert.False(t, a.IsEmpty()) a = makeTS(0, 1) assert.False(t, a.IsEmpty()) - a = makeTSWithFlags(0, 0, 3) + a = makeSynTS(0, 0) assert.False(t, a.IsEmpty()) } -func TestSetAndClearFlag(t *testing.T) { - a := Timestamp{} - assert.False(t, a.IsFlagSet(TimestampFlag_SYNTHETIC)) - a = a.SetFlag(TimestampFlag_UNKNOWN) - assert.False(t, a.IsFlagSet(TimestampFlag_SYNTHETIC)) - a = a.SetFlag(TimestampFlag_SYNTHETIC) - assert.True(t, a.IsFlagSet(TimestampFlag_SYNTHETIC)) - a = a.ClearFlag(TimestampFlag_SYNTHETIC) - assert.False(t, a.IsFlagSet(TimestampFlag_SYNTHETIC)) -} - func TestTimestampNext(t *testing.T) { testCases := []struct { ts, expNext Timestamp @@ -122,10 +104,10 @@ func TestTimestampNext(t *testing.T) { {makeTS(1, math.MaxInt32-1), makeTS(1, math.MaxInt32)}, {makeTS(1, math.MaxInt32), makeTS(2, 0)}, {makeTS(math.MaxInt32, math.MaxInt32), makeTS(math.MaxInt32+1, 0)}, - {makeTSWithFlags(1, 2, 3), makeTSWithFlags(1, 3, 3)}, - {makeTSWithFlags(1, math.MaxInt32-1, 3), makeTSWithFlags(1, math.MaxInt32, 3)}, - {makeTSWithFlags(1, math.MaxInt32, 3), makeTSWithFlags(2, 0, 3)}, - {makeTSWithFlags(math.MaxInt32, math.MaxInt32, 3), makeTSWithFlags(math.MaxInt32+1, 0, 3)}, + {makeSynTS(1, 2), makeSynTS(1, 3)}, + {makeSynTS(1, math.MaxInt32-1), makeSynTS(1, math.MaxInt32)}, + {makeSynTS(1, math.MaxInt32), makeSynTS(2, 0)}, + {makeSynTS(math.MaxInt32, math.MaxInt32), makeSynTS(math.MaxInt32+1, 0)}, } for _, c := range testCases { assert.Equal(t, c.expNext, c.ts.Next()) @@ -139,9 +121,9 @@ func TestTimestampPrev(t *testing.T) { {makeTS(1, 2), makeTS(1, 1)}, {makeTS(1, 1), makeTS(1, 0)}, {makeTS(1, 0), makeTS(0, math.MaxInt32)}, - {makeTSWithFlags(1, 2, 3), makeTSWithFlags(1, 1, 3)}, - {makeTSWithFlags(1, 1, 3), makeTSWithFlags(1, 0, 3)}, - {makeTSWithFlags(1, 0, 3), makeTSWithFlags(0, math.MaxInt32, 3)}, + {makeSynTS(1, 2), makeSynTS(1, 1)}, + {makeSynTS(1, 1), makeSynTS(1, 0)}, + {makeSynTS(1, 0), makeSynTS(0, math.MaxInt32)}, } for _, c := range testCases { assert.Equal(t, c.expPrev, c.ts.Prev()) @@ -156,10 +138,10 @@ func TestTimestampFloorPrev(t *testing.T) { {makeTS(1, 2), makeTS(1, 1)}, {makeTS(1, 1), makeTS(1, 0)}, {makeTS(1, 0), makeTS(0, 0)}, - {makeTSWithFlags(2, 0, 3), makeTSWithFlags(1, 0, 3)}, - {makeTSWithFlags(1, 2, 3), makeTSWithFlags(1, 1, 3)}, - {makeTSWithFlags(1, 1, 3), makeTSWithFlags(1, 0, 3)}, - {makeTSWithFlags(1, 0, 3), makeTSWithFlags(0, 0, 3)}, + {makeSynTS(2, 0), makeSynTS(1, 0)}, + {makeSynTS(1, 2), makeSynTS(1, 1)}, + {makeSynTS(1, 1), makeSynTS(1, 0)}, + {makeSynTS(1, 0), makeSynTS(0, 0)}, } for _, c := range testCases { assert.Equal(t, c.expPrev, c.ts.FloorPrev()) @@ -167,7 +149,6 @@ func TestTimestampFloorPrev(t *testing.T) { } func TestTimestampForward(t *testing.T) { - flagSyn := TimestampFlag_SYNTHETIC testCases := []struct { ts, arg Timestamp expFwd Timestamp @@ -178,21 +159,21 @@ func TestTimestampForward(t *testing.T) { {makeTS(2, 0), makeTS(2, 0), makeTS(2, 0), false}, {makeTS(2, 0), makeTS(2, 1), makeTS(2, 1), true}, {makeTS(2, 0), makeTS(3, 0), makeTS(3, 0), true}, - {makeTSWithFlags(2, 0, flagSyn), makeTS(1, 0), makeTSWithFlags(2, 0, flagSyn), false}, - {makeTSWithFlags(2, 0, flagSyn), makeTS(1, 1), makeTSWithFlags(2, 0, flagSyn), false}, - {makeTSWithFlags(2, 0, flagSyn), makeTS(2, 0), makeTS(2, 0), false}, - {makeTSWithFlags(2, 0, flagSyn), makeTS(2, 1), makeTS(2, 1), true}, - {makeTSWithFlags(2, 0, flagSyn), makeTS(3, 0), makeTS(3, 0), true}, - {makeTS(2, 0), makeTSWithFlags(1, 0, flagSyn), makeTS(2, 0), false}, - {makeTS(2, 0), makeTSWithFlags(1, 1, flagSyn), makeTS(2, 0), false}, - {makeTS(2, 0), makeTSWithFlags(2, 0, flagSyn), makeTS(2, 0), false}, - {makeTS(2, 0), makeTSWithFlags(2, 1, flagSyn), makeTSWithFlags(2, 1, flagSyn), true}, - {makeTS(2, 0), makeTSWithFlags(3, 0, flagSyn), makeTSWithFlags(3, 0, flagSyn), true}, - {makeTSWithFlags(2, 0, flagSyn), makeTSWithFlags(1, 0, flagSyn), makeTSWithFlags(2, 0, flagSyn), false}, - {makeTSWithFlags(2, 0, flagSyn), makeTSWithFlags(1, 1, flagSyn), makeTSWithFlags(2, 0, flagSyn), false}, - {makeTSWithFlags(2, 0, flagSyn), makeTSWithFlags(2, 0, flagSyn), makeTSWithFlags(2, 0, flagSyn), false}, - {makeTSWithFlags(2, 0, flagSyn), makeTSWithFlags(2, 1, flagSyn), makeTSWithFlags(2, 1, flagSyn), true}, - {makeTSWithFlags(2, 0, flagSyn), makeTSWithFlags(3, 0, flagSyn), makeTSWithFlags(3, 0, flagSyn), true}, + {makeSynTS(2, 0), makeTS(1, 0), makeSynTS(2, 0), false}, + {makeSynTS(2, 0), makeTS(1, 1), makeSynTS(2, 0), false}, + {makeSynTS(2, 0), makeTS(2, 0), makeTS(2, 0), false}, + {makeSynTS(2, 0), makeTS(2, 1), makeTS(2, 1), true}, + {makeSynTS(2, 0), makeTS(3, 0), makeTS(3, 0), true}, + {makeTS(2, 0), makeSynTS(1, 0), makeTS(2, 0), false}, + {makeTS(2, 0), makeSynTS(1, 1), makeTS(2, 0), false}, + {makeTS(2, 0), makeSynTS(2, 0), makeTS(2, 0), false}, + {makeTS(2, 0), makeSynTS(2, 1), makeSynTS(2, 1), true}, + {makeTS(2, 0), makeSynTS(3, 0), makeSynTS(3, 0), true}, + {makeSynTS(2, 0), makeSynTS(1, 0), makeSynTS(2, 0), false}, + {makeSynTS(2, 0), makeSynTS(1, 1), makeSynTS(2, 0), false}, + {makeSynTS(2, 0), makeSynTS(2, 0), makeSynTS(2, 0), false}, + {makeSynTS(2, 0), makeSynTS(2, 1), makeSynTS(2, 1), true}, + {makeSynTS(2, 0), makeSynTS(3, 0), makeSynTS(3, 0), true}, } for _, c := range testCases { ts := c.ts @@ -202,7 +183,6 @@ func TestTimestampForward(t *testing.T) { } func TestTimestampBackward(t *testing.T) { - flagSyn := TimestampFlag_SYNTHETIC testCases := []struct { ts, arg, expBwd Timestamp }{ @@ -211,21 +191,21 @@ func TestTimestampBackward(t *testing.T) { {makeTS(2, 0), makeTS(2, 0), makeTS(2, 0)}, {makeTS(2, 0), makeTS(2, 1), makeTS(2, 0)}, {makeTS(2, 0), makeTS(3, 0), makeTS(2, 0)}, - {makeTSWithFlags(2, 0, flagSyn), makeTS(1, 0), makeTS(1, 0)}, - {makeTSWithFlags(2, 0, flagSyn), makeTS(1, 1), makeTS(1, 1)}, - {makeTSWithFlags(2, 0, flagSyn), makeTS(2, 0), makeTS(2, 0)}, - {makeTSWithFlags(2, 0, flagSyn), makeTS(2, 1), makeTS(2, 0)}, - {makeTSWithFlags(2, 0, flagSyn), makeTS(3, 0), makeTS(2, 0)}, - {makeTS(2, 0), makeTSWithFlags(1, 0, flagSyn), makeTS(1, 0)}, - {makeTS(2, 0), makeTSWithFlags(1, 1, flagSyn), makeTS(1, 1)}, - {makeTS(2, 0), makeTSWithFlags(2, 0, flagSyn), makeTS(2, 0)}, - {makeTS(2, 0), makeTSWithFlags(2, 1, flagSyn), makeTS(2, 0)}, - {makeTS(2, 0), makeTSWithFlags(3, 0, flagSyn), makeTS(2, 0)}, - {makeTSWithFlags(2, 0, flagSyn), makeTSWithFlags(1, 0, flagSyn), makeTSWithFlags(1, 0, flagSyn)}, - {makeTSWithFlags(2, 0, flagSyn), makeTSWithFlags(1, 1, flagSyn), makeTSWithFlags(1, 1, flagSyn)}, - {makeTSWithFlags(2, 0, flagSyn), makeTSWithFlags(2, 0, flagSyn), makeTSWithFlags(2, 0, flagSyn)}, - {makeTSWithFlags(2, 0, flagSyn), makeTSWithFlags(2, 1, flagSyn), makeTSWithFlags(2, 0, flagSyn)}, - {makeTSWithFlags(2, 0, flagSyn), makeTSWithFlags(3, 0, flagSyn), makeTSWithFlags(2, 0, flagSyn)}, + {makeSynTS(2, 0), makeTS(1, 0), makeTS(1, 0)}, + {makeSynTS(2, 0), makeTS(1, 1), makeTS(1, 1)}, + {makeSynTS(2, 0), makeTS(2, 0), makeTS(2, 0)}, + {makeSynTS(2, 0), makeTS(2, 1), makeTS(2, 0)}, + {makeSynTS(2, 0), makeTS(3, 0), makeTS(2, 0)}, + {makeTS(2, 0), makeSynTS(1, 0), makeTS(1, 0)}, + {makeTS(2, 0), makeSynTS(1, 1), makeTS(1, 1)}, + {makeTS(2, 0), makeSynTS(2, 0), makeTS(2, 0)}, + {makeTS(2, 0), makeSynTS(2, 1), makeTS(2, 0)}, + {makeTS(2, 0), makeSynTS(3, 0), makeTS(2, 0)}, + {makeSynTS(2, 0), makeSynTS(1, 0), makeSynTS(1, 0)}, + {makeSynTS(2, 0), makeSynTS(1, 1), makeSynTS(1, 1)}, + {makeSynTS(2, 0), makeSynTS(2, 0), makeSynTS(2, 0)}, + {makeSynTS(2, 0), makeSynTS(2, 1), makeSynTS(2, 0)}, + {makeSynTS(2, 0), makeSynTS(3, 0), makeSynTS(2, 0)}, } for _, c := range testCases { ts := c.ts @@ -268,10 +248,9 @@ func TestTimestampFormatParseRoundTrip(t *testing.T) { {makeTS(-1234567890, 0), "-1.234567890,0"}, {makeTS(6661234567890, 0), "6661.234567890,0"}, {makeTS(-6661234567890, 0), "-6661.234567890,0"}, - {makeTSWithFlags(0, 0, TimestampFlag_SYNTHETIC), "0,0[syn]"}, - {makeTSWithFlags(0, 123, TimestampFlag_SYNTHETIC), "0,123[syn]"}, - {makeTSWithFlags(1, 0, TimestampFlag_SYNTHETIC), "0.000000001,0[syn]"}, - {makeTSWithFlags(1, 123, TimestampFlag_SYNTHETIC), "0.000000001,123[syn]"}, + {makeSynTS(0, 123), "0,123?"}, + {makeSynTS(1, 0), "0.000000001,0?"}, + {makeSynTS(1, 123), "0.000000001,123?"}, } for _, c := range testCases { str := c.ts.String() @@ -307,10 +286,10 @@ func TestTimestampParseFormatNonRoundTrip(t *testing.T) { // Other cases. {"0.000000001", makeTS(1, 0), "0.000000001,0"}, {"99.000000001", makeTS(99000000001, 0), "99.000000001,0"}, - {"0[syn]", makeTSWithFlags(0, 0, TimestampFlag_SYNTHETIC), "0,0[syn]"}, - {"99[syn]", makeTSWithFlags(99000000000, 0, TimestampFlag_SYNTHETIC), "99.000000000,0[syn]"}, - {"0.000000001[syn]", makeTSWithFlags(1, 0, TimestampFlag_SYNTHETIC), "0.000000001,0[syn]"}, - {"99.000000001[syn]", makeTSWithFlags(99000000001, 0, TimestampFlag_SYNTHETIC), "99.000000001,0[syn]"}, + {"0?", makeSynTS(0, 0), "0,0?"}, + {"99?", makeSynTS(99000000000, 0), "99.000000000,0?"}, + {"0.000000001?", makeSynTS(1, 0), "0.000000001,0?"}, + {"99.000000001?", makeSynTS(99000000001, 0), "99.000000001,0?"}, } for _, c := range testCases { parsed, err := ParseTimestamp(c.s) @@ -344,20 +323,12 @@ func TestTimestampParseError(t *testing.T) { "failed to parse \"1.9999999999999999999,0\" as Timestamp: strconv.ParseInt: parsing \"9999999999999999999\": value out of range", }, { - "0,123[]", - "failed to parse \"0,123\\[\\]\" as Timestamp", - }, - { - "0,123[bad]", - "failed to parse \"0,123\\[bad\\]\" as Timestamp: unknown flag \"bad\" provided", - }, - { - "0,123[syn,]", - "failed to parse \"0,123\\[syn,\\]\" as Timestamp: empty flag provided", + "0,123[?]", + "failed to parse \"0,123\\[\\?\\]\" as Timestamp", }, { - "0,123[syn,syn]", - "failed to parse \"0,123\\[syn,syn\\]\" as Timestamp: duplicate flag \"syn\" provided", + "0,123??", + "failed to parse \"0,123\\?\\?\" as Timestamp", }, } { _, err := ParseTimestamp(c.s) @@ -375,8 +346,8 @@ func BenchmarkTimestampString(b *testing.B) { } } -func BenchmarkTimestampStringWithFlags(b *testing.B) { - ts := makeTSWithFlags(-6661234567890, 0, TimestampFlag_SYNTHETIC) +func BenchmarkTimestampStringSynthetic(b *testing.B) { + ts := makeSynTS(-6661234567890, 0) for i := 0; i < b.N; i++ { _ = ts.String() From 924423f89a5afb67b5312031d7526c3534a96303 Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Thu, 31 Dec 2020 01:09:28 -0500 Subject: [PATCH 6/7] kv: delete StoreTestingKnobs.ClockBeforeSend This was only used in one very old test (added in 1123717), which could be easily adapted to no longer need the very specific hook. --- pkg/kv/kvserver/store_send.go | 4 --- pkg/kv/kvserver/testing_knobs.go | 4 --- pkg/sql/txn_restart_test.go | 55 +++++++++++++------------------- 3 files changed, 22 insertions(+), 41 deletions(-) diff --git a/pkg/kv/kvserver/store_send.go b/pkg/kv/kvserver/store_send.go index e49dd6419afc..bc913550bd35 100644 --- a/pkg/kv/kvserver/store_send.go +++ b/pkg/kv/kvserver/store_send.go @@ -78,10 +78,6 @@ func (s *Store) Send( return nil, roachpb.NewError(err) } - if s.cfg.TestingKnobs.ClockBeforeSend != nil { - s.cfg.TestingKnobs.ClockBeforeSend(s.cfg.Clock, ba) - } - // 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. diff --git a/pkg/kv/kvserver/testing_knobs.go b/pkg/kv/kvserver/testing_knobs.go index f909ff699908..0cb375ad49de 100644 --- a/pkg/kv/kvserver/testing_knobs.go +++ b/pkg/kv/kvserver/testing_knobs.go @@ -75,10 +75,6 @@ type StoreTestingKnobs struct { // or data. TestingRangefeedFilter kvserverbase.ReplicaRangefeedFilter - // A hack to manipulate the clock before sending a batch request to a replica. - // TODO(kaneda): This hook is not encouraged to use. Get rid of it once - // we make TestServer take a ManualClock. - ClockBeforeSend func(*hlc.Clock, roachpb.BatchRequest) // MaxOffset, if set, overrides the server clock's MaxOffset at server // creation time. // See also DisableMaxOffsetCheck. diff --git a/pkg/sql/txn_restart_test.go b/pkg/sql/txn_restart_test.go index d7350e5d707c..c7961e7af9f3 100644 --- a/pkg/sql/txn_restart_test.go +++ b/pkg/sql/txn_restart_test.go @@ -1145,31 +1145,12 @@ func TestReacquireLeaseOnRestart(t *testing.T) { var cmdFilters tests.CommandFilters cmdFilters.AppendFilter(tests.CheckEndTxnTrigger, true) - var clockUpdate int32 testKey := []byte("test_key") storeTestingKnobs := &kvserver.StoreTestingKnobs{ EvalKnobs: kvserverbase.BatchEvalTestingKnobs{ TestingEvalFilter: cmdFilters.RunFilters, }, DisableMaxOffsetCheck: true, - ClockBeforeSend: func(c *hlc.Clock, ba roachpb.BatchRequest) { - if atomic.LoadInt32(&clockUpdate) > 0 { - return - } - - // Hack to advance the transaction timestamp on a transaction restart. - for _, union := range ba.Requests { - if req, ok := union.GetInner().(*roachpb.ScanRequest); ok { - if bytes.Contains(req.Key, testKey) && !kv.TestingIsRangeLookupRequest(req) { - atomic.AddInt32(&clockUpdate, 1) - now := c.NowAsClockTimestamp() - now.WallTime += advancement.Nanoseconds() - c.Update(now) - break - } - } - } - }, } const refreshAttempts = 3 @@ -1183,24 +1164,32 @@ func TestReacquireLeaseOnRestart(t *testing.T) { s, sqlDB, _ := serverutils.StartServer(t, params) defer s.Stopper().Stop(context.Background()) - var restartDone int32 + var clockUpdate, restartDone int32 cleanupFilter := cmdFilters.AppendFilter( func(args kvserverbase.FilterArgs) *roachpb.Error { - // Allow a set number of restarts so that the auto retry on the - // first few uncertainty interval errors also fails. - if atomic.LoadInt32(&restartDone) > refreshAttempts { - return nil - } - if req, ok := args.Req.(*roachpb.ScanRequest); ok { if bytes.Contains(req.Key, testKey) && !kv.TestingIsRangeLookupRequest(req) { - atomic.AddInt32(&restartDone, 1) - // Return ReadWithinUncertaintyIntervalError to update the transaction timestamp on retry. - txn := args.Hdr.Txn - txn.ResetObservedTimestamps() - now := s.Clock().NowAsClockTimestamp() - txn.UpdateObservedTimestamp(s.(*server.TestServer).Gossip().NodeID.Get(), now) - return roachpb.NewErrorWithTxn(roachpb.NewReadWithinUncertaintyIntervalError(now.ToTimestamp(), now.ToTimestamp(), txn), txn) + if atomic.LoadInt32(&clockUpdate) == 0 { + atomic.AddInt32(&clockUpdate, 1) + // Hack to advance the transaction timestamp on a + // transaction restart. + now := s.Clock().NowAsClockTimestamp() + now.WallTime += advancement.Nanoseconds() + s.Clock().Update(now) + } + + // Allow a set number of restarts so that the auto retry on + // the first few uncertainty interval errors also fails. + if atomic.LoadInt32(&restartDone) <= refreshAttempts { + atomic.AddInt32(&restartDone, 1) + // Return ReadWithinUncertaintyIntervalError to update + // the transaction timestamp on retry. + txn := args.Hdr.Txn + txn.ResetObservedTimestamps() + now := s.Clock().NowAsClockTimestamp() + txn.UpdateObservedTimestamp(s.(*server.TestServer).Gossip().NodeID.Get(), now) + return roachpb.NewErrorWithTxn(roachpb.NewReadWithinUncertaintyIntervalError(now.ToTimestamp(), now.ToTimestamp(), txn), txn) + } } } return nil From b96f85c426365c02626429013583fcb7371521c2 Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Mon, 4 Jan 2021 15:58:52 -0500 Subject: [PATCH 7/7] sql/pgwire: fix references to fmt.Stringer These methods were implementing the `fmt.Stringer` interface, not the `fmt.Formatter` interface. --- pkg/sql/pgwire/hba/hba.go | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/pkg/sql/pgwire/hba/hba.go b/pkg/sql/pgwire/hba/hba.go index 901492b7cd07..c56d55009705 100644 --- a/pkg/sql/pgwire/hba/hba.go +++ b/pkg/sql/pgwire/hba/hba.go @@ -79,7 +79,7 @@ const ( ConnAny = ConnHostAny | ConnLocal ) -// String implements the fmt.Formatter interface. +// String implements the fmt.Stringer interface. func (t ConnType) String() string { switch t { case ConnLocal: @@ -95,7 +95,7 @@ func (t ConnType) String() string { } } -// String implements the fmt.Formatter interface. +// String implements the fmt.Stringer interface. func (c Conf) String() string { if len(c.Entries) == 0 { return "# (empty configuration)\n" @@ -138,7 +138,7 @@ func (c Conf) String() string { // the "Address" field. type AnyAddr struct{} -// String implements the fmt.Formatter interface. +// String implements the fmt.Stringer interface. func (AnyAddr) String() string { return "all" } // GetOption returns the value of option name if there is exactly one @@ -290,7 +290,7 @@ func (h Entry) OptionsString() string { return sb.String() } -// String implements the fmt.Formatter interface. +// String implements the fmt.Stringer interface. func (h Entry) String() string { return Conf{Entries: []Entry{h}}.String() } @@ -301,7 +301,7 @@ type String struct { Quoted bool } -// String implements the fmt.Formatter interface. +// String implements the fmt.Stringer interface. func (s String) String() string { if s.Quoted { return `"` + s.Value + `"`