diff --git a/pkg/ccl/importccl/import_stmt.go b/pkg/ccl/importccl/import_stmt.go index 4840b479508e..d236d8cbd288 100644 --- a/pkg/ccl/importccl/import_stmt.go +++ b/pkg/ccl/importccl/import_stmt.go @@ -48,6 +48,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/storage/cloud" "github.com/cockroachdb/cockroach/pkg/storage/cloudimpl" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -1662,7 +1663,7 @@ func (r *importResumer) dropTables( if details.Walltime == 0 { return errors.Errorf("invalid pre-IMPORT time to rollback") } - ts := hlc.Timestamp{WallTime: details.Walltime}.Prev() + ts := enginepb.TxnTimestamp{WallTime: details.Walltime}.Prev() if err := sql.RevertTables(ctx, txn.DB(), execCfg, revert, ts, sql.RevertTableDefaultBatchSize); err != nil { return errors.Wrap(err, "rolling back partially completed IMPORT") } diff --git a/pkg/ccl/importccl/import_table_creation.go b/pkg/ccl/importccl/import_table_creation.go index 6c1b41f6348f..9713bce57241 100644 --- a/pkg/ccl/importccl/import_table_creation.go +++ b/pkg/ccl/importccl/import_table_creation.go @@ -27,6 +27,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/storage/cloud" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/errors" @@ -158,7 +159,7 @@ func MakeSimpleTableDescriptor( parentID, parentSchemaID, tableID, - hlc.Timestamp{WallTime: walltime}, + enginepb.TxnTimestamp(hlc.Timestamp{WallTime: walltime}), descpb.NewDefaultPrivilegeDescriptor(security.AdminRoleName()), affected, semaCtx, diff --git a/pkg/cli/debug.go b/pkg/cli/debug.go index 56490a488317..f97e3fe0e814 100644 --- a/pkg/cli/debug.go +++ b/pkg/cli/debug.go @@ -46,6 +46,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/row" "github.com/cockroachdb/cockroach/pkg/storage" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/envutil" "github.com/cockroachdb/cockroach/pkg/util/flagutil" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -189,7 +190,7 @@ func runDebugKeys(cmd *cobra.Command, args []string) error { if err := protoutil.Unmarshal(bytes, &desc); err != nil { return err } - table := tabledesc.NewImmutable(*descpb.TableFromDescriptor(&desc, hlc.Timestamp{})) + table := tabledesc.NewImmutable(*descpb.TableFromDescriptor(&desc, enginepb.TxnTimestamp{})) fn := func(kv storage.MVCCKeyValue) (string, error) { var v roachpb.Value @@ -471,7 +472,7 @@ Decode and print a hexadecimal-encoded key-value pair. } k = storage.MVCCKey{ Key: bs[0], - Timestamp: hlc.Timestamp{WallTime: 987654321}, + Timestamp: enginepb.TxnTimestamp(hlc.Timestamp{WallTime: 987654321}), } } @@ -591,7 +592,7 @@ func runDebugGCCmd(cmd *cobra.Command, args []string) error { var descs []roachpb.RangeDescriptor - if _, err := storage.MVCCIterate(context.Background(), db, start, end, hlc.MaxTimestamp, + if _, err := storage.MVCCIterate(context.Background(), db, start, end, enginepb.TxnTimestamp(hlc.MaxTimestamp), storage.MVCCScanOptions{Inconsistent: true}, func(kv roachpb.KeyValue) error { var desc roachpb.RangeDescriptor _, suffix, _, err := keys.DecodeRangeKey(kv.Key) @@ -1075,7 +1076,7 @@ func removeDeadReplicas( // A crude form of the intent resolution process: abort the // transaction by deleting its record. txnKey := keys.TransactionKey(intent.Txn.Key, intent.Txn.ID) - if err := storage.MVCCDelete(ctx, batch, &ms, txnKey, hlc.Timestamp{}, nil); err != nil { + if err := storage.MVCCDelete(ctx, batch, &ms, txnKey, enginepb.TxnTimestamp{}, nil); err != nil { return nil, err } update := roachpb.LockUpdate{ diff --git a/pkg/gossip/infostore.go b/pkg/gossip/infostore.go index 85ca4f6895ec..6eac02988fc1 100644 --- a/pkg/gossip/infostore.go +++ b/pkg/gossip/infostore.go @@ -21,8 +21,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util" - "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/stop" "github.com/cockroachdb/cockroach/pkg/util/syncutil" @@ -209,7 +209,7 @@ func (is *infoStore) newInfo(val []byte, ttl time.Duration) *Info { if ttl == 0 { ttlStamp = math.MaxInt64 } - v := roachpb.MakeValueFromBytesAndTimestamp(val, hlc.Timestamp{WallTime: now}) + v := roachpb.MakeValueFromBytesAndTimestamp(val, enginepb.TxnTimestamp{WallTime: now}) return &Info{ Value: v, TTLStamp: ttlStamp, diff --git a/pkg/jobs/jobsprotectedts/jobs_protected_ts.go b/pkg/jobs/jobsprotectedts/jobs_protected_ts.go index f80833d221b1..b873e697aabf 100644 --- a/pkg/jobs/jobsprotectedts/jobs_protected_ts.go +++ b/pkg/jobs/jobsprotectedts/jobs_protected_ts.go @@ -19,7 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptreconcile" "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" ) @@ -53,7 +53,7 @@ func MakeStatusFunc(jr *jobs.Registry) ptreconcile.StatusFunc { // MakeRecord makes a protected timestamp record to protect a timestamp on // behalf of this job. func MakeRecord( - id uuid.UUID, jobID int64, tsToProtect hlc.Timestamp, spans []roachpb.Span, + id uuid.UUID, jobID int64, tsToProtect enginepb.TxnTimestamp, spans []roachpb.Span, ) *ptpb.Record { return &ptpb.Record{ ID: id, diff --git a/pkg/kv/bulk/buffering_adder.go b/pkg/kv/bulk/buffering_adder.go index afe4acfdb0cc..b49f386ecb56 100644 --- a/pkg/kv/bulk/buffering_adder.go +++ b/pkg/kv/bulk/buffering_adder.go @@ -20,7 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/storage" - "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/mon" "github.com/cockroachdb/cockroach/pkg/util/timeutil" @@ -33,7 +33,7 @@ import ( type BufferingAdder struct { sink SSTBatcher // timestamp applied to mvcc keys created from keys during SST construction. - timestamp hlc.Timestamp + timestamp enginepb.TxnTimestamp // threshold at which buffered entries will be flushed to SSTBatcher. curBufferSize int64 @@ -74,7 +74,7 @@ func MakeBulkAdder( db SSTSender, rangeCache *rangecache.RangeCache, settings *cluster.Settings, - timestamp hlc.Timestamp, + timestamp enginepb.TxnTimestamp, opts kvserverbase.BulkAdderOptions, bulkMon *mon.BytesMonitor, ) (*BufferingAdder, error) { diff --git a/pkg/kv/db.go b/pkg/kv/db.go index 89a2c001e695..a7e4c144d0d8 100644 --- a/pkg/kv/db.go +++ b/pkg/kv/db.go @@ -335,13 +335,13 @@ func (db *DB) GetProto(ctx context.Context, key interface{}, msg protoutil.Messa // key can be either a byte slice or a string. func (db *DB) GetProtoTs( ctx context.Context, key interface{}, msg protoutil.Message, -) (hlc.Timestamp, error) { +) (enginepb.TxnTimestamp, error) { r, err := db.Get(ctx, key) if err != nil { - return hlc.Timestamp{}, err + return enginepb.TxnTimestamp{}, err } if err := r.ValueProto(msg); err != nil || r.Value == nil { - return hlc.Timestamp{}, err + return enginepb.TxnTimestamp{}, err } return r.Value.Timestamp, nil } diff --git a/pkg/kv/kvclient/kvcoord/dist_sender.go b/pkg/kv/kvclient/kvcoord/dist_sender.go index 78d998f91a2c..b91abb797b0d 100644 --- a/pkg/kv/kvclient/kvcoord/dist_sender.go +++ b/pkg/kv/kvclient/kvcoord/dist_sender.go @@ -30,6 +30,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/rpc/nodedialer" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/grpcutil" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -579,7 +580,8 @@ func (ds *DistSender) initAndVerifyBatch( // In the event that timestamp isn't set and read consistency isn't // required, set the timestamp using the local clock. if ba.ReadConsistency != roachpb.CONSISTENT && ba.Timestamp.IsEmpty() { - ba.Timestamp = ds.clock.Now() + // TODO(nvanbenschoten): split this... + ba.Timestamp = enginepb.TxnTimestamp(ds.clock.Now()) } if len(ba.Requests) < 1 { @@ -1022,7 +1024,8 @@ func (ds *DistSender) detectIntentMissingDueToIntentResolution( ctx context.Context, txn *roachpb.Transaction, ) (bool, error) { ba := roachpb.BatchRequest{} - ba.Timestamp = ds.clock.Now() + // TODO(nvanbenschoten): how does the QueryTxn work in this case? + ba.Timestamp = enginepb.TxnTimestamp(ds.clock.Now()) ba.Add(&roachpb.QueryTxnRequest{ RequestHeader: roachpb.RequestHeader{ Key: txn.TxnMeta.Key, diff --git a/pkg/kv/kvclient/kvcoord/dist_sender_rangefeed.go b/pkg/kv/kvclient/kvcoord/dist_sender_rangefeed.go index 768e694cc610..af70b3cdb823 100644 --- a/pkg/kv/kvclient/kvcoord/dist_sender_rangefeed.go +++ b/pkg/kv/kvclient/kvcoord/dist_sender_rangefeed.go @@ -19,9 +19,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangecache" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" "github.com/cockroachdb/cockroach/pkg/util/grpcutil" - "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/retry" "github.com/cockroachdb/cockroach/pkg/util/timeutil" @@ -31,7 +31,7 @@ import ( type singleRangeInfo struct { rs roachpb.RSpan - ts hlc.Timestamp + ts enginepb.TxnTimestamp token rangecache.EvictionToken } @@ -44,7 +44,7 @@ type singleRangeInfo struct { func (ds *DistSender) RangeFeed( ctx context.Context, span roachpb.Span, - ts hlc.Timestamp, + ts enginepb.TxnTimestamp, withDiff bool, eventCh chan<- *roachpb.RangeFeedEvent, ) error { @@ -84,7 +84,7 @@ func (ds *DistSender) RangeFeed( } func (ds *DistSender) divideAndSendRangeFeedToRanges( - ctx context.Context, rs roachpb.RSpan, ts hlc.Timestamp, rangeCh chan<- singleRangeInfo, + ctx context.Context, rs roachpb.RSpan, ts enginepb.TxnTimestamp, rangeCh chan<- singleRangeInfo, ) error { // As RangeIterator iterates, it can return overlapping descriptors (and // during splits, this happens frequently), but divideAndSendRangeFeedToRanges @@ -211,11 +211,11 @@ func (ds *DistSender) partialRangeFeed( func (ds *DistSender) singleRangeFeed( ctx context.Context, span roachpb.Span, - ts hlc.Timestamp, + ts enginepb.TxnTimestamp, withDiff bool, desc *roachpb.RangeDescriptor, eventCh chan<- *roachpb.RangeFeedEvent, -) (hlc.Timestamp, error) { +) (enginepb.TxnTimestamp, error) { args := roachpb.RangeFeedRequest{ Span: span, Header: roachpb.Header{ diff --git a/pkg/kv/kvclient/kvcoord/txn_coord_sender.go b/pkg/kv/kvclient/kvcoord/txn_coord_sender.go index 2233813822c6..0a6c40d9eb3e 100644 --- a/pkg/kv/kvclient/kvcoord/txn_coord_sender.go +++ b/pkg/kv/kvclient/kvcoord/txn_coord_sender.go @@ -22,7 +22,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/duration" - "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/cockroach/pkg/util/uuid" @@ -402,7 +401,7 @@ func (tc *TxnCoordSender) DisablePipelining() error { } func generateTxnDeadlineExceededErr( - txn *roachpb.Transaction, deadline hlc.Timestamp, + txn *roachpb.Transaction, deadline enginepb.TxnTimestamp, ) *roachpb.Error { exceededBy := txn.WriteTimestamp.GoTime().Sub(deadline.GoTime()) extraMsg := fmt.Sprintf( @@ -888,21 +887,21 @@ func (tc *TxnCoordSender) String() string { } // ReadTimestamp is part of the client.TxnSender interface. -func (tc *TxnCoordSender) ReadTimestamp() hlc.Timestamp { +func (tc *TxnCoordSender) ReadTimestamp() enginepb.TxnTimestamp { tc.mu.Lock() defer tc.mu.Unlock() return tc.mu.txn.ReadTimestamp } // ProvisionalCommitTimestamp is part of the client.TxnSender interface. -func (tc *TxnCoordSender) ProvisionalCommitTimestamp() hlc.Timestamp { +func (tc *TxnCoordSender) ProvisionalCommitTimestamp() enginepb.TxnTimestamp { tc.mu.Lock() defer tc.mu.Unlock() return tc.mu.txn.WriteTimestamp } // CommitTimestamp is part of the client.TxnSender interface. -func (tc *TxnCoordSender) CommitTimestamp() hlc.Timestamp { +func (tc *TxnCoordSender) CommitTimestamp() enginepb.TxnTimestamp { tc.mu.Lock() defer tc.mu.Unlock() txn := &tc.mu.txn @@ -918,7 +917,7 @@ func (tc *TxnCoordSender) CommitTimestampFixed() bool { } // SetFixedTimestamp is part of the client.TxnSender interface. -func (tc *TxnCoordSender) SetFixedTimestamp(ctx context.Context, ts hlc.Timestamp) { +func (tc *TxnCoordSender) SetFixedTimestamp(ctx context.Context, ts enginepb.TxnTimestamp) { tc.mu.Lock() defer tc.mu.Unlock() tc.mu.txn.ReadTimestamp = ts @@ -933,7 +932,7 @@ func (tc *TxnCoordSender) SetFixedTimestamp(ctx context.Context, ts hlc.Timestam // ManualRestart is part of the client.TxnSender interface. func (tc *TxnCoordSender) ManualRestart( - ctx context.Context, pri roachpb.UserPriority, ts hlc.Timestamp, + ctx context.Context, pri roachpb.UserPriority, ts enginepb.TxnTimestamp, ) { tc.mu.Lock() defer tc.mu.Unlock() diff --git a/pkg/kv/kvclient/kvcoord/txn_interceptor_span_refresher.go b/pkg/kv/kvclient/kvcoord/txn_interceptor_span_refresher.go index 6043188d86f8..54a04570f7b7 100644 --- a/pkg/kv/kvclient/kvcoord/txn_interceptor_span_refresher.go +++ b/pkg/kv/kvclient/kvcoord/txn_interceptor_span_refresher.go @@ -16,7 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/metric" "github.com/cockroachdb/errors" @@ -123,7 +123,7 @@ type txnSpanRefresher struct { // don't fail on (i.e. if we'll refresh, we'll refreshFrom timestamp onwards). // After every epoch bump, it is initialized to the timestamp of the first // batch. It is then bumped after every successful refresh. - refreshedTimestamp hlc.Timestamp + refreshedTimestamp enginepb.TxnTimestamp // canAutoRetry is set if the txnSpanRefresher is allowed to auto-retry. canAutoRetry bool diff --git a/pkg/kv/kvserver/abortspan/abortspan.go b/pkg/kv/kvserver/abortspan/abortspan.go index f9dd6ad27464..eae8ad248fec 100644 --- a/pkg/kv/kvserver/abortspan/abortspan.go +++ b/pkg/kv/kvserver/abortspan/abortspan.go @@ -18,7 +18,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" - "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" @@ -95,7 +94,7 @@ func (sc *AbortSpan) Get( ) (bool, error) { // Pull response from disk and read into reply if available. key := keys.AbortSpanKey(sc.rangeID, txnID) - ok, err := storage.MVCCGetProto(ctx, reader, key, hlc.Timestamp{}, entry, storage.MVCCGetOptions{}) + ok, err := storage.MVCCGetProto(ctx, reader, key, enginepb.TxnTimestamp{}, entry, storage.MVCCGetOptions{}) return ok, err } @@ -104,7 +103,7 @@ func (sc *AbortSpan) Get( func (sc *AbortSpan) Iterate( ctx context.Context, reader storage.Reader, f func(roachpb.Key, roachpb.AbortSpanEntry) error, ) error { - _, err := storage.MVCCIterate(ctx, reader, sc.min(), sc.max(), hlc.Timestamp{}, storage.MVCCScanOptions{}, + _, err := storage.MVCCIterate(ctx, reader, sc.min(), sc.max(), enginepb.TxnTimestamp{}, storage.MVCCScanOptions{}, func(kv roachpb.KeyValue) error { var entry roachpb.AbortSpanEntry if _, err := keys.DecodeAbortSpanKey(kv.Key, nil); err != nil { @@ -123,7 +122,7 @@ func (sc *AbortSpan) Del( ctx context.Context, reader storage.ReadWriter, ms *enginepb.MVCCStats, txnID uuid.UUID, ) error { key := keys.AbortSpanKey(sc.rangeID, txnID) - return storage.MVCCDelete(ctx, reader, ms, key, hlc.Timestamp{}, nil /* txn */) + return storage.MVCCDelete(ctx, reader, ms, key, enginepb.TxnTimestamp{}, nil /* txn */) } // Put writes an entry for the specified transaction ID. @@ -135,7 +134,7 @@ func (sc *AbortSpan) Put( entry *roachpb.AbortSpanEntry, ) error { key := keys.AbortSpanKey(sc.rangeID, txnID) - return storage.MVCCPutProto(ctx, readWriter, ms, key, hlc.Timestamp{}, nil /* txn */, entry) + return storage.MVCCPutProto(ctx, readWriter, ms, key, enginepb.TxnTimestamp{}, nil /* txn */, entry) } // CopyTo copies the abort span entries to the abort span for the range @@ -151,7 +150,7 @@ func (sc *AbortSpan) CopyTo( r storage.Reader, w storage.ReadWriter, ms *enginepb.MVCCStats, - ts hlc.Timestamp, + ts enginepb.TxnTimestamp, newRangeID roachpb.RangeID, ) error { var abortSpanCopyCount, abortSpanSkipCount int @@ -180,7 +179,7 @@ func (sc *AbortSpan) CopyTo( } return storage.MVCCPutProto(ctx, w, ms, keys.AbortSpanKey(newRangeID, txnID), - hlc.Timestamp{}, nil, &entry, + enginepb.TxnTimestamp{}, nil, &entry, ) }); err != nil { return roachpb.NewReplicaCorruptionError(errors.Wrap(err, "AbortSpan.CopyTo")) diff --git a/pkg/kv/kvserver/batcheval/cmd_clear_range.go b/pkg/kv/kvserver/batcheval/cmd_clear_range.go index 1400ccfbb283..9ee2548f0e65 100644 --- a/pkg/kv/kvserver/batcheval/cmd_clear_range.go +++ b/pkg/kv/kvserver/batcheval/cmd_clear_range.go @@ -69,7 +69,7 @@ func ClearRange( var pd result.Result if !args.Deadline.IsEmpty() { - if now := cArgs.EvalCtx.Clock().Now(); args.Deadline.LessEq(now) { + if now := cArgs.EvalCtx.Clock().Now(); args.Deadline.LessEq(enginepb.TxnTimestamp(now)) { return result.Result{}, errors.Errorf("ClearRange has deadline %s <= %s", args.Deadline, now) } } diff --git a/pkg/kv/kvserver/batcheval/cmd_delete_range.go b/pkg/kv/kvserver/batcheval/cmd_delete_range.go index 85d19b71ab88..3bf679abdc66 100644 --- a/pkg/kv/kvserver/batcheval/cmd_delete_range.go +++ b/pkg/kv/kvserver/batcheval/cmd_delete_range.go @@ -17,7 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" - "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" ) func init() { @@ -47,7 +47,7 @@ func DeleteRange( h := cArgs.Header reply := resp.(*roachpb.DeleteRangeResponse) - var timestamp hlc.Timestamp + var timestamp enginepb.TxnTimestamp if !args.Inline { timestamp = h.Timestamp } diff --git a/pkg/kv/kvserver/batcheval/cmd_end_transaction.go b/pkg/kv/kvserver/batcheval/cmd_end_transaction.go index df228fd6dae3..3d9dd511b025 100644 --- a/pkg/kv/kvserver/batcheval/cmd_end_transaction.go +++ b/pkg/kv/kvserver/batcheval/cmd_end_transaction.go @@ -29,7 +29,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" - "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/tracing" "github.com/cockroachdb/errors" @@ -63,7 +62,7 @@ func declareKeysEndTxn( ) { et := req.(*roachpb.EndTxnRequest) declareKeysWriteTransaction(desc, header, req, latchSpans) - var minTxnTS hlc.Timestamp + var minTxnTS enginepb.TxnTimestamp if header.Txn != nil { header.Txn.AssertInitialized(context.TODO()) minTxnTS = header.Txn.MinTimestamp @@ -197,7 +196,7 @@ func EndTxn( // Fetch existing transaction. var existingTxn roachpb.Transaction if ok, err := storage.MVCCGetProto( - ctx, readWriter, key, hlc.Timestamp{}, &existingTxn, storage.MVCCGetOptions{}, + ctx, readWriter, key, enginepb.TxnTimestamp{}, &existingTxn, storage.MVCCGetOptions{}, ); err != nil { return result.Result{}, err } else if !ok { @@ -380,7 +379,7 @@ func EndTxn( // IsEndTxnExceedingDeadline returns true if the transaction exceeded its // deadline. -func IsEndTxnExceedingDeadline(t hlc.Timestamp, args *roachpb.EndTxnRequest) bool { +func IsEndTxnExceedingDeadline(t enginepb.TxnTimestamp, args *roachpb.EndTxnRequest) bool { return args.Deadline != nil && args.Deadline.LessEq(t) } @@ -533,7 +532,7 @@ func updateStagingTxn( txn.LockSpans = args.LockSpans txn.InFlightWrites = args.InFlightWrites txnRecord := txn.AsRecord() - return storage.MVCCPutProto(ctx, readWriter, ms, key, hlc.Timestamp{}, nil /* txn */, &txnRecord) + return storage.MVCCPutProto(ctx, readWriter, ms, key, enginepb.TxnTimestamp{}, nil /* txn */, &txnRecord) } // updateFinalizedTxn persists the COMMITTED or ABORTED transaction record with @@ -553,12 +552,12 @@ func updateFinalizedTxn( if log.V(2) { log.Infof(ctx, "auto-gc'ed %s (%d locks)", txn.Short(), len(args.LockSpans)) } - return storage.MVCCDelete(ctx, readWriter, ms, key, hlc.Timestamp{}, nil /* txn */) + return storage.MVCCDelete(ctx, readWriter, ms, key, enginepb.TxnTimestamp{}, nil /* txn */) } txn.LockSpans = externalLocks txn.InFlightWrites = nil txnRecord := txn.AsRecord() - return storage.MVCCPutProto(ctx, readWriter, ms, key, hlc.Timestamp{}, nil /* txn */, &txnRecord) + return storage.MVCCPutProto(ctx, readWriter, ms, key, enginepb.TxnTimestamp{}, nil /* txn */, &txnRecord) } // RunCommitTrigger runs the commit trigger from an end transaction request. @@ -808,7 +807,7 @@ func splitTrigger( batch storage.Batch, bothDeltaMS enginepb.MVCCStats, split *roachpb.SplitTrigger, - ts hlc.Timestamp, + ts enginepb.TxnTimestamp, ) (enginepb.MVCCStats, result.Result, error) { // TODO(andrei): should this span be a child of the ctx's (if any)? sp := rec.ClusterSettings().Tracer.StartSpan("split", tracing.WithCtxLogTags(ctx)) @@ -853,7 +852,7 @@ func splitTriggerHelper( batch storage.Batch, statsInput splitStatsHelperInput, split *roachpb.SplitTrigger, - ts hlc.Timestamp, + ts enginepb.TxnTimestamp, ) (enginepb.MVCCStats, result.Result, error) { // TODO(d4l3k): we should check which side of the split is smaller // and compute stats for it instead of having a constraint that the @@ -869,7 +868,7 @@ func splitTriggerHelper( if err != nil { return enginepb.MVCCStats{}, result.Result{}, errors.Wrap(err, "unable to fetch last replica GC timestamp") } - if err := storage.MVCCPutProto(ctx, batch, nil, keys.RangeLastReplicaGCTimestampKey(split.RightDesc.RangeID), hlc.Timestamp{}, nil, &replicaGCTS); err != nil { + if err := storage.MVCCPutProto(ctx, batch, nil, keys.RangeLastReplicaGCTimestampKey(split.RightDesc.RangeID), enginepb.TxnTimestamp{}, nil, &replicaGCTS); err != nil { return enginepb.MVCCStats{}, result.Result{}, errors.Wrap(err, "unable to copy last replica GC timestamp") } @@ -963,7 +962,7 @@ func splitTriggerHelper( ctx, batch, keys.RaftTruncatedStateLegacyKey(rec.GetRangeID()), - hlc.Timestamp{}, + enginepb.TxnTimestamp{}, nil, storage.MVCCGetOptions{}, ); err != nil { @@ -1032,7 +1031,7 @@ func mergeTrigger( batch storage.Batch, ms *enginepb.MVCCStats, merge *roachpb.MergeTrigger, - ts hlc.Timestamp, + ts enginepb.TxnTimestamp, ) (result.Result, error) { desc := rec.Desc() if !bytes.Equal(desc.StartKey, merge.LeftDesc.StartKey) { diff --git a/pkg/kv/kvserver/batcheval/cmd_heartbeat_txn.go b/pkg/kv/kvserver/batcheval/cmd_heartbeat_txn.go index 96fb31572245..e847ef5fcf47 100644 --- a/pkg/kv/kvserver/batcheval/cmd_heartbeat_txn.go +++ b/pkg/kv/kvserver/batcheval/cmd_heartbeat_txn.go @@ -19,7 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" - "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" ) func init() { @@ -57,7 +57,7 @@ func HeartbeatTxn( var txn roachpb.Transaction if ok, err := storage.MVCCGetProto( - ctx, readWriter, key, hlc.Timestamp{}, &txn, storage.MVCCGetOptions{}, + ctx, readWriter, key, enginepb.TxnTimestamp{}, &txn, storage.MVCCGetOptions{}, ); err != nil { return result.Result{}, err } else if !ok { @@ -77,7 +77,7 @@ func HeartbeatTxn( // is up for debate. txn.LastHeartbeat.Forward(args.Now) txnRecord := txn.AsRecord() - if err := storage.MVCCPutProto(ctx, readWriter, cArgs.Stats, key, hlc.Timestamp{}, nil, &txnRecord); err != nil { + if err := storage.MVCCPutProto(ctx, readWriter, cArgs.Stats, key, enginepb.TxnTimestamp{}, nil, &txnRecord); err != nil { return result.Result{}, err } } diff --git a/pkg/kv/kvserver/batcheval/cmd_push_txn.go b/pkg/kv/kvserver/batcheval/cmd_push_txn.go index b7163c9dfd5d..446549017a3d 100644 --- a/pkg/kv/kvserver/batcheval/cmd_push_txn.go +++ b/pkg/kv/kvserver/batcheval/cmd_push_txn.go @@ -20,7 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/txnwait" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" - "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" ) @@ -127,7 +127,7 @@ func PushTxn( return result.Result{}, errors.Errorf("request timestamp %s less than pushee txn timestamp %s", h.Timestamp, args.PusheeTxn.WriteTimestamp) } now := cArgs.EvalCtx.Clock().Now() - if now.Less(h.Timestamp) { + if enginepb.TxnTimestamp(now).Less(h.Timestamp) { // The batch's timestamp should have been used to update the clock. return result.Result{}, errors.Errorf("request timestamp %s less than current clock time %s", h.Timestamp, now) } @@ -138,7 +138,7 @@ func PushTxn( // Fetch existing transaction; if missing, we're allowed to abort. var existTxn roachpb.Transaction - ok, err := storage.MVCCGetProto(ctx, readWriter, key, hlc.Timestamp{}, &existTxn, storage.MVCCGetOptions{}) + ok, err := storage.MVCCGetProto(ctx, readWriter, key, enginepb.TxnTimestamp{}, &existTxn, storage.MVCCGetOptions{}) if err != nil { return result.Result{}, err } else if !ok { @@ -279,7 +279,7 @@ func PushTxn( // If the transaction record was already present, forward the timestamp // to accommodate AbortSpan GC. See method comment for details. if ok { - reply.PusheeTxn.WriteTimestamp.Forward(reply.PusheeTxn.LastActive()) + reply.PusheeTxn.WriteTimestamp.Forward(enginepb.TxnTimestamp(reply.PusheeTxn.LastActive())) } case roachpb.PUSH_TIMESTAMP: // Otherwise, update timestamp to be one greater than the request's @@ -301,7 +301,7 @@ func PushTxn( // in the timestamp cache. if ok { txnRecord := reply.PusheeTxn.AsRecord() - if err := storage.MVCCPutProto(ctx, readWriter, cArgs.Stats, key, hlc.Timestamp{}, nil, &txnRecord); err != nil { + if err := storage.MVCCPutProto(ctx, readWriter, cArgs.Stats, key, enginepb.TxnTimestamp{}, nil, &txnRecord); err != nil { return result.Result{}, err } } diff --git a/pkg/kv/kvserver/batcheval/cmd_put.go b/pkg/kv/kvserver/batcheval/cmd_put.go index caabd2b94be5..37b7f41a74bd 100644 --- a/pkg/kv/kvserver/batcheval/cmd_put.go +++ b/pkg/kv/kvserver/batcheval/cmd_put.go @@ -17,7 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" - "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" ) func init() { @@ -46,7 +46,7 @@ func Put( h := cArgs.Header ms := cArgs.Stats - var ts hlc.Timestamp + var ts enginepb.TxnTimestamp if !args.Inline { ts = h.Timestamp } diff --git a/pkg/kv/kvserver/batcheval/cmd_query_intent.go b/pkg/kv/kvserver/batcheval/cmd_query_intent.go index b4785f211382..67b33f607cf9 100644 --- a/pkg/kv/kvserver/batcheval/cmd_query_intent.go +++ b/pkg/kv/kvserver/batcheval/cmd_query_intent.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" ) @@ -56,7 +57,7 @@ func QueryIntent( // Read at the specified key at the maximum timestamp. This ensures that we // see an intent if one exists, regardless of what timestamp it is written // at. - _, intent, err := storage.MVCCGet(ctx, reader, args.Key, hlc.MaxTimestamp, storage.MVCCGetOptions{ + _, intent, err := storage.MVCCGet(ctx, reader, args.Key, enginepb.TxnTimestamp(hlc.MaxTimestamp), storage.MVCCGetOptions{ // Perform an inconsistent read so that intents are returned instead of // causing WriteIntentErrors. Inconsistent: true, diff --git a/pkg/kv/kvserver/batcheval/cmd_query_txn.go b/pkg/kv/kvserver/batcheval/cmd_query_txn.go index 1f935fb4d89c..6438b6dddc5c 100644 --- a/pkg/kv/kvserver/batcheval/cmd_query_txn.go +++ b/pkg/kv/kvserver/batcheval/cmd_query_txn.go @@ -18,7 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" - "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/errors" ) @@ -66,7 +66,7 @@ func QueryTxn( // Fetch transaction record; if missing, attempt to synthesize one. ok, err := storage.MVCCGetProto( - ctx, reader, key, hlc.Timestamp{}, &reply.QueriedTxn, storage.MVCCGetOptions{}, + ctx, reader, key, enginepb.TxnTimestamp{}, &reply.QueriedTxn, storage.MVCCGetOptions{}, ) if err != nil { return result.Result{}, err diff --git a/pkg/kv/kvserver/batcheval/cmd_recover_txn.go b/pkg/kv/kvserver/batcheval/cmd_recover_txn.go index d5ab6147b2e0..a2c3935f8483 100644 --- a/pkg/kv/kvserver/batcheval/cmd_recover_txn.go +++ b/pkg/kv/kvserver/batcheval/cmd_recover_txn.go @@ -19,7 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" - "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/errors" ) @@ -71,7 +71,7 @@ func RecoverTxn( // Fetch transaction record; if missing, attempt to synthesize one. if ok, err := storage.MVCCGetProto( - ctx, readWriter, key, hlc.Timestamp{}, &reply.RecoveredTxn, storage.MVCCGetOptions{}, + ctx, readWriter, key, enginepb.TxnTimestamp{}, &reply.RecoveredTxn, storage.MVCCGetOptions{}, ); err != nil { return result.Result{}, err } else if !ok { @@ -217,7 +217,7 @@ func RecoverTxn( reply.RecoveredTxn.Status = roachpb.ABORTED } txnRecord := reply.RecoveredTxn.AsRecord() - if err := storage.MVCCPutProto(ctx, readWriter, cArgs.Stats, key, hlc.Timestamp{}, nil, &txnRecord); err != nil { + if err := storage.MVCCPutProto(ctx, readWriter, cArgs.Stats, key, enginepb.TxnTimestamp{}, nil, &txnRecord); err != nil { return result.Result{}, err } diff --git a/pkg/kv/kvserver/batcheval/cmd_resolve_intent.go b/pkg/kv/kvserver/batcheval/cmd_resolve_intent.go index 072ed2f7bbe3..cc1ab44b1471 100644 --- a/pkg/kv/kvserver/batcheval/cmd_resolve_intent.go +++ b/pkg/kv/kvserver/batcheval/cmd_resolve_intent.go @@ -18,7 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" - "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/uuid" ) @@ -31,7 +31,7 @@ func declareKeysResolveIntentCombined( ) { var status roachpb.TransactionStatus var txnID uuid.UUID - var minTxnTS hlc.Timestamp + var minTxnTS enginepb.TxnTimestamp switch t := req.(type) { case *roachpb.ResolveIntentRequest: status = t.Status diff --git a/pkg/kv/kvserver/batcheval/cmd_revert_range.go b/pkg/kv/kvserver/batcheval/cmd_revert_range.go index 81320380e294..61798ee14eee 100644 --- a/pkg/kv/kvserver/batcheval/cmd_revert_range.go +++ b/pkg/kv/kvserver/batcheval/cmd_revert_range.go @@ -18,7 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" - "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" ) @@ -42,7 +42,7 @@ func declareKeysRevertRange( // isEmptyKeyTimeRange checks if the span has no writes in (since,until]. func isEmptyKeyTimeRange( - readWriter storage.ReadWriter, from, to roachpb.Key, since, until hlc.Timestamp, + readWriter storage.ReadWriter, from, to roachpb.Key, since, until enginepb.TxnTimestamp, ) (bool, error) { // Use a TBI to check if there is anything to delete -- the first key Seek hits // may not be in the time range but the fact the TBI found any key indicates diff --git a/pkg/kv/kvserver/batcheval/cmd_truncate_log.go b/pkg/kv/kvserver/batcheval/cmd_truncate_log.go index 5c84910c37cf..2a93a8e4f9c9 100644 --- a/pkg/kv/kvserver/batcheval/cmd_truncate_log.go +++ b/pkg/kv/kvserver/batcheval/cmd_truncate_log.go @@ -19,7 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" - "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" ) @@ -60,7 +60,7 @@ func TruncateLog( var legacyTruncatedState roachpb.RaftTruncatedState legacyKeyFound, err := storage.MVCCGetProto( ctx, readWriter, keys.RaftTruncatedStateLegacyKey(cArgs.EvalCtx.GetRangeID()), - hlc.Timestamp{}, &legacyTruncatedState, storage.MVCCGetOptions{}, + enginepb.TxnTimestamp{}, &legacyTruncatedState, storage.MVCCGetOptions{}, ) if err != nil { return result.Result{}, err @@ -140,7 +140,7 @@ func TruncateLog( // side effect) into the new unreplicated key. if err := storage.MVCCDelete( ctx, readWriter, cArgs.Stats, keys.RaftTruncatedStateLegacyKey(cArgs.EvalCtx.GetRangeID()), - hlc.Timestamp{}, nil, /* txn */ + enginepb.TxnTimestamp{}, nil, /* txn */ ); err != nil { return result.Result{}, err } diff --git a/pkg/kv/kvserver/batcheval/eval_context.go b/pkg/kv/kvserver/batcheval/eval_context.go index eb3db03ee689..39056f3b8e12 100644 --- a/pkg/kv/kvserver/batcheval/eval_context.go +++ b/pkg/kv/kvserver/batcheval/eval_context.go @@ -76,8 +76,8 @@ type EvalContext interface { // for the provided transaction information. See Replica.CanCreateTxnRecord // for details about its arguments, return values, and preconditions. CanCreateTxnRecord( - txnID uuid.UUID, txnKey []byte, txnMinTS hlc.Timestamp, - ) (ok bool, minCommitTS hlc.Timestamp, reason roachpb.TransactionAbortedReason) + txnID uuid.UUID, txnKey []byte, txnMinTS enginepb.TxnTimestamp, + ) (ok bool, minCommitTS enginepb.TxnTimestamp, reason roachpb.TransactionAbortedReason) // GetMVCCStats returns a snapshot of the MVCC stats for the range. // If called from a command that declares a read/write span on the @@ -92,7 +92,7 @@ type EvalContext interface { // setting is disabled. GetSplitQPS() float64 - GetGCThreshold() hlc.Timestamp + GetGCThreshold() enginepb.TxnTimestamp GetLastReplicaGCTimestamp(context.Context) (hlc.Timestamp, error) GetLease() (roachpb.Lease, roachpb.Lease) GetDescAndLease(context.Context) (roachpb.RangeDescriptor, roachpb.Lease) @@ -112,9 +112,9 @@ type MockEvalCtx struct { Stats enginepb.MVCCStats QPS float64 AbortSpan *abortspan.AbortSpan - GCThreshold hlc.Timestamp + GCThreshold enginepb.TxnTimestamp Term, FirstIndex uint64 - CanCreateTxn func() (bool, hlc.Timestamp, roachpb.TransactionAbortedReason) + CanCreateTxn func() (bool, enginepb.TxnTimestamp, roachpb.TransactionAbortedReason) Lease roachpb.Lease } @@ -197,11 +197,11 @@ func (m *mockEvalCtxImpl) GetSplitQPS() float64 { return m.QPS } func (m *mockEvalCtxImpl) CanCreateTxnRecord( - uuid.UUID, []byte, hlc.Timestamp, -) (bool, hlc.Timestamp, roachpb.TransactionAbortedReason) { + uuid.UUID, []byte, enginepb.TxnTimestamp, +) (bool, enginepb.TxnTimestamp, roachpb.TransactionAbortedReason) { return m.CanCreateTxn() } -func (m *mockEvalCtxImpl) GetGCThreshold() hlc.Timestamp { +func (m *mockEvalCtxImpl) GetGCThreshold() enginepb.TxnTimestamp { return m.GCThreshold } func (m *mockEvalCtxImpl) GetLastReplicaGCTimestamp(context.Context) (hlc.Timestamp, error) { diff --git a/pkg/kv/kvserver/batcheval/transaction.go b/pkg/kv/kvserver/batcheval/transaction.go index 08d394cdff84..70f64213d020 100644 --- a/pkg/kv/kvserver/batcheval/transaction.go +++ b/pkg/kv/kvserver/batcheval/transaction.go @@ -19,7 +19,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" - "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" ) @@ -167,7 +166,8 @@ func SynthesizeTxnFromMeta(rec EvalContext, txn enginepb.TxnMeta) roachpb.Transa // use this as an indication of client activity. Note that we cannot use // txn.WriteTimestamp for that purpose, as the WriteTimestamp could have // been bumped by other pushers. - LastHeartbeat: txn.MinTimestamp, + // TODO(nvanbenschoten): what to do here. + LastHeartbeat: txn.MinTimestamp.ToClockTimestampUnchecked(), } // If the transaction metadata's min timestamp is empty this intent must @@ -201,7 +201,7 @@ func HasTxnRecord( ctx context.Context, reader storage.Reader, txn *roachpb.Transaction, ) (bool, error) { key := keys.TransactionKey(txn.Key, txn.ID) - val, _, err := storage.MVCCGet(ctx, reader, key, hlc.Timestamp{}, storage.MVCCGetOptions{}) + val, _, err := storage.MVCCGet(ctx, reader, key, enginepb.TxnTimestamp{}, storage.MVCCGetOptions{}) if err != nil { return false, err } diff --git a/pkg/kv/kvserver/closedts/closedts.go b/pkg/kv/kvserver/closedts/closedts.go index 72fad6d821f3..8e07422d5ce7 100644 --- a/pkg/kv/kvserver/closedts/closedts.go +++ b/pkg/kv/kvserver/closedts/closedts.go @@ -43,6 +43,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/hlc" ) @@ -62,8 +63,8 @@ type ReleaseFunc func(context.Context, ctpb.Epoch, roachpb.RangeID, ctpb.LAI) // // The methods exposed on Tracker are safe for concurrent use. type TrackerI interface { - Close(next hlc.Timestamp, expCurEpoch ctpb.Epoch) (hlc.Timestamp, map[roachpb.RangeID]ctpb.LAI, bool) - Track(ctx context.Context) (hlc.Timestamp, ReleaseFunc) + Close(next enginepb.TxnTimestamp, expCurEpoch ctpb.Epoch) (enginepb.TxnTimestamp, map[roachpb.RangeID]ctpb.LAI, bool) + Track(ctx context.Context) (enginepb.TxnTimestamp, ReleaseFunc) FailedCloseAttempts() int64 } @@ -142,7 +143,7 @@ type Provider interface { Producer Notifyee Start() - MaxClosed(roachpb.NodeID, roachpb.RangeID, ctpb.Epoch, ctpb.LAI) hlc.Timestamp + MaxClosed(roachpb.NodeID, roachpb.RangeID, ctpb.Epoch, ctpb.LAI) enginepb.TxnTimestamp } // A ClientRegistry is the client component of the follower reads subsystem. It @@ -162,11 +163,11 @@ type ClientRegistry interface { // detailed description of the semantics. The final returned boolean indicates // whether tracked epoch matched the expCurEpoch and that returned information // may be used. -type CloseFn func(next hlc.Timestamp, expCurEpoch ctpb.Epoch) (hlc.Timestamp, map[roachpb.RangeID]ctpb.LAI, bool) +type CloseFn func(next enginepb.TxnTimestamp, expCurEpoch ctpb.Epoch) (enginepb.TxnTimestamp, map[roachpb.RangeID]ctpb.LAI, bool) // AsCloseFn uses the TrackerI as a CloseFn. func AsCloseFn(t TrackerI) CloseFn { - return func(next hlc.Timestamp, expCurEpoch ctpb.Epoch) (hlc.Timestamp, map[roachpb.RangeID]ctpb.LAI, bool) { + return func(next enginepb.TxnTimestamp, expCurEpoch ctpb.Epoch) (enginepb.TxnTimestamp, map[roachpb.RangeID]ctpb.LAI, bool) { return t.Close(next, expCurEpoch) } } diff --git a/pkg/kv/kvserver/closedts/container/noop.go b/pkg/kv/kvserver/closedts/container/noop.go index abeec20f4872..f74619a4e922 100644 --- a/pkg/kv/kvserver/closedts/container/noop.go +++ b/pkg/kv/kvserver/closedts/container/noop.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/errors" ) @@ -54,12 +55,12 @@ func (noopEverything) Get(client ctpb.InboundClient) error { return errors.New("closed timestamps disabled") } func (noopEverything) Close( - next hlc.Timestamp, expCurEpoch ctpb.Epoch, -) (hlc.Timestamp, map[roachpb.RangeID]ctpb.LAI, bool) { - return hlc.Timestamp{}, nil, false + next enginepb.TxnTimestamp, expCurEpoch ctpb.Epoch, +) (enginepb.TxnTimestamp, map[roachpb.RangeID]ctpb.LAI, bool) { + return enginepb.TxnTimestamp{}, nil, false } -func (noopEverything) Track(ctx context.Context) (hlc.Timestamp, closedts.ReleaseFunc) { - return hlc.Timestamp{}, func(context.Context, ctpb.Epoch, roachpb.RangeID, ctpb.LAI) {} +func (noopEverything) Track(ctx context.Context) (enginepb.TxnTimestamp, closedts.ReleaseFunc) { + return enginepb.TxnTimestamp{}, func(context.Context, ctpb.Epoch, roachpb.RangeID, ctpb.LAI) {} } func (noopEverything) FailedCloseAttempts() int64 { return 0 @@ -75,8 +76,8 @@ func (noopEverything) Subscribe(context.Context, chan<- ctpb.Entry) {} func (noopEverything) Start() {} func (noopEverything) MaxClosed( roachpb.NodeID, roachpb.RangeID, ctpb.Epoch, ctpb.LAI, -) hlc.Timestamp { - return hlc.Timestamp{} +) enginepb.TxnTimestamp { + return enginepb.TxnTimestamp{} } func (noopEverything) Request(roachpb.NodeID, roachpb.RangeID) {} func (noopEverything) EnsureClient(roachpb.NodeID) {} diff --git a/pkg/kv/kvserver/closedts/ctpb/entry.pb.go b/pkg/kv/kvserver/closedts/ctpb/entry.pb.go index 96e381c0a392..24d2f79809ef 100644 --- a/pkg/kv/kvserver/closedts/ctpb/entry.pb.go +++ b/pkg/kv/kvserver/closedts/ctpb/entry.pb.go @@ -6,8 +6,8 @@ package ctpb import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" -import hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" +import github_com_cockroachdb_cockroach_pkg_storage_enginepb "github.com/cockroachdb/cockroach/pkg/storage/enginepb" import github_com_cockroachdb_cockroach_pkg_roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" import github_com_gogo_protobuf_sortkeys "github.com/gogo/protobuf/sortkeys" @@ -36,9 +36,9 @@ const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package // are incremental. An incremental update represents the implicit promise that // the state accumulated since the last full Entry is the true full state. type Entry struct { - Epoch Epoch `protobuf:"varint,1,opt,name=epoch,proto3,casttype=Epoch" json:"epoch,omitempty"` - ClosedTimestamp hlc.Timestamp `protobuf:"bytes,2,opt,name=closed_timestamp,json=closedTimestamp,proto3" json:"closed_timestamp"` - MLAI map[github_com_cockroachdb_cockroach_pkg_roachpb.RangeID]LAI `protobuf:"bytes,3,rep,name=mlai,proto3,castkey=github.com/cockroachdb/cockroach/pkg/roachpb.RangeID,castvalue=LAI" json:"mlai,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` + Epoch Epoch `protobuf:"varint,1,opt,name=epoch,proto3,casttype=Epoch" json:"epoch,omitempty"` + ClosedTimestamp github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp `protobuf:"bytes,2,opt,name=closed_timestamp,json=closedTimestamp,proto3,casttype=github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp" json:"closed_timestamp"` + MLAI map[github_com_cockroachdb_cockroach_pkg_roachpb.RangeID]LAI `protobuf:"bytes,3,rep,name=mlai,proto3,castkey=github.com/cockroachdb/cockroach/pkg/roachpb.RangeID,castvalue=LAI" json:"mlai,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` // Full is true if the emitter promises that any future write to any range // mentioned in this Entry will be reflected in a subsequent Entry before any // stale follower reads are possible. For example, if range 1 is assigned an @@ -55,7 +55,7 @@ type Entry struct { func (m *Entry) Reset() { *m = Entry{} } func (*Entry) ProtoMessage() {} func (*Entry) Descriptor() ([]byte, []int) { - return fileDescriptor_entry_cedfda89c463dabb, []int{0} + return fileDescriptor_entry_8e5a18907dc908d1, []int{0} } func (m *Entry) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -92,7 +92,7 @@ type Reaction struct { func (m *Reaction) Reset() { *m = Reaction{} } func (*Reaction) ProtoMessage() {} func (*Reaction) Descriptor() ([]byte, []int) { - return fileDescriptor_entry_cedfda89c463dabb, []int{1} + return fileDescriptor_entry_8e5a18907dc908d1, []int{1} } func (m *Reaction) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -726,37 +726,39 @@ var ( ) func init() { - proto.RegisterFile("kv/kvserver/closedts/ctpb/entry.proto", fileDescriptor_entry_cedfda89c463dabb) + proto.RegisterFile("kv/kvserver/closedts/ctpb/entry.proto", fileDescriptor_entry_8e5a18907dc908d1) } -var fileDescriptor_entry_cedfda89c463dabb = []byte{ - // 442 bytes of a gzipped FileDescriptorProto +var fileDescriptor_entry_8e5a18907dc908d1 = []byte{ + // 467 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x92, 0x3f, 0x6f, 0xd4, 0x30, - 0x18, 0xc6, 0xe3, 0x4b, 0x82, 0x7a, 0xee, 0x40, 0x15, 0x75, 0x88, 0x4e, 0xe0, 0x44, 0x95, 0x40, - 0x99, 0x6c, 0xa9, 0x20, 0x51, 0x75, 0x6b, 0x44, 0x85, 0x4e, 0x6a, 0x19, 0x2c, 0xc4, 0xc0, 0x82, - 0x1c, 0x9f, 0x49, 0xa2, 0xf8, 0xe2, 0x90, 0x38, 0x91, 0xba, 0x32, 0x22, 0x06, 0x46, 0x46, 0x3e, - 0xce, 0x8d, 0x1d, 0x2b, 0x86, 0x2b, 0xe4, 0xbe, 0x45, 0x27, 0x14, 0x87, 0xde, 0x6d, 0x0c, 0xdd, - 0x7e, 0x79, 0xff, 0x3d, 0x4f, 0xde, 0xd7, 0xf0, 0x59, 0xd1, 0x91, 0xa2, 0x6b, 0x44, 0xdd, 0x89, - 0x9a, 0x70, 0xa9, 0x1a, 0xb1, 0xd0, 0x0d, 0xe1, 0xba, 0x4a, 0x88, 0x28, 0x75, 0x7d, 0x85, 0xab, - 0x5a, 0x69, 0xe5, 0x21, 0xae, 0x78, 0x51, 0x2b, 0xc6, 0x33, 0x5c, 0x74, 0xf8, 0xbe, 0x01, 0x73, - 0xdd, 0x56, 0x0b, 0xa6, 0xc5, 0xec, 0x30, 0x55, 0xa9, 0x32, 0xa5, 0x64, 0xa0, 0xb1, 0x6b, 0xf6, - 0x24, 0x55, 0x2a, 0x95, 0x82, 0xb0, 0x2a, 0x27, 0xac, 0x2c, 0x95, 0x66, 0x3a, 0x57, 0x65, 0xf3, - 0x2f, 0xeb, 0xb7, 0x3a, 0x97, 0x24, 0x93, 0x9c, 0xe8, 0x7c, 0x29, 0x1a, 0xcd, 0x96, 0xd5, 0x98, - 0x39, 0xfa, 0x35, 0x81, 0xee, 0xf9, 0xa0, 0xee, 0x05, 0xd0, 0x15, 0x95, 0xe2, 0x99, 0x0f, 0x42, - 0x10, 0xd9, 0xf1, 0xf4, 0x6e, 0x1d, 0xb8, 0xe7, 0x43, 0x80, 0x8e, 0x71, 0xef, 0x2d, 0x3c, 0x18, - 0x5d, 0x7f, 0xdc, 0x0e, 0xf1, 0x27, 0x21, 0x88, 0xf6, 0x8f, 0x9f, 0xe2, 0x9d, 0xe7, 0x41, 0x09, - 0x67, 0x92, 0xe3, 0x77, 0xf7, 0x45, 0xb1, 0xb3, 0x5a, 0x07, 0x16, 0x7d, 0x3c, 0x36, 0x6f, 0xc3, - 0xde, 0x37, 0x00, 0x9d, 0xa5, 0x64, 0xb9, 0x6f, 0x87, 0x76, 0xb4, 0x7f, 0x4c, 0xf0, 0xff, 0x7f, - 0x1c, 0x1b, 0x9b, 0xf8, 0x52, 0xb2, 0xdc, 0x50, 0xfc, 0xa6, 0x5f, 0x07, 0xce, 0xe5, 0xc5, 0xd9, - 0xfc, 0xcb, 0x6d, 0xf0, 0x32, 0xcd, 0x75, 0xd6, 0x26, 0x98, 0xab, 0x25, 0xd9, 0x8e, 0x59, 0x24, - 0x3b, 0x26, 0x55, 0x91, 0x12, 0x43, 0x55, 0x82, 0x29, 0x2b, 0x53, 0x31, 0x7f, 0xfd, 0xf5, 0x36, - 0xb0, 0x2f, 0xce, 0xe6, 0xd4, 0xb8, 0xf0, 0x3c, 0xe8, 0x7c, 0x6a, 0xa5, 0xf4, 0x9d, 0x10, 0x44, - 0x7b, 0xd4, 0xf0, 0xec, 0x15, 0x9c, 0x6e, 0xf5, 0xbc, 0x03, 0x68, 0x17, 0xe2, 0xca, 0xac, 0xc7, - 0xa5, 0x03, 0x7a, 0x87, 0xd0, 0xed, 0x98, 0x6c, 0x85, 0x59, 0x83, 0x4d, 0xc7, 0x8f, 0xd3, 0xc9, - 0x09, 0x38, 0x75, 0x7e, 0xfc, 0x0c, 0xac, 0xa3, 0x0c, 0xee, 0x51, 0xc1, 0xf8, 0x70, 0x09, 0xef, - 0x3d, 0x9c, 0x52, 0xf1, 0xb9, 0x15, 0x8d, 0x16, 0x0b, 0x1f, 0x84, 0x76, 0xe4, 0xc6, 0x27, 0x77, - 0xeb, 0x87, 0x19, 0xa7, 0xbb, 0x51, 0xa3, 0x52, 0xfc, 0x7c, 0xf5, 0x07, 0x59, 0xab, 0x1e, 0x81, - 0xeb, 0x1e, 0x81, 0x9b, 0x1e, 0x81, 0xdf, 0x3d, 0x02, 0xdf, 0x37, 0xc8, 0xba, 0xde, 0x20, 0xeb, - 0x66, 0x83, 0xac, 0x0f, 0xce, 0xf0, 0xd0, 0x92, 0x47, 0xe6, 0xea, 0x2f, 0xfe, 0x06, 0x00, 0x00, - 0xff, 0xff, 0x39, 0x68, 0xb2, 0x7f, 0x8c, 0x02, 0x00, 0x00, + 0x18, 0xc6, 0xe3, 0x26, 0x41, 0x3d, 0x77, 0xa0, 0x8a, 0x3a, 0x44, 0x27, 0x48, 0x4e, 0x95, 0x40, + 0x99, 0x6c, 0xa9, 0x20, 0x51, 0x75, 0x6b, 0x44, 0x85, 0x0e, 0xb5, 0x8b, 0x55, 0x31, 0xb0, 0x20, + 0xc7, 0x67, 0x92, 0x28, 0xbe, 0x38, 0x24, 0x4e, 0x44, 0x57, 0x46, 0xc4, 0xc0, 0xc0, 0xc0, 0x08, + 0xdf, 0xe6, 0xc6, 0x8e, 0x9d, 0x52, 0xc8, 0x7d, 0x8b, 0x9b, 0x50, 0x1c, 0x9a, 0xdb, 0x10, 0xea, + 0xf6, 0xe4, 0xfd, 0xf7, 0xfc, 0xf2, 0xfa, 0x85, 0x4f, 0xb2, 0x06, 0x67, 0x4d, 0xc5, 0xcb, 0x86, + 0x97, 0x98, 0x09, 0x59, 0xf1, 0x85, 0xaa, 0x30, 0x53, 0x45, 0x84, 0x79, 0xae, 0xca, 0x2b, 0x54, + 0x94, 0x52, 0x49, 0xc7, 0x63, 0x92, 0x65, 0xa5, 0xa4, 0x2c, 0x41, 0x59, 0x83, 0xee, 0x1a, 0x10, + 0x53, 0x75, 0xb1, 0xa0, 0x8a, 0x4f, 0x0f, 0x62, 0x19, 0x4b, 0x5d, 0x8a, 0x7b, 0x35, 0x74, 0x4d, + 0x1f, 0xc5, 0x52, 0xc6, 0x82, 0x63, 0x5a, 0xa4, 0x98, 0xe6, 0xb9, 0x54, 0x54, 0xa5, 0x32, 0xaf, + 0xfe, 0x66, 0xdd, 0x5a, 0xa5, 0x02, 0x27, 0x82, 0x61, 0x95, 0x2e, 0x79, 0xa5, 0xe8, 0xb2, 0x18, + 0x32, 0x87, 0x3f, 0x4d, 0x68, 0x9f, 0xf5, 0xee, 0x8e, 0x0f, 0x6d, 0x5e, 0x48, 0x96, 0xb8, 0x60, + 0x06, 0x02, 0x33, 0x9c, 0x6c, 0x5a, 0xdf, 0x3e, 0xeb, 0x03, 0x64, 0x88, 0x3b, 0xdf, 0x00, 0xdc, + 0x1f, 0xb0, 0xdf, 0x8d, 0x53, 0xdc, 0x9d, 0x19, 0x08, 0xf6, 0x8e, 0x1e, 0xa3, 0x2d, 0x74, 0x6f, + 0x85, 0x12, 0xc1, 0xd0, 0xe5, 0x5d, 0x51, 0xf8, 0x7a, 0xd5, 0xfa, 0xc6, 0xa6, 0xf5, 0xc3, 0x38, + 0x55, 0x49, 0x1d, 0x21, 0x26, 0x97, 0x78, 0x6c, 0x58, 0x44, 0x5b, 0x8d, 0x8b, 0x2c, 0xc6, 0x95, + 0x92, 0x25, 0x8d, 0x39, 0xe6, 0x79, 0x9c, 0xe6, 0xbc, 0x88, 0xd0, 0xe5, 0xc7, 0x7c, 0x9c, 0x45, + 0x1e, 0x0e, 0x08, 0x63, 0xc0, 0xf9, 0x02, 0xa0, 0xb5, 0x14, 0x34, 0x75, 0xcd, 0x99, 0x19, 0xec, + 0x1d, 0x61, 0xf4, 0xef, 0xfd, 0x21, 0xfd, 0xb7, 0xe8, 0x42, 0xd0, 0x54, 0xab, 0xf0, 0x55, 0xd7, + 0xfa, 0xd6, 0xc5, 0xf9, 0xe9, 0xfc, 0xd3, 0xad, 0xff, 0xfc, 0xbf, 0x00, 0xb5, 0x2a, 0x22, 0x44, + 0x68, 0x1e, 0xf3, 0xf9, 0xcb, 0xcf, 0xb7, 0xbe, 0x79, 0x7e, 0x3a, 0x27, 0x9a, 0xc2, 0x71, 0xa0, + 0xf5, 0xbe, 0x16, 0xc2, 0xb5, 0x66, 0x20, 0xd8, 0x25, 0x5a, 0x4f, 0x5f, 0xc0, 0xc9, 0xe8, 0xe7, + 0xec, 0x43, 0x33, 0xe3, 0x57, 0x7a, 0xcb, 0x36, 0xe9, 0xa5, 0x73, 0x00, 0xed, 0x86, 0x8a, 0x9a, + 0xeb, 0x65, 0x9a, 0x64, 0xf8, 0x38, 0xd9, 0x39, 0x06, 0x27, 0xd6, 0xf7, 0x1f, 0xbe, 0x71, 0x98, + 0xc0, 0x5d, 0xc2, 0x29, 0xeb, 0x1f, 0xd4, 0x79, 0x03, 0x27, 0x84, 0x7f, 0xa8, 0x79, 0xa5, 0xf8, + 0xc2, 0x05, 0x33, 0x33, 0xb0, 0xc3, 0xe3, 0x4d, 0x7b, 0x3f, 0x70, 0xb2, 0x1d, 0x35, 0x38, 0x85, + 0x4f, 0x57, 0xbf, 0x3d, 0x63, 0xd5, 0x79, 0xe0, 0xba, 0xf3, 0xc0, 0x4d, 0xe7, 0x81, 0x5f, 0x9d, + 0x07, 0xbe, 0xae, 0x3d, 0xe3, 0x7a, 0xed, 0x19, 0x37, 0x6b, 0xcf, 0x78, 0x6b, 0xf5, 0xf7, 0x1a, + 0x3d, 0xd0, 0xc7, 0xf3, 0xec, 0x4f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x2c, 0x10, 0x7b, 0x5c, 0xd3, + 0x02, 0x00, 0x00, } diff --git a/pkg/kv/kvserver/closedts/ctpb/entry.proto b/pkg/kv/kvserver/closedts/ctpb/entry.proto index 47335f424626..0b4c8f2f7bc6 100644 --- a/pkg/kv/kvserver/closedts/ctpb/entry.proto +++ b/pkg/kv/kvserver/closedts/ctpb/entry.proto @@ -31,7 +31,8 @@ message Entry { option (gogoproto.goproto_stringer) = false; int64 epoch = 1 [(gogoproto.casttype) = "Epoch"]; - util.hlc.Timestamp closed_timestamp = 2 [(gogoproto.nullable) = false]; + util.hlc.Timestamp closed_timestamp = 2 [(gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp"]; map mlai = 3 [(gogoproto.castkey) = "github.com/cockroachdb/cockroach/pkg/roachpb.RangeID", (gogoproto.castvalue) = "LAI", (gogoproto.customname) = "MLAI"]; diff --git a/pkg/kv/kvserver/closedts/minprop/tracker.go b/pkg/kv/kvserver/closedts/minprop/tracker.go index df5cd39ca368..73609d708dce 100644 --- a/pkg/kv/kvserver/closedts/minprop/tracker.go +++ b/pkg/kv/kvserver/closedts/minprop/tracker.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/syncutil" @@ -28,7 +29,7 @@ type Tracker struct { mu struct { syncutil.Mutex // closed is the most recently closed timestamp. - closed hlc.Timestamp + closed enginepb.TxnTimestamp closedEpoch ctpb.Epoch // The variables below track required information for the next closed @@ -78,7 +79,7 @@ type Tracker struct { // later epoch than is currently tracked will result in the current data // corresponding to the prior epoch to be evicted. - next hlc.Timestamp + next enginepb.TxnTimestamp leftMLAI, rightMLAI map[roachpb.RangeID]ctpb.LAI leftRef, rightRef int leftEpoch, rightEpoch ctpb.Epoch @@ -99,7 +100,7 @@ func NewTracker() *Tracker { t.mu.closedEpoch = initialEpoch t.mu.leftEpoch = initialEpoch t.mu.rightEpoch = initialEpoch - t.mu.next = hlc.Timestamp{Logical: 1} + t.mu.next = enginepb.TxnTimestamp(hlc.Timestamp{Logical: 1}) t.mu.leftMLAI = map[roachpb.RangeID]ctpb.LAI{} t.mu.rightMLAI = map[roachpb.RangeID]ctpb.LAI{} return t @@ -191,8 +192,8 @@ func (t *Tracker) String() string { // tracked data the state of the tracker is progressed but zero values are // returned. func (t *Tracker) Close( - next hlc.Timestamp, expCurEpoch ctpb.Epoch, -) (ts hlc.Timestamp, mlai map[roachpb.RangeID]ctpb.LAI, ok bool) { + next enginepb.TxnTimestamp, expCurEpoch ctpb.Epoch, +) (ts enginepb.TxnTimestamp, mlai map[roachpb.RangeID]ctpb.LAI, ok bool) { t.mu.Lock() defer t.mu.Unlock() defer func() { @@ -259,7 +260,7 @@ func (t *Tracker) Close( } if t.mu.closedEpoch != expCurEpoch { - return hlc.Timestamp{}, nil, false + return enginepb.TxnTimestamp{}, nil, false } return t.mu.closed, mlai, true } @@ -275,7 +276,7 @@ func (t *Tracker) Close( // // The ReleaseFunc is not thread safe. For convenience, it may be called with // zero arguments once after a regular call. -func (t *Tracker) Track(ctx context.Context) (hlc.Timestamp, closedts.ReleaseFunc) { +func (t *Tracker) Track(ctx context.Context) (enginepb.TxnTimestamp, closedts.ReleaseFunc) { shouldLog := log.V(3) t.mu.Lock() @@ -316,7 +317,7 @@ func (t *Tracker) FailedCloseAttempts() int64 { // Track. func (t *Tracker) release( ctx context.Context, - minProp hlc.Timestamp, + minProp enginepb.TxnTimestamp, epoch ctpb.Epoch, rangeID roachpb.RangeID, lai ctpb.LAI, @@ -370,7 +371,7 @@ func releaseProposal( ctx context.Context, side string, shouldLog bool, - minProp hlc.Timestamp, + minProp enginepb.TxnTimestamp, rangeID roachpb.RangeID, lai ctpb.LAI, refs *int, diff --git a/pkg/kv/kvserver/closedts/provider/provider.go b/pkg/kv/kvserver/closedts/provider/provider.go index 33fb0b4243d8..25d35a6dbaee 100644 --- a/pkg/kv/kvserver/closedts/provider/provider.go +++ b/pkg/kv/kvserver/closedts/provider/provider.go @@ -20,7 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/ctpb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/stop" "github.com/cockroachdb/cockroach/pkg/util/syncutil" @@ -161,7 +161,7 @@ func (p *Provider) runCloser(ctx context.Context) { everBeenLive = true // Close may fail if the data being closed does not correspond to the // current liveAtEpoch. - closed, m, ok := p.cfg.Close(next, liveAtEpoch) + closed, m, ok := p.cfg.Close(enginepb.TxnTimestamp(next), liveAtEpoch) if !ok { if log.V(1) { log.Infof(ctx, "failed to close %v due to liveness epoch mismatch at %v", @@ -343,8 +343,8 @@ func (p *Provider) Subscribe(ctx context.Context, ch chan<- ctpb.Entry) { // MaxClosed implements closedts.Provider. func (p *Provider) MaxClosed( nodeID roachpb.NodeID, rangeID roachpb.RangeID, epoch ctpb.Epoch, lai ctpb.LAI, -) hlc.Timestamp { - var maxTS hlc.Timestamp +) enginepb.TxnTimestamp { + var maxTS enginepb.TxnTimestamp p.cfg.Storage.VisitDescending(nodeID, func(entry ctpb.Entry) (done bool) { if mlai, found := entry.MLAI[rangeID]; found { if entry.Epoch == epoch && mlai <= lai { diff --git a/pkg/kv/kvserver/concurrency/concurrency_control.go b/pkg/kv/kvserver/concurrency/concurrency_control.go index de7e90222cd8..b2734fb60a3b 100644 --- a/pkg/kv/kvserver/concurrency/concurrency_control.go +++ b/pkg/kv/kvserver/concurrency/concurrency_control.go @@ -22,7 +22,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/txnwait" "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/uuid" ) @@ -318,7 +317,7 @@ type Request struct { // The timestamp that the request should evaluate at. // Should be set to Txn.ReadTimestamp if Txn is non-nil. - Timestamp hlc.Timestamp + Timestamp enginepb.TxnTimestamp // The priority of the request. Only set if Txn is nil. Priority roachpb.UserPriority diff --git a/pkg/kv/kvserver/concurrency/concurrency_manager.go b/pkg/kv/kvserver/concurrency/concurrency_manager.go index 2e5e53a8d5d4..678f75c8718c 100644 --- a/pkg/kv/kvserver/concurrency/concurrency_manager.go +++ b/pkg/kv/kvserver/concurrency/concurrency_manager.go @@ -415,7 +415,7 @@ func (r *Request) txnMeta() *enginepb.TxnMeta { // for all locks acquired by other transactions at or below this timestamp // to be released. All locks acquired by other transactions above this // timestamp are ignored. -func (r *Request) readConflictTimestamp() hlc.Timestamp { +func (r *Request) readConflictTimestamp() enginepb.TxnTimestamp { ts := r.Timestamp if r.Txn != nil { ts = r.Txn.ReadTimestamp @@ -427,7 +427,7 @@ func (r *Request) readConflictTimestamp() hlc.Timestamp { // writeConflictTimestamp returns the minimum timestamp at which the request // acquires locks when performing mutations. All writes performed by the // requests must take place at or above this timestamp. -func (r *Request) writeConflictTimestamp() hlc.Timestamp { +func (r *Request) writeConflictTimestamp() enginepb.TxnTimestamp { ts := r.Timestamp if r.Txn != nil { ts = r.Txn.WriteTimestamp diff --git a/pkg/kv/kvserver/concurrency/lock_table.go b/pkg/kv/kvserver/concurrency/lock_table.go index 554268f83d53..8d446d595ae2 100644 --- a/pkg/kv/kvserver/concurrency/lock_table.go +++ b/pkg/kv/kvserver/concurrency/lock_table.go @@ -23,7 +23,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" "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/syncutil" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" @@ -260,8 +259,8 @@ type lockTableGuardImpl struct { // Information about this request. txn *enginepb.TxnMeta spans *spanset.SpanSet - readTS hlc.Timestamp - writeTS hlc.Timestamp + readTS enginepb.TxnTimestamp + writeTS enginepb.TxnTimestamp // Snapshots of the trees for which this request has some spans. Note that // the lockStates in these snapshots may have been removed from @@ -511,7 +510,7 @@ type lockHolderInfo struct { seqs []enginepb.TxnSeq // The timestamp at which the lock is held. - ts hlc.Timestamp + ts enginepb.TxnTimestamp } func (lh *lockHolderInfo) isEmpty() bool { @@ -751,12 +750,12 @@ func (l *lockState) Format(buf *strings.Builder) { fmt.Fprintln(buf, " empty") return } - writeResInfo := func(b *strings.Builder, txn *enginepb.TxnMeta, ts hlc.Timestamp) { + writeResInfo := func(b *strings.Builder, txn *enginepb.TxnMeta, ts enginepb.TxnTimestamp) { // TODO(sbhola): strip the leading 0 bytes from the UUID string since tests are assigning // UUIDs using a counter and makes this output more readable. fmt.Fprintf(b, "txn: %v, ts: %v, seq: %v\n", txn.ID, ts, txn.Sequence) } - writeHolderInfo := func(b *strings.Builder, txn *enginepb.TxnMeta, ts hlc.Timestamp) { + writeHolderInfo := func(b *strings.Builder, txn *enginepb.TxnMeta, ts enginepb.TxnTimestamp) { fmt.Fprintf(b, " holder: txn: %v, ts: %v, info: ", txn.ID, ts) first := true for i := range l.holder.holder { @@ -991,9 +990,9 @@ func (l *lockState) isLockedBy(id uuid.UUID) bool { // Returns information about the current lock holder if the lock is held, else // returns nil. // REQUIRES: l.mu is locked. -func (l *lockState) getLockHolder() (*enginepb.TxnMeta, hlc.Timestamp) { +func (l *lockState) getLockHolder() (*enginepb.TxnMeta, enginepb.TxnTimestamp) { if !l.holder.locked { - return nil, hlc.Timestamp{} + return nil, enginepb.TxnTimestamp{} } // If the lock is held as both replicated and unreplicated we want to @@ -1185,7 +1184,7 @@ func (l *lockState) tryActiveWait(g *lockTableGuardImpl, sa spanset.SpanAccess, // that is acquiring the lock. // Acquires l.mu. func (l *lockState) acquireLock( - _ lock.Strength, durability lock.Durability, txn *enginepb.TxnMeta, ts hlc.Timestamp, + _ lock.Strength, durability lock.Durability, txn *enginepb.TxnMeta, ts enginepb.TxnTimestamp, ) error { l.mu.Lock() defer l.mu.Unlock() @@ -1315,7 +1314,7 @@ func (l *lockState) acquireLock( // where g is trying to access this key with access sa. // Acquires l.mu. func (l *lockState) discoveredLock( - txn *enginepb.TxnMeta, ts hlc.Timestamp, g *lockTableGuardImpl, sa spanset.SpanAccess, + txn *enginepb.TxnMeta, ts enginepb.TxnTimestamp, g *lockTableGuardImpl, sa spanset.SpanAccess, ) error { l.mu.Lock() defer l.mu.Unlock() @@ -1575,7 +1574,7 @@ func (l *lockState) tryUpdateLock(up *roachpb.LockUpdate) (gc bool, err error) { // The lock holder timestamp has increased. Some of the waiters may no longer // need to wait. // REQUIRES: l.mu is locked. -func (l *lockState) increasedLockTs(newTs hlc.Timestamp) { +func (l *lockState) increasedLockTs(newTs enginepb.TxnTimestamp) { distinguishedRemoved := false for e := l.waitingReaders.Front(); e != nil; { g := e.Value.(*lockTableGuardImpl) diff --git a/pkg/kv/kvserver/gc/gc.go b/pkg/kv/kvserver/gc/gc.go index 0f76a4ad88af..dfbe0a9423f8 100644 --- a/pkg/kv/kvserver/gc/gc.go +++ b/pkg/kv/kvserver/gc/gc.go @@ -50,16 +50,16 @@ const ( // CalculateThreshold calculates the GC threshold given the policy and the // current view of time. -func CalculateThreshold(now hlc.Timestamp, policy zonepb.GCPolicy) (threshold hlc.Timestamp) { +func CalculateThreshold(now hlc.Timestamp, policy zonepb.GCPolicy) (threshold enginepb.TxnTimestamp) { ttlNanos := int64(policy.TTLSeconds) * time.Second.Nanoseconds() - return now.Add(-ttlNanos, 0) + return enginepb.TxnTimestamp(now.Add(-ttlNanos, 0)) } // TimestampForThreshold inverts CalculateThreshold. It returns the timestamp // which should be used for now to arrive at the passed threshold. -func TimestampForThreshold(threshold hlc.Timestamp, policy zonepb.GCPolicy) (ts hlc.Timestamp) { +func TimestampForThreshold(threshold enginepb.TxnTimestamp, policy zonepb.GCPolicy) (ts hlc.Timestamp) { ttlNanos := int64(policy.TTLSeconds) * time.Second.Nanoseconds() - return threshold.Add(ttlNanos, 0) + return hlc.Timestamp(threshold.Add(ttlNanos, 0)) } // Thresholder is part of the GCer interface. @@ -89,9 +89,10 @@ func (NoopGCer) SetGCThreshold(context.Context, Threshold) error { return nil } // GC implements storage.GCer. func (NoopGCer) GC(context.Context, []roachpb.GCRequest_GCKey) error { return nil } -// Threshold holds the key and txn span GC thresholds, respectively. +// Threshold holds the GC threshold. +// TODO(nvanbenschoten): remove the Txn field. It's unused. type Threshold struct { - Key hlc.Timestamp + Key enginepb.TxnTimestamp Txn hlc.Timestamp } @@ -126,7 +127,7 @@ type Info struct { // this cycle. ResolveTotal int // Threshold is the computed expiration timestamp. Equal to `Now - Policy`. - Threshold hlc.Timestamp + Threshold enginepb.TxnTimestamp // AffectedVersionsKeyBytes is the number of (fully encoded) bytes deleted from keys in the storage engine. // Note that this does not account for compression that the storage engine uses to store data on disk. Real // space savings tends to be smaller due to this compression, and space may be released only at a later point @@ -158,7 +159,8 @@ func Run( ctx context.Context, desc *roachpb.RangeDescriptor, snap storage.Reader, - now, newThreshold hlc.Timestamp, + now hlc.Timestamp, + newThreshold enginepb.TxnTimestamp, policy zonepb.GCPolicy, gcer GCer, cleanupIntentsFn CleanupIntentsFunc, @@ -199,7 +201,7 @@ func Run( // Clean up the AbortSpan. log.Event(ctx, "processing AbortSpan") - if err := processAbortSpan(ctx, snap, desc.RangeID, txnExp, &info, gcer); err != nil { + if err := processAbortSpan(ctx, snap, desc.RangeID, enginepb.TxnTimestamp(txnExp), &info, gcer); err != nil { if errors.Is(err, ctx.Err()) { return Info{}, err } @@ -232,7 +234,7 @@ func processReplicatedKeyRange( desc *roachpb.RangeDescriptor, snap storage.Reader, now hlc.Timestamp, - threshold hlc.Timestamp, + threshold enginepb.TxnTimestamp, gcer GCer, txnMap map[uuid.UUID]*roachpb.Transaction, intentKeyMap map[uuid.UUID][]roachpb.Key, @@ -287,7 +289,7 @@ func processReplicatedKeyRange( batchGCKeys []roachpb.GCRequest_GCKey batchGCKeysBytes int64 haveGarbageForThisKey bool - gcTimestampForThisKey hlc.Timestamp + gcTimestampForThisKey enginepb.TxnTimestamp sentBatchForThisKey bool ) it := makeGCIterator(desc, snap) @@ -327,7 +329,7 @@ func processReplicatedKeyRange( Timestamp: gcTimestampForThisKey, }) haveGarbageForThisKey = false - gcTimestampForThisKey = hlc.Timestamp{} + gcTimestampForThisKey = enginepb.TxnTimestamp{} // Mark that we sent a batch for this key so we know that we had garbage // even if it turns out that there's no more garbage for this key. @@ -373,7 +375,7 @@ func processReplicatedKeyRange( // guaranteed as described above. However if this were the only rule, then if // the most recent write was a delete, it would never be removed. Thus, when a // deleted value is the most recent before expiration, it can be deleted. -func isGarbage(threshold hlc.Timestamp, cur, next *storage.MVCCKeyValue, isNewest bool) bool { +func isGarbage(threshold enginepb.TxnTimestamp, cur, next *storage.MVCCKeyValue, isNewest bool) bool { // If the value is not at or below the threshold then it's not garbage. if belowThreshold := cur.Key.Timestamp.LessEq(threshold); !belowThreshold { return false @@ -483,7 +485,7 @@ func processLocalKeyRange( startKey := keys.MakeRangeKeyPrefix(desc.StartKey) endKey := keys.MakeRangeKeyPrefix(desc.EndKey) - _, err := storage.MVCCIterate(ctx, snap, startKey, endKey, hlc.Timestamp{}, storage.MVCCScanOptions{}, + _, err := storage.MVCCIterate(ctx, snap, startKey, endKey, enginepb.TxnTimestamp{}, storage.MVCCScanOptions{}, func(kv roachpb.KeyValue) error { return handleOne(kv) }) @@ -499,7 +501,7 @@ func processAbortSpan( ctx context.Context, snap storage.Reader, rangeID roachpb.RangeID, - threshold hlc.Timestamp, + threshold enginepb.TxnTimestamp, info *Info, gcer PureGCer, ) error { diff --git a/pkg/kv/kvserver/gc_queue.go b/pkg/kv/kvserver/gc_queue.go index 52df3b6179a3..489defd29719 100644 --- a/pkg/kv/kvserver/gc_queue.go +++ b/pkg/kv/kvserver/gc_queue.go @@ -277,7 +277,7 @@ func makeGCQueueScoreImpl( now hlc.Timestamp, ms enginepb.MVCCStats, policy zonepb.GCPolicy, - gcThreshold hlc.Timestamp, + gcThreshold enginepb.TxnTimestamp, ) gcQueueScore { ms.Forward(now.WallTime) var r gcQueueScore @@ -380,7 +380,7 @@ func (r *replicaGCer) send(ctx context.Context, req roachpb.GCRequest) error { // Technically not needed since we're talking directly to the Replica. ba.RangeID = r.repl.Desc().RangeID - ba.Timestamp = r.repl.Clock().Now() + ba.Timestamp = enginepb.TxnTimestamp(r.repl.Clock().Now()) ba.Add(&req) if _, pErr := r.repl.Send(ctx, ba); pErr != nil { @@ -392,7 +392,7 @@ func (r *replicaGCer) send(ctx context.Context, req roachpb.GCRequest) error { func (r *replicaGCer) SetGCThreshold(ctx context.Context, thresh gc.Threshold) error { req := r.template() - req.Threshold = thresh.Key + req.Threshold = enginepb.TxnTimestamp(thresh.Key) return r.send(ctx, req) } @@ -460,6 +460,7 @@ func (gcq *gcQueue) process( info, err := gc.Run(ctx, desc, snap, gcTimestamp, newThreshold, *zone.GC, &replicaGCer{repl: repl}, func(ctx context.Context, intents []roachpb.Intent) error { + // TODO(nvanbenshoten): this is wrong. intentCount, err := repl.store.intentResolver. CleanupIntents(ctx, intents, gcTimestamp, roachpb.PUSH_ABORT) if err == nil { @@ -468,6 +469,7 @@ func (gcq *gcQueue) process( return err }, func(ctx context.Context, txn *roachpb.Transaction) error { + // TODO(nvanbenshoten): this is wrong. err := repl.store.intentResolver. CleanupTxnIntentsOnGCAsync(ctx, repl.RangeID, txn, gcTimestamp, func(pushed, succeeded bool) { diff --git a/pkg/kv/kvserver/intentresolver/intent_resolver.go b/pkg/kv/kvserver/intentresolver/intent_resolver.go index 6dc5808ee9f7..5a5ef720d080 100644 --- a/pkg/kv/kvserver/intentresolver/intent_resolver.go +++ b/pkg/kv/kvserver/intentresolver/intent_resolver.go @@ -358,7 +358,8 @@ func (ir *IntentResolver) MaybePushTransactions( // Attempt to push the transaction(s). b := &kv.Batch{} - b.Header.Timestamp = ir.clock.Now() + // TODO(nvanbenschoten): how does the PushTxn work in this case? + b.Header.Timestamp = enginepb.TxnTimestamp(ir.clock.Now()) for _, pushTxn := range pushTxns { b.AddRawRequest(&roachpb.PushTxnRequest{ RequestHeader: roachpb.RequestHeader{ @@ -455,7 +456,8 @@ func (ir *IntentResolver) CleanupIntentsAsync( func (ir *IntentResolver) CleanupIntents( ctx context.Context, intents []roachpb.Intent, now hlc.Timestamp, pushType roachpb.PushTxnType, ) (int, error) { - h := roachpb.Header{Timestamp: now} + // TODO(nvanbenschoten): how does the PushTxn work in this case? + h := roachpb.Header{Timestamp: enginepb.TxnTimestamp(now)} // All transactions in MaybePushTransactions will be sent in a single batch. // In order to ensure that progress is made, we want to ensure that this @@ -627,7 +629,8 @@ func (ir *IntentResolver) CleanupTxnIntentsOnGCAsync( return } b := &kv.Batch{} - b.Header.Timestamp = now + // TODO(nvanbenschoten): how does the PushTxn work in this case? + b.Header.Timestamp = enginepb.TxnTimestamp(now) b.AddRawRequest(&roachpb.PushTxnRequest{ RequestHeader: roachpb.RequestHeader{Key: txn.Key}, PusherTxn: roachpb.Transaction{ @@ -770,7 +773,7 @@ type ResolveOptions struct { Poison bool // The original transaction timestamp from the earliest txn epoch; if // supplied, resolution of intent ranges can be optimized in some cases. - MinTimestamp hlc.Timestamp + MinTimestamp enginepb.TxnTimestamp } // lookupRangeID maps a key to a RangeID for best effort batching of intent diff --git a/pkg/kv/kvserver/kvserverbase/bulk_adder.go b/pkg/kv/kvserver/kvserverbase/bulk_adder.go index 563054da7d3d..e585a76738f5 100644 --- a/pkg/kv/kvserver/kvserverbase/bulk_adder.go +++ b/pkg/kv/kvserver/kvserverbase/bulk_adder.go @@ -16,7 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" ) // BulkAdderOptions is used to configure the behavior of a BulkAdder. @@ -66,7 +66,7 @@ const DisableExplicitSplits = -1 // BulkAdderFactory describes a factory function for BulkAdders. type BulkAdderFactory func( - ctx context.Context, db *kv.DB, timestamp hlc.Timestamp, opts BulkAdderOptions, + ctx context.Context, db *kv.DB, timestamp enginepb.TxnTimestamp, opts BulkAdderOptions, ) (BulkAdder, error) // BulkAdder describes a bulk-adding helper that can be used to add lots of KVs. diff --git a/pkg/kv/kvserver/kvserverpb/proposer_kv.pb.go b/pkg/kv/kvserver/kvserverpb/proposer_kv.pb.go index d0ed4bce4e06..30ea54debdf4 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_storage_enginepb "github.com/cockroachdb/cockroach/pkg/storage/enginepb" 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_8ad072a2f3dc105d, []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_8ad072a2f3dc105d, []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_8ad072a2f3dc105d, []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_8ad072a2f3dc105d, []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_8ad072a2f3dc105d, []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_8ad072a2f3dc105d, []int{5} } func (m *SuggestedCompaction) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -286,7 +287,7 @@ type ReplicatedEvalResult struct { // Duplicates BatchRequest.Timestamp for proposer-evaluated KV. Used // to verify the validity of the command (for lease coverage and GC // threshold). - Timestamp hlc.Timestamp `protobuf:"bytes,8,opt,name=timestamp,proto3" json:"timestamp"` + Timestamp github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp `protobuf:"bytes,8,opt,name=timestamp,proto3,casttype=github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp" json:"timestamp"` // The stats delta corresponding to the data in this WriteBatch. On // a split, contains only the contributions to the left-hand side. DeprecatedDelta *enginepb.MVCCStats `protobuf:"bytes,10,opt,name=deprecated_delta,json=deprecatedDelta,proto3" json:"deprecated_delta,omitempty"` @@ -304,7 +305,7 @@ 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_8ad072a2f3dc105d, []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_8ad072a2f3dc105d, []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_8ad072a2f3dc105d, []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_8ad072a2f3dc105d, []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_8ad072a2f3dc105d, []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_8ad072a2f3dc105d, []int{10} } func (m *RaftCommandFooter) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3476,97 +3477,99 @@ 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_8ad072a2f3dc105d) } -var fileDescriptor_proposer_kv_ed289abcb229e5c7 = []byte{ - // 1404 bytes of a gzipped FileDescriptorProto +var fileDescriptor_proposer_kv_8ad072a2f3dc105d = []byte{ + // 1426 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, 0x34, 0x92, 0x93, 0xc6, 0x8a, + 0xed, 0x26, 0x23, 0x27, 0x2d, 0xd2, 0x03, 0x31, 0x22, 0x27, 0x14, 0x2b, 0x8a, 0x64, 0x86, 0x23, + 0xc5, 0xfe, 0x04, 0xbd, 0xb6, 0x40, 0x0f, 0x3d, 0xb5, 0x39, 0xf6, 0x8b, 0x14, 0xc8, 0xa5, 0x40, + 0x8e, 0x41, 0x0f, 0x42, 0xe3, 0x5c, 0xfa, 0x19, 0x72, 0x2a, 0x66, 0x38, 0x94, 0xe4, 0xc0, 0xa9, + 0x95, 0xf6, 0x36, 0x7c, 0x33, 0xbf, 0xdf, 0x1b, 0xbe, 0x3f, 0xbf, 0x47, 0xc2, 0x46, 0x6f, 0x58, + 0xef, 0x0d, 0x63, 0xca, 0x86, 0x94, 0x8d, 0x17, 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, 0x8f, 0x9a, 0x31, 0x0f, 0x19, 0x71, 0x69, 0xd4, 0x59, 0x5b, 0x91, 0x9b, 0x51, + 0xa7, 0x4e, 0x22, 0x2f, 0xc1, 0xac, 0xa1, 0xd4, 0xe4, 0x10, 0x4e, 0x94, 0xed, 0x42, 0x6a, 0xeb, + 0x53, 0x4e, 0xa6, 0xec, 0x7f, 0x55, 0x4c, 0x75, 0x1a, 0xb8, 0x5e, 0x40, 0xc5, 0x81, 0xa1, 0x6d, + 0xab, 0xcd, 0xbf, 0x9d, 0xba, 0xb9, 0xa5, 0x76, 0x6b, 0xef, 0x78, 0x89, 0x98, 0x13, 0x4e, 0xd5, + 0x19, 0x63, 0xc0, 0x3d, 0xbf, 0xde, 0xf5, 0xed, 0x3a, 0xf7, 0xfa, 0x34, 0xe6, 0xa4, 0x1f, 0xa9, + 0x9d, 0x55, 0x37, 0x74, 0x43, 0xb9, 0xac, 0x8b, 0x55, 0x62, 0xad, 0xfd, 0x90, 0x81, 0x7c, 0x3b, + 0xf2, 0x3d, 0x8e, 0x9a, 0xb0, 0xc8, 0x99, 0xe7, 0xba, 0x94, 0x19, 0x99, 0xf5, 0xcc, 0x46, 0x71, + 0xb3, 0x6a, 0x4e, 0x42, 0xa1, 0x5e, 0xc6, 0x94, 0x47, 0x0f, 0x92, 0x63, 0x0d, 0xed, 0xf9, 0xa8, + 0x3a, 0xf7, 0x62, 0x54, 0xcd, 0xe0, 0x14, 0x89, 0x0e, 0xa0, 0xc0, 0xba, 0xb1, 0xe5, 0x50, 0x9f, + 0x13, 0x63, 0x5e, 0xd2, 0xfc, 0x73, 0x8a, 0x46, 0xbd, 0x9e, 0x99, 0xbe, 0x9e, 0xb9, 0xf7, 0xb0, + 0xd9, 0x6c, 0x73, 0xc2, 0xe3, 0x86, 0x2e, 0xc8, 0x8e, 0x47, 0x55, 0x0d, 0xdf, 0x69, 0x6f, 0x0b, + 0x38, 0xd6, 0x58, 0x37, 0x96, 0xab, 0x6b, 0xb9, 0x5f, 0x9f, 0x55, 0x33, 0x35, 0x0c, 0xf9, 0x3d, + 0xca, 0x5c, 0x3a, 0xdb, 0x4d, 0xe5, 0xd1, 0x77, 0xdf, 0x54, 0x71, 0x3a, 0x50, 0x6a, 0x76, 0x49, + 0xe0, 0x52, 0x4c, 0x23, 0xdf, 0xb3, 0x49, 0x8c, 0x76, 0xdf, 0x26, 0xdf, 0x38, 0x85, 0xfc, 0x24, + 0xe6, 0xf7, 0xbc, 0x7c, 0xfb, 0xac, 0x3a, 0x57, 0x7b, 0x35, 0x0f, 0xe5, 0x66, 0xd8, 0x8f, 0x06, + 0x9c, 0x36, 0xbb, 0xd4, 0xee, 0xc5, 0x83, 0x3e, 0xfa, 0x02, 0x8a, 0xb6, 0x5a, 0x5b, 0x9e, 0x23, + 0x7d, 0x2d, 0x35, 0x76, 0x04, 0xc3, 0xcf, 0xa3, 0xea, 0x96, 0xeb, 0xf1, 0xee, 0xa0, 0x63, 0xda, + 0x61, 0xbf, 0x3e, 0xf6, 0xee, 0x74, 0x26, 0xeb, 0x7a, 0xd4, 0x73, 0xeb, 0x32, 0xd5, 0x83, 0x81, + 0xe7, 0x98, 0x0f, 0x1e, 0xec, 0x6c, 0x1f, 0x8f, 0xaa, 0x90, 0xb2, 0xef, 0x6c, 0x63, 0x48, 0xd9, + 0x77, 0x1c, 0xf4, 0x0f, 0x58, 0x8e, 0xc9, 0x90, 0x5a, 0x71, 0x40, 0xa2, 0xb8, 0x1b, 0x72, 0x99, + 0x19, 0x0d, 0x2f, 0x09, 0x63, 0x5b, 0xd9, 0xd0, 0x16, 0xe4, 0xfa, 0xa1, 0x43, 0x8d, 0xec, 0x7a, + 0x66, 0xa3, 0x74, 0x6a, 0x48, 0x53, 0xf6, 0xbd, 0xd0, 0xa1, 0x58, 0x1e, 0x46, 0x15, 0x48, 0xfc, + 0x44, 0xa1, 0x17, 0x70, 0x23, 0x27, 0x69, 0xa7, 0x2c, 0xc8, 0x80, 0xc5, 0x21, 0x65, 0xb1, 0x17, + 0x06, 0x46, 0x7e, 0x3d, 0xb3, 0xb1, 0x8c, 0xd3, 0x47, 0x74, 0x07, 0x0a, 0x9c, 0xb2, 0xbe, 0x17, + 0x10, 0x4e, 0x8d, 0x85, 0xf5, 0xec, 0x46, 0x71, 0xf3, 0xd2, 0x29, 0x3e, 0x55, 0x8c, 0xb7, 0x69, + 0x6c, 0x33, 0x2f, 0xe2, 0x21, 0x6b, 0xe4, 0x44, 0x8c, 0xf0, 0x04, 0xac, 0x32, 0xf9, 0x10, 0x40, + 0x84, 0x98, 0xd8, 0x5c, 0xb0, 0xaf, 0x42, 0xbe, 0x73, 0xc4, 0x69, 0x2c, 0xe3, 0x9a, 0xc5, 0xc9, + 0x03, 0xba, 0x02, 0x28, 0x1e, 0xb8, 0x2e, 0x8d, 0x39, 0x75, 0x2c, 0xc2, 0xad, 0x80, 0x04, 0x61, + 0x2c, 0x83, 0x91, 0xc5, 0xfa, 0x78, 0xe7, 0x26, 0xdf, 0x17, 0x76, 0xc5, 0xfb, 0xcd, 0x3c, 0x9c, + 0x6b, 0xa7, 0x5b, 0x53, 0x1e, 0xee, 0x43, 0x21, 0xe6, 0x84, 0x71, 0xab, 0x47, 0x8f, 0x54, 0xf6, + 0xfe, 0xf7, 0x66, 0x54, 0xbd, 0x3a, 0x53, 0xe6, 0xd2, 0xb7, 0xbb, 0x4b, 0x8f, 0xb0, 0x26, 0x69, + 0xee, 0xd2, 0x23, 0xb4, 0x07, 0x8b, 0x34, 0x70, 0x24, 0xe1, 0xfc, 0x9f, 0x20, 0x5c, 0xa0, 0x81, + 0x23, 0xe8, 0x1e, 0x00, 0xd8, 0xe3, 0xfb, 0xca, 0xb4, 0x16, 0x37, 0xff, 0x63, 0x9e, 0x21, 0x6f, + 0xe6, 0xe4, 0x15, 0xa7, 0xea, 0x79, 0x8a, 0x48, 0x85, 0xe5, 0x47, 0x0d, 0x56, 0x55, 0x6e, 0x38, + 0x75, 0x6e, 0x0d, 0x89, 0x8f, 0x69, 0x3c, 0xf0, 0x85, 0x8c, 0xe4, 0xa5, 0x1e, 0xa9, 0xee, 0xff, + 0xef, 0x99, 0x0e, 0x15, 0x8b, 0x50, 0x01, 0x8a, 0x13, 0x2c, 0xba, 0x01, 0xf9, 0x58, 0x28, 0x8d, + 0xba, 0xf5, 0xe5, 0x33, 0x49, 0xa4, 0x2e, 0xe1, 0x04, 0x24, 0xd0, 0x7d, 0xd1, 0xfd, 0xb2, 0x1e, + 0x67, 0x41, 0x4b, 0xad, 0xc0, 0x09, 0x08, 0x6d, 0x80, 0xee, 0xc5, 0x96, 0x4f, 0x49, 0x4c, 0x2d, + 0x46, 0x9f, 0x0c, 0x68, 0xcc, 0x8d, 0x05, 0x59, 0xd8, 0x25, 0x2f, 0xde, 0x15, 0x66, 0x9c, 0x58, + 0xd1, 0x97, 0x19, 0x28, 0x8c, 0x55, 0xd6, 0xd0, 0xa4, 0xb3, 0xbf, 0x4f, 0x39, 0x13, 0xfd, 0x69, + 0x76, 0x7d, 0xdb, 0x3c, 0x48, 0x0f, 0x35, 0x5a, 0x22, 0xa4, 0x6f, 0x46, 0xd5, 0xc6, 0x4c, 0x59, + 0x7d, 0x7b, 0x1a, 0x98, 0x07, 0x87, 0xc1, 0x98, 0x0b, 0x4f, 0x7c, 0xa3, 0x7b, 0xa0, 0x3b, 0x34, + 0x62, 0x54, 0x26, 0x43, 0xa9, 0x2f, 0xbc, 0x87, 0xfa, 0xe2, 0xf2, 0x04, 0x2e, 0x25, 0x17, 0x7d, + 0x06, 0x65, 0x5b, 0x8a, 0x9c, 0xc5, 0x94, 0xca, 0x19, 0x4b, 0x92, 0xb0, 0x7e, 0x76, 0x05, 0x9d, + 0x10, 0x47, 0x5c, 0xb2, 0x4f, 0x0a, 0xec, 0x25, 0x28, 0x31, 0xf2, 0x98, 0x5b, 0x7e, 0xe8, 0xaa, + 0x9b, 0x2e, 0xcb, 0x06, 0x5c, 0x12, 0xd6, 0xdd, 0xd0, 0x4d, 0xfc, 0x3f, 0x81, 0x22, 0x71, 0x1c, + 0x2b, 0x8e, 0x39, 0xe9, 0xf8, 0xd4, 0x58, 0x91, 0xbe, 0x3f, 0x9a, 0xb5, 0x98, 0x4e, 0x94, 0xa4, + 0x79, 0xd3, 0x71, 0xda, 0xed, 0x03, 0xc1, 0xd3, 0x28, 0x09, 0x95, 0x9c, 0x3c, 0x63, 0x20, 0x8e, + 0xd3, 0x4e, 0x7c, 0xa0, 0xdb, 0x90, 0x4f, 0xee, 0x83, 0xa4, 0xb3, 0x7f, 0xcf, 0x14, 0x39, 0x79, + 0x5b, 0xa5, 0x49, 0x09, 0x1c, 0xed, 0xc1, 0xb9, 0x88, 0xd1, 0xa1, 0x2a, 0xa1, 0xe4, 0x0b, 0x83, + 0xf8, 0xc6, 0xea, 0x0c, 0xf5, 0x81, 0x57, 0x04, 0x52, 0x16, 0xd9, 0x3d, 0x85, 0x43, 0x9f, 0x83, + 0x6e, 0x27, 0xb3, 0xc3, 0x4a, 0x25, 0xdd, 0x38, 0x2f, 0xb9, 0xae, 0xce, 0xd4, 0xcc, 0x53, 0x43, + 0x07, 0x97, 0xed, 0x93, 0x86, 0xb5, 0x8f, 0x61, 0x2a, 0x1a, 0x08, 0x41, 0x4e, 0x7c, 0xa9, 0x24, + 0x72, 0x86, 0xe5, 0x1a, 0x55, 0x21, 0x6f, 0x33, 0x7b, 0x6b, 0x53, 0xf6, 0xf3, 0x72, 0xa3, 0x70, + 0x3c, 0xaa, 0xe6, 0x9b, 0xb8, 0xb9, 0xb5, 0x89, 0x13, 0x7b, 0xa2, 0x07, 0xad, 0x9c, 0x96, 0xd1, + 0xe7, 0x5b, 0x39, 0x2d, 0xaf, 0x2f, 0xb4, 0x72, 0xda, 0xa2, 0xae, 0xb5, 0x72, 0x5a, 0x41, 0x87, + 0x56, 0x4e, 0x2b, 0xe9, 0xe5, 0x56, 0x4e, 0x2b, 0xeb, 0x7a, 0x2b, 0xa7, 0xe9, 0xfa, 0x4a, 0x2b, + 0xa7, 0x9d, 0xd3, 0x57, 0x5b, 0x0b, 0xda, 0xd7, 0xfb, 0xfa, 0x77, 0xfb, 0xb5, 0x75, 0x80, 0x4f, + 0x99, 0xc7, 0x69, 0x83, 0x70, 0xbb, 0x7b, 0xda, 0x05, 0x6a, 0xf7, 0x61, 0x69, 0x37, 0x74, 0x3d, + 0x9b, 0xf8, 0x9f, 0x44, 0xbb, 0xa1, 0x8b, 0x6e, 0x42, 0x36, 0x8c, 0x84, 0xb0, 0x8b, 0x91, 0xf1, + 0xaf, 0xb3, 0x92, 0x34, 0x86, 0xaa, 0x1c, 0x09, 0x6c, 0xed, 0xa7, 0x3c, 0x14, 0x31, 0x79, 0xcc, + 0x9b, 0x61, 0xbf, 0x4f, 0x02, 0x07, 0x5d, 0x86, 0x72, 0x9f, 0x1c, 0xaa, 0x84, 0x79, 0x81, 0x43, + 0x0f, 0xa5, 0x74, 0xe4, 0xf0, 0x72, 0x9f, 0x1c, 0xca, 0x6c, 0xec, 0x08, 0x23, 0x3a, 0x80, 0xbf, + 0x4c, 0xb5, 0xd9, 0xf8, 0xdb, 0x51, 0xe2, 0xe4, 0x7c, 0x2b, 0x6e, 0x1a, 0xa7, 0xcc, 0xb0, 0x44, + 0x34, 0x2e, 0x4e, 0xa0, 0xf7, 0x14, 0x52, 0x6e, 0xa0, 0x21, 0x5c, 0x3c, 0x49, 0x65, 0xc5, 0x42, + 0x60, 0x02, 0x9b, 0x4a, 0xdd, 0xc9, 0x36, 0x3e, 0x7c, 0x33, 0xaa, 0x5e, 0x7b, 0xaf, 0x31, 0x20, + 0x89, 0xdb, 0x8a, 0x05, 0x9f, 0x8f, 0xa6, 0xfd, 0xa5, 0x66, 0xf4, 0x04, 0x2e, 0xb0, 0x71, 0xbb, + 0x58, 0x74, 0x48, 0x7c, 0x8b, 0xc9, 0x86, 0x91, 0x0d, 0x59, 0xdc, 0xfc, 0xff, 0x1f, 0xea, 0x36, + 0x15, 0xe7, 0x55, 0x76, 0xda, 0x70, 0xd8, 0x85, 0xe2, 0x53, 0x91, 0x6d, 0xab, 0x23, 0xd2, 0x6d, + 0x94, 0x66, 0x9c, 0x49, 0x93, 0x0a, 0xc1, 0xf0, 0x74, 0x52, 0x2d, 0x6d, 0x28, 0xf9, 0x49, 0x7a, + 0xad, 0x30, 0x12, 0x7a, 0x62, 0x94, 0x67, 0x9c, 0x39, 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, 0x10, 0xf0, 0x6d, 0xc2, 0xc9, 0xad, 0x80, 0xb3, 0x23, 0x5c, 0xe0, 0xe9, 0xf3, + 0xda, 0x0d, 0x28, 0x9d, 0xdc, 0x44, 0x3a, 0x64, 0xd3, 0xef, 0x87, 0x02, 0x16, 0x4b, 0xf1, 0xe5, + 0x32, 0x24, 0xfe, 0x20, 0x99, 0x9f, 0x05, 0x9c, 0x3c, 0x5c, 0x9b, 0xff, 0x40, 0xb4, 0x58, 0x56, + 0xcf, 0x8d, 0x1b, 0x6d, 0x5e, 0xcf, 0x26, 0x4d, 0xf4, 0xfd, 0x7e, 0xed, 0x3a, 0xac, 0x4c, 0x5d, + 0xe0, 0x76, 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, 0xab, 0xd7, + 0x95, 0xb9, 0x17, 0xaf, 0x2b, 0x73, 0x2f, 0x5f, 0x57, 0xe6, 0x1e, 0xc1, 0xe4, 0x5f, 0xa3, 0xb3, + 0x20, 0x7f, 0x1b, 0xb6, 0x7e, 0x0b, 0x00, 0x00, 0xff, 0xff, 0xa8, 0x5e, 0xae, 0xd8, 0x51, 0x0d, + 0x00, 0x00, } diff --git a/pkg/kv/kvserver/kvserverpb/proposer_kv.proto b/pkg/kv/kvserver/kvserverpb/proposer_kv.proto index d7ccd83e4e93..3f9b37b56a43 100644 --- a/pkg/kv/kvserver/kvserverpb/proposer_kv.proto +++ b/pkg/kv/kvserver/kvserverpb/proposer_kv.proto @@ -127,7 +127,8 @@ message ReplicatedEvalResult { // Duplicates BatchRequest.Timestamp for proposer-evaluated KV. Used // to verify the validity of the command (for lease coverage and GC // threshold). - util.hlc.Timestamp timestamp = 8 [(gogoproto.nullable) = false]; + util.hlc.Timestamp timestamp = 8 [(gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp"]; // The stats delta corresponding to the data in this WriteBatch. On // a split, contains only the contributions to the left-hand side. storage.enginepb.MVCCStats deprecated_delta = 10; // See #18828 diff --git a/pkg/kv/kvserver/kvserverpb/state.pb.go b/pkg/kv/kvserver/kvserverpb/state.pb.go index 46d4982b269e..bcb2f0ea1341 100644 --- a/pkg/kv/kvserver/kvserverpb/state.pb.go +++ b/pkg/kv/kvserver/kvserverpb/state.pb.go @@ -8,8 +8,8 @@ import fmt "fmt" import math "math" import roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" import enginepb "github.com/cockroachdb/cockroach/pkg/storage/enginepb" -import hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" +import github_com_cockroachdb_cockroach_pkg_storage_enginepb "github.com/cockroachdb/cockroach/pkg/storage/enginepb" import github_com_cockroachdb_cockroach_pkg_roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" import io "io" @@ -60,8 +60,8 @@ type ReplicaState struct { // gcThreshold is the GC threshold of the Range, typically updated when keys // are garbage collected. Reads and writes at timestamps <= this time will // not be served. - GCThreshold *hlc.Timestamp `protobuf:"bytes,6,opt,name=gc_threshold,json=gcThreshold,proto3" json:"gc_threshold,omitempty"` - Stats *enginepb.MVCCStats `protobuf:"bytes,7,opt,name=stats,proto3" json:"stats,omitempty"` + GCThreshold *github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp `protobuf:"bytes,6,opt,name=gc_threshold,json=gcThreshold,proto3,casttype=github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp" json:"gc_threshold,omitempty"` + Stats *enginepb.MVCCStats `protobuf:"bytes,7,opt,name=stats,proto3" json:"stats,omitempty"` // using_applied_state_key specifies whether the Range has been upgraded // to begin using the RangeAppliedState key. This key holds a combination // of the Raft applied index, the lease applied index, and the MVCC stats. @@ -77,7 +77,7 @@ func (m *ReplicaState) Reset() { *m = ReplicaState{} } func (m *ReplicaState) String() string { return proto.CompactTextString(m) } func (*ReplicaState) ProtoMessage() {} func (*ReplicaState) Descriptor() ([]byte, []int) { - return fileDescriptor_state_32aca65b437d2266, []int{0} + return fileDescriptor_state_3b902eacbbe4c5ca, []int{0} } func (m *ReplicaState) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -135,7 +135,7 @@ type RangeInfo struct { // closed timestamp information received from other nodes, among other things. // In practice, this should not usually trail newest_closed_timestamp except // for a short moment after newest_closed_timestamp gets updated. - ActiveClosedTimestamp hlc.Timestamp `protobuf:"bytes,12,opt,name=active_closed_timestamp,json=activeClosedTimestamp,proto3" json:"active_closed_timestamp"` + ActiveClosedTimestamp github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp `protobuf:"bytes,12,opt,name=active_closed_timestamp,json=activeClosedTimestamp,proto3,casttype=github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp" json:"active_closed_timestamp"` // The number of Rangefeed registrations attached to the Replica. RangefeedRegistrations int64 `protobuf:"varint,13,opt,name=rangefeed_registrations,json=rangefeedRegistrations,proto3" json:"rangefeed_registrations,omitempty"` // TenantID is the ID of the tenant for which this range contains data. @@ -149,7 +149,7 @@ func (m *RangeInfo) Reset() { *m = RangeInfo{} } func (m *RangeInfo) String() string { return proto.CompactTextString(m) } func (*RangeInfo) ProtoMessage() {} func (*RangeInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_state_32aca65b437d2266, []int{1} + return fileDescriptor_state_3b902eacbbe4c5ca, []int{1} } func (m *RangeInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -175,17 +175,17 @@ func (m *RangeInfo) XXX_DiscardUnknown() { var xxx_messageInfo_RangeInfo proto.InternalMessageInfo type RangeInfo_CTEntry struct { - NodeID github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,1,opt,name=node_id,json=nodeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"node_id,omitempty"` - ClosedTimestamp hlc.Timestamp `protobuf:"bytes,2,opt,name=closed_timestamp,json=closedTimestamp,proto3" json:"closed_timestamp"` - MLAI int64 `protobuf:"varint,3,opt,name=mlai,proto3" json:"mlai,omitempty"` - Epoch int64 `protobuf:"varint,4,opt,name=epoch,proto3" json:"epoch,omitempty"` + NodeID github_com_cockroachdb_cockroach_pkg_roachpb.NodeID `protobuf:"varint,1,opt,name=node_id,json=nodeId,proto3,casttype=github.com/cockroachdb/cockroach/pkg/roachpb.NodeID" json:"node_id,omitempty"` + ClosedTimestamp github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp `protobuf:"bytes,2,opt,name=closed_timestamp,json=closedTimestamp,proto3,casttype=github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp" json:"closed_timestamp"` + MLAI int64 `protobuf:"varint,3,opt,name=mlai,proto3" json:"mlai,omitempty"` + Epoch int64 `protobuf:"varint,4,opt,name=epoch,proto3" json:"epoch,omitempty"` } func (m *RangeInfo_CTEntry) Reset() { *m = RangeInfo_CTEntry{} } func (m *RangeInfo_CTEntry) String() string { return proto.CompactTextString(m) } func (*RangeInfo_CTEntry) ProtoMessage() {} func (*RangeInfo_CTEntry) Descriptor() ([]byte, []int) { - return fileDescriptor_state_32aca65b437d2266, []int{1, 0} + return fileDescriptor_state_3b902eacbbe4c5ca, []int{1, 0} } func (m *RangeInfo_CTEntry) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -221,7 +221,7 @@ func (m *LatchManagerInfo) Reset() { *m = LatchManagerInfo{} } func (m *LatchManagerInfo) String() string { return proto.CompactTextString(m) } func (*LatchManagerInfo) ProtoMessage() {} func (*LatchManagerInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_state_32aca65b437d2266, []int{2} + return fileDescriptor_state_3b902eacbbe4c5ca, []int{2} } func (m *LatchManagerInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1027,7 +1027,7 @@ func (m *ReplicaState) Unmarshal(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.GCThreshold == nil { - m.GCThreshold = &hlc.Timestamp{} + m.GCThreshold = &github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp{} } if err := m.GCThreshold.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err @@ -1871,74 +1871,76 @@ var ( ) func init() { - proto.RegisterFile("kv/kvserver/kvserverpb/state.proto", fileDescriptor_state_32aca65b437d2266) + proto.RegisterFile("kv/kvserver/kvserverpb/state.proto", fileDescriptor_state_3b902eacbbe4c5ca) } -var fileDescriptor_state_32aca65b437d2266 = []byte{ - // 1040 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x55, 0x4f, 0x6f, 0xdb, 0xc6, - 0x13, 0x35, 0x23, 0xca, 0xa6, 0x56, 0x76, 0xac, 0xec, 0xcf, 0x89, 0x19, 0x27, 0x96, 0x0c, 0x01, - 0xbf, 0xc2, 0x05, 0x52, 0x12, 0x75, 0xd0, 0x16, 0xfd, 0x03, 0x14, 0x96, 0x5c, 0x14, 0x72, 0x6d, - 0x23, 0xde, 0x08, 0x3d, 0xa4, 0x07, 0x62, 0x45, 0x8e, 0x29, 0x42, 0x14, 0x97, 0x59, 0x2e, 0x15, - 0x3b, 0xd7, 0x7e, 0x81, 0x7e, 0x84, 0x7e, 0x1c, 0xa3, 0x27, 0x1f, 0x73, 0x12, 0x5a, 0xf9, 0xd2, - 0x5b, 0xef, 0x3d, 0x15, 0xbb, 0x4b, 0xc9, 0x92, 0x1d, 0x20, 0xb9, 0x2d, 0xdf, 0x9b, 0x19, 0x0e, - 0xe7, 0xbd, 0x1d, 0xa2, 0xe6, 0x60, 0xe4, 0x0e, 0x46, 0x19, 0xf0, 0x11, 0xf0, 0xd9, 0x21, 0xed, - 0xb9, 0x99, 0xa0, 0x02, 0x9c, 0x94, 0x33, 0xc1, 0x70, 0xc3, 0x67, 0xfe, 0x80, 0x33, 0xea, 0xf7, - 0x9d, 0xc1, 0xc8, 0x99, 0x06, 0x39, 0x99, 0x60, 0x9c, 0x86, 0x90, 0xf6, 0xb6, 0x9e, 0x14, 0x47, - 0x17, 0x92, 0x30, 0x4a, 0x20, 0xed, 0xb9, 0xc3, 0x91, 0xef, 0xeb, 0xec, 0xad, 0x27, 0x2a, 0x33, - 0xed, 0xb9, 0x51, 0x22, 0x80, 0x27, 0x34, 0xf6, 0x38, 0x3d, 0x13, 0x05, 0xf9, 0x68, 0x4a, 0x0e, - 0x41, 0xd0, 0x80, 0x0a, 0x5a, 0xe0, 0x78, 0x8a, 0xcf, 0x61, 0x76, 0x2e, 0xa2, 0xd8, 0xed, 0xc7, - 0xbe, 0x2b, 0xa2, 0x21, 0x64, 0x82, 0x0e, 0xd3, 0x82, 0xd9, 0x08, 0x59, 0xc8, 0xd4, 0xd1, 0x95, - 0x27, 0x8d, 0x36, 0x7f, 0x35, 0xd1, 0x2a, 0x81, 0x34, 0x8e, 0x7c, 0xfa, 0x52, 0x7e, 0x0d, 0x7e, - 0x86, 0xb0, 0x7c, 0xb5, 0x47, 0xd3, 0x34, 0x8e, 0x20, 0xf0, 0xa2, 0x24, 0x80, 0x73, 0xdb, 0xd8, - 0x31, 0x76, 0x4d, 0x52, 0x93, 0xcc, 0xbe, 0x26, 0x3a, 0x12, 0xc7, 0x0e, 0xfa, 0x5f, 0x0c, 0x34, - 0x83, 0x5b, 0xe1, 0xf7, 0x54, 0xf8, 0x03, 0x45, 0x2d, 0xc4, 0x7f, 0x89, 0xcc, 0x00, 0x32, 0xdf, - 0x2e, 0xed, 0x18, 0xbb, 0xd5, 0xbd, 0xa6, 0x73, 0x33, 0xb4, 0xe2, 0x5b, 0x1c, 0x42, 0x93, 0x10, - 0x0e, 0x20, 0xf3, 0x79, 0x94, 0x0a, 0xc6, 0x89, 0x8a, 0xc7, 0x0e, 0x2a, 0xab, 0x62, 0xb6, 0xa9, - 0x12, 0xed, 0xf7, 0x24, 0x1e, 0x49, 0x9e, 0xe8, 0x30, 0x7c, 0x82, 0xd6, 0x05, 0xcf, 0x13, 0x9f, - 0x0a, 0x08, 0x3c, 0x25, 0x93, 0x5d, 0x56, 0x99, 0xff, 0x7f, 0xef, 0x2b, 0xcf, 0x44, 0x77, 0x1a, - 0xad, 0xa6, 0x40, 0xee, 0x8b, 0x85, 0x67, 0x7c, 0x8a, 0x56, 0x43, 0xdf, 0x13, 0x7d, 0x0e, 0x59, - 0x9f, 0xc5, 0x81, 0xbd, 0xac, 0x8a, 0x6d, 0xcf, 0x15, 0x93, 0x73, 0x77, 0xfa, 0xb1, 0xef, 0x74, - 0xa7, 0x73, 0x6f, 0xad, 0x4f, 0xc6, 0x8d, 0xea, 0x8f, 0xed, 0xee, 0x34, 0x8b, 0x54, 0x43, 0x7f, - 0xf6, 0x80, 0xbf, 0x45, 0x65, 0xd9, 0x58, 0x66, 0xaf, 0xdc, 0x69, 0xac, 0x70, 0x8a, 0x33, 0x75, - 0x8a, 0x73, 0xfc, 0x73, 0xbb, 0x2d, 0x1b, 0xc9, 0x88, 0xce, 0xc1, 0x5f, 0xa0, 0xcd, 0x3c, 0x8b, - 0x92, 0x70, 0x36, 0x77, 0xf5, 0x8d, 0xde, 0x00, 0x2e, 0xec, 0xea, 0x8e, 0xb1, 0x6b, 0x91, 0x0d, - 0x45, 0x17, 0xb3, 0x57, 0xdf, 0xf0, 0x13, 0x5c, 0x7c, 0x63, 0xfe, 0xfd, 0x7b, 0xc3, 0x38, 0x34, - 0x2d, 0xab, 0x56, 0x39, 0x34, 0xad, 0x4a, 0x0d, 0x1d, 0x9a, 0x16, 0xaa, 0x55, 0x9b, 0x7f, 0x58, - 0xa8, 0xa2, 0x06, 0xdf, 0x49, 0xce, 0x18, 0x3e, 0xd6, 0x9d, 0x81, 0x52, 0xbd, 0xba, 0xf7, 0x99, - 0xf3, 0x01, 0x6b, 0x3b, 0xf3, 0x06, 0x6a, 0x59, 0x97, 0xe3, 0xc6, 0xd2, 0xd5, 0xb8, 0x61, 0xe8, - 0x5e, 0x01, 0x6f, 0x23, 0x14, 0xd3, 0x4c, 0x2c, 0x58, 0xa3, 0x22, 0x11, 0x6d, 0x89, 0x06, 0xaa, - 0x26, 0xf9, 0xd0, 0x4b, 0x21, 0x09, 0xa2, 0x24, 0x54, 0xce, 0x30, 0x09, 0x4a, 0xf2, 0xe1, 0x0b, - 0x8d, 0x4c, 0x03, 0x02, 0xce, 0xd2, 0x14, 0x02, 0xa5, 0xa3, 0x0e, 0x38, 0xd0, 0x08, 0x6e, 0xa2, - 0x35, 0x65, 0xd9, 0x98, 0x85, 0x5e, 0x16, 0xbd, 0x05, 0xa5, 0x4e, 0x89, 0x54, 0x25, 0x78, 0xc4, - 0xc2, 0x97, 0xd1, 0x5b, 0xc0, 0xdf, 0xa1, 0x2d, 0x9a, 0xa6, 0x9c, 0x9d, 0x47, 0x43, 0x39, 0xa8, - 0x94, 0xb3, 0x94, 0x65, 0x34, 0xf6, 0x5e, 0xe7, 0x4c, 0x50, 0x25, 0x41, 0x89, 0xd8, 0x73, 0x11, - 0x2f, 0x8a, 0x80, 0x53, 0xc9, 0xe3, 0x4f, 0xd0, 0x3a, 0x97, 0xe3, 0xf1, 0x86, 0xf4, 0xdc, 0xeb, - 0x5d, 0x08, 0xc8, 0x6c, 0x4b, 0xa5, 0xac, 0x29, 0xf8, 0x98, 0x9e, 0xb7, 0x24, 0x88, 0x3f, 0x47, - 0x0f, 0x17, 0x3a, 0xf1, 0x04, 0xcf, 0x33, 0x01, 0x81, 0x8d, 0x94, 0x28, 0x78, 0xae, 0xa3, 0xae, - 0x66, 0x70, 0x8a, 0x36, 0x13, 0x78, 0x03, 0x99, 0xf0, 0xfc, 0x98, 0x65, 0x10, 0x78, 0xb3, 0x7b, - 0xab, 0x94, 0xac, 0xee, 0xed, 0x7d, 0x78, 0xfc, 0x53, 0xe5, 0x9c, 0x76, 0xf7, 0x87, 0x44, 0xf0, - 0x8b, 0x96, 0x29, 0x35, 0x20, 0x0f, 0x75, 0xe1, 0xb6, 0xaa, 0x3b, 0xb3, 0x25, 0xfe, 0x05, 0x6d, - 0x52, 0x5f, 0x44, 0x23, 0xb8, 0xfb, 0xc6, 0xd5, 0x8f, 0xb1, 0x75, 0x51, 0x5c, 0xd7, 0xb8, 0x5d, - 0xfc, 0x2b, 0xb4, 0xa9, 0x46, 0x72, 0x06, 0x10, 0x78, 0x1c, 0xc2, 0x28, 0x13, 0x9c, 0x8a, 0x88, - 0x25, 0x99, 0xbd, 0xa6, 0x26, 0xf6, 0x68, 0x46, 0x93, 0x79, 0x16, 0x7f, 0x8d, 0x1e, 0x2f, 0x8a, - 0xe2, 0xf5, 0xe4, 0x5e, 0xd1, 0xa6, 0xb9, 0xaf, 0x53, 0xd3, 0x79, 0x51, 0x5a, 0x34, 0x03, 0xed, - 0xa0, 0xef, 0xd1, 0xd3, 0x5b, 0xa9, 0x1c, 0xf4, 0x56, 0x7a, 0x9d, 0x43, 0x0e, 0xf6, 0xfa, 0x4e, - 0x69, 0xb7, 0x44, 0x1e, 0x2f, 0x64, 0x13, 0x1d, 0x71, 0x2a, 0x03, 0xf0, 0xa7, 0xa8, 0x22, 0x20, - 0xa1, 0x89, 0xf0, 0xa2, 0xc0, 0xae, 0x49, 0x7f, 0xb5, 0x56, 0x27, 0xe3, 0x86, 0xd5, 0x55, 0x60, - 0xe7, 0x80, 0x58, 0x9a, 0xee, 0x04, 0xca, 0xcc, 0xcc, 0x1f, 0x78, 0x82, 0xf6, 0x62, 0xb0, 0x1f, - 0xec, 0x18, 0xbb, 0x15, 0x52, 0x91, 0x48, 0x57, 0x02, 0x5b, 0xff, 0x18, 0x68, 0xa5, 0x10, 0x01, - 0xbf, 0x42, 0x2b, 0x09, 0x0b, 0x40, 0xd6, 0x94, 0x17, 0xa9, 0xdc, 0xda, 0x9f, 0x8c, 0x1b, 0xcb, - 0x27, 0x2c, 0x80, 0xce, 0xc1, 0xbf, 0xe3, 0xc6, 0xf3, 0x30, 0x12, 0xfd, 0xbc, 0xe7, 0xf8, 0x6c, - 0xe8, 0xce, 0xe6, 0x1d, 0xf4, 0x6e, 0xce, 0x6e, 0x3a, 0x08, 0xdd, 0xe9, 0x8e, 0xd2, 0x69, 0x64, - 0x59, 0x56, 0xec, 0x04, 0xf8, 0x04, 0xd5, 0xee, 0x88, 0x77, 0xef, 0xe3, 0xc5, 0x5b, 0xf7, 0x6f, - 0xc9, 0xf6, 0x14, 0x99, 0xc3, 0x98, 0x46, 0xea, 0xf6, 0x95, 0x5a, 0xd6, 0x64, 0xdc, 0x30, 0x8f, - 0x8f, 0xf6, 0x3b, 0x44, 0xa1, 0x78, 0x03, 0x95, 0x21, 0x65, 0x7e, 0x5f, 0x6d, 0xdf, 0x12, 0xd1, - 0x0f, 0x7a, 0x99, 0xcc, 0x56, 0x8a, 0x59, 0x2b, 0xeb, 0x95, 0xd2, 0x24, 0xa8, 0x76, 0x44, 0x85, - 0xdf, 0x3f, 0xa6, 0x09, 0x0d, 0x81, 0xab, 0x95, 0xb2, 0x8d, 0x10, 0x07, 0x1a, 0x78, 0x3e, 0xcb, - 0x13, 0xa1, 0xc6, 0x51, 0x22, 0x15, 0x89, 0xb4, 0x25, 0x20, 0xaf, 0xf8, 0x1b, 0x1e, 0x09, 0x28, - 0xf8, 0x7b, 0x8a, 0x47, 0x0a, 0x52, 0x01, 0xad, 0x67, 0x97, 0x7f, 0xd5, 0x97, 0x2e, 0x27, 0x75, - 0xe3, 0x6a, 0x52, 0x37, 0xde, 0x4d, 0xea, 0xc6, 0x9f, 0x93, 0xba, 0xf1, 0xdb, 0x75, 0x7d, 0xe9, - 0xea, 0xba, 0xbe, 0xf4, 0xee, 0xba, 0xbe, 0xf4, 0x0a, 0xdd, 0xfc, 0x97, 0x7b, 0xcb, 0xea, 0xdf, - 0xf6, 0xfc, 0xbf, 0x00, 0x00, 0x00, 0xff, 0xff, 0x1b, 0x72, 0x76, 0x6f, 0xb8, 0x07, 0x00, 0x00, +var fileDescriptor_state_3b902eacbbe4c5ca = []byte{ + // 1067 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x56, 0xcd, 0x6e, 0x1b, 0x37, + 0x10, 0xf6, 0x5a, 0x2b, 0x5b, 0xa2, 0xec, 0x58, 0x61, 0xed, 0x78, 0xe3, 0xc4, 0x92, 0x21, 0xa0, + 0x85, 0x0b, 0xa4, 0x2b, 0xd4, 0x41, 0x5b, 0xf4, 0x07, 0x28, 0x2c, 0xb9, 0x28, 0xe4, 0xda, 0x41, + 0xc2, 0xa8, 0x3d, 0xe4, 0xb2, 0xa0, 0x76, 0xc7, 0xab, 0x85, 0x56, 0xe4, 0x86, 0xcb, 0x55, 0xec, + 0x3c, 0x45, 0x0f, 0x05, 0x7a, 0x6d, 0x5f, 0xa2, 0xcf, 0x60, 0xf4, 0xe4, 0xa3, 0x4f, 0x42, 0x2b, + 0x5f, 0xfa, 0x0c, 0x39, 0x15, 0x24, 0x25, 0x59, 0xb2, 0x03, 0x24, 0x87, 0xa2, 0x37, 0xea, 0xfb, + 0x66, 0x86, 0xe4, 0x7c, 0x1f, 0x67, 0x85, 0x6a, 0xbd, 0x41, 0xbd, 0x37, 0x48, 0x41, 0x0c, 0x40, + 0x4c, 0x17, 0x49, 0xa7, 0x9e, 0x4a, 0x2a, 0xc1, 0x4d, 0x04, 0x97, 0x1c, 0x57, 0x7d, 0xee, 0xf7, + 0x04, 0xa7, 0x7e, 0xd7, 0xed, 0x0d, 0xdc, 0x49, 0x90, 0x9b, 0x4a, 0x2e, 0x68, 0x08, 0x49, 0x67, + 0xeb, 0xc1, 0x78, 0x59, 0x07, 0x16, 0x46, 0x0c, 0x92, 0x4e, 0xbd, 0x3f, 0xf0, 0x7d, 0x93, 0xbd, + 0xf5, 0x40, 0x67, 0x26, 0x9d, 0x7a, 0xc4, 0x24, 0x08, 0x46, 0x63, 0x4f, 0xd0, 0x13, 0x39, 0x26, + 0xef, 0x4d, 0xc8, 0x3e, 0x48, 0x1a, 0x50, 0x49, 0xc7, 0x38, 0x9e, 0xe0, 0x33, 0x98, 0x93, 0xc9, + 0x28, 0xae, 0x77, 0x63, 0xbf, 0x2e, 0xa3, 0x3e, 0xa4, 0x92, 0xf6, 0x93, 0x31, 0xb3, 0x1e, 0xf2, + 0x90, 0xeb, 0x65, 0x5d, 0xad, 0x0c, 0x5a, 0xfb, 0xd3, 0x46, 0x2b, 0x04, 0x92, 0x38, 0xf2, 0xe9, + 0x73, 0x75, 0x1b, 0xfc, 0x08, 0x61, 0xb5, 0xb5, 0x47, 0x93, 0x24, 0x8e, 0x20, 0xf0, 0x22, 0x16, + 0xc0, 0xa9, 0x63, 0xed, 0x58, 0xbb, 0x36, 0x29, 0x2b, 0x66, 0xdf, 0x10, 0x2d, 0x85, 0x63, 0x17, + 0x7d, 0x10, 0x03, 0x4d, 0xe1, 0x46, 0xf8, 0xa2, 0x0e, 0xbf, 0xab, 0xa9, 0xb9, 0xf8, 0xcf, 0x91, + 0x1d, 0x40, 0xea, 0x3b, 0xb9, 0x1d, 0x6b, 0xb7, 0xb4, 0x57, 0x73, 0xaf, 0x9b, 0x36, 0xbe, 0x8b, + 0x4b, 0x28, 0x0b, 0xe1, 0x00, 0x52, 0x5f, 0x44, 0x89, 0xe4, 0x82, 0xe8, 0x78, 0xec, 0xa2, 0xbc, + 0x2e, 0xe6, 0xd8, 0x3a, 0xd1, 0x79, 0x4b, 0xe2, 0x91, 0xe2, 0x89, 0x09, 0xc3, 0x4f, 0xd0, 0x9a, + 0x14, 0x19, 0xf3, 0xa9, 0x84, 0xc0, 0xd3, 0x32, 0x39, 0x79, 0x9d, 0xf9, 0xe1, 0x5b, 0xb7, 0x3c, + 0x91, 0xed, 0x49, 0xb4, 0xee, 0x02, 0xb9, 0x23, 0xe7, 0x7e, 0xe3, 0x5f, 0x2d, 0xb4, 0x12, 0xfa, + 0x9e, 0xec, 0x0a, 0x48, 0xbb, 0x3c, 0x0e, 0x9c, 0x25, 0x5d, 0x6d, 0x7b, 0xa6, 0x9a, 0x6a, 0xbc, + 0xdb, 0x8d, 0x7d, 0xb7, 0x3d, 0x69, 0x7c, 0xe3, 0xc7, 0xd1, 0xb0, 0x5a, 0xfa, 0xbe, 0xd9, 0x9e, + 0x64, 0xbd, 0x19, 0x56, 0x1b, 0x61, 0x24, 0xbb, 0x59, 0xc7, 0xf5, 0x79, 0xbf, 0x3e, 0xcd, 0x0d, + 0x3a, 0xd7, 0xeb, 0x7a, 0xd2, 0x0b, 0xeb, 0x37, 0xad, 0xe2, 0xb6, 0x4f, 0xd9, 0xb4, 0x2c, 0x29, + 0x85, 0xfe, 0xb4, 0x24, 0xfe, 0x1a, 0xe5, 0xd5, 0xfd, 0x52, 0x67, 0xf9, 0xd6, 0xfd, 0xc6, 0x55, + 0xdc, 0x69, 0x95, 0xe3, 0x9f, 0x9a, 0x4d, 0x75, 0x9f, 0x94, 0x98, 0x1c, 0xfc, 0x19, 0xda, 0xcc, + 0xd2, 0x88, 0x85, 0x53, 0xf9, 0x74, 0xab, 0xbc, 0x1e, 0x9c, 0x39, 0xa5, 0x1d, 0x6b, 0xb7, 0x40, + 0xd6, 0x35, 0x3d, 0x96, 0x50, 0xb7, 0xe2, 0x07, 0x38, 0xfb, 0xca, 0xfe, 0xe7, 0xb7, 0xaa, 0x75, + 0x68, 0x17, 0x0a, 0xe5, 0xe2, 0xa1, 0x5d, 0x28, 0x96, 0xd1, 0xa1, 0x5d, 0x40, 0xe5, 0x52, 0xed, + 0xb2, 0x88, 0x8a, 0x5a, 0xbf, 0x16, 0x3b, 0xe1, 0xf8, 0xd8, 0x9c, 0x0c, 0xb4, 0x79, 0x4a, 0x7b, + 0x9f, 0xb8, 0xef, 0x78, 0x21, 0xee, 0xac, 0x0f, 0x1b, 0x85, 0xf3, 0x61, 0x75, 0xe1, 0x62, 0x58, + 0xb5, 0xcc, 0x59, 0x01, 0x6f, 0x23, 0x14, 0xd3, 0x54, 0xce, 0x39, 0xac, 0xa8, 0x10, 0xe3, 0xac, + 0x2a, 0x2a, 0xb1, 0xac, 0xef, 0x25, 0xc0, 0x82, 0x88, 0x85, 0xda, 0x60, 0x36, 0x41, 0x2c, 0xeb, + 0x3f, 0x35, 0xc8, 0x24, 0x20, 0x10, 0x3c, 0x49, 0x20, 0xd0, 0x76, 0x30, 0x01, 0x07, 0x06, 0xc1, + 0x35, 0xb4, 0xaa, 0x9d, 0x1f, 0xf3, 0xd0, 0x4b, 0xa3, 0xd7, 0xa0, 0x35, 0xce, 0x91, 0x92, 0x02, + 0x8f, 0x78, 0xf8, 0x3c, 0x7a, 0x0d, 0xf8, 0x1b, 0xb4, 0x45, 0x93, 0x44, 0xf0, 0xd3, 0xa8, 0xaf, + 0x1a, 0x95, 0x08, 0x9e, 0xf0, 0x94, 0xc6, 0xde, 0xcb, 0x8c, 0x4b, 0xaa, 0x25, 0xc8, 0x11, 0x67, + 0x26, 0xe2, 0xe9, 0x38, 0xe0, 0x99, 0xe2, 0xf1, 0x47, 0x68, 0x4d, 0xa8, 0xf6, 0x78, 0x7d, 0x7a, + 0xea, 0x75, 0xce, 0x24, 0xa4, 0x4e, 0x41, 0xa7, 0xac, 0x6a, 0xf8, 0x98, 0x9e, 0x36, 0x14, 0x88, + 0x3f, 0x45, 0x1b, 0x73, 0x27, 0xf1, 0xa4, 0xc8, 0x52, 0x09, 0x81, 0x83, 0xb4, 0x28, 0x78, 0xe6, + 0x44, 0x6d, 0xc3, 0xe0, 0x04, 0x6d, 0x32, 0x78, 0x05, 0xa9, 0xf4, 0xfc, 0x98, 0xa7, 0x10, 0x78, + 0xd3, 0xe7, 0xaf, 0x95, 0x2c, 0xed, 0xed, 0xbd, 0xbb, 0xfd, 0x13, 0xe5, 0xdc, 0x66, 0xfb, 0x3b, + 0x26, 0xc5, 0x59, 0xc3, 0x56, 0x1a, 0x90, 0x0d, 0x53, 0xb8, 0xa9, 0xeb, 0x4e, 0x5d, 0x88, 0x7f, + 0xb7, 0xd0, 0x26, 0xf5, 0x65, 0x34, 0x80, 0xdb, 0x5b, 0xae, 0xbc, 0xcf, 0xeb, 0x38, 0x54, 0xd5, + 0xff, 0xa3, 0x27, 0xb1, 0x61, 0x4e, 0x72, 0xf3, 0x8c, 0x5f, 0xa0, 0x4d, 0xdd, 0xd9, 0x13, 0x80, + 0xc0, 0x13, 0x10, 0x46, 0xa9, 0x14, 0x54, 0x46, 0x9c, 0xa5, 0xce, 0xaa, 0x6e, 0xfc, 0xbd, 0x29, + 0x4d, 0x66, 0x59, 0xfc, 0x25, 0xba, 0x3f, 0xaf, 0xad, 0xd7, 0x51, 0x53, 0xce, 0x78, 0xef, 0x8e, + 0x49, 0x4d, 0x66, 0xb5, 0x6d, 0xd0, 0x14, 0x8c, 0x11, 0xbf, 0x45, 0x0f, 0x6f, 0xa4, 0x0a, 0x30, + 0x33, 0xf2, 0x65, 0x06, 0x19, 0x38, 0x6b, 0x3b, 0xb9, 0xdd, 0x1c, 0xb9, 0x3f, 0x97, 0x4d, 0x4c, + 0xc4, 0x33, 0x15, 0x80, 0x3f, 0x46, 0x45, 0x09, 0x8c, 0x32, 0xe9, 0x45, 0x81, 0x53, 0x56, 0x36, + 0x6d, 0xac, 0x8c, 0x86, 0xd5, 0x42, 0x5b, 0x83, 0xad, 0x03, 0x52, 0x30, 0x74, 0x2b, 0xd0, 0x6f, + 0x82, 0xfb, 0x3d, 0x4f, 0xd2, 0x4e, 0x0c, 0xce, 0xdd, 0x1d, 0x6b, 0xb7, 0x48, 0x8a, 0x0a, 0x69, + 0x2b, 0x60, 0xeb, 0x8f, 0x45, 0xb4, 0x3c, 0xd6, 0x12, 0xbf, 0x40, 0xcb, 0x8c, 0x07, 0xa0, 0x6a, + 0xaa, 0xf7, 0x98, 0x6f, 0xec, 0x8f, 0x86, 0xd5, 0xa5, 0x27, 0x3c, 0x80, 0xd6, 0xc1, 0x9b, 0x61, + 0xf5, 0xf1, 0x7b, 0x89, 0x30, 0x99, 0x98, 0x26, 0x8d, 0x2c, 0xa9, 0x8a, 0xad, 0x00, 0xff, 0x62, + 0xa1, 0xf2, 0x2d, 0x0f, 0x2c, 0xfe, 0xdf, 0x1e, 0x58, 0xf3, 0x6f, 0xa8, 0xff, 0x10, 0xd9, 0xfd, + 0x98, 0x46, 0x7a, 0x16, 0xe4, 0x1a, 0x85, 0xd1, 0xb0, 0x6a, 0x1f, 0x1f, 0xed, 0xb7, 0x88, 0x46, + 0xf1, 0x3a, 0xca, 0x43, 0xc2, 0xfd, 0xae, 0xfe, 0xa4, 0xe4, 0x88, 0xf9, 0x61, 0x46, 0xdb, 0x74, + 0xc0, 0xd9, 0xe5, 0xbc, 0x19, 0x70, 0x35, 0x82, 0xca, 0x47, 0x54, 0xfa, 0xdd, 0x63, 0xca, 0x68, + 0x08, 0x42, 0x0f, 0xb8, 0x6d, 0x84, 0x04, 0xd0, 0xc0, 0xf3, 0x79, 0xc6, 0xa4, 0xee, 0x6a, 0x8e, + 0x14, 0x15, 0xd2, 0x54, 0x80, 0x1a, 0x38, 0xaf, 0x44, 0x24, 0x61, 0xcc, 0x2f, 0x6a, 0x1e, 0x69, + 0x48, 0x07, 0x34, 0x1e, 0x9d, 0xff, 0x5d, 0x59, 0x38, 0x1f, 0x55, 0xac, 0x8b, 0x51, 0xc5, 0xba, + 0x1c, 0x55, 0xac, 0xbf, 0x46, 0x15, 0xeb, 0xe7, 0xab, 0xca, 0xc2, 0xc5, 0x55, 0x65, 0xe1, 0xf2, + 0xaa, 0xb2, 0xf0, 0x02, 0x5d, 0xff, 0xd9, 0xe8, 0x2c, 0xe9, 0x0f, 0xf6, 0xe3, 0x7f, 0x03, 0x00, + 0x00, 0xff, 0xff, 0x8b, 0xa3, 0x87, 0x5a, 0x8d, 0x08, 0x00, 0x00, } diff --git a/pkg/kv/kvserver/kvserverpb/state.proto b/pkg/kv/kvserver/kvserverpb/state.proto index bfd0dd45d65b..68728acc180b 100644 --- a/pkg/kv/kvserver/kvserverpb/state.proto +++ b/pkg/kv/kvserver/kvserverpb/state.proto @@ -57,7 +57,8 @@ message ReplicaState { // gcThreshold is the GC threshold of the Range, typically updated when keys // are garbage collected. Reads and writes at timestamps <= this time will // not be served. - util.hlc.Timestamp gc_threshold = 6 [(gogoproto.customname) = "GCThreshold"]; + util.hlc.Timestamp gc_threshold = 6 [(gogoproto.customname) = "GCThreshold", + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp"]; storage.enginepb.MVCCStats stats = 7; // using_applied_state_key specifies whether the Range has been upgraded // to begin using the RangeAppliedState key. This key holds a combination @@ -101,7 +102,8 @@ message RangeInfo { option (gogoproto.equal) = true; int32 node_id = 1 [(gogoproto.customname) = "NodeID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.NodeID"]; - util.hlc.Timestamp closed_timestamp = 2 [(gogoproto.nullable) = false]; + util.hlc.Timestamp closed_timestamp = 2 [(gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp"]; int64 mlai = 3 [(gogoproto.customname) = "MLAI"]; int64 epoch = 4; } @@ -117,7 +119,8 @@ message RangeInfo { // closed timestamp information received from other nodes, among other things. // In practice, this should not usually trail newest_closed_timestamp except // for a short moment after newest_closed_timestamp gets updated. - util.hlc.Timestamp active_closed_timestamp = 12 [(gogoproto.nullable) = false]; + util.hlc.Timestamp active_closed_timestamp = 12 [(gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp"]; // The number of Rangefeed registrations attached to the Replica. int64 rangefeed_registrations = 13; // TenantID is the ID of the tenant for which this range contains data. diff --git a/pkg/kv/kvserver/log.go b/pkg/kv/kvserver/log.go index eab9082d61b0..ea3a7547691f 100644 --- a/pkg/kv/kvserver/log.go +++ b/pkg/kv/kvserver/log.go @@ -20,7 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" - "github.com/cockroachdb/cockroach/pkg/util/hlc" + enginepb "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" ) @@ -213,7 +213,7 @@ func (s *Store) logChange( // assigned database timestamp. However, in the case of our tests log events // *are* the first action in a transaction, and we must elect to use the store's // physical time instead. -func selectEventTimestamp(s *Store, input hlc.Timestamp) time.Time { +func selectEventTimestamp(s *Store, input enginepb.TxnTimestamp) time.Time { if input.IsEmpty() { return s.Clock().PhysicalTime() } diff --git a/pkg/kv/kvserver/observedts/limit.go b/pkg/kv/kvserver/observedts/limit.go index 484d52807acd..29470a5a8c0e 100644 --- a/pkg/kv/kvserver/observedts/limit.go +++ b/pkg/kv/kvserver/observedts/limit.go @@ -15,6 +15,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/log" ) @@ -63,17 +64,19 @@ func LimitTxnMaxTimestamp( // incorrect assumptions about when data was written, in absolute time on a // different node, which held the lease before this replica acquired it. obsTS.Forward(status.Lease.Start) + // Convert the observed timestamp to the transaction time domain. + obsTxnTS := enginepb.TxnTimestamp(obsTS) // If the observed timestamp reduces the transaction's uncertainty interval, // update the transacion proto. - if obsTS.Less(txn.MaxTimestamp) { + if obsTxnTS.Less(txn.MaxTimestamp) { // Copy-on-write to protect others we might be sharing the Txn with. txnClone := txn.Clone() // The uncertainty window is [ReadTimestamp, maxTS), so if that window // is empty, there won't be any uncertainty restarts. - if obsTS.LessEq(txn.ReadTimestamp) { + if obsTxnTS.LessEq(txn.ReadTimestamp) { log.Event(ctx, "read has no clock uncertainty") } - txnClone.MaxTimestamp.Backward(obsTS) + txnClone.MaxTimestamp.Backward(obsTxnTS) txn = txnClone } return txn diff --git a/pkg/kv/kvserver/protectedts/ptcache/cache.go b/pkg/kv/kvserver/protectedts/ptcache/cache.go index b4102872520f..7a1f9ca5daa5 100644 --- a/pkg/kv/kvserver/protectedts/ptcache/cache.go +++ b/pkg/kv/kvserver/protectedts/ptcache/cache.go @@ -217,7 +217,7 @@ func (c *Cache) doUpdate(ctx context.Context) error { // to change its read timestamp. defer func() { if err == nil { - ts = txn.ReadTimestamp() + ts = txn.ReadTimestamp().ToClockTimestampUnchecked() } }() md, err := c.storage.GetMetadata(ctx, txn) diff --git a/pkg/kv/kvserver/protectedts/ptpb/protectedts.pb.go b/pkg/kv/kvserver/protectedts/ptpb/protectedts.pb.go index 5504b48ebc99..8673e2c4f45b 100644 --- a/pkg/kv/kvserver/protectedts/ptpb/protectedts.pb.go +++ b/pkg/kv/kvserver/protectedts/ptpb/protectedts.pb.go @@ -7,9 +7,9 @@ import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" import roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" -import hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" import github_com_cockroachdb_cockroach_pkg_util_uuid "github.com/cockroachdb/cockroach/pkg/util/uuid" +import github_com_cockroachdb_cockroach_pkg_storage_enginepb "github.com/cockroachdb/cockroach/pkg/storage/enginepb" import io "io" @@ -44,7 +44,7 @@ func (x ProtectionMode) String() string { return proto.EnumName(ProtectionMode_name, int32(x)) } func (ProtectionMode) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_protectedts_5535080d2f6ed5b9, []int{0} + return fileDescriptor_protectedts_e4a0069de36f267a, []int{0} } // Metadata is the system metadata. The metadata is stored explicitly and all @@ -73,7 +73,7 @@ func (m *Metadata) Reset() { *m = Metadata{} } func (m *Metadata) String() string { return proto.CompactTextString(m) } func (*Metadata) ProtoMessage() {} func (*Metadata) Descriptor() ([]byte, []int) { - return fileDescriptor_protectedts_5535080d2f6ed5b9, []int{0} + return fileDescriptor_protectedts_e4a0069de36f267a, []int{0} } func (m *Metadata) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -103,7 +103,7 @@ type Record struct { // ID uniquely identifies this row. ID github_com_cockroachdb_cockroach_pkg_util_uuid.UUID `protobuf:"bytes,1,opt,name=id,proto3,customtype=github.com/cockroachdb/cockroach/pkg/util/uuid.UUID" json:"id"` // Timestamp is the timestamp which is protected. - Timestamp hlc.Timestamp `protobuf:"bytes,2,opt,name=timestamp,proto3" json:"timestamp"` + Timestamp github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp `protobuf:"bytes,2,opt,name=timestamp,proto3,casttype=github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp" json:"timestamp"` // Mode specifies whether this record protects all values live at timestamp // or all values live at or after that timestamp. Mode ProtectionMode `protobuf:"varint,3,opt,name=mode,proto3,enum=cockroach.protectedts.ProtectionMode" json:"mode,omitempty"` @@ -132,7 +132,7 @@ func (m *Record) Reset() { *m = Record{} } func (m *Record) String() string { return proto.CompactTextString(m) } func (*Record) ProtoMessage() {} func (*Record) Descriptor() ([]byte, []int) { - return fileDescriptor_protectedts_5535080d2f6ed5b9, []int{1} + return fileDescriptor_protectedts_e4a0069de36f267a, []int{1} } func (m *Record) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -167,7 +167,7 @@ func (m *State) Reset() { *m = State{} } func (m *State) String() string { return proto.CompactTextString(m) } func (*State) ProtoMessage() {} func (*State) Descriptor() ([]byte, []int) { - return fileDescriptor_protectedts_5535080d2f6ed5b9, []int{2} + return fileDescriptor_protectedts_e4a0069de36f267a, []int{2} } func (m *State) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1023,44 +1023,46 @@ var ( ) func init() { - proto.RegisterFile("kv/kvserver/protectedts/ptpb/protectedts.proto", fileDescriptor_protectedts_5535080d2f6ed5b9) + proto.RegisterFile("kv/kvserver/protectedts/ptpb/protectedts.proto", fileDescriptor_protectedts_e4a0069de36f267a) } -var fileDescriptor_protectedts_5535080d2f6ed5b9 = []byte{ - // 555 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x74, 0x53, 0xcf, 0x8f, 0xd2, 0x40, - 0x14, 0x6e, 0xa1, 0x40, 0x19, 0x74, 0xa3, 0x13, 0x8d, 0x0d, 0x6a, 0x4b, 0x48, 0x34, 0xe8, 0xa1, - 0x93, 0xc0, 0xc9, 0x83, 0x07, 0x70, 0x31, 0xd9, 0xc3, 0xc6, 0x75, 0x96, 0xbd, 0x78, 0x21, 0xd3, - 0x76, 0x2c, 0x13, 0x68, 0xa7, 0x69, 0xa7, 0x24, 0x9c, 0xbd, 0x78, 0x31, 0xf1, 0x7f, 0xf0, 0x9f, - 0xe1, 0xc8, 0x71, 0xa3, 0x09, 0x51, 0xf8, 0x47, 0xcc, 0x4c, 0xf9, 0xa5, 0xd1, 0xdb, 0x9b, 0xf7, - 0xbe, 0xef, 0xcd, 0x7c, 0xdf, 0x7b, 0x03, 0xdc, 0xe9, 0x1c, 0x4d, 0xe7, 0x19, 0x4d, 0xe7, 0x34, - 0x45, 0x49, 0xca, 0x05, 0xf5, 0x05, 0x0d, 0x44, 0x86, 0x12, 0x91, 0x78, 0xa7, 0x09, 0x57, 0xc6, - 0x1c, 0x3e, 0xf4, 0xb9, 0x3f, 0x4d, 0x39, 0xf1, 0x27, 0xee, 0x49, 0xb1, 0xf9, 0x20, 0xe4, 0x21, - 0x57, 0x08, 0x24, 0xa3, 0x02, 0xdc, 0x7c, 0x12, 0x72, 0x1e, 0xce, 0x28, 0x22, 0x09, 0x43, 0x24, - 0x8e, 0xb9, 0x20, 0x82, 0xf1, 0x78, 0xd7, 0xaa, 0x09, 0x55, 0x9b, 0xc4, 0x43, 0x01, 0x11, 0x64, - 0x97, 0xb3, 0x72, 0xc1, 0x66, 0x68, 0x32, 0xf3, 0x91, 0x60, 0x11, 0xcd, 0x04, 0x89, 0x92, 0xa2, - 0xd2, 0xfe, 0xa4, 0x03, 0xf3, 0x92, 0x0a, 0x22, 0xc1, 0xd0, 0x02, 0xb5, 0x39, 0x4d, 0x33, 0xc6, - 0x63, 0x4b, 0x6f, 0xe9, 0x1d, 0x03, 0xef, 0x8f, 0xd0, 0x01, 0x8d, 0x38, 0x8f, 0xc6, 0x29, 0xf5, - 0x79, 0x1a, 0x64, 0x56, 0x49, 0x55, 0x41, 0x9c, 0x47, 0xb8, 0xc8, 0xc0, 0xc7, 0xa0, 0x2e, 0x01, - 0x59, 0x42, 0xe2, 0xcc, 0x2a, 0xab, 0xb2, 0x19, 0xe7, 0xd1, 0xb5, 0x3c, 0x4b, 0xb6, 0xe0, 0x82, - 0xcc, 0xc6, 0xde, 0x42, 0xd0, 0xcc, 0x32, 0x0a, 0xb6, 0x4a, 0x0d, 0x64, 0xa6, 0xfd, 0xa3, 0x04, - 0xaa, 0x45, 0x27, 0xf8, 0x1e, 0x94, 0x58, 0xa0, 0xae, 0xbf, 0x33, 0xe8, 0x2f, 0xd7, 0x8e, 0xf6, - 0x7d, 0xed, 0xf4, 0x42, 0x26, 0x26, 0xb9, 0xe7, 0xfa, 0x3c, 0x42, 0x07, 0xa3, 0x02, 0xef, 0x18, - 0xa3, 0x64, 0x1a, 0x22, 0xa5, 0x31, 0xcf, 0x59, 0xe0, 0xde, 0xdc, 0x5c, 0x9c, 0x6f, 0xd6, 0x4e, - 0xe9, 0xe2, 0x1c, 0x97, 0x58, 0x00, 0xfb, 0xa0, 0x7e, 0x90, 0xad, 0x9e, 0xde, 0xe8, 0x3e, 0x75, - 0x8f, 0x86, 0x4b, 0x9e, 0x3b, 0x99, 0xf9, 0xee, 0x68, 0x0f, 0x1a, 0x18, 0xf2, 0x62, 0x7c, 0x64, - 0xc1, 0x57, 0xc0, 0x88, 0x78, 0x40, 0x95, 0xb2, 0xb3, 0xee, 0x33, 0xf7, 0x9f, 0xe3, 0x72, 0xaf, - 0x8a, 0x98, 0xf1, 0xf8, 0x92, 0x07, 0x14, 0x2b, 0x8a, 0x74, 0x26, 0xa2, 0x82, 0x8c, 0xc5, 0x22, - 0xa1, 0x4a, 0x7a, 0x1d, 0x9b, 0x32, 0x31, 0x5a, 0x24, 0x14, 0x42, 0x60, 0xc8, 0xd8, 0xaa, 0x48, - 0xbd, 0x58, 0xc5, 0xb0, 0x09, 0xcc, 0x39, 0x4d, 0xd9, 0x47, 0x46, 0x03, 0xab, 0xda, 0xd2, 0x3b, - 0x26, 0x3e, 0x9c, 0x61, 0x0f, 0x54, 0x0a, 0x8b, 0x6b, 0xad, 0x72, 0xa7, 0xd1, 0x7d, 0x74, 0xf2, - 0x90, 0xdd, 0xd8, 0x5d, 0x69, 0xf9, 0x4e, 0x40, 0x81, 0x6d, 0x7f, 0xd1, 0x41, 0xe5, 0x5a, 0x10, - 0x41, 0xe1, 0x10, 0xa8, 0xab, 0xe5, 0xb0, 0x95, 0xc5, 0x8d, 0xae, 0xf3, 0x1f, 0x29, 0xfb, 0x9d, - 0x18, 0x98, 0xb2, 0xd3, 0x6a, 0xed, 0xe8, 0xf8, 0x40, 0x85, 0xaf, 0x41, 0xed, 0xb8, 0x09, 0xe5, - 0xbf, 0xec, 0x3c, 0xed, 0x52, 0xcc, 0x74, 0xf7, 0x9a, 0x3d, 0xe7, 0xe5, 0x0b, 0x70, 0xf6, 0xa7, - 0x53, 0xf0, 0x3e, 0xb8, 0x7b, 0x85, 0xdf, 0x8d, 0x86, 0x6f, 0x46, 0xe3, 0xfe, 0xdb, 0xd1, 0x10, - 0xdf, 0xd3, 0x9a, 0xc6, 0xe7, 0x6f, 0xb6, 0x36, 0x78, 0xbe, 0xfc, 0x65, 0x6b, 0xcb, 0x8d, 0xad, - 0xaf, 0x36, 0xb6, 0x7e, 0xbb, 0xb1, 0xf5, 0x9f, 0x1b, 0x5b, 0xff, 0xba, 0xb5, 0xb5, 0xd5, 0xd6, - 0xd6, 0x6e, 0xb7, 0xb6, 0xf6, 0xc1, 0x90, 0x3f, 0xca, 0xab, 0xaa, 0x6d, 0xee, 0xfd, 0x0e, 0x00, - 0x00, 0xff, 0xff, 0xf9, 0x47, 0xda, 0x7a, 0x78, 0x03, 0x00, 0x00, +var fileDescriptor_protectedts_e4a0069de36f267a = []byte{ + // 588 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x53, 0x4f, 0x6f, 0xd4, 0x3e, + 0x10, 0x4d, 0x76, 0xd3, 0x36, 0xf5, 0xfe, 0x7e, 0x15, 0x58, 0x20, 0xa2, 0x05, 0x92, 0xaa, 0x12, + 0xa8, 0x70, 0xb0, 0xa5, 0xed, 0x89, 0x03, 0x87, 0x86, 0x16, 0xa9, 0x48, 0x15, 0xc5, 0x4d, 0x2f, + 0x5c, 0x56, 0x4e, 0x62, 0xb2, 0xd6, 0x6e, 0xec, 0x28, 0x71, 0x56, 0xf4, 0xcc, 0x01, 0x2e, 0x48, + 0x7c, 0x06, 0xf8, 0x32, 0x3d, 0xf6, 0x58, 0x71, 0x58, 0xc1, 0xf6, 0x5b, 0x70, 0x42, 0x76, 0xf6, + 0x1f, 0x08, 0x24, 0x6e, 0xe3, 0x99, 0xf7, 0xde, 0xd8, 0x6f, 0xc6, 0x00, 0x0d, 0xc7, 0x78, 0x38, + 0xae, 0x58, 0x39, 0x66, 0x25, 0x2e, 0x4a, 0xa9, 0x58, 0xa2, 0x58, 0xaa, 0x2a, 0x5c, 0xa8, 0x22, + 0x5e, 0x4d, 0x20, 0x1d, 0x4b, 0x78, 0x3b, 0x91, 0xc9, 0xb0, 0x94, 0x34, 0x19, 0xa0, 0x95, 0x62, + 0xf7, 0x56, 0x26, 0x33, 0x69, 0x10, 0x58, 0x47, 0x0d, 0xb8, 0x7b, 0x2f, 0x93, 0x32, 0x1b, 0x31, + 0x4c, 0x0b, 0x8e, 0xa9, 0x10, 0x52, 0x51, 0xc5, 0xa5, 0x98, 0x49, 0x75, 0xa1, 0x91, 0x29, 0x62, + 0x9c, 0x52, 0x45, 0x67, 0x39, 0xaf, 0x56, 0x7c, 0x84, 0x07, 0xa3, 0x04, 0x2b, 0x9e, 0xb3, 0x4a, + 0xd1, 0xbc, 0x68, 0x2a, 0x3b, 0xef, 0x6c, 0xe0, 0x1e, 0x33, 0x45, 0x35, 0x18, 0x7a, 0x60, 0x63, + 0xcc, 0xca, 0x8a, 0x4b, 0xe1, 0xd9, 0xdb, 0xf6, 0xae, 0x43, 0xe6, 0x47, 0x18, 0x80, 0x8e, 0xa8, + 0xf3, 0x7e, 0xc9, 0x12, 0x59, 0xa6, 0x95, 0xd7, 0x32, 0x55, 0x20, 0xea, 0x9c, 0x34, 0x19, 0x78, + 0x17, 0x6c, 0x6a, 0x40, 0x55, 0x50, 0x51, 0x79, 0x6d, 0x53, 0x76, 0x45, 0x9d, 0x9f, 0xea, 0xb3, + 0x66, 0x2b, 0xa9, 0xe8, 0xa8, 0x1f, 0x9f, 0x2b, 0x56, 0x79, 0x4e, 0xc3, 0x36, 0xa9, 0x50, 0x67, + 0x76, 0x3e, 0xb7, 0xc1, 0x7a, 0xa3, 0x04, 0x5f, 0x81, 0x16, 0x4f, 0x4d, 0xfb, 0xff, 0xc2, 0xfd, + 0x8b, 0x49, 0x60, 0x7d, 0x9d, 0x04, 0x7b, 0x19, 0x57, 0x83, 0x3a, 0x46, 0x89, 0xcc, 0xf1, 0xc2, + 0xa8, 0x34, 0x5e, 0xc6, 0xb8, 0x18, 0x66, 0xd8, 0xbc, 0xb1, 0xae, 0x79, 0x8a, 0xce, 0xce, 0x8e, + 0x0e, 0xa6, 0x93, 0xa0, 0x75, 0x74, 0x40, 0x5a, 0x3c, 0x85, 0xef, 0x6d, 0xb0, 0xb9, 0x78, 0xb7, + 0xb9, 0x7b, 0xa7, 0x77, 0x1f, 0x2d, 0x1d, 0xd7, 0x44, 0x34, 0x18, 0x25, 0x28, 0x9a, 0x83, 0xc2, + 0x17, 0xba, 0xf3, 0x8f, 0x49, 0x10, 0xfe, 0x53, 0xe7, 0x4a, 0xc9, 0x92, 0x66, 0x0c, 0x33, 0x91, + 0x71, 0xc1, 0x8a, 0x18, 0x45, 0x6f, 0xc5, 0x42, 0x8b, 0x2c, 0x7b, 0xc3, 0x27, 0xc0, 0xc9, 0x65, + 0xca, 0x8c, 0x41, 0x5b, 0xbd, 0x07, 0xe8, 0x8f, 0x53, 0x47, 0x27, 0x4d, 0xcc, 0xa5, 0x38, 0x96, + 0x29, 0x23, 0x86, 0xa2, 0x0d, 0xce, 0x99, 0xa2, 0x7d, 0x75, 0x5e, 0x30, 0xe3, 0xe0, 0x26, 0x71, + 0x75, 0x22, 0x3a, 0x2f, 0x18, 0x84, 0xc0, 0xd1, 0xb1, 0xb7, 0xa6, 0x6d, 0x23, 0x26, 0x86, 0x5d, + 0xe0, 0x8e, 0x59, 0xc9, 0xdf, 0x70, 0x96, 0x7a, 0xeb, 0xdb, 0xf6, 0xae, 0x4b, 0x16, 0x67, 0xb8, + 0x07, 0xd6, 0x9a, 0x49, 0x6d, 0x6c, 0xb7, 0x77, 0x3b, 0xbd, 0x3b, 0x2b, 0x17, 0x99, 0x6d, 0x0f, + 0xd2, 0x93, 0x0b, 0x1d, 0x6d, 0x03, 0x69, 0xb0, 0x3b, 0x1f, 0x6d, 0xb0, 0x76, 0xaa, 0xa8, 0x62, + 0xf0, 0x10, 0x98, 0xd6, 0x7a, 0x67, 0xcc, 0xa4, 0x3a, 0xbd, 0xe0, 0x2f, 0x4f, 0x99, 0xaf, 0x56, + 0xe8, 0x6a, 0xa5, 0xcb, 0x49, 0x60, 0x93, 0x05, 0x15, 0x3e, 0x05, 0x1b, 0xcb, 0x85, 0x6a, 0xff, + 0x36, 0x94, 0x55, 0x95, 0x66, 0x35, 0x66, 0xb7, 0x99, 0x73, 0x1e, 0x3f, 0x02, 0x5b, 0xbf, 0x3a, + 0x05, 0x6f, 0x82, 0xff, 0x4f, 0xc8, 0xcb, 0xe8, 0xf0, 0x59, 0xd4, 0xdf, 0x7f, 0x1e, 0x1d, 0x92, + 0x1b, 0x56, 0xd7, 0xf9, 0xf0, 0xc5, 0xb7, 0xc2, 0x87, 0x17, 0xdf, 0x7d, 0xeb, 0x62, 0xea, 0xdb, + 0x97, 0x53, 0xdf, 0xbe, 0x9a, 0xfa, 0xf6, 0xb7, 0xa9, 0x6f, 0x7f, 0xba, 0xf6, 0xad, 0xcb, 0x6b, + 0xdf, 0xba, 0xba, 0xf6, 0xad, 0xd7, 0x8e, 0xfe, 0x98, 0xf1, 0xba, 0xf9, 0x14, 0x7b, 0x3f, 0x03, + 0x00, 0x00, 0xff, 0xff, 0x97, 0xc2, 0xaa, 0x7e, 0xbf, 0x03, 0x00, 0x00, } diff --git a/pkg/kv/kvserver/protectedts/ptpb/protectedts.proto b/pkg/kv/kvserver/protectedts/ptpb/protectedts.proto index 9db249c14f43..639790975a02 100644 --- a/pkg/kv/kvserver/protectedts/ptpb/protectedts.proto +++ b/pkg/kv/kvserver/protectedts/ptpb/protectedts.proto @@ -93,7 +93,7 @@ message Record { bytes id = 1 [(gogoproto.customtype) = "github.com/cockroachdb/cockroach/pkg/util/uuid.UUID", (gogoproto.nullable) = false, (gogoproto.customname) = "ID"]; // Timestamp is the timestamp which is protected. - util.hlc.Timestamp timestamp = 2 [(gogoproto.nullable) = false]; + util.hlc.Timestamp timestamp = 2 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp", (gogoproto.nullable) = false]; // Mode specifies whether this record protects all values live at timestamp // or all values live at or after that timestamp. diff --git a/pkg/kv/kvserver/protectedts/ptstorage/storage.go b/pkg/kv/kvserver/protectedts/ptstorage/storage.go index 2b4fdc0008eb..28b2c71349db 100644 --- a/pkg/kv/kvserver/protectedts/ptstorage/storage.go +++ b/pkg/kv/kvserver/protectedts/ptstorage/storage.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/uuid" @@ -227,7 +228,7 @@ func rowToRecord(ctx context.Context, row tree.Datums, r *ptpb.Record) error { if err != nil { return errors.Wrapf(err, "failed to parse timestamp for %v", r.ID) } - r.Timestamp = ts + r.Timestamp = enginepb.TxnTimestamp(ts) r.MetaType = string(*row[2].(*tree.DString)) if row[3] != tree.DNull { diff --git a/pkg/kv/kvserver/protectedts/ptverifier/verifier.go b/pkg/kv/kvserver/protectedts/ptverifier/verifier.go index c12b6eba6281..fa877dfb40f0 100644 --- a/pkg/kv/kvserver/protectedts/ptverifier/verifier.go +++ b/pkg/kv/kvserver/protectedts/ptverifier/verifier.go @@ -69,7 +69,7 @@ func getRecordWithTimestamp( ) (r *ptpb.Record, readAt hlc.Timestamp, err error) { if err = db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { r, err = s.GetRecord(ctx, txn, id) - readAt = txn.ReadTimestamp() + readAt = txn.ReadTimestamp().ToClockTimestampUnchecked() return err }); err != nil { return nil, hlc.Timestamp{}, err diff --git a/pkg/kv/kvserver/rangefeed/processor.go b/pkg/kv/kvserver/rangefeed/processor.go index 0d56a46f0e63..d56b8bb235bb 100644 --- a/pkg/kv/kvserver/rangefeed/processor.go +++ b/pkg/kv/kvserver/rangefeed/processor.go @@ -145,7 +145,7 @@ func putPooledEvent(ev *event) { // channel, which is necessary to prevent reordering. type event struct { ops []enginepb.MVCCLogicalOp - ct hlc.Timestamp + ct enginepb.TxnTimestamp initRTS bool syncC chan struct{} // This setting is used in conjunction with syncC in tests in order to ensure @@ -294,7 +294,7 @@ func (p *Processor) run( continue } - now := p.Clock.Now() + now := enginepb.TxnTimestamp(p.Clock.Now()) before := now.Add(-p.PushTxnsAge.Nanoseconds(), 0) oldTxns := p.rts.intentQ.Before(before) @@ -389,7 +389,7 @@ func (p *Processor) sendStop(pErr *roachpb.Error) { // NOT safe to call on nil Processor. func (p *Processor) Register( span roachpb.RSpan, - startTS hlc.Timestamp, + startTS enginepb.TxnTimestamp, catchupIterConstructor IteratorConstructor, withDiff bool, stream Stream, @@ -467,7 +467,7 @@ func (p *Processor) ConsumeLogicalOps(ops ...enginepb.MVCCLogicalOp) bool { // EventChanTimeout configuration. If the method returns false, the processor // will have been stopped, so calling Stop is not necessary. Safe to call on // nil Processor. -func (p *Processor) ForwardClosedTS(closedTS hlc.Timestamp) bool { +func (p *Processor) ForwardClosedTS(closedTS enginepb.TxnTimestamp) bool { if p == nil { return true } @@ -594,7 +594,7 @@ func (p *Processor) consumeLogicalOps(ctx context.Context, ops []enginepb.MVCCLo } } -func (p *Processor) forwardClosedTS(ctx context.Context, newClosedTS hlc.Timestamp) { +func (p *Processor) forwardClosedTS(ctx context.Context, newClosedTS enginepb.TxnTimestamp) { if p.rts.ForwardClosedTS(newClosedTS) { p.publishCheckpoint(ctx) } @@ -607,7 +607,7 @@ func (p *Processor) initResolvedTS(ctx context.Context) { } func (p *Processor) publishValue( - ctx context.Context, key roachpb.Key, timestamp hlc.Timestamp, value, prevValue []byte, + ctx context.Context, key roachpb.Key, timestamp enginepb.TxnTimestamp, value, prevValue []byte, ) { if !p.Span.ContainsKey(roachpb.RKey(key)) { log.Fatalf(ctx, "key %v not in Processor's key range %v", key, p.Span) diff --git a/pkg/kv/kvserver/rangefeed/registry.go b/pkg/kv/kvserver/rangefeed/registry.go index 4138574c15fa..c591920cf500 100644 --- a/pkg/kv/kvserver/rangefeed/registry.go +++ b/pkg/kv/kvserver/rangefeed/registry.go @@ -54,7 +54,7 @@ type Stream interface { type registration struct { // Input. span roachpb.Span - catchupTimestamp hlc.Timestamp + catchupTimestamp enginepb.TxnTimestamp catchupIterConstructor func() storage.SimpleMVCCIterator withDiff bool metrics *Metrics @@ -85,7 +85,7 @@ type registration struct { func newRegistration( span roachpb.Span, - startTS hlc.Timestamp, + startTS enginepb.TxnTimestamp, catchupIterConstructor func() storage.SimpleMVCCIterator, withDiff bool, bufferSz int, @@ -348,7 +348,7 @@ func (r *registration) maybeRunCatchupScan() error { // immediately after) the provisional key. catchupIter.SeekGE(storage.MVCCKey{ Key: unsafeKey.Key, - Timestamp: meta.Timestamp.ToTimestamp().Prev(), + Timestamp: meta.TxnTimestamp().Prev(), }) continue } @@ -467,7 +467,7 @@ func (reg *registry) nextID() int64 { func (reg *registry) PublishToOverlapping(span roachpb.Span, event *roachpb.RangeFeedEvent) { // Determine the earliest starting timestamp that a registration // can have while still needing to hear about this event. - var minTS hlc.Timestamp + var minTS enginepb.TxnTimestamp switch t := event.GetValue().(type) { case *roachpb.RangeFeedValue: // Only publish values to registrations with starting @@ -479,7 +479,7 @@ func (reg *registry) PublishToOverlapping(span roachpb.Span, event *roachpb.Rang // // TODO(dan): It's unclear if this is the right contract, it's certainly // surprising. Revisit this once RangeFeed has more users. - minTS = hlc.MaxTimestamp + minTS = enginepb.TxnTimestamp(hlc.MaxTimestamp) default: panic(fmt.Sprintf("unexpected RangeFeedEvent variant: %v", t)) } diff --git a/pkg/kv/kvserver/rangefeed/resolved_timestamp.go b/pkg/kv/kvserver/rangefeed/resolved_timestamp.go index 120cab958357..3865960d5892 100644 --- a/pkg/kv/kvserver/rangefeed/resolved_timestamp.go +++ b/pkg/kv/kvserver/rangefeed/resolved_timestamp.go @@ -17,7 +17,6 @@ import ( "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/uuid" "github.com/cockroachdb/errors" ) @@ -77,8 +76,8 @@ import ( // handled carefully and in a total order. type resolvedTimestamp struct { init bool - closedTS hlc.Timestamp - resolvedTS hlc.Timestamp + closedTS enginepb.TxnTimestamp + resolvedTS enginepb.TxnTimestamp intentQ unresolvedIntentQueue } @@ -89,7 +88,7 @@ func makeResolvedTimestamp() resolvedTimestamp { } // Get returns the current value of the resolved timestamp. -func (rts *resolvedTimestamp) Get() hlc.Timestamp { +func (rts *resolvedTimestamp) Get() enginepb.TxnTimestamp { return rts.resolvedTS } @@ -116,7 +115,7 @@ func (rts *resolvedTimestamp) IsInit() bool { // ForwardClosedTS indicates that the closed timestamp that serves as the basis // for the resolved timestamp has advanced. The method returns whether this // caused the resolved timestamp to move forward. -func (rts *resolvedTimestamp) ForwardClosedTS(newClosedTS hlc.Timestamp) bool { +func (rts *resolvedTimestamp) ForwardClosedTS(newClosedTS enginepb.TxnTimestamp) bool { if rts.closedTS.Forward(newClosedTS) { return rts.recompute() } @@ -158,7 +157,7 @@ func (rts *resolvedTimestamp) consumeLogicalOp(op enginepb.MVCCLogicalOp) bool { // that was written only in an earlier epoch being resolved after its // transaction committed in a later epoch. Don't make any assumptions // about the transaction other than to decrement its reference count. - return rts.intentQ.DecrRef(t.TxnID, hlc.Timestamp{}) + return rts.intentQ.DecrRef(t.TxnID, enginepb.TxnTimestamp{}) case *enginepb.MVCCAbortTxnOp: // Unlike the previous case, an aborted transaction does indicate @@ -246,7 +245,7 @@ func (rts *resolvedTimestamp) assertNoChange() { // assertOpAboveTimestamp asserts that this operation is at a larger timestamp // than the current resolved timestamp. A violation of this assertion would // indicate a failure of the closed timestamp mechanism. -func (rts *resolvedTimestamp) assertOpAboveRTS(op enginepb.MVCCLogicalOp, opTS hlc.Timestamp) { +func (rts *resolvedTimestamp) assertOpAboveRTS(op enginepb.MVCCLogicalOp, opTS enginepb.TxnTimestamp) { if opTS.LessEq(rts.resolvedTS) { panic(fmt.Sprintf("resolved timestamp %s equal to or above timestamp of operation %v", rts.resolvedTS, op)) @@ -282,8 +281,8 @@ func (rts *resolvedTimestamp) assertOpAboveRTS(op enginepb.MVCCLogicalOp, opTS h type unresolvedTxn struct { txnID uuid.UUID txnKey roachpb.Key - txnMinTimestamp hlc.Timestamp - timestamp hlc.Timestamp + txnMinTimestamp enginepb.TxnTimestamp + timestamp enginepb.TxnTimestamp refCount int // count of unresolved intents // The index of the item in the unresolvedTxnHeap, maintained by the @@ -385,7 +384,7 @@ func (uiq *unresolvedIntentQueue) Oldest() *unresolvedTxn { // timestamp. It does so in O(n) time, where n is the number of matching // transactions, NOT the total number of transactions being tracked. The // resulting transactions will not be in sorted order. -func (uiq *unresolvedIntentQueue) Before(ts hlc.Timestamp) []*unresolvedTxn { +func (uiq *unresolvedIntentQueue) Before(ts enginepb.TxnTimestamp) []*unresolvedTxn { var txns []*unresolvedTxn var collect func(int) collect = func(i int) { @@ -403,7 +402,7 @@ func (uiq *unresolvedIntentQueue) Before(ts hlc.Timestamp) []*unresolvedTxn { // returns whether the update advanced the timestamp of the oldest transaction // in the queue. func (uiq *unresolvedIntentQueue) IncRef( - txnID uuid.UUID, txnKey roachpb.Key, txnMinTS, ts hlc.Timestamp, + txnID uuid.UUID, txnKey roachpb.Key, txnMinTS, ts enginepb.TxnTimestamp, ) bool { return uiq.updateTxn(txnID, txnKey, txnMinTS, ts, +1) } @@ -411,19 +410,19 @@ func (uiq *unresolvedIntentQueue) IncRef( // DecrRef decrements the reference count of the specified transaction. It // returns whether the update advanced the timestamp of the oldest transaction // in the queue. -func (uiq *unresolvedIntentQueue) DecrRef(txnID uuid.UUID, ts hlc.Timestamp) bool { - return uiq.updateTxn(txnID, nil, hlc.Timestamp{}, ts, -1) +func (uiq *unresolvedIntentQueue) DecrRef(txnID uuid.UUID, ts enginepb.TxnTimestamp) bool { + return uiq.updateTxn(txnID, nil, enginepb.TxnTimestamp{}, ts, -1) } // UpdateTS updates the timestamp of the specified transaction without modifying // its intent reference count. It returns whether the update advanced the // timestamp of the oldest transaction in the queue. -func (uiq *unresolvedIntentQueue) UpdateTS(txnID uuid.UUID, ts hlc.Timestamp) bool { - return uiq.updateTxn(txnID, nil, hlc.Timestamp{}, ts, 0) +func (uiq *unresolvedIntentQueue) UpdateTS(txnID uuid.UUID, ts enginepb.TxnTimestamp) bool { + return uiq.updateTxn(txnID, nil, enginepb.TxnTimestamp{}, ts, 0) } func (uiq *unresolvedIntentQueue) updateTxn( - txnID uuid.UUID, txnKey roachpb.Key, txnMinTS, ts hlc.Timestamp, delta int, + txnID uuid.UUID, txnKey roachpb.Key, txnMinTS, ts enginepb.TxnTimestamp, delta int, ) bool { txn, ok := uiq.txns[txnID] if !ok { @@ -474,7 +473,7 @@ func (uiq *unresolvedIntentQueue) updateTxn( func (uiq *unresolvedIntentQueue) Del(txnID uuid.UUID) bool { // This implementation is logically equivalent to the following, but // it avoids underflow conditions: - // return uiq.updateTxn(txnID, nil, hlc.Timestamp{}, hlc.Timestamp{}, math.MinInt64) + // return uiq.updateTxn(txnID, nil, enginepb.TxnTimestamp{}, enginepb.TxnTimestamp{}, math.MinInt64) txn, ok := uiq.txns[txnID] if !ok { diff --git a/pkg/kv/kvserver/rangefeed/task.go b/pkg/kv/kvserver/rangefeed/task.go index ccf0f11783b7..a35038a65e73 100644 --- a/pkg/kv/kvserver/rangefeed/task.go +++ b/pkg/kv/kvserver/rangefeed/task.go @@ -16,7 +16,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" - "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/errors" @@ -116,7 +115,7 @@ func (s *initResolvedTSScan) Cancel() { type TxnPusher interface { // PushTxns attempts to push the specified transactions to a new // timestamp. It returns the resulting transaction protos. - PushTxns(context.Context, []enginepb.TxnMeta, hlc.Timestamp) ([]*roachpb.Transaction, error) + PushTxns(context.Context, []enginepb.TxnMeta, enginepb.TxnTimestamp) ([]*roachpb.Transaction, error) // ResolveIntents resolves the specified intents. ResolveIntents(ctx context.Context, intents []roachpb.LockUpdate) error } @@ -141,12 +140,12 @@ type TxnPusher interface { type txnPushAttempt struct { p *Processor txns []enginepb.TxnMeta - ts hlc.Timestamp + ts enginepb.TxnTimestamp doneC chan struct{} } func newTxnPushAttempt( - p *Processor, txns []enginepb.TxnMeta, ts hlc.Timestamp, doneC chan struct{}, + p *Processor, txns []enginepb.TxnMeta, ts enginepb.TxnTimestamp, doneC chan struct{}, ) runnable { return &txnPushAttempt{ p: p, diff --git a/pkg/kv/kvserver/replica.go b/pkg/kv/kvserver/replica.go index 984a72708d04..8e69cc6b93aa 100644 --- a/pkg/kv/kvserver/replica.go +++ b/pkg/kv/kvserver/replica.go @@ -388,7 +388,7 @@ type Replica struct { lastReplicaAddedTime time.Time // initialMaxClosed is the initial maxClosed timestamp for the replica as known // from its left-hand-side upon creation. - initialMaxClosed hlc.Timestamp + initialMaxClosed enginepb.TxnTimestamp // The most recently updated time for each follower of this range. This is updated // every time a Raft message is received from a peer. @@ -583,7 +583,7 @@ type Replica struct { // pendingGCThreshold holds a timestamp which is being proposed as a new // GC threshold for the range. - pendingGCThreshold hlc.Timestamp + pendingGCThreshold enginepb.TxnTimestamp } } @@ -792,7 +792,7 @@ func (r *Replica) GetRangeID() roachpb.RangeID { } // GetGCThreshold returns the GC threshold. -func (r *Replica) GetGCThreshold() hlc.Timestamp { +func (r *Replica) GetGCThreshold() enginepb.TxnTimestamp { r.mu.RLock() defer r.mu.RUnlock() return *r.mu.state.GCThreshold @@ -805,7 +805,7 @@ func (r *Replica) GetGCThreshold() hlc.Timestamp { // contains data outside of the user keyspace, we return the true GC threshold. func (r *Replica) getImpliedGCThresholdRLocked( st *kvserverpb.LeaseStatus, isAdmin bool, -) hlc.Timestamp { +) enginepb.TxnTimestamp { threshold := *r.mu.state.GCThreshold // The GC threshold is the oldest value we can return here. @@ -958,7 +958,7 @@ func (r *Replica) ContainsKeyRange(start, end roachpb.Key) bool { func (r *Replica) GetLastReplicaGCTimestamp(ctx context.Context) (hlc.Timestamp, error) { key := keys.RangeLastReplicaGCTimestampKey(r.RangeID) var timestamp hlc.Timestamp - _, err := storage.MVCCGetProto(ctx, r.store.Engine(), key, hlc.Timestamp{}, ×tamp, + _, err := storage.MVCCGetProto(ctx, r.store.Engine(), key, enginepb.TxnTimestamp{}, ×tamp, storage.MVCCGetOptions{}) if err != nil { return hlc.Timestamp{}, err @@ -968,7 +968,7 @@ func (r *Replica) GetLastReplicaGCTimestamp(ctx context.Context) (hlc.Timestamp, func (r *Replica) setLastReplicaGCTimestamp(ctx context.Context, timestamp hlc.Timestamp) error { key := keys.RangeLastReplicaGCTimestampKey(r.RangeID) - return storage.MVCCPutProto(ctx, r.store.Engine(), nil, key, hlc.Timestamp{}, nil, ×tamp) + return storage.MVCCPutProto(ctx, r.store.Engine(), nil, key, enginepb.TxnTimestamp{}, nil, ×tamp) } // getQueueLastProcessed returns the last processed timestamp for the @@ -977,7 +977,7 @@ func (r *Replica) getQueueLastProcessed(ctx context.Context, queue string) (hlc. key := keys.QueueLastProcessedKey(r.Desc().StartKey, queue) var timestamp hlc.Timestamp if r.store != nil { - _, err := storage.MVCCGetProto(ctx, r.store.Engine(), key, hlc.Timestamp{}, ×tamp, + _, err := storage.MVCCGetProto(ctx, r.store.Engine(), key, enginepb.TxnTimestamp{}, ×tamp, storage.MVCCGetOptions{}) if err != nil { log.VErrEventf(ctx, 2, "last processed timestamp unavailable: %s", err) @@ -1155,7 +1155,7 @@ func (r *Replica) checkExecutionCanProceed( // checkExecutionCanProceedForRangeFeed returns an error if a rangefeed request // cannot be executed by the Replica. func (r *Replica) checkExecutionCanProceedForRangeFeed( - ctx context.Context, rSpan roachpb.RSpan, ts hlc.Timestamp, + ctx context.Context, rSpan roachpb.RSpan, ts enginepb.TxnTimestamp, ) error { now := r.Clock().Now() r.mu.RLock() @@ -1191,7 +1191,7 @@ func (r *Replica) checkSpanInRangeRLocked(ctx context.Context, rspan roachpb.RSp // checkTSAboveGCThresholdRLocked returns an error if a request (identified // by its MVCC timestamp) can be run on the replica. func (r *Replica) checkTSAboveGCThresholdRLocked( - ts hlc.Timestamp, st *kvserverpb.LeaseStatus, isAdmin bool, + ts enginepb.TxnTimestamp, st *kvserverpb.LeaseStatus, isAdmin bool, ) error { threshold := r.getImpliedGCThresholdRLocked(st, isAdmin) if threshold.Less(ts) { @@ -1226,7 +1226,8 @@ func (r *Replica) shouldWaitForPendingMergeRLocked( if ba.Txn != nil { ts.Forward(ba.Txn.MaxTimestamp) } - if ts.Less(freezeStart) { + // TODO(nvanbenschoten): what's the deal here? + if ts.ToClockTimestampUnchecked().Less(freezeStart) { // When the max timestamp of a read request is less than the subsumption // time recorded by this Range (freezeStart), we're guaranteed that none // of the writes accepted by the leaseholder for the keyspan (which could @@ -1407,7 +1408,7 @@ func (ec *endCmds) done( func (r *Replica) maybeWatchForMerge(ctx context.Context, freezeStart hlc.Timestamp) error { desc := r.Desc() descKey := keys.RangeDescriptorKey(desc.StartKey) - _, intent, err := storage.MVCCGet(ctx, r.Engine(), descKey, r.Clock().Now(), + _, intent, err := storage.MVCCGet(ctx, r.Engine(), descKey, enginepb.TxnTimestamp(r.Clock().Now()), storage.MVCCGetOptions{Inconsistent: true}) if err != nil { return err @@ -1462,7 +1463,8 @@ func (r *Replica) maybeWatchForMerge(ctx context.Context, freezeStart hlc.Timest // returns immediately and causes us to spin hot, whereas // roachpb.PUSH_ABORT efficiently blocks until the transaction completes. b := &kv.Batch{} - b.Header.Timestamp = r.Clock().Now() + // TODO(nvanbenschoten): how does the PushTxnRequest work in this case? + b.Header.Timestamp = enginepb.TxnTimestamp(r.Clock().Now()) b.AddRawRequest(&roachpb.PushTxnRequest{ RequestHeader: roachpb.RequestHeader{Key: intent.Txn.Key}, PusherTxn: roachpb.Transaction{ diff --git a/pkg/kv/kvserver/replica_application_result.go b/pkg/kv/kvserver/replica_application_result.go index dea3a95ce7e1..35c19ffd47a2 100644 --- a/pkg/kv/kvserver/replica_application_result.go +++ b/pkg/kv/kvserver/replica_application_result.go @@ -18,7 +18,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "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/log" ) @@ -66,7 +65,7 @@ func isTrivial(r *kvserverpb.ReplicatedEvalResult) bool { // it is trivial. allowlist := *r allowlist.Delta = enginepb.MVCCStatsDelta{} - allowlist.Timestamp = hlc.Timestamp{} + allowlist.Timestamp = enginepb.TxnTimestamp{} allowlist.DeprecatedDelta = nil allowlist.PrevLeaseProposal = nil allowlist.State = nil @@ -84,7 +83,7 @@ func clearTrivialReplicatedEvalResultFields(r *kvserverpb.ReplicatedEvalResult) // they don't trigger an assertion at the end of the application process // (which checks that all fields were handled). r.IsLeaseRequest = false - r.Timestamp = hlc.Timestamp{} + r.Timestamp = enginepb.TxnTimestamp{} r.PrevLeaseProposal = nil // The state fields cleared here were already applied to the in-memory view of // replica state for this batch. @@ -284,7 +283,7 @@ func (r *Replica) handleTruncatedStateResult( return -size } -func (r *Replica) handleGCThresholdResult(ctx context.Context, thresh *hlc.Timestamp) { +func (r *Replica) handleGCThresholdResult(ctx context.Context, thresh *enginepb.TxnTimestamp) { if thresh.IsEmpty() { return } diff --git a/pkg/kv/kvserver/replica_application_state_machine.go b/pkg/kv/kvserver/replica_application_state_machine.go index ad02664c0565..f2f7bb6ea8b9 100644 --- a/pkg/kv/kvserver/replica_application_state_machine.go +++ b/pkg/kv/kvserver/replica_application_state_machine.go @@ -21,7 +21,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" - "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/errors" @@ -377,7 +376,7 @@ type replicaAppBatch struct { stats enginepb.MVCCStats // maxTS is the maximum timestamp that any command that was staged in this // batch was evaluated at. - maxTS hlc.Timestamp + maxTS enginepb.TxnTimestamp // 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. @@ -821,7 +820,9 @@ func (b *replicaAppBatch) ApplyToStateMachine(ctx context.Context) error { // received ops without a timestamp specified are guaranteed one higher than // any op already executed for overlapping keys. r := b.r - r.store.Clock().Update(b.maxTS) + // TODO(nvanbenschoten): what's up with this? Why do we need to update the + // clock here? + // r.store.Clock().Update(b.maxTS) // Add the replica applied state key to the write batch if this change // doesn't remove us. diff --git a/pkg/kv/kvserver/replica_batch_updates.go b/pkg/kv/kvserver/replica_batch_updates.go index 0dd63311b408..95baded1230f 100644 --- a/pkg/kv/kvserver/replica_batch_updates.go +++ b/pkg/kv/kvserver/replica_batch_updates.go @@ -16,7 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/util/hlc" + enginepb "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" ) @@ -201,7 +201,7 @@ func maybeBumpReadTimestampToWriteTimestamp( // Returns true if the timestamp was bumped. Returns false if the timestamp could // not be bumped. func tryBumpBatchTimestamp( - ctx context.Context, ba *roachpb.BatchRequest, ts hlc.Timestamp, latchSpans *spanset.SpanSet, + ctx context.Context, ba *roachpb.BatchRequest, ts enginepb.TxnTimestamp, latchSpans *spanset.SpanSet, ) bool { if latchSpans.MaxProtectedTimestamp().Less(ts) { // If the batch acquired any read latches with bounded (MVCC) timestamps diff --git a/pkg/kv/kvserver/replica_command.go b/pkg/kv/kvserver/replica_command.go index 26143c0d0704..655f4fc68184 100644 --- a/pkg/kv/kvserver/replica_command.go +++ b/pkg/kv/kvserver/replica_command.go @@ -29,6 +29,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/rpc/nodedialer" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/storage" + enginepb "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/contextutil" "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -2049,7 +2050,7 @@ func (r *Replica) sendSnapshot( } usesReplicatedTruncatedState, err := storage.MVCCGetProto( - ctx, snap.EngineSnap, keys.RaftTruncatedStateLegacyKey(r.RangeID), hlc.Timestamp{}, nil, storage.MVCCGetOptions{}, + ctx, snap.EngineSnap, keys.RaftTruncatedStateLegacyKey(r.RangeID), enginepb.TxnTimestamp{}, nil, storage.MVCCGetOptions{}, ) if err != nil { return errors.Wrap(err, "loading legacy truncated state") diff --git a/pkg/kv/kvserver/replica_consistency.go b/pkg/kv/kvserver/replica_consistency.go index f0dd22c55f17..9d8586825706 100644 --- a/pkg/kv/kvserver/replica_consistency.go +++ b/pkg/kv/kvserver/replica_consistency.go @@ -669,7 +669,7 @@ func (r *Replica) sha512( if snapshot != nil { // Add LeaseAppliedState to the diff. kv := roachpb.RaftSnapshotData_KeyValue{ - Timestamp: hlc.Timestamp{}, + Timestamp: enginepb.TxnTimestamp{}, } kv.Key = keys.RangeAppliedStateKey(desc.RangeID) var v roachpb.Value diff --git a/pkg/kv/kvserver/replica_consistency_diff.go b/pkg/kv/kvserver/replica_consistency_diff.go index 36e3190bce9d..80d146d8e3fd 100644 --- a/pkg/kv/kvserver/replica_consistency_diff.go +++ b/pkg/kv/kvserver/replica_consistency_diff.go @@ -15,7 +15,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" - "github.com/cockroachdb/cockroach/pkg/util/hlc" + enginepb "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/redact" ) @@ -26,7 +26,7 @@ type ReplicaSnapshotDiff struct { // holder. LeaseHolder bool Key roachpb.Key - Timestamp hlc.Timestamp + Timestamp enginepb.TxnTimestamp Value []byte } diff --git a/pkg/kv/kvserver/replica_destroy.go b/pkg/kv/kvserver/replica_destroy.go index ea00dc61ab4d..5f749fed3c19 100644 --- a/pkg/kv/kvserver/replica_destroy.go +++ b/pkg/kv/kvserver/replica_destroy.go @@ -20,7 +20,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" - "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/timeutil" ) @@ -245,5 +244,5 @@ func writeTombstoneKey( } // "Blind" because ms == nil and timestamp.IsEmpty(). return storage.MVCCBlindPutProto(ctx, writer, nil, tombstoneKey, - hlc.Timestamp{}, tombstone, nil) + enginepb.TxnTimestamp{}, tombstone, nil) } diff --git a/pkg/kv/kvserver/replica_eval_context_span.go b/pkg/kv/kvserver/replica_eval_context_span.go index 60b5cb2f6fca..86cc6e23babb 100644 --- a/pkg/kv/kvserver/replica_eval_context_span.go +++ b/pkg/kv/kvserver/replica_eval_context_span.go @@ -157,8 +157,8 @@ func (rec SpanSetReplicaEvalContext) GetSplitQPS() float64 { // for the provided transaction information. See Replica.CanCreateTxnRecord // for details about its arguments, return values, and preconditions. func (rec SpanSetReplicaEvalContext) CanCreateTxnRecord( - txnID uuid.UUID, txnKey []byte, txnMinTS hlc.Timestamp, -) (bool, hlc.Timestamp, roachpb.TransactionAbortedReason) { + txnID uuid.UUID, txnKey []byte, txnMinTS enginepb.TxnTimestamp, +) (bool, enginepb.TxnTimestamp, roachpb.TransactionAbortedReason) { rec.ss.AssertAllowed(spanset.SpanReadOnly, roachpb.Span{Key: keys.TransactionKey(txnKey, txnID)}, ) @@ -168,7 +168,7 @@ func (rec SpanSetReplicaEvalContext) CanCreateTxnRecord( // GetGCThreshold returns the GC threshold of the Range, typically updated when // keys are garbage collected. Reads and writes at timestamps <= this time will // not be served. -func (rec SpanSetReplicaEvalContext) GetGCThreshold() hlc.Timestamp { +func (rec SpanSetReplicaEvalContext) GetGCThreshold() enginepb.TxnTimestamp { rec.ss.AssertAllowed(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeLastGCKey(rec.GetRangeID())}, ) diff --git a/pkg/kv/kvserver/replica_evaluate.go b/pkg/kv/kvserver/replica_evaluate.go index 75a3e09e7955..9e3576dcd822 100644 --- a/pkg/kv/kvserver/replica_evaluate.go +++ b/pkg/kv/kvserver/replica_evaluate.go @@ -22,7 +22,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/errorutil" - "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" "github.com/kr/pretty" @@ -533,7 +532,7 @@ func canDoServersideRetry( ba *roachpb.BatchRequest, br *roachpb.BatchResponse, latchSpans *spanset.SpanSet, - deadline *hlc.Timestamp, + deadline *enginepb.TxnTimestamp, ) bool { if ba.Txn != nil { if !ba.CanForwardReadTimestamp { @@ -547,7 +546,7 @@ func canDoServersideRetry( deadline = et.Deadline } } - var newTimestamp hlc.Timestamp + var newTimestamp enginepb.TxnTimestamp if pErr != nil { switch tErr := pErr.GetDetail().(type) { diff --git a/pkg/kv/kvserver/replica_follower_read.go b/pkg/kv/kvserver/replica_follower_read.go index b295bc689768..3a26e5157189 100644 --- a/pkg/kv/kvserver/replica_follower_read.go +++ b/pkg/kv/kvserver/replica_follower_read.go @@ -18,7 +18,7 @@ import ( ctstorage "github.com/cockroachdb/cockroach/pkg/kv/kvserver/closedts/storage" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" - "github.com/cockroachdb/cockroach/pkg/util/hlc" + enginepb "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/log" ) @@ -118,18 +118,18 @@ func (r *Replica) canServeFollowerRead( // uses an expiration-based lease. Expiration-based leases do not support the // closed timestamp subsystem. A zero-value timestamp will be returned if ok // is false. -func (r *Replica) maxClosed(ctx context.Context) (_ hlc.Timestamp, ok bool) { +func (r *Replica) maxClosed(ctx context.Context) (_ enginepb.TxnTimestamp, ok bool) { r.mu.RLock() lai := r.mu.state.LeaseAppliedIndex lease := *r.mu.state.Lease initialMaxClosed := r.mu.initialMaxClosed r.mu.RUnlock() if lease.Expiration != nil { - return hlc.Timestamp{}, false + return enginepb.TxnTimestamp{}, false } maxClosed := r.store.cfg.ClosedTimestamp.Provider.MaxClosed( lease.Replica.NodeID, r.RangeID, ctpb.Epoch(lease.Epoch), ctpb.LAI(lai)) - maxClosed.Forward(lease.Start) + maxClosed.Forward(enginepb.TxnTimestamp(lease.Start)) maxClosed.Forward(initialMaxClosed) return maxClosed, true } diff --git a/pkg/kv/kvserver/replica_gossip.go b/pkg/kv/kvserver/replica_gossip.go index e914e40dab23..3c404129ce56 100644 --- a/pkg/kv/kvserver/replica_gossip.go +++ b/pkg/kv/kvserver/replica_gossip.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/roachpb" + enginepb "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" ) @@ -148,7 +149,7 @@ func (r *Replica) MaybeGossipNodeLiveness(ctx context.Context, span roachpb.Span } ba := roachpb.BatchRequest{} - ba.Timestamp = r.store.Clock().Now() + ba.Timestamp = enginepb.TxnTimestamp(r.store.Clock().Now()) ba.Add(&roachpb.ScanRequest{RequestHeader: roachpb.RequestHeaderFromSpan(span)}) // Call evaluateBatch instead of Send to avoid reacquiring latches. rec := NewReplicaEvalContext(r, todoSpanSet) @@ -200,7 +201,7 @@ var errSystemConfigIntent = errors.New("must retry later due to intent on System func (r *Replica) loadSystemConfig(ctx context.Context) (*config.SystemConfigEntries, error) { ba := roachpb.BatchRequest{} ba.ReadConsistency = roachpb.INCONSISTENT - ba.Timestamp = r.store.Clock().Now() + ba.Timestamp = enginepb.TxnTimestamp(r.store.Clock().Now()) ba.Add(&roachpb.ScanRequest{RequestHeader: roachpb.RequestHeaderFromSpan(keys.SystemConfigSpan)}) // Call evaluateBatch instead of Send to avoid reacquiring latches. rec := NewReplicaEvalContext(r, todoSpanSet) diff --git a/pkg/kv/kvserver/replica_protected_timestamp.go b/pkg/kv/kvserver/replica_protected_timestamp.go index 212dd655bb1f..db5433797311 100644 --- a/pkg/kv/kvserver/replica_protected_timestamp.go +++ b/pkg/kv/kvserver/replica_protected_timestamp.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptpb" "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/log" "github.com/cockroachdb/errors" @@ -233,7 +234,7 @@ func (r *Replica) protectedTimestampRecordCurrentlyApplies( // the new gc threshold itself. func (r *Replica) checkProtectedTimestampsForGC( ctx context.Context, policy zonepb.GCPolicy, -) (canGC bool, cacheTimestamp, gcTimestamp, newThreshold hlc.Timestamp) { +) (canGC bool, cacheTimestamp, gcTimestamp hlc.Timestamp, newThreshold enginepb.TxnTimestamp) { // We may be reading the protected timestamp cache while we're holding // the Replica.mu for reading. If we do so and find newer state in the cache @@ -266,7 +267,7 @@ func (r *Replica) checkProtectedTimestampsForGC( if gcTimestamp.Less(lease.Start) { log.VEventf(ctx, 1, "not gc'ing replica %v due to new lease %v started after %v", r, lease, gcTimestamp) - return false, hlc.Timestamp{}, hlc.Timestamp{}, hlc.Timestamp{} + return false, hlc.Timestamp{}, hlc.Timestamp{}, enginepb.TxnTimestamp{} } newThreshold = gc.CalculateThreshold(gcTimestamp, policy) @@ -274,7 +275,7 @@ func (r *Replica) checkProtectedTimestampsForGC( // If we've already GC'd right up to this record, there's no reason to // gc again. if newThreshold.Equal(gcThreshold) { - return false, hlc.Timestamp{}, hlc.Timestamp{}, hlc.Timestamp{} + return false, hlc.Timestamp{}, hlc.Timestamp{}, enginepb.TxnTimestamp{} } return true, read.readAt, gcTimestamp, newThreshold @@ -288,7 +289,7 @@ func (r *Replica) checkProtectedTimestampsForGC( // verification request arrives which applies under a later cache state and then // the gc queue, acting on older cache state, attempts to set the gc threshold // above a successfully verified record. -func (r *Replica) markPendingGC(readAt, newThreshold hlc.Timestamp) error { +func (r *Replica) markPendingGC(readAt hlc.Timestamp, newThreshold enginepb.TxnTimestamp) error { r.protectedTimestampMu.Lock() defer r.protectedTimestampMu.Unlock() if readAt.Less(r.protectedTimestampMu.minStateReadTimestamp) { diff --git a/pkg/kv/kvserver/replica_raft.go b/pkg/kv/kvserver/replica_raft.go index 99cae0ee4884..648bb597f4fd 100644 --- a/pkg/kv/kvserver/replica_raft.go +++ b/pkg/kv/kvserver/replica_raft.go @@ -26,9 +26,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/stateloader" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" + enginepb "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/encoding" - "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/humanizeutil" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/protoutil" @@ -1722,7 +1722,7 @@ func handleTruncatedStateBelowRaft( if err := storage.MVCCPutProto( ctx, readWriter, nil /* ms */, prefixBuf.RaftTruncatedStateKey(), - hlc.Timestamp{}, nil /* txn */, newTruncatedState, + enginepb.TxnTimestamp{}, nil /* txn */, newTruncatedState, ); err != nil { return false, errors.Wrap(err, "unable to migrate RaftTruncatedState") } diff --git a/pkg/kv/kvserver/replica_raftstorage.go b/pkg/kv/kvserver/replica_raftstorage.go index 0ff166dd5c14..05cc1908965a 100644 --- a/pkg/kv/kvserver/replica_raftstorage.go +++ b/pkg/kv/kvserver/replica_raftstorage.go @@ -237,7 +237,7 @@ func iterateEntries( ctx, reader, keys.RaftLogKey(rangeID, lo), keys.RaftLogKey(rangeID, hi), - hlc.Timestamp{}, + enginepb.TxnTimestamp{}, storage.MVCCScanOptions{}, scanFunc, ) @@ -543,7 +543,7 @@ func snapshot( // know they cannot be committed yet; operations that modify range // descriptors resolve their own intents when they commit. ok, err := storage.MVCCGetProto(ctx, snap, keys.RangeDescriptorKey(startKey), - hlc.MaxTimestamp, &desc, storage.MVCCGetOptions{Inconsistent: true}) + enginepb.TxnTimestamp(hlc.MaxTimestamp), &desc, storage.MVCCGetOptions{Inconsistent: true}) if err != nil { return OutgoingSnapshot{}, errors.Errorf("failed to get desc: %s", err) } @@ -629,7 +629,7 @@ func (r *Replica) append( value.InitChecksum(key) var err error if ent.Index > prevLastIndex { - err = storage.MVCCBlindPut(ctx, writer, &diff, key, hlc.Timestamp{}, value, nil /* txn */) + err = storage.MVCCBlindPut(ctx, writer, &diff, key, enginepb.TxnTimestamp{}, value, nil /* txn */) } else { // We type assert `writer` to also be an engine.ReadWriter only in // the case where we're replacing existing entries. @@ -637,7 +637,7 @@ func (r *Replica) append( if !ok { panic("expected writer to be a engine.ReadWriter when overwriting log entries") } - err = storage.MVCCPut(ctx, eng, &diff, key, hlc.Timestamp{}, value, nil /* txn */) + err = storage.MVCCPut(ctx, eng, &diff, key, enginepb.TxnTimestamp{}, value, nil /* txn */) } if err != nil { return 0, 0, 0, err @@ -658,7 +658,7 @@ func (r *Replica) append( // Note that the caller is in charge of deleting any sideloaded payloads // (which they must only do *after* the batch has committed). err := storage.MVCCDelete(ctx, eng, &diff, r.raftMu.stateLoader.RaftLogKey(i), - hlc.Timestamp{}, nil /* txn */) + enginepb.TxnTimestamp{}, nil /* txn */) if err != nil { return 0, 0, 0, err } diff --git a/pkg/kv/kvserver/replica_range_lease.go b/pkg/kv/kvserver/replica_range_lease.go index dafc1af911c5..c8f707c63ba0 100644 --- a/pkg/kv/kvserver/replica_range_lease.go +++ b/pkg/kv/kvserver/replica_range_lease.go @@ -52,6 +52,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" "github.com/cockroachdb/cockroach/pkg/roachpb" + enginepb "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/timeutil" @@ -396,7 +397,7 @@ func (p *pendingLeaseRequest) requestLeaseAsync( // lease to be applied. if pErr == nil { ba := roachpb.BatchRequest{} - ba.Timestamp = p.repl.store.Clock().Now() + ba.Timestamp = enginepb.TxnTimestamp(p.repl.store.Clock().Now()) ba.RangeID = p.repl.RangeID ba.Add(leaseReq) _, pErr = p.repl.Send(ctx, ba) diff --git a/pkg/kv/kvserver/replica_rangefeed.go b/pkg/kv/kvserver/replica_rangefeed.go index fe82511f03ba..d1245fed4d9f 100644 --- a/pkg/kv/kvserver/replica_rangefeed.go +++ b/pkg/kv/kvserver/replica_rangefeed.go @@ -26,7 +26,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" - "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil" @@ -70,7 +69,7 @@ type rangefeedTxnPusher struct { // high-priority push at the specified timestamp to each of the specified // transactions. func (tp *rangefeedTxnPusher) PushTxns( - ctx context.Context, txns []enginepb.TxnMeta, ts hlc.Timestamp, + ctx context.Context, txns []enginepb.TxnMeta, ts enginepb.TxnTimestamp, ) ([]*roachpb.Transaction, error) { pushTxnMap := make(map[uuid.UUID]*enginepb.TxnMeta, len(txns)) for i := range txns { @@ -155,7 +154,7 @@ func (r *Replica) RangeFeed( if checkTS.IsEmpty() { // If no timestamp was provided then we're not going to run a catch-up // scan, so make sure the GCThreshold in requestCanProceed succeeds. - checkTS = r.Clock().Now() + checkTS = enginepb.TxnTimestamp(r.Clock().Now()) } lockedStream := &lockedRangefeedStream{wrapped: stream} @@ -300,7 +299,7 @@ const defaultEventChanCap = 4096 func (r *Replica) registerWithRangefeedRaftMuLocked( ctx context.Context, span roachpb.RSpan, - startTS hlc.Timestamp, + startTS enginepb.TxnTimestamp, catchupIter rangefeed.IteratorConstructor, withDiff bool, stream rangefeed.Stream, @@ -450,7 +449,7 @@ func (r *Replica) populatePrevValsInLogicalOpLogRaftMuLocked( // Read from the Reader to populate the PrevValue fields. for _, op := range ops.Ops { var key []byte - var ts hlc.Timestamp + var ts enginepb.TxnTimestamp var prevValPtr *[]byte switch t := op.GetValue().(type) { case *enginepb.MVCCWriteValueOp: @@ -524,7 +523,7 @@ func (r *Replica) handleLogicalOpLogRaftMuLocked( // fully populated. Read from the Reader to populate all fields. for _, op := range ops.Ops { var key []byte - var ts hlc.Timestamp + var ts enginepb.TxnTimestamp var valPtr *[]byte switch t := op.GetValue().(type) { case *enginepb.MVCCWriteValueOp: diff --git a/pkg/kv/kvserver/replica_tscache.go b/pkg/kv/kvserver/replica_tscache.go index 1fe1aebf6c42..7088b2f2c543 100644 --- a/pkg/kv/kvserver/replica_tscache.go +++ b/pkg/kv/kvserver/replica_tscache.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/tscache" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/server/telemetry" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -48,9 +49,12 @@ func (r *Replica) updateTimestampCache( // Inconsistent reads are excluded from the timestamp cache. return } - addToTSCache := r.store.tsCache.Add + addToTSCache2 := r.store.tsCache.Add if util.RaceEnabled { - addToTSCache = checkedTSCacheUpdate(r.store.Clock().Now(), r.store.tsCache, ba, br, pErr) + addToTSCache2 = checkedTSCacheUpdate(r.store.Clock().Now(), r.store.tsCache, ba, br, pErr) + } + addToTSCache := func(start, end roachpb.Key, ts enginepb.TxnTimestamp, txnID uuid.UUID) { + addToTSCache2(start, end, ts.ToClockTimestampUnchecked(), txnID) } // Update the timestamp cache using the timestamp at which the batch // was executed. Note this may have moved forward from ba.Timestamp, @@ -260,7 +264,7 @@ func init() { // minReadTS, minReadTS (without an associated txn id) will be used instead to // adjust the batch's timestamp. func (r *Replica) applyTimestampCache( - ctx context.Context, ba *roachpb.BatchRequest, minReadTS hlc.Timestamp, + ctx context.Context, ba *roachpb.BatchRequest, minReadTS enginepb.TxnTimestamp, ) bool { // bumpedDueToMinReadTS is set to true if the highest timestamp bump encountered // below is due to the minReadTS. @@ -274,7 +278,8 @@ func (r *Replica) applyTimestampCache( header := args.Header() // Forward the timestamp if there's been a more recent read (by someone else). - rTS, rTxnID := r.store.tsCache.GetMax(header.Key, header.EndKey) + rTS2, rTxnID := r.store.tsCache.GetMax(header.Key, header.EndKey) + rTS := enginepb.TxnTimestamp(rTS2) var forwardedToMinReadTS bool if rTS.Forward(minReadTS) { forwardedToMinReadTS = true @@ -451,8 +456,8 @@ func (r *Replica) applyTimestampCache( // system. // func (r *Replica) CanCreateTxnRecord( - txnID uuid.UUID, txnKey []byte, txnMinTS hlc.Timestamp, -) (ok bool, minCommitTS hlc.Timestamp, reason roachpb.TransactionAbortedReason) { + txnID uuid.UUID, txnKey []byte, txnMinTS enginepb.TxnTimestamp, +) (ok bool, minCommitTS enginepb.TxnTimestamp, reason roachpb.TransactionAbortedReason) { // Consult the timestamp cache with the transaction's key. The timestamp // cache is used in two ways for transactions without transaction records. // The timestamp cache is used to push the timestamp of transactions @@ -470,7 +475,8 @@ func (r *Replica) CanCreateTxnRecord( // transaction, which indicates the minimum timestamp that the transaction // can commit at. This is used by pushers to push the timestamp of a // transaction that hasn't yet written its transaction record. - minCommitTS, _ = r.store.tsCache.GetMax(pushKey, nil /* end */) + minCommitTS2, _ := r.store.tsCache.GetMax(pushKey, nil /* end */) + minCommitTS = enginepb.TxnTimestamp(minCommitTS2) // Also look in the timestamp cache to see if there is a tombstone entry for // this transaction, which would indicate this transaction has already been @@ -479,7 +485,8 @@ func (r *Replica) CanCreateTxnRecord( // then the error will be transformed into an ambiguous one higher up. // Otherwise, if the client is still waiting for a result, then this cannot // be a "replay" of any sort. - tombstoneTimestamp, tombstomeTxnID := r.store.tsCache.GetMax(tombstoneKey, nil /* end */) + tombstoneTimestamp2, tombstomeTxnID := r.store.tsCache.GetMax(tombstoneKey, nil /* end */) + tombstoneTimestamp := enginepb.TxnTimestamp(tombstoneTimestamp2) // Compare against the minimum timestamp that the transaction could have // written intents at. if txnMinTS.LessEq(tombstoneTimestamp) { @@ -497,23 +504,23 @@ func (r *Replica) CanCreateTxnRecord( // If there were other requests in the EndTxn batch, then the client would // still have trouble reconstructing the result, but at least it could // provide a non-ambiguous error to the application. - return false, hlc.Timestamp{}, + return false, enginepb.TxnTimestamp{}, roachpb.ABORT_REASON_ALREADY_COMMITTED_OR_ROLLED_BACK_POSSIBLE_REPLAY case uuid.Nil: lease, _ /* nextLease */ := r.GetLease() // Recognize the case where a lease started recently. Lease transfers bump // the ts cache low water mark. - if tombstoneTimestamp == lease.Start { - return false, hlc.Timestamp{}, roachpb.ABORT_REASON_NEW_LEASE_PREVENTS_TXN + if tombstoneTimestamp == enginepb.TxnTimestamp(lease.Start) { + return false, enginepb.TxnTimestamp{}, roachpb.ABORT_REASON_NEW_LEASE_PREVENTS_TXN } - return false, hlc.Timestamp{}, roachpb.ABORT_REASON_TIMESTAMP_CACHE_REJECTED + return false, enginepb.TxnTimestamp{}, roachpb.ABORT_REASON_TIMESTAMP_CACHE_REJECTED default: // If we find another transaction's ID then that transaction has // aborted us before our transaction record was written. It obeyed // the restriction that it couldn't create a transaction record for // us, so it recorded a tombstone cache instead to prevent us // from ever creating a transaction record. - return false, hlc.Timestamp{}, roachpb.ABORT_REASON_ABORTED_RECORD_FOUND + return false, enginepb.TxnTimestamp{}, roachpb.ABORT_REASON_ABORTED_RECORD_FOUND } } return true, minCommitTS, 0 diff --git a/pkg/kv/kvserver/replica_write.go b/pkg/kv/kvserver/replica_write.go index 1c8edfceec9e..ce9eeae8f2e3 100644 --- a/pkg/kv/kvserver/replica_write.go +++ b/pkg/kv/kvserver/replica_write.go @@ -27,7 +27,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util" - "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/errors" @@ -555,7 +554,7 @@ func (r *Replica) evaluateWriteBatchWithServersideRefreshes( ms *enginepb.MVCCStats, ba *roachpb.BatchRequest, latchSpans *spanset.SpanSet, - deadline *hlc.Timestamp, + deadline *enginepb.TxnTimestamp, ) (batch storage.Batch, br *roachpb.BatchResponse, res result.Result, pErr *roachpb.Error) { goldenMS := *ms for retries := 0; ; retries++ { diff --git a/pkg/kv/kvserver/spanlatch/manager.go b/pkg/kv/kvserver/spanlatch/manager.go index e95a76e2a565..6bcf467265f0 100644 --- a/pkg/kv/kvserver/spanlatch/manager.go +++ b/pkg/kv/kvserver/spanlatch/manager.go @@ -19,7 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/metric" "github.com/cockroachdb/cockroach/pkg/util/stop" @@ -85,7 +85,7 @@ func Make(stopper *stop.Stopper, slowReqs *metric.Gauge) Manager { type latch struct { id uint64 span roachpb.Span - ts hlc.Timestamp + ts enginepb.TxnTimestamp done *signal next, prev *latch // readSet linked-list. } @@ -311,11 +311,11 @@ func (m *Manager) nextIDLocked() uint64 { // This is also disabled in the global scope if either of the timestamps are // empty. In those cases, we consider the latch without a timestamp to be a // non-MVCC operation that affects all timestamps in the key range. -type ignoreFn func(ts, other hlc.Timestamp) bool +type ignoreFn func(ts, other enginepb.TxnTimestamp) bool -func ignoreLater(ts, other hlc.Timestamp) bool { return !ts.IsEmpty() && ts.Less(other) } -func ignoreEarlier(ts, other hlc.Timestamp) bool { return !other.IsEmpty() && other.Less(ts) } -func ignoreNothing(ts, other hlc.Timestamp) bool { return false } +func ignoreLater(ts, other enginepb.TxnTimestamp) bool { return !ts.IsEmpty() && ts.Less(other) } +func ignoreEarlier(ts, other enginepb.TxnTimestamp) bool { return !other.IsEmpty() && other.Less(ts) } +func ignoreNothing(ts, other enginepb.TxnTimestamp) bool { return false } // wait waits for all interfering latches in the provided snapshot to complete // before returning. diff --git a/pkg/kv/kvserver/spanset/batch.go b/pkg/kv/kvserver/spanset/batch.go index 7b62135f5c7d..66a1230dc160 100644 --- a/pkg/kv/kvserver/spanset/batch.go +++ b/pkg/kv/kvserver/spanset/batch.go @@ -14,7 +14,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" - "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/uuid" ) @@ -32,7 +31,7 @@ type MVCCIterator struct { // Timestamp the access is taking place. If timestamp is zero, access is // considered non-MVCC. If spansOnly is set to true, ts is not consulted. - ts hlc.Timestamp + ts enginepb.TxnTimestamp // Seeking to an invalid key puts the iterator in an error state. err error @@ -52,7 +51,7 @@ func NewIterator(iter storage.MVCCIterator, spans *SpanSet) *MVCCIterator { // NewIteratorAt constructs an iterator that verifies access of the underlying // iterator against the given SpanSet at the given timestamp. -func NewIteratorAt(iter storage.MVCCIterator, spans *SpanSet, ts hlc.Timestamp) *MVCCIterator { +func NewIteratorAt(iter storage.MVCCIterator, spans *SpanSet, ts enginepb.TxnTimestamp) *MVCCIterator { return &MVCCIterator{i: iter, spans: spans, ts: ts} } @@ -327,7 +326,7 @@ type spanSetReader struct { spans *SpanSet spansOnly bool - ts hlc.Timestamp + ts enginepb.TxnTimestamp } var _ storage.Reader = spanSetReader{} @@ -343,7 +342,7 @@ func (s spanSetReader) Closed() bool { // ExportMVCCToSst is part of the engine.Reader interface. func (s spanSetReader) ExportMVCCToSst( startKey, endKey roachpb.Key, - startTS, endTS hlc.Timestamp, + startTS, endTS enginepb.TxnTimestamp, exportAllRevisions bool, targetSize, maxSize uint64, io storage.IterOptions, @@ -442,7 +441,7 @@ type spanSetWriter struct { spans *SpanSet spansOnly bool - ts hlc.Timestamp + ts enginepb.TxnTimestamp } var _ storage.Writer = spanSetWriter{} @@ -620,7 +619,7 @@ func makeSpanSetReadWriter(rw storage.ReadWriter, spans *SpanSet) ReadWriter { } } -func makeSpanSetReadWriterAt(rw storage.ReadWriter, spans *SpanSet, ts hlc.Timestamp) ReadWriter { +func makeSpanSetReadWriterAt(rw storage.ReadWriter, spans *SpanSet, ts enginepb.TxnTimestamp) ReadWriter { return ReadWriter{ spanSetReader: spanSetReader{r: rw, spans: spans, ts: ts}, spanSetWriter: spanSetWriter{w: rw, spans: spans, ts: ts}, @@ -636,7 +635,7 @@ func NewReadWriter(rw storage.ReadWriter, spans *SpanSet) storage.ReadWriter { // NewReadWriterAt returns an engine.ReadWriter that asserts access of the // underlying ReadWriter against the given SpanSet at a given timestamp. // If zero timestamp is provided, accesses are considered non-MVCC. -func NewReadWriterAt(rw storage.ReadWriter, spans *SpanSet, ts hlc.Timestamp) storage.ReadWriter { +func NewReadWriterAt(rw storage.ReadWriter, spans *SpanSet, ts enginepb.TxnTimestamp) storage.ReadWriter { return makeSpanSetReadWriterAt(rw, spans, ts) } @@ -646,7 +645,7 @@ type spanSetBatch struct { spans *SpanSet spansOnly bool - ts hlc.Timestamp + ts enginepb.TxnTimestamp } var _ storage.Batch = spanSetBatch{} @@ -689,7 +688,7 @@ func NewBatch(b storage.Batch, spans *SpanSet) storage.Batch { // NewBatchAt returns an engine.Batch that asserts access of the underlying // Batch against the given SpanSet at the given timestamp. // If the zero timestamp is used, all accesses are considered non-MVCC. -func NewBatchAt(b storage.Batch, spans *SpanSet, ts hlc.Timestamp) storage.Batch { +func NewBatchAt(b storage.Batch, spans *SpanSet, ts enginepb.TxnTimestamp) storage.Batch { return &spanSetBatch{ ReadWriter: makeSpanSetReadWriterAt(b, spans, ts), b: b, diff --git a/pkg/kv/kvserver/spanset/spanset.go b/pkg/kv/kvserver/spanset/spanset.go index 4ba50823bcc5..8c6cafe5e89e 100644 --- a/pkg/kv/kvserver/spanset/spanset.go +++ b/pkg/kv/kvserver/spanset/spanset.go @@ -18,6 +18,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/log" "github.com/cockroachdb/errors" @@ -71,7 +72,7 @@ func (a SpanScope) String() string { // timestamp. A zero timestamp indicates it's a non-MVCC access. type Span struct { roachpb.Span - Timestamp hlc.Timestamp + Timestamp enginepb.TxnTimestamp } // SpanSet tracks the set of key spans touched by a command, broken into MVCC @@ -124,16 +125,16 @@ func (s *SpanSet) Reserve(access SpanAccess, scope SpanScope, n int) { // AddNonMVCC adds a non-MVCC span to the span set. This should typically // local keys. func (s *SpanSet) AddNonMVCC(access SpanAccess, span roachpb.Span) { - s.AddMVCC(access, span, hlc.Timestamp{}) + s.AddMVCC(access, span, enginepb.TxnTimestamp{}) } // AddMVCC adds an MVCC span to the span set to be accessed at the given // timestamp. This should typically be used for MVCC keys, user keys for e.g. -func (s *SpanSet) AddMVCC(access SpanAccess, span roachpb.Span, timestamp hlc.Timestamp) { +func (s *SpanSet) AddMVCC(access SpanAccess, span roachpb.Span, timestamp enginepb.TxnTimestamp) { scope := SpanGlobal if keys.IsLocal(span.Key) { scope = SpanLocal - timestamp = hlc.Timestamp{} + timestamp = enginepb.TxnTimestamp{} } s.spans[access][scope] = append(s.spans[access][scope], Span{Span: span, Timestamp: timestamp}) @@ -183,8 +184,8 @@ func (s *SpanSet) BoundarySpan(scope SpanScope) roachpb.Span { // declared timestamp forward, so they have no maximum protect timestamp. // However, ReadOnly are protected only up to their declared timestamp and // are not protected at later timestamps. -func (s *SpanSet) MaxProtectedTimestamp() hlc.Timestamp { - maxTS := hlc.MaxTimestamp +func (s *SpanSet) MaxProtectedTimestamp() enginepb.TxnTimestamp { + maxTS := enginepb.TxnTimestamp(hlc.MaxTimestamp) for ss := SpanScope(0); ss < NumSpanScope; ss++ { for _, cur := range s.GetSpans(SpanReadOnly, ss) { curTS := cur.Timestamp @@ -248,7 +249,7 @@ func (s *SpanSet) CheckAllowed(access SpanAccess, span roachpb.Span) error { // CheckAllowedAt is like CheckAllowed, except it returns an error if the access // is not allowed over the given keyspan at the given timestamp. func (s *SpanSet) CheckAllowedAt( - access SpanAccess, span roachpb.Span, timestamp hlc.Timestamp, + access SpanAccess, span roachpb.Span, timestamp enginepb.TxnTimestamp, ) error { mvcc := !timestamp.IsEmpty() return s.checkAllowed(access, span, func(declAccess SpanAccess, declSpan Span) bool { diff --git a/pkg/kv/kvserver/stateloader/initial.go b/pkg/kv/kvserver/stateloader/initial.go index 51002edcc6d6..8c02b3cb3501 100644 --- a/pkg/kv/kvserver/stateloader/initial.go +++ b/pkg/kv/kvserver/stateloader/initial.go @@ -17,7 +17,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" - "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" ) @@ -44,7 +43,7 @@ func WriteInitialReplicaState( ms enginepb.MVCCStats, desc roachpb.RangeDescriptor, lease roachpb.Lease, - gcThreshold hlc.Timestamp, + gcThreshold enginepb.TxnTimestamp, truncStateType TruncatedStateType, ) (enginepb.MVCCStats, error) { rsl := Make(desc.RangeID) @@ -92,7 +91,7 @@ func WriteInitialState( ms enginepb.MVCCStats, desc roachpb.RangeDescriptor, lease roachpb.Lease, - gcThreshold hlc.Timestamp, + gcThreshold enginepb.TxnTimestamp, truncStateType TruncatedStateType, ) (enginepb.MVCCStats, error) { newMS, err := WriteInitialReplicaState( diff --git a/pkg/kv/kvserver/stateloader/stateloader.go b/pkg/kv/kvserver/stateloader/stateloader.go index d03b5d8173b8..c6dde47fba39 100644 --- a/pkg/kv/kvserver/stateloader/stateloader.go +++ b/pkg/kv/kvserver/stateloader/stateloader.go @@ -20,7 +20,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/encoding" - "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/errors" @@ -174,7 +173,7 @@ func (rsl StateLoader) LoadLease( ) (roachpb.Lease, error) { var lease roachpb.Lease _, err := storage.MVCCGetProto(ctx, reader, rsl.RangeLeaseKey(), - hlc.Timestamp{}, &lease, storage.MVCCGetOptions{}) + enginepb.TxnTimestamp{}, &lease, storage.MVCCGetOptions{}) return lease, err } @@ -183,7 +182,7 @@ func (rsl StateLoader) SetLease( ctx context.Context, readWriter storage.ReadWriter, ms *enginepb.MVCCStats, lease roachpb.Lease, ) error { return storage.MVCCPutProto(ctx, readWriter, ms, rsl.RangeLeaseKey(), - hlc.Timestamp{}, nil, &lease) + enginepb.TxnTimestamp{}, nil, &lease) } // LoadRangeAppliedState loads the Range applied state. The returned pointer @@ -192,7 +191,7 @@ func (rsl StateLoader) LoadRangeAppliedState( ctx context.Context, reader storage.Reader, ) (*enginepb.RangeAppliedState, error) { var as enginepb.RangeAppliedState - found, err := storage.MVCCGetProto(ctx, reader, rsl.RangeAppliedStateKey(), hlc.Timestamp{}, &as, + found, err := storage.MVCCGetProto(ctx, reader, rsl.RangeAppliedStateKey(), enginepb.TxnTimestamp{}, &as, storage.MVCCGetOptions{}) if !found { return nil, err @@ -225,7 +224,7 @@ func (rsl StateLoader) LoadAppliedIndex( // index and the lease applied index keys. This is where these indices were // stored before the range applied state was introduced. v, _, err := storage.MVCCGet(ctx, reader, rsl.RaftAppliedIndexLegacyKey(), - hlc.Timestamp{}, storage.MVCCGetOptions{}) + enginepb.TxnTimestamp{}, storage.MVCCGetOptions{}) if err != nil { return 0, 0, err } @@ -238,7 +237,7 @@ func (rsl StateLoader) LoadAppliedIndex( } // TODO(tschottdorf): code duplication. v, _, err = storage.MVCCGet(ctx, reader, rsl.LeaseAppliedIndexLegacyKey(), - hlc.Timestamp{}, storage.MVCCGetOptions{}) + enginepb.TxnTimestamp{}, storage.MVCCGetOptions{}) if err != nil { return 0, 0, err } @@ -267,7 +266,7 @@ func (rsl StateLoader) LoadMVCCStats( // key. This is where stats were stored before the range applied // state was introduced. var ms enginepb.MVCCStats - _, err := storage.MVCCGetProto(ctx, reader, rsl.RangeStatsLegacyKey(), hlc.Timestamp{}, &ms, + _, err := storage.MVCCGetProto(ctx, reader, rsl.RangeStatsLegacyKey(), enginepb.TxnTimestamp{}, &ms, storage.MVCCGetOptions{}) return ms, err } @@ -292,7 +291,7 @@ func (rsl StateLoader) SetRangeAppliedState( // The RangeAppliedStateKey is not included in stats. This is also reflected // in C.MVCCComputeStats and ComputeStatsForRange. ms := (*enginepb.MVCCStats)(nil) - return storage.MVCCPutProto(ctx, readWriter, ms, rsl.RangeAppliedStateKey(), hlc.Timestamp{}, nil, &as) + return storage.MVCCPutProto(ctx, readWriter, ms, rsl.RangeAppliedStateKey(), enginepb.TxnTimestamp{}, nil, &as) } // MigrateToRangeAppliedStateKey deletes the keys that were replaced by the @@ -300,7 +299,7 @@ func (rsl StateLoader) SetRangeAppliedState( func (rsl StateLoader) MigrateToRangeAppliedStateKey( ctx context.Context, readWriter storage.ReadWriter, ms *enginepb.MVCCStats, ) error { - noTS := hlc.Timestamp{} + noTS := enginepb.TxnTimestamp{} if err := storage.MVCCDelete(ctx, readWriter, ms, rsl.RaftAppliedIndexLegacyKey(), noTS, nil); err != nil { return err } @@ -329,7 +328,7 @@ func (rsl StateLoader) SetLegacyAppliedIndex( value.SetInt(int64(appliedIndex)) if err := storage.MVCCPut(ctx, readWriter, ms, rsl.RaftAppliedIndexLegacyKey(), - hlc.Timestamp{}, + enginepb.TxnTimestamp{}, value, nil /* txn */); err != nil { return err @@ -337,7 +336,7 @@ func (rsl StateLoader) SetLegacyAppliedIndex( value.SetInt(int64(leaseAppliedIndex)) return storage.MVCCPut(ctx, readWriter, ms, rsl.LeaseAppliedIndexLegacyKey(), - hlc.Timestamp{}, + enginepb.TxnTimestamp{}, value, nil /* txn */) } @@ -364,7 +363,7 @@ func (rsl StateLoader) SetLegacyAppliedIndexBlind( value.SetInt(int64(appliedIndex)) if err := storage.MVCCBlindPut(ctx, readWriter, ms, rsl.RaftAppliedIndexLegacyKey(), - hlc.Timestamp{}, + enginepb.TxnTimestamp{}, value, nil /* txn */); err != nil { return err @@ -372,7 +371,7 @@ func (rsl StateLoader) SetLegacyAppliedIndexBlind( value.SetInt(int64(leaseAppliedIndex)) return storage.MVCCBlindPut(ctx, readWriter, ms, rsl.LeaseAppliedIndexLegacyKey(), - hlc.Timestamp{}, + enginepb.TxnTimestamp{}, value, nil /* txn */) } @@ -435,7 +434,7 @@ func (rsl StateLoader) writeLegacyMVCCStatsInternal( SysBytes: newMS.SysBytes, SysCount: newMS.SysCount, } - return storage.MVCCPutProto(ctx, readWriter, nil, rsl.RangeStatsLegacyKey(), hlc.Timestamp{}, nil, &legacyMS) + return storage.MVCCPutProto(ctx, readWriter, nil, rsl.RangeStatsLegacyKey(), enginepb.TxnTimestamp{}, nil, &legacyMS) } // SetLegacyMVCCStats overwrites the legacy MVCC stats key. @@ -478,16 +477,16 @@ func (rsl StateLoader) SetLegacyRaftTruncatedState( return errors.New("cannot persist empty RaftTruncatedState") } return storage.MVCCPutProto(ctx, readWriter, ms, - rsl.RaftTruncatedStateLegacyKey(), hlc.Timestamp{}, nil, truncState) + rsl.RaftTruncatedStateLegacyKey(), enginepb.TxnTimestamp{}, nil, truncState) } // LoadGCThreshold loads the GC threshold. func (rsl StateLoader) LoadGCThreshold( ctx context.Context, reader storage.Reader, -) (*hlc.Timestamp, error) { - var t hlc.Timestamp +) (*enginepb.TxnTimestamp, error) { + var t enginepb.TxnTimestamp _, err := storage.MVCCGetProto(ctx, reader, rsl.RangeLastGCKey(), - hlc.Timestamp{}, &t, storage.MVCCGetOptions{}) + enginepb.TxnTimestamp{}, &t, storage.MVCCGetOptions{}) return &t, err } @@ -496,13 +495,13 @@ func (rsl StateLoader) SetGCThreshold( ctx context.Context, readWriter storage.ReadWriter, ms *enginepb.MVCCStats, - threshold *hlc.Timestamp, + threshold *enginepb.TxnTimestamp, ) error { if threshold == nil { return errors.New("cannot persist nil GCThreshold") } return storage.MVCCPutProto(ctx, readWriter, ms, - rsl.RangeLastGCKey(), hlc.Timestamp{}, nil, threshold) + rsl.RangeLastGCKey(), enginepb.TxnTimestamp{}, nil, threshold) } // The rest is not technically part of ReplicaState. @@ -547,7 +546,7 @@ func (rsl StateLoader) LoadRaftTruncatedState( ) (_ roachpb.RaftTruncatedState, isLegacy bool, _ error) { var truncState roachpb.RaftTruncatedState if found, err := storage.MVCCGetProto( - ctx, reader, rsl.RaftTruncatedStateKey(), hlc.Timestamp{}, &truncState, storage.MVCCGetOptions{}, + ctx, reader, rsl.RaftTruncatedStateKey(), enginepb.TxnTimestamp{}, &truncState, storage.MVCCGetOptions{}, ); err != nil { return roachpb.RaftTruncatedState{}, false, err } else if found { @@ -560,7 +559,7 @@ func (rsl StateLoader) LoadRaftTruncatedState( // // See VersionUnreplicatedRaftTruncatedState. legacyFound, err := storage.MVCCGetProto( - ctx, reader, rsl.RaftTruncatedStateLegacyKey(), hlc.Timestamp{}, &truncState, storage.MVCCGetOptions{}, + ctx, reader, rsl.RaftTruncatedStateLegacyKey(), enginepb.TxnTimestamp{}, &truncState, storage.MVCCGetOptions{}, ) if err != nil { return roachpb.RaftTruncatedState{}, false, err @@ -581,7 +580,7 @@ func (rsl StateLoader) SetRaftTruncatedState( writer, nil, /* ms */ rsl.RaftTruncatedStateKey(), - hlc.Timestamp{}, /* timestamp */ + enginepb.TxnTimestamp{}, /* timestamp */ truncState, nil, /* txn */ ) @@ -593,7 +592,7 @@ func (rsl StateLoader) LoadHardState( ) (raftpb.HardState, error) { var hs raftpb.HardState found, err := storage.MVCCGetProto(ctx, reader, rsl.RaftHardStateKey(), - hlc.Timestamp{}, &hs, storage.MVCCGetOptions{}) + enginepb.TxnTimestamp{}, &hs, storage.MVCCGetOptions{}) if !found || err != nil { return raftpb.HardState{}, err @@ -611,7 +610,7 @@ func (rsl StateLoader) SetHardState( writer, nil, /* ms */ rsl.RaftHardStateKey(), - hlc.Timestamp{}, /* timestamp */ + enginepb.TxnTimestamp{}, /* timestamp */ &hs, nil, /* txn */ ) diff --git a/pkg/kv/kvserver/store.go b/pkg/kv/kvserver/store.go index a6b6a6bd00f8..8e5b35d15cdc 100644 --- a/pkg/kv/kvserver/store.go +++ b/pkg/kv/kvserver/store.go @@ -1296,7 +1296,7 @@ func IterateIDPrefixKeys( } ok, err := storage.MVCCGetProto( - ctx, reader, unsafeKey.Key, hlc.Timestamp{}, msg, storage.MVCCGetOptions{}) + ctx, reader, unsafeKey.Key, enginepb.TxnTimestamp{}, msg, storage.MVCCGetOptions{}) if err != nil { return err } @@ -1350,7 +1350,7 @@ func IterateRangeDescriptors( return err } - _, err := storage.MVCCIterate(ctx, reader, start, end, hlc.MaxTimestamp, + _, err := storage.MVCCIterate(ctx, reader, start, end, enginepb.TxnTimestamp(hlc.MaxTimestamp), storage.MVCCScanOptions{Inconsistent: true}, kvToDesc) log.Eventf(ctx, "iterated over %d keys to find %d range descriptors (by suffix: %v)", allCount, matchCount, bySuffix) @@ -1363,7 +1363,7 @@ func IterateRangeDescriptors( func ReadStoreIdent(ctx context.Context, eng storage.Engine) (roachpb.StoreIdent, error) { var ident roachpb.StoreIdent ok, err := storage.MVCCGetProto( - ctx, eng, keys.StoreIdentKey(), hlc.Timestamp{}, &ident, storage.MVCCGetOptions{}) + ctx, eng, keys.StoreIdentKey(), enginepb.TxnTimestamp{}, &ident, storage.MVCCGetOptions{}) if err != nil { return roachpb.StoreIdent{}, err } else if !ok { @@ -2076,7 +2076,7 @@ func (s *Store) WriteLastUpTimestamp(ctx context.Context, time hlc.Timestamp) er s.engine, nil, keys.StoreLastUpKey(), - hlc.Timestamp{}, + enginepb.TxnTimestamp{}, nil, &time, ) @@ -2089,7 +2089,7 @@ func (s *Store) WriteLastUpTimestamp(ctx context.Context, time hlc.Timestamp) er // timestamp is returned instead. func (s *Store) ReadLastUpTimestamp(ctx context.Context) (hlc.Timestamp, error) { var timestamp hlc.Timestamp - ok, err := storage.MVCCGetProto(ctx, s.Engine(), keys.StoreLastUpKey(), hlc.Timestamp{}, + ok, err := storage.MVCCGetProto(ctx, s.Engine(), keys.StoreLastUpKey(), enginepb.TxnTimestamp{}, ×tamp, storage.MVCCGetOptions{}) if err != nil { return hlc.Timestamp{}, err @@ -2111,7 +2111,7 @@ func (s *Store) WriteHLCUpperBound(ctx context.Context, time int64) error { batch, nil, keys.StoreHLCUpperBoundKey(), - hlc.Timestamp{}, + enginepb.TxnTimestamp{}, nil, &ts, ); err != nil { @@ -2128,7 +2128,7 @@ func (s *Store) WriteHLCUpperBound(ctx context.Context, time int64) error { // If this value does not exist 0 is returned func ReadHLCUpperBound(ctx context.Context, e storage.Engine) (int64, error) { var timestamp hlc.Timestamp - ok, err := storage.MVCCGetProto(ctx, e, keys.StoreHLCUpperBoundKey(), hlc.Timestamp{}, + ok, err := storage.MVCCGetProto(ctx, e, keys.StoreHLCUpperBoundKey(), enginepb.TxnTimestamp{}, ×tamp, storage.MVCCGetOptions{}) if err != nil { return 0, err @@ -2509,7 +2509,7 @@ func (s *Store) updateReplicationGauges(ctx context.Context) error { } clusterNodes := s.ClusterNodeCount() - var minMaxClosedTS hlc.Timestamp + var minMaxClosedTS enginepb.TxnTimestamp newStoreReplicaVisitor(s).Visit(func(rep *Replica) bool { metrics := rep.Metrics(ctx, timestamp, livenessMap, clusterNodes) if metrics.Leader { @@ -2775,7 +2775,7 @@ func (s *Store) ManuallyEnqueue( func WriteClusterVersion( ctx context.Context, eng storage.Engine, cv clusterversion.ClusterVersion, ) error { - return storage.MVCCPutProto(ctx, eng, nil, keys.StoreClusterVersionKey(), hlc.Timestamp{}, nil, &cv) + return storage.MVCCPutProto(ctx, eng, nil, keys.StoreClusterVersionKey(), enginepb.TxnTimestamp{}, nil, &cv) } // ReadClusterVersion reads the cluster version from the store-local version @@ -2784,7 +2784,7 @@ func ReadClusterVersion( ctx context.Context, reader storage.Reader, ) (clusterversion.ClusterVersion, error) { var cv clusterversion.ClusterVersion - _, err := storage.MVCCGetProto(ctx, reader, keys.StoreClusterVersionKey(), hlc.Timestamp{}, + _, err := storage.MVCCGetProto(ctx, reader, keys.StoreClusterVersionKey(), enginepb.TxnTimestamp{}, &cv, storage.MVCCGetOptions{}) return cv, err } diff --git a/pkg/kv/kvserver/store_create_replica.go b/pkg/kv/kvserver/store_create_replica.go index 2044d4d31b51..65a17e124925 100644 --- a/pkg/kv/kvserver/store_create_replica.go +++ b/pkg/kv/kvserver/store_create_replica.go @@ -18,7 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" - "github.com/cockroachdb/cockroach/pkg/util/hlc" + enginepb "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/retry" "github.com/cockroachdb/errors" @@ -142,7 +142,7 @@ func (s *Store) tryGetOrCreateReplica( tombstoneKey := keys.RangeTombstoneKey(rangeID) var tombstone roachpb.RangeTombstone if ok, err := storage.MVCCGetProto( - ctx, s.Engine(), tombstoneKey, hlc.Timestamp{}, &tombstone, storage.MVCCGetOptions{}, + ctx, s.Engine(), tombstoneKey, enginepb.TxnTimestamp{}, &tombstone, storage.MVCCGetOptions{}, ); err != nil { return nil, false, err } else if ok && replicaID != 0 && replicaID < tombstone.NextReplicaID { @@ -202,7 +202,7 @@ func (s *Store) tryGetOrCreateReplica( // again now, we make sure to synchronize with any goroutine that wrote // a tombstone and then removed an old replica from the Range map. if ok, err := storage.MVCCGetProto( - ctx, s.Engine(), tombstoneKey, hlc.Timestamp{}, &tombstone, storage.MVCCGetOptions{}, + ctx, s.Engine(), tombstoneKey, enginepb.TxnTimestamp{}, &tombstone, storage.MVCCGetOptions{}, ); err != nil { return err } else if ok && replicaID < tombstone.NextReplicaID { diff --git a/pkg/kv/kvserver/store_init.go b/pkg/kv/kvserver/store_init.go index 7dc9ec747632..1c69dcba2ec1 100644 --- a/pkg/kv/kvserver/store_init.go +++ b/pkg/kv/kvserver/store_init.go @@ -50,7 +50,7 @@ func InitEngine(ctx context.Context, eng storage.Engine, ident roachpb.StoreIden batch, nil, keys.StoreIdentKey(), - hlc.Timestamp{}, + enginepb.TxnTimestamp{}, nil, &ident, ); err != nil { @@ -180,10 +180,10 @@ func WriteInitialClusterData( batch := eng.NewBatch() defer batch.Close() - now := hlc.Timestamp{ + now := enginepb.TxnTimestamp(hlc.Timestamp{ WallTime: nowNanos, Logical: 0, - } + }) // NOTE: We don't do stats computations in any of the puts below. Instead, // we write everything and then compute the stats over the whole range. @@ -199,7 +199,7 @@ func WriteInitialClusterData( // Replica GC timestamp. if err := storage.MVCCPutProto( ctx, batch, nil /* ms */, keys.RangeLastReplicaGCTimestampKey(desc.RangeID), - hlc.Timestamp{}, nil /* txn */, &now, + enginepb.TxnTimestamp{}, nil /* txn */, &now, ); err != nil { return err } @@ -240,7 +240,7 @@ func WriteInitialClusterData( enginepb.MVCCStats{}, *desc, lease, - hlc.Timestamp{}, /* gcThreshold */ + enginepb.TxnTimestamp{}, /* gcThreshold */ truncStateType, ) if err != nil { diff --git a/pkg/kv/kvserver/store_send.go b/pkg/kv/kvserver/store_send.go index 1ee43455b2dc..ba2b2b4d22c8 100644 --- a/pkg/kv/kvserver/store_send.go +++ b/pkg/kv/kvserver/store_send.go @@ -85,16 +85,17 @@ 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) - } - } + // TODO(nvanbenschoten) + // 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) + // } + // } defer func() { if r := recover(); r != nil { @@ -118,17 +119,17 @@ 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 ba.Timestamp.Less(br.Txn.WriteTimestamp) { + // s.cfg.Clock.Update(br.Txn.WriteTimestamp) + // } } } else { if pErr == nil { // 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 ba.Timestamp.Less(br.Timestamp) { + // s.cfg.Clock.Update(br.Timestamp) + // } } } diff --git a/pkg/kv/kvserver/store_snapshot.go b/pkg/kv/kvserver/store_snapshot.go index 4d93249e8d2e..fff66513a557 100644 --- a/pkg/kv/kvserver/store_snapshot.go +++ b/pkg/kv/kvserver/store_snapshot.go @@ -922,7 +922,7 @@ func SendEmptySnapshot( var ms enginepb.MVCCStats // Seed an empty range into the new engine. if err := storage.MVCCPutProto( - ctx, eng, &ms, keys.RangeDescriptorKey(desc.StartKey), now, nil /* txn */, &desc, + ctx, eng, &ms, keys.RangeDescriptorKey(desc.StartKey), enginepb.TxnTimestamp(now), nil /* txn */, &desc, ); err != nil { return err } @@ -932,7 +932,7 @@ func SendEmptySnapshot( ms, desc, roachpb.Lease{}, - hlc.Timestamp{}, // gcThreshold + enginepb.TxnTimestamp{}, // gcThreshold stateloader.TruncatedStateUnreplicated, ) if err != nil { diff --git a/pkg/kv/kvserver/stores.go b/pkg/kv/kvserver/stores.go index 885b27036f03..e3ece987d4e7 100644 --- a/pkg/kv/kvserver/stores.go +++ b/pkg/kv/kvserver/stores.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" + enginepb "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/protoutil" @@ -219,7 +220,7 @@ func (ls *Stores) ReadBootstrapInfo(bi *gossip.BootstrapInfo) error { s := (*Store)(v) var storeBI gossip.BootstrapInfo var ok bool - ok, err = storage.MVCCGetProto(ctx, s.engine, keys.StoreGossipKey(), hlc.Timestamp{}, &storeBI, + ok, err = storage.MVCCGetProto(ctx, s.engine, keys.StoreGossipKey(), enginepb.TxnTimestamp{}, &storeBI, storage.MVCCGetOptions{}) if err != nil { return false @@ -268,7 +269,7 @@ func (ls *Stores) updateBootstrapInfoLocked(bi *gossip.BootstrapInfo) error { var err error ls.storeMap.Range(func(k int64, v unsafe.Pointer) bool { s := (*Store)(v) - err = storage.MVCCPutProto(ctx, s.engine, nil, keys.StoreGossipKey(), hlc.Timestamp{}, nil, bi) + err = storage.MVCCPutProto(ctx, s.engine, nil, keys.StoreGossipKey(), enginepb.TxnTimestamp{}, nil, bi) return err == nil }) return err diff --git a/pkg/kv/kvserver/txnrecovery/manager.go b/pkg/kv/kvserver/txnrecovery/manager.go index df3fe03b4be9..a475d9208da9 100644 --- a/pkg/kv/kvserver/txnrecovery/manager.go +++ b/pkg/kv/kvserver/txnrecovery/manager.go @@ -16,6 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "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/log" "github.com/cockroachdb/cockroach/pkg/util/stop" @@ -239,7 +240,8 @@ func (m *manager) resolveIndeterminateCommitForTxnProbe( // write is prevented, or we run out of in-flight writes to query. for len(queryIntentReqs) > 0 { var b kv.Batch - b.Header.Timestamp = m.clock.Now() + // TODO(nvanbenschoten): how does the QueryIntent work in this case? + b.Header.Timestamp = enginepb.TxnTimestamp(m.clock.Now()) b.AddRawRequest(&queryTxnReq) for i := 0; i < defaultBatchSize && len(queryIntentReqs) > 0; i++ { b.AddRawRequest(&queryIntentReqs[0]) @@ -292,7 +294,8 @@ func (m *manager) resolveIndeterminateCommitForTxnRecover( ctx context.Context, txn *roachpb.Transaction, preventedIntent bool, ) (*roachpb.Transaction, error) { var b kv.Batch - b.Header.Timestamp = m.clock.Now() + // TODO(nvanbenschoten): how does the RecoverTxn work in this case? + b.Header.Timestamp = enginepb.TxnTimestamp(m.clock.Now()) b.AddRawRequest(&roachpb.RecoverTxnRequest{ RequestHeader: roachpb.RequestHeader{ Key: txn.Key, diff --git a/pkg/kv/kvserver/txnwait/queue.go b/pkg/kv/kvserver/txnwait/queue.go index 1e071922892e..acb9a43bc950 100644 --- a/pkg/kv/kvserver/txnwait/queue.go +++ b/pkg/kv/kvserver/txnwait/queue.go @@ -862,7 +862,8 @@ func (q *Queue) queryTxnStatus( now hlc.Timestamp, ) (*roachpb.Transaction, []uuid.UUID, *roachpb.Error) { b := &kv.Batch{} - b.Header.Timestamp = q.cfg.Clock.Now() + // TODO(nvanbenschoten): how does the QueryTxn work in this case? + b.Header.Timestamp = enginepb.TxnTimestamp(q.cfg.Clock.Now()) b.AddRawRequest(&roachpb.QueryTxnRequest{ RequestHeader: roachpb.RequestHeader{ Key: txnMeta.Key, @@ -918,7 +919,8 @@ func (q *Queue) forcePushAbort( forcePush.Force = true forcePush.PushType = roachpb.PUSH_ABORT b := &kv.Batch{} - b.Header.Timestamp = q.cfg.Clock.Now() + // TODO(nvanbenschoten): how does the QueryTxn work in this case? + b.Header.Timestamp = enginepb.TxnTimestamp(q.cfg.Clock.Now()) b.AddRawRequest(&forcePush) if err := q.cfg.DB.Run(ctx, b); err != nil { return nil, b.MustPErr() diff --git a/pkg/kv/mock_transactional_sender.go b/pkg/kv/mock_transactional_sender.go index b39957455395..d6f7566c66a2 100644 --- a/pkg/kv/mock_transactional_sender.go +++ b/pkg/kv/mock_transactional_sender.go @@ -15,7 +15,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" - "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/errors" ) @@ -93,17 +92,17 @@ func (m *MockTransactionalSender) String() string { } // ReadTimestamp is part of the TxnSender interface. -func (m *MockTransactionalSender) ReadTimestamp() hlc.Timestamp { +func (m *MockTransactionalSender) ReadTimestamp() enginepb.TxnTimestamp { return m.txn.ReadTimestamp } // ProvisionalCommitTimestamp is part of the TxnSender interface. -func (m *MockTransactionalSender) ProvisionalCommitTimestamp() hlc.Timestamp { +func (m *MockTransactionalSender) ProvisionalCommitTimestamp() enginepb.TxnTimestamp { return m.txn.WriteTimestamp } // CommitTimestamp is part of the TxnSender interface. -func (m *MockTransactionalSender) CommitTimestamp() hlc.Timestamp { +func (m *MockTransactionalSender) CommitTimestamp() enginepb.TxnTimestamp { return m.txn.ReadTimestamp } @@ -113,7 +112,7 @@ func (m *MockTransactionalSender) CommitTimestampFixed() bool { } // SetFixedTimestamp is part of the TxnSender interface. -func (m *MockTransactionalSender) SetFixedTimestamp(_ context.Context, ts hlc.Timestamp) { +func (m *MockTransactionalSender) SetFixedTimestamp(_ context.Context, ts enginepb.TxnTimestamp) { m.txn.WriteTimestamp = ts m.txn.ReadTimestamp = ts m.txn.MaxTimestamp = ts @@ -126,7 +125,7 @@ func (m *MockTransactionalSender) SetFixedTimestamp(_ context.Context, ts hlc.Ti // ManualRestart is part of the TxnSender interface. func (m *MockTransactionalSender) ManualRestart( - ctx context.Context, pri roachpb.UserPriority, ts hlc.Timestamp, + ctx context.Context, pri roachpb.UserPriority, ts enginepb.TxnTimestamp, ) { m.txn.Restart(pri, 0 /* upgradePriority */, ts) } diff --git a/pkg/kv/sender.go b/pkg/kv/sender.go index fe20e2559d56..99e7bfa6a15d 100644 --- a/pkg/kv/sender.go +++ b/pkg/kv/sender.go @@ -15,7 +15,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" - "github.com/cockroachdb/cockroach/pkg/util/hlc" ) // TxnType specifies whether a transaction is the root (parent) @@ -174,7 +173,7 @@ type TxnSender interface { // queries and backups). This method must be called on every // transaction retry (but note that retries should be rare for // read-only queries with no clock uncertainty). - SetFixedTimestamp(ctx context.Context, ts hlc.Timestamp) + SetFixedTimestamp(ctx context.Context, ts enginepb.TxnTimestamp) // ManualRestart bumps the transactions epoch, and can upgrade the // timestamp and priority. @@ -187,7 +186,7 @@ type TxnSender interface { // when a retryable error is seen. // TODO(andrei): this second use should go away once we move to a // TxnAttempt model. - ManualRestart(context.Context, roachpb.UserPriority, hlc.Timestamp) + ManualRestart(context.Context, roachpb.UserPriority, enginepb.TxnTimestamp) // UpdateStateOnRemoteRetryableErr updates the txn in response to an // error encountered when running a request through the txn. @@ -204,7 +203,7 @@ type TxnSender interface { // Note a transaction can be internally pushed forward in time // before committing so this is not guaranteed to be the commit // timestamp. Use CommitTimestamp() when needed. - ReadTimestamp() hlc.Timestamp + ReadTimestamp() enginepb.TxnTimestamp // CommitTimestamp returns the transaction's start timestamp. // @@ -218,7 +217,7 @@ type TxnSender interface { // likelihood that a retry error will bubble up to a client. // // See CommitTimestampFixed() below. - CommitTimestamp() hlc.Timestamp + CommitTimestamp() enginepb.TxnTimestamp // CommitTimestampFixed returns true if the commit timestamp has // been fixed to the start timestamp and cannot be pushed forward. @@ -228,7 +227,7 @@ type TxnSender interface { // commit timestamp. This can move forward throughout the txn's // lifetime. See the explanatory comments for the WriteTimestamp // field on TxnMeta. - ProvisionalCommitTimestamp() hlc.Timestamp + ProvisionalCommitTimestamp() enginepb.TxnTimestamp // IsSerializablePushAndRefreshNotPossible returns true if the // transaction is serializable, its timestamp has been pushed and diff --git a/pkg/kv/txn.go b/pkg/kv/txn.go index 460d44a125c3..811e9dd2e448 100644 --- a/pkg/kv/txn.go +++ b/pkg/kv/txn.go @@ -71,7 +71,7 @@ type Txn struct { // The txn has to be committed by this deadline. A nil value indicates no // deadline. - deadline *hlc.Timestamp + deadline *enginepb.TxnTimestamp } } @@ -281,13 +281,13 @@ func (txn *Txn) String() string { // Note a transaction can be internally pushed forward in time before // committing so this is not guaranteed to be the commit timestamp. // Use CommitTimestamp() when needed. -func (txn *Txn) ReadTimestamp() hlc.Timestamp { +func (txn *Txn) ReadTimestamp() enginepb.TxnTimestamp { txn.mu.Lock() defer txn.mu.Unlock() return txn.readTimestampLocked() } -func (txn *Txn) readTimestampLocked() hlc.Timestamp { +func (txn *Txn) readTimestampLocked() enginepb.TxnTimestamp { return txn.mu.sender.ReadTimestamp() } @@ -295,7 +295,7 @@ func (txn *Txn) readTimestampLocked() hlc.Timestamp { // The start timestamp can get pushed but the use of this // method will guarantee that if a timestamp push is needed // the commit will fail with a retryable error. -func (txn *Txn) CommitTimestamp() hlc.Timestamp { +func (txn *Txn) CommitTimestamp() enginepb.TxnTimestamp { txn.mu.Lock() defer txn.mu.Unlock() return txn.mu.sender.CommitTimestamp() @@ -304,7 +304,7 @@ func (txn *Txn) CommitTimestamp() hlc.Timestamp { // ProvisionalCommitTimestamp returns the transaction's provisional // commit timestamp. This can evolve throughout a txn's lifecycle. See // the comment on the WriteTimestamp field of TxnMeta for details. -func (txn *Txn) ProvisionalCommitTimestamp() hlc.Timestamp { +func (txn *Txn) ProvisionalCommitTimestamp() enginepb.TxnTimestamp { txn.mu.Lock() defer txn.mu.Unlock() return txn.mu.sender.ProvisionalCommitTimestamp() @@ -387,13 +387,13 @@ func (txn *Txn) GetProto(ctx context.Context, key interface{}, msg protoutil.Mes // key can be either a byte slice or a string. func (txn *Txn) GetProtoTs( ctx context.Context, key interface{}, msg protoutil.Message, -) (hlc.Timestamp, error) { +) (enginepb.TxnTimestamp, error) { r, err := txn.Get(ctx, key) if err != nil { - return hlc.Timestamp{}, err + return enginepb.TxnTimestamp{}, err } if err := r.ValueProto(msg); err != nil || r.Value == nil { - return hlc.Timestamp{}, err + return enginepb.TxnTimestamp{}, err } return r.Value.Timestamp, nil } @@ -671,7 +671,7 @@ func (txn *Txn) CommitOrCleanup(ctx context.Context) error { // current one (if any) and the passed value. // // The deadline cannot be lower than txn.ReadTimestamp. -func (txn *Txn) UpdateDeadlineMaybe(ctx context.Context, deadline hlc.Timestamp) bool { +func (txn *Txn) UpdateDeadlineMaybe(ctx context.Context, deadline enginepb.TxnTimestamp) bool { if txn.typ != RootTxn { panic(errors.WithContextTags(errors.AssertionFailedf("UpdateDeadlineMaybe() called on leaf txn"), ctx)) } @@ -685,7 +685,7 @@ func (txn *Txn) UpdateDeadlineMaybe(ctx context.Context, deadline hlc.Timestamp) "txn has would have no change to commit. Deadline: %s. Read timestamp: %s.", deadline, readTimestamp) } - txn.mu.deadline = new(hlc.Timestamp) + txn.mu.deadline = new(enginepb.TxnTimestamp) *txn.mu.deadline = deadline return true } @@ -769,7 +769,7 @@ func (txn *Txn) AddCommitTrigger(trigger func(ctx context.Context)) { txn.commitTriggers = append(txn.commitTriggers, trigger) } -func endTxnReq(commit bool, deadline *hlc.Timestamp, hasTrigger bool) roachpb.Request { +func endTxnReq(commit bool, deadline *enginepb.TxnTimestamp, hasTrigger bool) roachpb.Request { req := &roachpb.EndTxnRequest{ Commit: commit, Deadline: deadline, @@ -1114,7 +1114,7 @@ func (txn *Txn) recordPreviousTxnIDLocked(prevTxnID uuid.UUID) { // This is used to support historical queries (AS OF SYSTEM TIME queries and // backups). This method must be called on every transaction retry (but note // that retries should be rare for read-only queries with no clock uncertainty). -func (txn *Txn) SetFixedTimestamp(ctx context.Context, ts hlc.Timestamp) { +func (txn *Txn) SetFixedTimestamp(ctx context.Context, ts enginepb.TxnTimestamp) { if txn.typ != RootTxn { panic(errors.WithContextTags( errors.AssertionFailedf("SetFixedTimestamp() called on leaf txn"), ctx)) @@ -1138,7 +1138,7 @@ func (txn *Txn) GenerateForcedRetryableError(ctx context.Context, msg string) er txn.mu.Lock() defer txn.mu.Unlock() now := txn.db.clock.Now() - txn.mu.sender.ManualRestart(ctx, txn.mu.userPriority, now) + txn.mu.sender.ManualRestart(ctx, txn.mu.userPriority, enginepb.TxnTimestamp(now)) txn.resetDeadlineLocked() return roachpb.NewTransactionRetryWithProtoRefreshError( msg, @@ -1176,7 +1176,7 @@ func (txn *Txn) PrepareRetryableError(ctx context.Context, msg string) error { // retryable error is seen. // TODO(andrei): this second use should go away once we move to a TxnAttempt // model. -func (txn *Txn) ManualRestart(ctx context.Context, ts hlc.Timestamp) { +func (txn *Txn) ManualRestart(ctx context.Context, ts enginepb.TxnTimestamp) { if txn.typ != RootTxn { panic(errors.WithContextTags( errors.AssertionFailedf("ManualRestart() called on leaf txn"), ctx)) @@ -1216,7 +1216,7 @@ func (txn *Txn) TestingCloneTxn() *roachpb.Transaction { return txn.mu.sender.TestingCloneTxn() } -func (txn *Txn) deadline() *hlc.Timestamp { +func (txn *Txn) deadline() *enginepb.TxnTimestamp { txn.mu.Lock() defer txn.mu.Unlock() return txn.mu.deadline diff --git a/pkg/roachpb/api.pb.go b/pkg/roachpb/api.pb.go index 8806b751f919..83f3e87ee6f0 100644 --- a/pkg/roachpb/api.pb.go +++ b/pkg/roachpb/api.pb.go @@ -73,7 +73,7 @@ func (x ReadConsistencyType) String() string { return proto.EnumName(ReadConsistencyType_name, int32(x)) } func (ReadConsistencyType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{0} + return fileDescriptor_api_e319494d56330d3c, []int{0} } // ScanFormat is an enumeration of the available response formats for MVCCScan @@ -101,7 +101,7 @@ func (x ScanFormat) String() string { return proto.EnumName(ScanFormat_name, int32(x)) } func (ScanFormat) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{1} + return fileDescriptor_api_e319494d56330d3c, []int{1} } type ChecksumMode int32 @@ -148,7 +148,7 @@ func (x ChecksumMode) String() string { return proto.EnumName(ChecksumMode_name, int32(x)) } func (ChecksumMode) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{2} + return fileDescriptor_api_e319494d56330d3c, []int{2} } // PushTxnType determines what action to take when pushing a transaction. @@ -179,7 +179,7 @@ func (x PushTxnType) String() string { return proto.EnumName(PushTxnType_name, int32(x)) } func (PushTxnType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{3} + return fileDescriptor_api_e319494d56330d3c, []int{3} } type ExternalStorageProvider int32 @@ -220,7 +220,7 @@ func (x ExternalStorageProvider) String() string { return proto.EnumName(ExternalStorageProvider_name, int32(x)) } func (ExternalStorageProvider) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{4} + return fileDescriptor_api_e319494d56330d3c, []int{4} } type MVCCFilter int32 @@ -243,7 +243,7 @@ func (x MVCCFilter) String() string { return proto.EnumName(MVCCFilter_name, int32(x)) } func (MVCCFilter) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{5} + return fileDescriptor_api_e319494d56330d3c, []int{5} } type ResponseHeader_ResumeReason int32 @@ -269,7 +269,7 @@ func (x ResponseHeader_ResumeReason) String() string { return proto.EnumName(ResponseHeader_ResumeReason_name, int32(x)) } func (ResponseHeader_ResumeReason) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{1, 0} + return fileDescriptor_api_e319494d56330d3c, []int{1, 0} } type CheckConsistencyResponse_Status int32 @@ -311,7 +311,7 @@ func (x CheckConsistencyResponse_Status) String() string { return proto.EnumName(CheckConsistencyResponse_Status_name, int32(x)) } func (CheckConsistencyResponse_Status) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{25, 0} + return fileDescriptor_api_e319494d56330d3c, []int{25, 0} } // RequestHeader is supplied with every storage node request. @@ -332,7 +332,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_290a2f113c3e6c17, []int{0} + return fileDescriptor_api_e319494d56330d3c, []int{0} } func (m *RequestHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -403,7 +403,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_290a2f113c3e6c17, []int{1} + return fileDescriptor_api_e319494d56330d3c, []int{1} } func (m *ResponseHeader) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -437,7 +437,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_290a2f113c3e6c17, []int{2} + return fileDescriptor_api_e319494d56330d3c, []int{2} } func (m *GetRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -480,7 +480,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_290a2f113c3e6c17, []int{3} + return fileDescriptor_api_e319494d56330d3c, []int{3} } func (m *GetResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -523,7 +523,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_290a2f113c3e6c17, []int{4} + return fileDescriptor_api_e319494d56330d3c, []int{4} } func (m *PutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -557,7 +557,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_290a2f113c3e6c17, []int{5} + return fileDescriptor_api_e319494d56330d3c, []int{5} } func (m *PutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -639,7 +639,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_290a2f113c3e6c17, []int{6} + return fileDescriptor_api_e319494d56330d3c, []int{6} } func (m *ConditionalPutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -674,7 +674,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_290a2f113c3e6c17, []int{7} + return fileDescriptor_api_e319494d56330d3c, []int{7} } func (m *ConditionalPutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -720,7 +720,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_290a2f113c3e6c17, []int{8} + return fileDescriptor_api_e319494d56330d3c, []int{8} } func (m *InitPutRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -754,7 +754,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_290a2f113c3e6c17, []int{9} + return fileDescriptor_api_e319494d56330d3c, []int{9} } func (m *InitPutResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -794,7 +794,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_290a2f113c3e6c17, []int{10} + return fileDescriptor_api_e319494d56330d3c, []int{10} } func (m *IncrementRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -831,7 +831,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_290a2f113c3e6c17, []int{11} + return fileDescriptor_api_e319494d56330d3c, []int{11} } func (m *IncrementResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -865,7 +865,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_290a2f113c3e6c17, []int{12} + return fileDescriptor_api_e319494d56330d3c, []int{12} } func (m *DeleteRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -899,7 +899,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_290a2f113c3e6c17, []int{13} + return fileDescriptor_api_e319494d56330d3c, []int{13} } func (m *DeleteResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -951,7 +951,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_290a2f113c3e6c17, []int{14} + return fileDescriptor_api_e319494d56330d3c, []int{14} } func (m *DeleteRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -988,7 +988,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_290a2f113c3e6c17, []int{15} + return fileDescriptor_api_e319494d56330d3c, []int{15} } func (m *DeleteRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1036,14 +1036,14 @@ type ClearRangeRequest struct { // that time (plus max offset) to have passed at which point they can reuse // the span they cleared without fear of this request being replayed later and // clearing subsequent writes. - Deadline hlc.Timestamp `protobuf:"bytes,2,opt,name=deadline,proto3" json:"deadline"` + Deadline github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp `protobuf:"bytes,2,opt,name=deadline,proto3,casttype=github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp" json:"deadline"` } 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_290a2f113c3e6c17, []int{16} + return fileDescriptor_api_e319494d56330d3c, []int{16} } func (m *ClearRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1077,7 +1077,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_290a2f113c3e6c17, []int{17} + return fileDescriptor_api_e319494d56330d3c, []int{17} } func (m *ClearRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1111,14 +1111,14 @@ type RevertRangeRequest struct { // any MVCC key with a strictly higher timestamp. TargetTime must be higher // than the GC Threshold for the replica - so that it is assured that the keys // for that time are still there — or the request will fail. - TargetTime hlc.Timestamp `protobuf:"bytes,2,opt,name=target_time,json=targetTime,proto3" json:"target_time"` + TargetTime github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp `protobuf:"bytes,2,opt,name=target_time,json=targetTime,proto3,casttype=github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp" json:"target_time"` } 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_290a2f113c3e6c17, []int{18} + return fileDescriptor_api_e319494d56330d3c, []int{18} } func (m *RevertRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1152,7 +1152,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_290a2f113c3e6c17, []int{19} + return fileDescriptor_api_e319494d56330d3c, []int{19} } func (m *RevertRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1203,7 +1203,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_290a2f113c3e6c17, []int{20} + return fileDescriptor_api_e319494d56330d3c, []int{20} } func (m *ScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1255,7 +1255,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_290a2f113c3e6c17, []int{21} + return fileDescriptor_api_e319494d56330d3c, []int{21} } func (m *ScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1306,7 +1306,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_290a2f113c3e6c17, []int{22} + return fileDescriptor_api_e319494d56330d3c, []int{22} } func (m *ReverseScanRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1358,7 +1358,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_290a2f113c3e6c17, []int{23} + return fileDescriptor_api_e319494d56330d3c, []int{23} } func (m *ReverseScanResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1411,7 +1411,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_290a2f113c3e6c17, []int{24} + return fileDescriptor_api_e319494d56330d3c, []int{24} } func (m *CheckConsistencyRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1448,7 +1448,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_290a2f113c3e6c17, []int{25} + return fileDescriptor_api_e319494d56330d3c, []int{25} } func (m *CheckConsistencyResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1492,7 +1492,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_290a2f113c3e6c17, []int{25, 0} + return fileDescriptor_api_e319494d56330d3c, []int{25, 0} } func (m *CheckConsistencyResponse_Result) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1540,7 +1540,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_290a2f113c3e6c17, []int{26} + return fileDescriptor_api_e319494d56330d3c, []int{26} } func (m *RecomputeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1576,7 +1576,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_290a2f113c3e6c17, []int{27} + return fileDescriptor_api_e319494d56330d3c, []int{27} } func (m *RecomputeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1613,7 +1613,7 @@ type EndTxnRequest struct { // If EndTxn(Commit=true) finds that the txn's timestamp has been pushed above // this deadline, an error will be returned and the client is supposed to // rollback the txn. - Deadline *hlc.Timestamp `protobuf:"bytes,3,opt,name=deadline,proto3" json:"deadline,omitempty"` + Deadline *github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp `protobuf:"bytes,3,opt,name=deadline,proto3,casttype=github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp" json:"deadline,omitempty"` // commit triggers. Note that commit triggers are for // internal use only and will cause an error if requested through the // external-facing KV API. @@ -1687,7 +1687,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_290a2f113c3e6c17, []int{28} + return fileDescriptor_api_e319494d56330d3c, []int{28} } func (m *EndTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1726,14 +1726,14 @@ type EndTxnResponse struct { OnePhaseCommit bool `protobuf:"varint,4,opt,name=one_phase_commit,json=onePhaseCommit,proto3" json:"one_phase_commit,omitempty"` // The commit timestamp of the STAGING transaction record written // by the request. Only set if the transaction record was staged. - StagingTimestamp hlc.Timestamp `protobuf:"bytes,5,opt,name=staging_timestamp,json=stagingTimestamp,proto3" json:"staging_timestamp"` + StagingTimestamp github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp `protobuf:"bytes,5,opt,name=staging_timestamp,json=stagingTimestamp,proto3,casttype=github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp" json:"staging_timestamp"` } 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_290a2f113c3e6c17, []int{29} + return fileDescriptor_api_e319494d56330d3c, []int{29} } func (m *EndTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1794,7 +1794,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_290a2f113c3e6c17, []int{30} + return fileDescriptor_api_e319494d56330d3c, []int{30} } func (m *AdminSplitRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1829,7 +1829,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_290a2f113c3e6c17, []int{31} + return fileDescriptor_api_e319494d56330d3c, []int{31} } func (m *AdminSplitResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1868,7 +1868,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_290a2f113c3e6c17, []int{32} + return fileDescriptor_api_e319494d56330d3c, []int{32} } func (m *AdminUnsplitRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1903,7 +1903,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_290a2f113c3e6c17, []int{33} + return fileDescriptor_api_e319494d56330d3c, []int{33} } func (m *AdminUnsplitResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1946,7 +1946,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_290a2f113c3e6c17, []int{34} + return fileDescriptor_api_e319494d56330d3c, []int{34} } func (m *AdminMergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1981,7 +1981,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_290a2f113c3e6c17, []int{35} + return fileDescriptor_api_e319494d56330d3c, []int{35} } func (m *AdminMergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2019,7 +2019,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_290a2f113c3e6c17, []int{36} + return fileDescriptor_api_e319494d56330d3c, []int{36} } func (m *AdminTransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2052,7 +2052,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_290a2f113c3e6c17, []int{37} + return fileDescriptor_api_e319494d56330d3c, []int{37} } func (m *AdminTransferLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2087,7 +2087,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_290a2f113c3e6c17, []int{38} + return fileDescriptor_api_e319494d56330d3c, []int{38} } func (m *ReplicationChange) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2145,7 +2145,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_290a2f113c3e6c17, []int{39} + return fileDescriptor_api_e319494d56330d3c, []int{39} } func (m *AdminChangeReplicasRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2180,7 +2180,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_290a2f113c3e6c17, []int{40} + return fileDescriptor_api_e319494d56330d3c, []int{40} } func (m *AdminChangeReplicasResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2217,7 +2217,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_290a2f113c3e6c17, []int{41} + return fileDescriptor_api_e319494d56330d3c, []int{41} } func (m *AdminRelocateRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2250,7 +2250,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_290a2f113c3e6c17, []int{42} + return fileDescriptor_api_e319494d56330d3c, []int{42} } func (m *AdminRelocateRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2289,7 +2289,7 @@ 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_290a2f113c3e6c17, []int{43} + return fileDescriptor_api_e319494d56330d3c, []int{43} } func (m *HeartbeatTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2326,7 +2326,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_290a2f113c3e6c17, []int{44} + return fileDescriptor_api_e319494d56330d3c, []int{44} } func (m *HeartbeatTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2357,14 +2357,14 @@ type GCRequest struct { RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` Keys []GCRequest_GCKey `protobuf:"bytes,3,rep,name=keys,proto3" json:"keys"` // Threshold is the expiration timestamp. - Threshold hlc.Timestamp `protobuf:"bytes,4,opt,name=threshold,proto3" json:"threshold"` + Threshold github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp `protobuf:"bytes,4,opt,name=threshold,proto3,casttype=github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp" json:"threshold"` } 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_290a2f113c3e6c17, []int{45} + return fileDescriptor_api_e319494d56330d3c, []int{45} } func (m *GCRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2390,15 +2390,15 @@ func (m *GCRequest) XXX_DiscardUnknown() { var xxx_messageInfo_GCRequest proto.InternalMessageInfo type GCRequest_GCKey struct { - Key Key `protobuf:"bytes,1,opt,name=key,proto3,casttype=Key" json:"key,omitempty"` - Timestamp hlc.Timestamp `protobuf:"bytes,2,opt,name=timestamp,proto3" json:"timestamp"` + Key Key `protobuf:"bytes,1,opt,name=key,proto3,casttype=Key" json:"key,omitempty"` + Timestamp github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp `protobuf:"bytes,2,opt,name=timestamp,proto3,casttype=github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp" json:"timestamp"` } 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_290a2f113c3e6c17, []int{45, 0} + return fileDescriptor_api_e319494d56330d3c, []int{45, 0} } func (m *GCRequest_GCKey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2432,7 +2432,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_290a2f113c3e6c17, []int{46} + return fileDescriptor_api_e319494d56330d3c, []int{46} } func (m *GCResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2485,7 +2485,7 @@ type PushTxnRequest struct { // PushTo is the timestamp which PusheeTxn should be pushed to. During // conflict resolution, it should be set just after the timestamp of the // conflicting read or write. - PushTo hlc.Timestamp `protobuf:"bytes,4,opt,name=push_to,json=pushTo,proto3" json:"push_to"` + PushTo github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp `protobuf:"bytes,4,opt,name=push_to,json=pushTo,proto3,casttype=github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp" json:"push_to"` // Readers set this to PUSH_TIMESTAMP to move pushee_txn's provisional // commit timestamp forward. Writers set this to PUSH_ABORT to request // that pushee_txn be aborted if possible. Inconsistent readers set @@ -2501,7 +2501,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_290a2f113c3e6c17, []int{47} + return fileDescriptor_api_e319494d56330d3c, []int{47} } func (m *PushTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2544,7 +2544,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_290a2f113c3e6c17, []int{48} + return fileDescriptor_api_e319494d56330d3c, []int{48} } func (m *PushTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2591,7 +2591,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_290a2f113c3e6c17, []int{49} + return fileDescriptor_api_e319494d56330d3c, []int{49} } func (m *RecoverTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2627,7 +2627,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_290a2f113c3e6c17, []int{50} + return fileDescriptor_api_e319494d56330d3c, []int{50} } func (m *RecoverTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2671,7 +2671,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_290a2f113c3e6c17, []int{51} + return fileDescriptor_api_e319494d56330d3c, []int{51} } func (m *QueryTxnRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2715,7 +2715,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_290a2f113c3e6c17, []int{52} + return fileDescriptor_api_e319494d56330d3c, []int{52} } func (m *QueryTxnResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2775,7 +2775,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_290a2f113c3e6c17, []int{53} + return fileDescriptor_api_e319494d56330d3c, []int{53} } func (m *QueryIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2811,7 +2811,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_290a2f113c3e6c17, []int{54} + return fileDescriptor_api_e319494d56330d3c, []int{54} } func (m *QueryIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2858,7 +2858,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_290a2f113c3e6c17, []int{55} + return fileDescriptor_api_e319494d56330d3c, []int{55} } func (m *ResolveIntentRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2893,7 +2893,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_290a2f113c3e6c17, []int{56} + return fileDescriptor_api_e319494d56330d3c, []int{56} } func (m *ResolveIntentResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2935,7 +2935,7 @@ type ResolveIntentRangeRequest struct { // The minimum timestamp for any intents written by this // transaction. If present, this value can be used to optimize the // iteration over the span to find intents to resolve. - MinTimestamp hlc.Timestamp `protobuf:"bytes,5,opt,name=min_timestamp,json=minTimestamp,proto3" json:"min_timestamp"` + MinTimestamp github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp `protobuf:"bytes,5,opt,name=min_timestamp,json=minTimestamp,proto3,casttype=github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp" json:"min_timestamp"` // The list of ignored seqnum ranges as per the Transaction record. IgnoredSeqNums []enginepb.IgnoredSeqNumRange `protobuf:"bytes,6,rep,name=ignored_seqnums,json=ignoredSeqnums,proto3" json:"ignored_seqnums"` } @@ -2944,7 +2944,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_290a2f113c3e6c17, []int{57} + return fileDescriptor_api_e319494d56330d3c, []int{57} } func (m *ResolveIntentRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2979,7 +2979,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_290a2f113c3e6c17, []int{58} + return fileDescriptor_api_e319494d56330d3c, []int{58} } func (m *ResolveIntentRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3016,7 +3016,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_290a2f113c3e6c17, []int{59} + return fileDescriptor_api_e319494d56330d3c, []int{59} } func (m *MergeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3050,7 +3050,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_290a2f113c3e6c17, []int{60} + return fileDescriptor_api_e319494d56330d3c, []int{60} } func (m *MergeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3095,7 +3095,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_290a2f113c3e6c17, []int{61} + return fileDescriptor_api_e319494d56330d3c, []int{61} } func (m *TruncateLogRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3129,7 +3129,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_290a2f113c3e6c17, []int{62} + return fileDescriptor_api_e319494d56330d3c, []int{62} } func (m *TruncateLogResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3173,7 +3173,7 @@ 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_290a2f113c3e6c17, []int{63} + return fileDescriptor_api_e319494d56330d3c, []int{63} } func (m *RequestLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3222,7 +3222,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_290a2f113c3e6c17, []int{64} + return fileDescriptor_api_e319494d56330d3c, []int{64} } func (m *TransferLeaseRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3259,7 +3259,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_290a2f113c3e6c17, []int{65} + return fileDescriptor_api_e319494d56330d3c, []int{65} } func (m *LeaseInfoRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3296,7 +3296,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_290a2f113c3e6c17, []int{66} + return fileDescriptor_api_e319494d56330d3c, []int{66} } func (m *LeaseInfoResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3331,7 +3331,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_290a2f113c3e6c17, []int{67} + return fileDescriptor_api_e319494d56330d3c, []int{67} } func (m *RequestLeaseResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3386,7 +3386,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_290a2f113c3e6c17, []int{68} + return fileDescriptor_api_e319494d56330d3c, []int{68} } func (m *ComputeChecksumRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3423,7 +3423,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_290a2f113c3e6c17, []int{69} + return fileDescriptor_api_e319494d56330d3c, []int{69} } func (m *ComputeChecksumResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3463,7 +3463,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_290a2f113c3e6c17, []int{70} + return fileDescriptor_api_e319494d56330d3c, []int{70} } func (m *ExternalStorage) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3497,7 +3497,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_290a2f113c3e6c17, []int{70, 0} + return fileDescriptor_api_e319494d56330d3c, []int{70, 0} } func (m *ExternalStorage_LocalFilePath) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3530,7 +3530,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_290a2f113c3e6c17, []int{70, 1} + return fileDescriptor_api_e319494d56330d3c, []int{70, 1} } func (m *ExternalStorage_Http) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3572,7 +3572,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_290a2f113c3e6c17, []int{70, 2} + return fileDescriptor_api_e319494d56330d3c, []int{70, 2} } func (m *ExternalStorage_S3) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3611,7 +3611,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_290a2f113c3e6c17, []int{70, 3} + return fileDescriptor_api_e319494d56330d3c, []int{70, 3} } func (m *ExternalStorage_GCS) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3647,7 +3647,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_290a2f113c3e6c17, []int{70, 4} + return fileDescriptor_api_e319494d56330d3c, []int{70, 4} } func (m *ExternalStorage_Azure) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3686,7 +3686,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_290a2f113c3e6c17, []int{70, 5} + return fileDescriptor_api_e319494d56330d3c, []int{70, 5} } func (m *ExternalStorage_Workload) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3728,7 +3728,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_290a2f113c3e6c17, []int{70, 6} + return fileDescriptor_api_e319494d56330d3c, []int{70, 6} } func (m *ExternalStorage_FileTable) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3768,7 +3768,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_290a2f113c3e6c17, []int{71} + return fileDescriptor_api_e319494d56330d3c, []int{71} } func (m *WriteBatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3802,7 +3802,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_290a2f113c3e6c17, []int{72} + return fileDescriptor_api_e319494d56330d3c, []int{72} } func (m *WriteBatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3838,7 +3838,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_290a2f113c3e6c17, []int{73} + return fileDescriptor_api_e319494d56330d3c, []int{73} } func (m *FileEncryptionOptions) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3867,9 +3867,9 @@ var xxx_messageInfo_FileEncryptionOptions proto.InternalMessageInfo // files under a basepath. type ExportRequest struct { RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` - Storage ExternalStorage `protobuf:"bytes,2,opt,name=storage,proto3" json:"storage"` - StartTime hlc.Timestamp `protobuf:"bytes,3,opt,name=start_time,json=startTime,proto3" json:"start_time"` - MVCCFilter MVCCFilter `protobuf:"varint,4,opt,name=mvcc_filter,json=mvccFilter,proto3,enum=cockroach.roachpb.MVCCFilter" json:"mvcc_filter,omitempty"` + Storage ExternalStorage `protobuf:"bytes,2,opt,name=storage,proto3" json:"storage"` + StartTime github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp `protobuf:"bytes,3,opt,name=start_time,json=startTime,proto3,casttype=github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp" json:"start_time"` + MVCCFilter MVCCFilter `protobuf:"varint,4,opt,name=mvcc_filter,json=mvccFilter,proto3,enum=cockroach.roachpb.MVCCFilter" json:"mvcc_filter,omitempty"` // Return the exported SST data in the response. ReturnSST bool `protobuf:"varint,5,opt,name=return_sst,json=returnSst,proto3" json:"return_sst,omitempty"` // OmitChecksum, if true, will skip checksumming the sst and leave the @@ -3908,7 +3908,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_290a2f113c3e6c17, []int{74} + return fileDescriptor_api_e319494d56330d3c, []int{74} } func (m *ExportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3960,7 +3960,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_290a2f113c3e6c17, []int{75} + return fileDescriptor_api_e319494d56330d3c, []int{75} } func (m *BulkOpSummary) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3988,15 +3988,15 @@ var xxx_messageInfo_BulkOpSummary proto.InternalMessageInfo // ExportResponse is the response to an Export() operation. type ExportResponse struct { ResponseHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` - Files []ExportResponse_File `protobuf:"bytes,2,rep,name=files,proto3" json:"files"` - StartTime hlc.Timestamp `protobuf:"bytes,3,opt,name=start_time,json=startTime,proto3" json:"start_time"` + Files []ExportResponse_File `protobuf:"bytes,2,rep,name=files,proto3" json:"files"` + StartTime github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp `protobuf:"bytes,3,opt,name=start_time,json=startTime,proto3,casttype=github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp" json:"start_time"` } 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_290a2f113c3e6c17, []int{76} + return fileDescriptor_api_e319494d56330d3c, []int{76} } func (m *ExportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4036,7 +4036,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_290a2f113c3e6c17, []int{76, 0} + return fileDescriptor_api_e319494d56330d3c, []int{76, 0} } func (m *ExportResponse_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4072,7 +4072,7 @@ type ImportRequest struct { DataSpan Span `protobuf:"bytes,3,opt,name=data_span,json=dataSpan,proto3" json:"data_span"` // EndTime, if not the zero value, will cause only entries before it to be // imported. - EndTime hlc.Timestamp `protobuf:"bytes,6,opt,name=end_time,json=endTime,proto3" json:"end_time"` + EndTime github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp `protobuf:"bytes,6,opt,name=end_time,json=endTime,proto3,casttype=github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp" json:"end_time"` // Rekeys contains the descriptors for the data being Imported and the // previous ID for each (which is the ID used in the source data pointed to by // `files`). @@ -4087,7 +4087,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_290a2f113c3e6c17, []int{77} + return fileDescriptor_api_e319494d56330d3c, []int{77} } func (m *ImportRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4122,7 +4122,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_290a2f113c3e6c17, []int{77, 0} + return fileDescriptor_api_e319494d56330d3c, []int{77, 0} } func (m *ImportRequest_File) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4158,7 +4158,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_290a2f113c3e6c17, []int{77, 1} + return fileDescriptor_api_e319494d56330d3c, []int{77, 1} } func (m *ImportRequest_TableRekey) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4193,7 +4193,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_290a2f113c3e6c17, []int{78} + return fileDescriptor_api_e319494d56330d3c, []int{78} } func (m *ImportResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4231,7 +4231,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_290a2f113c3e6c17, []int{79} + return fileDescriptor_api_e319494d56330d3c, []int{79} } func (m *AdminScatterRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4268,7 +4268,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_290a2f113c3e6c17, []int{80} + return fileDescriptor_api_e319494d56330d3c, []int{80} } func (m *AdminScatterResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4301,7 +4301,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_290a2f113c3e6c17, []int{80, 0} + return fileDescriptor_api_e319494d56330d3c, []int{80, 0} } func (m *AdminScatterResponse_Range) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4334,7 +4334,7 @@ type AdminVerifyProtectedTimestampRequest struct { // RecordID is the ID of the protected timestamp Record being verified. RecordID github_com_cockroachdb_cockroach_pkg_util_uuid.UUID `protobuf:"bytes,4,opt,name=record_id,json=recordId,proto3,customtype=github.com/cockroachdb/cockroach/pkg/util/uuid.UUID" json:"record_id"` // Protected is the timestamp at which the record with RecordID protects. - Protected hlc.Timestamp `protobuf:"bytes,2,opt,name=protected,proto3" json:"protected"` + Protected github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp `protobuf:"bytes,2,opt,name=protected,proto3,casttype=github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp" json:"protected"` // RecordAliveAt is a an hlc timestamp at which the record being verified is // known to exist. A value for RecordAliveAt is generally determined by // reading a Record from the database and using the timestamp at which that @@ -4346,7 +4346,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_290a2f113c3e6c17, []int{81} + return fileDescriptor_api_e319494d56330d3c, []int{81} } func (m *AdminVerifyProtectedTimestampRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4384,7 +4384,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_290a2f113c3e6c17, []int{82} + return fileDescriptor_api_e319494d56330d3c, []int{82} } func (m *AdminVerifyProtectedTimestampResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4437,7 +4437,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_290a2f113c3e6c17, []int{83} + return fileDescriptor_api_e319494d56330d3c, []int{83} } func (m *AddSSTableRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4471,7 +4471,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_290a2f113c3e6c17, []int{84} + return fileDescriptor_api_e319494d56330d3c, []int{84} } func (m *AddSSTableResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4508,14 +4508,14 @@ type RefreshRequest struct { RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` // refresh_from specifies the lower-bound of the verification. The request verifies that // there's no write in the range [refresh_from, txn.read_timestamp]. - RefreshFrom hlc.Timestamp `protobuf:"bytes,3,opt,name=refresh_from,json=refreshFrom,proto3" json:"refresh_from"` + RefreshFrom github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp `protobuf:"bytes,3,opt,name=refresh_from,json=refreshFrom,proto3,casttype=github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp" json:"refresh_from"` } 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_290a2f113c3e6c17, []int{85} + return fileDescriptor_api_e319494d56330d3c, []int{85} } func (m *RefreshRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4549,7 +4549,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_290a2f113c3e6c17, []int{86} + return fileDescriptor_api_e319494d56330d3c, []int{86} } func (m *RefreshResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4581,14 +4581,14 @@ type RefreshRangeRequest struct { RequestHeader `protobuf:"bytes,1,opt,name=header,proto3,embedded=header" json:"header"` // refresh_from specifies the lower-bound of the verification. The request verifies that // there's no write in the range [refresh_from, txn.read_timestamp]. - RefreshFrom hlc.Timestamp `protobuf:"bytes,3,opt,name=refresh_from,json=refreshFrom,proto3" json:"refresh_from"` + RefreshFrom github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp `protobuf:"bytes,3,opt,name=refresh_from,json=refreshFrom,proto3,casttype=github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp" json:"refresh_from"` } 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_290a2f113c3e6c17, []int{87} + return fileDescriptor_api_e319494d56330d3c, []int{87} } func (m *RefreshRangeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4622,7 +4622,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_290a2f113c3e6c17, []int{88} + return fileDescriptor_api_e319494d56330d3c, []int{88} } func (m *RefreshRangeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4671,7 +4671,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_290a2f113c3e6c17, []int{89} + return fileDescriptor_api_e319494d56330d3c, []int{89} } func (m *SubsumeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4720,7 +4720,7 @@ 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_290a2f113c3e6c17, []int{90} + return fileDescriptor_api_e319494d56330d3c, []int{90} } func (m *SubsumeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4755,7 +4755,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_290a2f113c3e6c17, []int{91} + return fileDescriptor_api_e319494d56330d3c, []int{91} } func (m *RangeStatsRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4797,7 +4797,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_290a2f113c3e6c17, []int{92} + return fileDescriptor_api_e319494d56330d3c, []int{92} } func (m *RangeStatsResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -4880,7 +4880,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_290a2f113c3e6c17, []int{93} + return fileDescriptor_api_e319494d56330d3c, []int{93} } func (m *RequestUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -6330,7 +6330,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_290a2f113c3e6c17, []int{94} + return fileDescriptor_api_e319494d56330d3c, []int{94} } func (m *ResponseUnion) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7707,7 +7707,7 @@ type Header struct { // the server to set it from txn.ReadTimestamp. Also, for transactional // requests, writes are performed at the provisional commit timestamp // (txn.WriteTimestamp). - Timestamp hlc.Timestamp `protobuf:"bytes,1,opt,name=timestamp,proto3" json:"timestamp"` + Timestamp github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp `protobuf:"bytes,1,opt,name=timestamp,proto3,casttype=github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp" json:"timestamp"` // replica specifies the destination of the request. Replica ReplicaDescriptor `protobuf:"bytes,2,opt,name=replica,proto3" json:"replica"` // range_id specifies the ID of the Raft consensus group which the key @@ -7865,7 +7865,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_290a2f113c3e6c17, []int{95} + return fileDescriptor_api_e319494d56330d3c, []int{95} } func (m *Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7903,7 +7903,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_290a2f113c3e6c17, []int{96} + return fileDescriptor_api_e319494d56330d3c, []int{96} } func (m *ClientRangeInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7939,7 +7939,7 @@ type BatchRequest struct { func (m *BatchRequest) Reset() { *m = BatchRequest{} } func (*BatchRequest) ProtoMessage() {} func (*BatchRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{97} + return fileDescriptor_api_e319494d56330d3c, []int{97} } func (m *BatchRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -7976,7 +7976,7 @@ type BatchResponse struct { func (m *BatchResponse) Reset() { *m = BatchResponse{} } func (*BatchResponse) ProtoMessage() {} func (*BatchResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_api_290a2f113c3e6c17, []int{98} + return fileDescriptor_api_e319494d56330d3c, []int{98} } func (m *BatchResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8022,7 +8022,7 @@ type BatchResponse_Header struct { // which is awkward. We could consider making this field optional and only // populate it for non-transactional requests. The timestamp cache would then // use an accessor on BatchResponse to pick the one that matters. - Timestamp hlc.Timestamp `protobuf:"bytes,2,opt,name=Timestamp,proto3" json:"Timestamp"` + Timestamp github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp `protobuf:"bytes,2,opt,name=Timestamp,proto3,casttype=github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp" json:"Timestamp"` // txn is non-nil if the request specified a non-nil // transaction. The transaction timestamp and/or priority may have // been updated, depending on the outcome of the request. @@ -8055,7 +8055,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_290a2f113c3e6c17, []int{98, 0} + return fileDescriptor_api_e319494d56330d3c, []int{98, 0} } func (m *BatchResponse_Header) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8093,7 +8093,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_290a2f113c3e6c17, []int{99} + return fileDescriptor_api_e319494d56330d3c, []int{99} } func (m *RangeLookupRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8131,7 +8131,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_290a2f113c3e6c17, []int{100} + return fileDescriptor_api_e319494d56330d3c, []int{100} } func (m *RangeLookupResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8170,7 +8170,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_290a2f113c3e6c17, []int{101} + return fileDescriptor_api_e319494d56330d3c, []int{101} } func (m *RangeFeedRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8211,7 +8211,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_290a2f113c3e6c17, []int{102} + return fileDescriptor_api_e319494d56330d3c, []int{102} } func (m *RangeFeedValue) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8244,15 +8244,15 @@ var xxx_messageInfo_RangeFeedValue proto.InternalMessageInfo // Note that these resolved timestamps may be lower than the timestamp used in // the RangeFeedRequest used to start the RangeFeed. type RangeFeedCheckpoint struct { - Span Span `protobuf:"bytes,1,opt,name=span,proto3" json:"span"` - ResolvedTS hlc.Timestamp `protobuf:"bytes,2,opt,name=resolved_ts,json=resolvedTs,proto3" json:"resolved_ts"` + Span Span `protobuf:"bytes,1,opt,name=span,proto3" json:"span"` + ResolvedTS github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp `protobuf:"bytes,2,opt,name=resolved_ts,json=resolvedTs,proto3,casttype=github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp" json:"resolved_ts"` } 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_290a2f113c3e6c17, []int{103} + return fileDescriptor_api_e319494d56330d3c, []int{103} } func (m *RangeFeedCheckpoint) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8289,7 +8289,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_290a2f113c3e6c17, []int{104} + return fileDescriptor_api_e319494d56330d3c, []int{104} } func (m *RangeFeedError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8326,7 +8326,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_290a2f113c3e6c17, []int{105} + return fileDescriptor_api_e319494d56330d3c, []int{105} } func (m *RangeFeedEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8370,7 +8370,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_290a2f113c3e6c17, []int{106} + return fileDescriptor_api_e319494d56330d3c, []int{106} } func (m *ResetQuorumRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8402,7 +8402,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_290a2f113c3e6c17, []int{107} + return fileDescriptor_api_e319494d56330d3c, []int{107} } func (m *ResetQuorumResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8443,7 +8443,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_290a2f113c3e6c17, []int{108} + return fileDescriptor_api_e319494d56330d3c, []int{108} } func (m *GossipSubscriptionRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8483,7 +8483,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_290a2f113c3e6c17, []int{109} + return fileDescriptor_api_e319494d56330d3c, []int{109} } func (m *GossipSubscriptionEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8519,7 +8519,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_290a2f113c3e6c17, []int{110} + return fileDescriptor_api_e319494d56330d3c, []int{110} } func (m *JoinNodeRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8558,7 +8558,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_290a2f113c3e6c17, []int{111} + return fileDescriptor_api_e319494d56330d3c, []int{111} } func (m *JoinNodeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -8601,7 +8601,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_290a2f113c3e6c17, []int{112} + return fileDescriptor_api_e319494d56330d3c, []int{112} } func (m *ContentionEvent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -22656,7 +22656,7 @@ func (m *EndTxnRequest) Unmarshal(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.Deadline == nil { - m.Deadline = &hlc.Timestamp{} + m.Deadline = &github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp{} } if err := m.Deadline.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err @@ -38505,516 +38505,520 @@ var ( ErrIntOverflowApi = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_290a2f113c3e6c17) } - -var fileDescriptor_api_290a2f113c3e6c17 = []byte{ - // 8124 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x7d, 0x5d, 0x6c, 0x23, 0x59, - 0x76, 0x9e, 0x8a, 0xa4, 0x24, 0xf2, 0x50, 0xfc, 0xd1, 0x95, 0xba, 0x9b, 0xad, 0x99, 0x69, 0xa9, - 0xab, 0xff, 0x7b, 0x67, 0xa4, 0xe9, 0xee, 0x1d, 0xcf, 0x78, 0x7a, 0x3c, 0x6b, 0x91, 0x62, 0x37, - 0x29, 0xb5, 0xd4, 0xea, 0x22, 0xd5, 0xed, 0x19, 0xaf, 0x53, 0x5b, 0xaa, 0xba, 0xa2, 0x6a, 0x45, - 0x56, 0xb1, 0xab, 0x8a, 0xfa, 0x19, 0x20, 0x40, 0x1c, 0x1b, 0x89, 0x81, 0x00, 0x0b, 0x3f, 0x38, - 0xc8, 0x1a, 0x4e, 0xe2, 0x75, 0x1c, 0xc7, 0x01, 0x82, 0x20, 0x01, 0x12, 0xc4, 0x41, 0x90, 0xd8, - 0x2f, 0x46, 0xb2, 0x08, 0x0c, 0x64, 0xfd, 0x14, 0x23, 0x40, 0x14, 0x5b, 0x9b, 0xb7, 0x04, 0x46, - 0x90, 0x97, 0x00, 0xf3, 0x10, 0x04, 0xf7, 0xa7, 0xfe, 0xc8, 0x22, 0x45, 0xf5, 0xd6, 0x20, 0x03, - 0xec, 0x8b, 0xc4, 0x3a, 0x75, 0xcf, 0xa9, 0x7b, 0xcf, 0xbd, 0xf7, 0xdc, 0xf3, 0xdd, 0x3a, 0xf7, - 0x14, 0xcc, 0x5a, 0xa6, 0xa2, 0xee, 0x77, 0x77, 0x57, 0x94, 0xae, 0xbe, 0xdc, 0xb5, 0x4c, 0xc7, - 0x44, 0xb3, 0xaa, 0xa9, 0x1e, 0x50, 0xf2, 0x32, 0xbf, 0xb9, 0x70, 0xff, 0xe0, 0x70, 0xe5, 0xe0, - 0xd0, 0xc6, 0xd6, 0x21, 0xb6, 0x56, 0x54, 0xd3, 0x50, 0x7b, 0x96, 0x85, 0x0d, 0xf5, 0x64, 0xa5, - 0x6d, 0xaa, 0x07, 0xf4, 0x8f, 0x6e, 0xb4, 0x18, 0xfb, 0x02, 0x72, 0x25, 0x6a, 0x8a, 0xa3, 0x70, - 0xda, 0xbc, 0x4b, 0xc3, 0x96, 0x65, 0x5a, 0x36, 0xa7, 0x5e, 0x76, 0xa9, 0x1d, 0xec, 0x28, 0x81, - 0xd2, 0x6f, 0xd9, 0x8e, 0x69, 0x29, 0x2d, 0xbc, 0x82, 0x8d, 0x96, 0x6e, 0x60, 0x52, 0xe0, 0x50, - 0x55, 0xf9, 0xcd, 0xb7, 0x23, 0x6f, 0x3e, 0xe2, 0x77, 0x4b, 0x3d, 0x47, 0x6f, 0xaf, 0xec, 0xb7, - 0xd5, 0x15, 0x47, 0xef, 0x60, 0xdb, 0x51, 0x3a, 0x5d, 0x7e, 0xe7, 0x3e, 0xbd, 0xe3, 0x58, 0x8a, - 0xaa, 0x1b, 0x2d, 0xf7, 0x7f, 0x77, 0x77, 0xc5, 0xc2, 0xaa, 0x69, 0x69, 0x58, 0x93, 0xed, 0xae, - 0x62, 0xb8, 0xd5, 0x6d, 0x99, 0x2d, 0x93, 0xfe, 0x5c, 0x21, 0xbf, 0x38, 0xf5, 0x5a, 0xcb, 0x34, - 0x5b, 0x6d, 0xbc, 0x42, 0xaf, 0x76, 0x7b, 0x7b, 0x2b, 0x5a, 0xcf, 0x52, 0x1c, 0xdd, 0xe4, 0x5c, - 0xe2, 0xbf, 0x12, 0x20, 0x27, 0xe1, 0xd7, 0x3d, 0x6c, 0x3b, 0x35, 0xac, 0x68, 0xd8, 0x42, 0x57, - 0x21, 0x79, 0x80, 0x4f, 0x4a, 0xc9, 0x25, 0xe1, 0xee, 0x4c, 0x79, 0xfa, 0xcb, 0xd3, 0xc5, 0xe4, - 0x06, 0x3e, 0x91, 0x08, 0x0d, 0x2d, 0xc1, 0x34, 0x36, 0x34, 0x99, 0xdc, 0x4e, 0x85, 0x6f, 0x4f, - 0x61, 0x43, 0xdb, 0xc0, 0x27, 0xe8, 0xdb, 0x90, 0xb6, 0x89, 0x34, 0x43, 0xc5, 0xa5, 0xc9, 0x25, - 0xe1, 0xee, 0x64, 0xf9, 0xe7, 0xbf, 0x3c, 0x5d, 0xfc, 0xa4, 0xa5, 0x3b, 0xfb, 0xbd, 0xdd, 0x65, - 0xd5, 0xec, 0xac, 0x78, 0xfd, 0xa4, 0xed, 0xfa, 0xbf, 0x57, 0xba, 0x07, 0xad, 0x95, 0x7e, 0x1d, - 0x2d, 0x37, 0x8f, 0x8d, 0x06, 0x7e, 0x2d, 0x79, 0x12, 0xd7, 0x53, 0x69, 0xa1, 0x98, 0x58, 0x4f, - 0xa5, 0x13, 0xc5, 0xa4, 0xf8, 0xbb, 0x49, 0xc8, 0x4b, 0xd8, 0xee, 0x9a, 0x86, 0x8d, 0x79, 0xcd, - 0xdf, 0x87, 0xa4, 0x73, 0x6c, 0xd0, 0x9a, 0x67, 0x1f, 0x5e, 0x5b, 0x1e, 0x18, 0x11, 0xcb, 0x4d, - 0x4b, 0x31, 0x6c, 0x45, 0x25, 0xcd, 0x97, 0x48, 0x51, 0xf4, 0x11, 0x64, 0x2d, 0x6c, 0xf7, 0x3a, - 0x98, 0x2a, 0x92, 0x36, 0x2a, 0xfb, 0xf0, 0x4a, 0x04, 0x67, 0xa3, 0xab, 0x18, 0x12, 0xb0, 0xb2, - 0xe4, 0x37, 0xba, 0x0a, 0x69, 0xa3, 0xd7, 0x21, 0xaa, 0xb0, 0x69, 0x43, 0x93, 0xd2, 0xb4, 0xd1, - 0xeb, 0x6c, 0xe0, 0x13, 0x1b, 0xfd, 0x02, 0x5c, 0xd6, 0x70, 0xd7, 0xc2, 0xaa, 0xe2, 0x60, 0x4d, - 0xb6, 0x14, 0xa3, 0x85, 0x65, 0xdd, 0xd8, 0x33, 0xed, 0xd2, 0xd4, 0x52, 0xf2, 0x6e, 0xf6, 0xe1, - 0xdb, 0x11, 0xf2, 0x25, 0x52, 0xaa, 0x6e, 0xec, 0x99, 0xe5, 0xd4, 0x0f, 0x4f, 0x17, 0x27, 0xa4, - 0x79, 0x5f, 0x82, 0x77, 0xcb, 0x46, 0x0d, 0xc8, 0xf1, 0xea, 0x5a, 0x58, 0xb1, 0x4d, 0xa3, 0x34, - 0xbd, 0x24, 0xdc, 0xcd, 0x3f, 0x5c, 0x8e, 0x12, 0x18, 0x52, 0x0d, 0xb9, 0xec, 0x75, 0xb0, 0x44, - 0xb9, 0xa4, 0x19, 0x2b, 0x70, 0x85, 0xde, 0x82, 0x0c, 0x69, 0xc9, 0xee, 0x89, 0x83, 0xed, 0x52, - 0x9a, 0x36, 0x85, 0x34, 0xad, 0x4c, 0xae, 0xc5, 0x4f, 0x61, 0x26, 0xc8, 0x8a, 0x10, 0xe4, 0xa5, - 0x6a, 0x63, 0x67, 0xb3, 0x2a, 0xef, 0x6c, 0x6d, 0x6c, 0x3d, 0x7f, 0xb5, 0x55, 0x9c, 0x40, 0xf3, - 0x50, 0xe4, 0xb4, 0x8d, 0xea, 0x67, 0xf2, 0xb3, 0xfa, 0x66, 0xbd, 0x59, 0x14, 0x16, 0x52, 0xbf, - 0xf6, 0xbb, 0xd7, 0x26, 0xc4, 0x6d, 0x80, 0xa7, 0xd8, 0xe1, 0x03, 0x0c, 0x95, 0x61, 0x6a, 0x9f, - 0xd6, 0xa7, 0x24, 0x50, 0x4d, 0x2f, 0x45, 0x56, 0x3c, 0x30, 0x18, 0xcb, 0x69, 0xa2, 0x8d, 0x1f, - 0x9d, 0x2e, 0x0a, 0x12, 0xe7, 0x14, 0xff, 0x48, 0x80, 0x2c, 0x15, 0xc9, 0xda, 0x87, 0x2a, 0x7d, - 0x32, 0xaf, 0x9f, 0xab, 0x8c, 0x41, 0xa1, 0x68, 0x19, 0x26, 0x0f, 0x95, 0x76, 0x0f, 0x97, 0x12, - 0x54, 0x46, 0x29, 0x42, 0xc6, 0x4b, 0x72, 0x5f, 0x62, 0xc5, 0xd0, 0x63, 0x98, 0xd1, 0x0d, 0x07, - 0x1b, 0x8e, 0xcc, 0xd8, 0x92, 0xe7, 0xb0, 0x65, 0x59, 0x69, 0x7a, 0x21, 0xfe, 0x4b, 0x01, 0x60, - 0xbb, 0x17, 0xa7, 0x52, 0xd0, 0x37, 0xc7, 0xac, 0x3f, 0x1f, 0x5d, 0xbc, 0x15, 0x97, 0x61, 0x4a, - 0x37, 0xda, 0xba, 0xc1, 0xea, 0x9f, 0x96, 0xf8, 0x15, 0x9a, 0x87, 0xc9, 0xdd, 0xb6, 0x6e, 0x68, - 0x74, 0x3e, 0xa4, 0x25, 0x76, 0x21, 0x4a, 0x90, 0xa5, 0xb5, 0x8e, 0x51, 0xef, 0xe2, 0xbf, 0x4f, - 0xc0, 0xa5, 0x8a, 0x69, 0x68, 0x3a, 0x99, 0x92, 0x4a, 0xfb, 0x6b, 0xa1, 0x95, 0x75, 0x08, 0x4c, - 0x3e, 0x19, 0x1f, 0x77, 0xc7, 0xec, 0x63, 0xe4, 0x73, 0x55, 0x8f, 0xbb, 0x94, 0x16, 0xad, 0x49, - 0xf4, 0x4d, 0xb8, 0xa2, 0xb4, 0xdb, 0xe6, 0x91, 0xac, 0xef, 0xc9, 0x9a, 0x89, 0x6d, 0xd9, 0x30, - 0x1d, 0x19, 0x1f, 0xeb, 0xb6, 0x43, 0x4d, 0x49, 0x5a, 0x9a, 0xa3, 0xb7, 0xeb, 0x7b, 0x6b, 0x26, - 0xb6, 0xb7, 0x4c, 0xa7, 0x4a, 0x6e, 0x91, 0x79, 0x4a, 0x2a, 0xc3, 0xe6, 0xe9, 0x14, 0x31, 0xbf, - 0x52, 0x1a, 0x1f, 0x77, 0xd9, 0x3c, 0xfd, 0x25, 0xb8, 0xdc, 0xaf, 0xc7, 0x38, 0xfb, 0xe9, 0x4f, - 0x04, 0xc8, 0xd7, 0x0d, 0xdd, 0xf9, 0x5a, 0x74, 0x90, 0xa7, 0xd4, 0x64, 0x50, 0xa9, 0xf7, 0xa1, - 0xb8, 0xa7, 0xe8, 0xed, 0xe7, 0x46, 0xd3, 0xec, 0xec, 0xda, 0x8e, 0x69, 0x60, 0x9b, 0x6b, 0x7d, - 0x80, 0x2e, 0xbe, 0x84, 0x82, 0xd7, 0x9a, 0x38, 0xd5, 0xe4, 0x40, 0xb1, 0x6e, 0xa8, 0x16, 0xee, - 0x60, 0x23, 0x56, 0x3d, 0xbd, 0x0d, 0x19, 0xdd, 0x95, 0x4b, 0x75, 0x95, 0x94, 0x7c, 0x82, 0xd8, - 0x83, 0xd9, 0xc0, 0x53, 0xe3, 0x34, 0x8b, 0x64, 0x69, 0xc0, 0x47, 0xb2, 0xdf, 0x47, 0x64, 0x69, - 0xc0, 0x47, 0xcc, 0x8c, 0x35, 0x20, 0xb7, 0x86, 0xdb, 0xd8, 0xc1, 0x71, 0x5a, 0xf7, 0x1d, 0xc8, - 0xbb, 0x42, 0xe3, 0xec, 0x98, 0xbf, 0x23, 0x00, 0xe2, 0x72, 0xc9, 0x6a, 0x1a, 0x67, 0xdf, 0x2c, - 0x12, 0x17, 0xc2, 0xe9, 0x59, 0x06, 0xf3, 0x05, 0xd8, 0x98, 0x04, 0x46, 0xa2, 0xee, 0x80, 0x6f, - 0x65, 0x53, 0x41, 0x2b, 0xcb, 0xdd, 0x98, 0x23, 0x98, 0x0b, 0x55, 0x2c, 0xde, 0xee, 0x4b, 0xd1, - 0x3a, 0x25, 0x96, 0x92, 0x41, 0x5f, 0x8d, 0x12, 0xc5, 0xef, 0x0b, 0x30, 0x5b, 0x69, 0x63, 0xc5, - 0x8a, 0x5d, 0x23, 0xdf, 0x82, 0xb4, 0x86, 0x15, 0x8d, 0x36, 0x99, 0x4d, 0xec, 0x77, 0x02, 0x52, - 0x88, 0x47, 0xbb, 0xbc, 0xdf, 0x56, 0x97, 0x9b, 0xae, 0xaf, 0xcb, 0x67, 0xb7, 0xc7, 0x24, 0x7e, - 0x06, 0x28, 0x58, 0xb3, 0x38, 0x07, 0xc2, 0xdf, 0x17, 0x00, 0x49, 0xf8, 0x10, 0x5b, 0x4e, 0xec, - 0xcd, 0x5e, 0x83, 0xac, 0xa3, 0x58, 0x2d, 0xec, 0xc8, 0xc4, 0x8b, 0xbf, 0x48, 0xcb, 0x81, 0xf1, - 0x11, 0xb2, 0xf8, 0x39, 0xcc, 0x85, 0xea, 0x17, 0x67, 0xe3, 0xff, 0x97, 0x00, 0xd9, 0x86, 0xaa, - 0x18, 0x71, 0xb6, 0xfa, 0x53, 0xc8, 0xda, 0xaa, 0x62, 0xc8, 0x7b, 0xa6, 0xd5, 0x51, 0x1c, 0x3a, - 0xc4, 0xf3, 0xa1, 0x56, 0x7b, 0x1e, 0xb4, 0xaa, 0x18, 0x4f, 0x68, 0x21, 0x09, 0x6c, 0xef, 0x37, - 0x7a, 0x01, 0xd9, 0x03, 0x7c, 0x22, 0x73, 0x34, 0x46, 0xd7, 0xbf, 0xfc, 0xc3, 0xf7, 0x03, 0xfc, - 0x07, 0x87, 0xcb, 0x2e, 0x88, 0x5b, 0x0e, 0x80, 0xb8, 0x65, 0xc2, 0xb1, 0xdc, 0x70, 0x2c, 0x6c, - 0xb4, 0x9c, 0x7d, 0x09, 0x0e, 0xf0, 0xc9, 0x33, 0x26, 0x83, 0x4d, 0xac, 0xf5, 0x54, 0x3a, 0x59, - 0x4c, 0x89, 0xff, 0x47, 0x80, 0x19, 0xd6, 0xe4, 0x38, 0x27, 0xd6, 0x07, 0x90, 0xb2, 0xcc, 0x23, - 0x36, 0xb1, 0xb2, 0x0f, 0xdf, 0x8a, 0x10, 0xb1, 0x81, 0x4f, 0x82, 0x2b, 0x17, 0x2d, 0x8e, 0xca, - 0xc0, 0xfd, 0x40, 0x99, 0x72, 0x27, 0xc7, 0xe5, 0x06, 0xc6, 0x25, 0x11, 0x19, 0x77, 0xa0, 0xb0, - 0xab, 0x38, 0xea, 0xbe, 0x6c, 0xf1, 0x4a, 0x92, 0x55, 0x2e, 0x79, 0x77, 0x46, 0xca, 0x53, 0xb2, - 0x5b, 0x75, 0x9b, 0xb4, 0x9c, 0x8d, 0x74, 0x1b, 0xff, 0x94, 0xf5, 0xf9, 0xff, 0x15, 0xf8, 0x1c, - 0x72, 0x5b, 0xfe, 0xd3, 0xd6, 0xf5, 0xbf, 0x99, 0x80, 0x2b, 0x95, 0x7d, 0xac, 0x1e, 0x54, 0x4c, - 0xc3, 0xd6, 0x6d, 0x87, 0xe8, 0x2e, 0xce, 0xfe, 0x7f, 0x0b, 0x32, 0x47, 0xba, 0xb3, 0x2f, 0x6b, - 0xfa, 0xde, 0x1e, 0xb5, 0x73, 0x69, 0x29, 0x4d, 0x08, 0x6b, 0xfa, 0xde, 0x1e, 0x7a, 0x04, 0xa9, - 0x8e, 0xa9, 0x31, 0x77, 0x39, 0xff, 0x70, 0x31, 0x42, 0x3c, 0xad, 0x9a, 0xdd, 0xeb, 0x6c, 0x9a, - 0x1a, 0x96, 0x68, 0x61, 0x74, 0x0d, 0x40, 0x25, 0xd4, 0xae, 0xa9, 0x1b, 0x0e, 0x5f, 0x27, 0x03, - 0x14, 0x54, 0x83, 0x8c, 0x83, 0xad, 0x8e, 0x6e, 0x28, 0x0e, 0x2e, 0x4d, 0x52, 0xe5, 0xdd, 0x8c, - 0xac, 0x78, 0xb7, 0xad, 0xab, 0xca, 0x1a, 0xb6, 0x55, 0x4b, 0xef, 0x3a, 0xa6, 0xc5, 0xb5, 0xe8, - 0x33, 0x8b, 0xdf, 0x4b, 0x41, 0x69, 0x50, 0x37, 0x71, 0x8e, 0x90, 0x6d, 0x98, 0x22, 0xf8, 0xba, - 0xed, 0xf0, 0x31, 0xf2, 0x70, 0x98, 0x0a, 0x22, 0x6a, 0x40, 0x71, 0x7a, 0xdb, 0xe1, 0xd5, 0xe6, - 0x72, 0x16, 0xfe, 0xad, 0x00, 0x53, 0xec, 0x06, 0x7a, 0x00, 0x69, 0xbe, 0xa1, 0xa0, 0xd1, 0x3a, - 0x26, 0xcb, 0x97, 0xcf, 0x4e, 0x17, 0xa7, 0xd9, 0x1e, 0xc1, 0xda, 0x97, 0xfe, 0x4f, 0x69, 0x9a, - 0x96, 0xab, 0x6b, 0xa4, 0xb7, 0x6c, 0x47, 0xb1, 0x1c, 0xba, 0x6d, 0x93, 0x60, 0xb8, 0x81, 0x12, - 0x36, 0xf0, 0x09, 0x5a, 0x87, 0x29, 0xdb, 0x51, 0x9c, 0x9e, 0xcd, 0xfb, 0xeb, 0x42, 0x95, 0x6d, - 0x50, 0x4e, 0x89, 0x4b, 0x20, 0x8e, 0x8e, 0x86, 0x1d, 0x45, 0x6f, 0xd3, 0x0e, 0xcc, 0x48, 0xfc, - 0x4a, 0xfc, 0x2d, 0x01, 0xa6, 0x58, 0x51, 0x74, 0x05, 0xe6, 0xa4, 0xd5, 0xad, 0xa7, 0x55, 0xb9, - 0xbe, 0xb5, 0x56, 0x6d, 0x56, 0xa5, 0xcd, 0xfa, 0xd6, 0x6a, 0xb3, 0x5a, 0x9c, 0x40, 0x97, 0x01, - 0xb9, 0x37, 0x2a, 0xcf, 0xb7, 0x1a, 0xf5, 0x46, 0xb3, 0xba, 0xd5, 0x2c, 0x0a, 0x74, 0x6f, 0x81, - 0xd2, 0x03, 0xd4, 0x04, 0xba, 0x09, 0x4b, 0xfd, 0x54, 0xb9, 0xd1, 0x5c, 0x6d, 0x36, 0xe4, 0x6a, - 0xa3, 0x59, 0xdf, 0x5c, 0x6d, 0x56, 0xd7, 0x8a, 0xc9, 0x11, 0xa5, 0xc8, 0x43, 0x24, 0xa9, 0x5a, - 0x69, 0x16, 0x53, 0xa2, 0x03, 0x97, 0x24, 0xac, 0x9a, 0x9d, 0x6e, 0xcf, 0xc1, 0xa4, 0x96, 0x76, - 0x9c, 0x33, 0xe5, 0x0a, 0x4c, 0x6b, 0xd6, 0x89, 0x6c, 0xf5, 0x0c, 0x3e, 0x4f, 0xa6, 0x34, 0xeb, - 0x44, 0xea, 0x19, 0xe2, 0x3f, 0x17, 0xe0, 0x72, 0xff, 0x63, 0xe3, 0x1c, 0x84, 0x2f, 0x20, 0xab, - 0x68, 0x1a, 0xd6, 0x64, 0x0d, 0xb7, 0x1d, 0x85, 0x3b, 0x23, 0xf7, 0x03, 0x92, 0xf8, 0x66, 0xdb, - 0xb2, 0xb7, 0xd9, 0xb6, 0xf9, 0xb2, 0x52, 0xa1, 0x15, 0x59, 0x23, 0x1c, 0xae, 0xf9, 0xa1, 0x42, - 0x28, 0x45, 0xfc, 0x9f, 0x29, 0xc8, 0x55, 0x0d, 0xad, 0x79, 0x1c, 0xeb, 0x5a, 0x72, 0x19, 0xa6, - 0x54, 0xb3, 0xd3, 0xd1, 0x1d, 0x57, 0x41, 0xec, 0x0a, 0xfd, 0x6c, 0xc0, 0x89, 0x4c, 0x8e, 0xe1, - 0x4a, 0xf9, 0xee, 0x23, 0xfa, 0x0e, 0x5c, 0x21, 0x56, 0xd3, 0x32, 0x94, 0xb6, 0xcc, 0xa4, 0xc9, - 0x8e, 0xa5, 0xb7, 0x5a, 0xd8, 0xe2, 0x1b, 0x7c, 0x77, 0x23, 0xea, 0x59, 0xe7, 0x1c, 0x15, 0xca, - 0xd0, 0x64, 0xe5, 0xa5, 0x4b, 0x7a, 0x14, 0x19, 0x7d, 0x02, 0x40, 0x96, 0x22, 0xba, 0x69, 0x68, - 0x73, 0x7b, 0x34, 0x6c, 0xd7, 0xd0, 0x35, 0x41, 0x84, 0x81, 0x5c, 0xdb, 0x68, 0x85, 0x20, 0x86, - 0xd7, 0x3d, 0xdd, 0xc2, 0xf2, 0x83, 0xae, 0x4a, 0xa1, 0x7c, 0xba, 0x9c, 0x3f, 0x3b, 0x5d, 0x04, - 0x89, 0x91, 0x1f, 0x6c, 0x57, 0x08, 0x82, 0x60, 0xbf, 0xbb, 0x2a, 0x7a, 0x05, 0xf7, 0x02, 0x3b, - 0x12, 0x64, 0xe5, 0xe5, 0xcd, 0x52, 0x1c, 0x79, 0x5f, 0x6f, 0xed, 0x63, 0x4b, 0xf6, 0x36, 0x8e, - 0xe9, 0x0e, 0x5e, 0x5a, 0xba, 0xe9, 0x33, 0x54, 0x14, 0x83, 0xd5, 0x7e, 0xd5, 0xa9, 0xd1, 0xc2, - 0x9e, 0xce, 0x88, 0xf2, 0xbb, 0xa6, 0x6e, 0x9b, 0x46, 0x29, 0xc3, 0x94, 0xcf, 0xae, 0xd0, 0x3d, - 0x28, 0x3a, 0xc7, 0x86, 0xbc, 0x8f, 0x15, 0xcb, 0xd9, 0xc5, 0x8a, 0x43, 0x56, 0x69, 0xa0, 0x25, - 0x0a, 0xce, 0xb1, 0x51, 0x0b, 0x90, 0xd1, 0x0b, 0x28, 0xea, 0x86, 0xbc, 0xd7, 0xd6, 0x5b, 0xfb, - 0x8e, 0x7c, 0x64, 0xe9, 0x0e, 0xb6, 0x4b, 0xb3, 0x54, 0x21, 0x51, 0xe3, 0xb6, 0xc1, 0x77, 0x72, - 0xb5, 0x57, 0xa4, 0x24, 0x57, 0x4d, 0x5e, 0x37, 0x9e, 0x50, 0x7e, 0x4a, 0xb4, 0xd7, 0x53, 0xe9, - 0xe9, 0x62, 0x5a, 0xfc, 0xaf, 0x02, 0xe4, 0xdd, 0xe1, 0x16, 0xe7, 0xcc, 0xb8, 0x0b, 0x45, 0xd3, - 0xc0, 0x72, 0x77, 0x5f, 0xb1, 0x31, 0xd7, 0x23, 0x5f, 0x70, 0xf2, 0xa6, 0x81, 0xb7, 0x09, 0x99, - 0xa9, 0x0b, 0x6d, 0xc3, 0xac, 0xed, 0x28, 0x2d, 0xdd, 0x68, 0x05, 0xd4, 0x3b, 0x39, 0xbe, 0x5b, - 0x5f, 0xe4, 0xdc, 0x1e, 0x3d, 0xe4, 0xa5, 0xfc, 0xa9, 0x00, 0xb3, 0xab, 0x5a, 0x47, 0x37, 0x1a, - 0xdd, 0xb6, 0x1e, 0xeb, 0x6e, 0xc1, 0x4d, 0xc8, 0xd8, 0x44, 0xa6, 0x6f, 0xf0, 0x7d, 0xec, 0x97, - 0xa6, 0x77, 0x88, 0xe5, 0x7f, 0x06, 0x05, 0x7c, 0xdc, 0xd5, 0xd9, 0xcb, 0x00, 0x06, 0x59, 0x52, - 0xe3, 0xb7, 0x2d, 0xef, 0xf3, 0x92, 0x5b, 0xbc, 0x4d, 0x9f, 0x01, 0x0a, 0x36, 0x29, 0x4e, 0xec, - 0xf2, 0x19, 0xcc, 0x51, 0xd1, 0x3b, 0x86, 0x1d, 0xb3, 0xbe, 0xc4, 0x5f, 0x84, 0xf9, 0xb0, 0xe8, - 0x38, 0xeb, 0xfd, 0x8a, 0xf7, 0xf2, 0x26, 0xb6, 0x62, 0x85, 0x9b, 0x9e, 0xae, 0xb9, 0xe0, 0x38, - 0xeb, 0xfc, 0xab, 0x02, 0x5c, 0xa5, 0xb2, 0xe9, 0xfb, 0x92, 0x3d, 0x6c, 0x3d, 0xc3, 0x8a, 0x1d, - 0x2b, 0x56, 0xbe, 0x01, 0x53, 0x0c, 0xf3, 0xd2, 0xf1, 0x39, 0x59, 0xce, 0x12, 0xcf, 0xa5, 0xe1, - 0x98, 0x16, 0xf1, 0x5c, 0xf8, 0x2d, 0x51, 0x81, 0x85, 0xa8, 0x5a, 0xc4, 0xbc, 0x1d, 0x30, 0xcb, - 0x9d, 0x46, 0x32, 0x94, 0x2b, 0xfb, 0xc4, 0x67, 0x42, 0x55, 0xc8, 0xaa, 0xf4, 0x97, 0xec, 0x9c, - 0x74, 0x31, 0x95, 0x9f, 0x1f, 0xe5, 0x6f, 0x32, 0xb6, 0xe6, 0x49, 0x17, 0x13, 0xa7, 0xd5, 0xfd, - 0x4d, 0x14, 0x15, 0x68, 0xe4, 0x48, 0x8f, 0x95, 0xce, 0x23, 0x5a, 0xd6, 0x75, 0xfd, 0xb8, 0x0e, - 0xfe, 0x45, 0x92, 0x2b, 0x81, 0x3d, 0x83, 0x17, 0x8f, 0xd5, 0x47, 0xf9, 0x3c, 0xf4, 0xba, 0x2a, - 0xd8, 0xf0, 0xc4, 0x05, 0x1a, 0x1e, 0xd8, 0x33, 0xf7, 0xa9, 0xe8, 0x33, 0x08, 0xec, 0x8a, 0xcb, - 0xac, 0x4d, 0x2e, 0xfa, 0xb9, 0x88, 0x3a, 0x66, 0x7d, 0x29, 0x8c, 0x6e, 0xa3, 0x0a, 0xa4, 0xf1, - 0x71, 0x57, 0xd6, 0xb0, 0xad, 0x72, 0xc3, 0x25, 0x0e, 0x7b, 0xaf, 0x36, 0x80, 0x07, 0xa6, 0xf1, - 0x71, 0x97, 0x10, 0xd1, 0x0e, 0x59, 0xbd, 0x5c, 0x57, 0x81, 0x56, 0xdb, 0x3e, 0x1f, 0x5e, 0xf8, - 0x23, 0x85, 0x8b, 0x2b, 0x78, 0x5e, 0x02, 0x13, 0x21, 0xfe, 0x40, 0x80, 0xb7, 0x22, 0x7b, 0x2d, - 0xce, 0x85, 0xec, 0x13, 0x48, 0xd1, 0xc6, 0x27, 0x2e, 0xd8, 0x78, 0xca, 0x25, 0xfe, 0x9e, 0x3b, - 0xc7, 0x25, 0xdc, 0x36, 0x89, 0x62, 0xbf, 0x82, 0xfd, 0xb0, 0x69, 0xb7, 0xc3, 0x13, 0x17, 0xee, - 0x70, 0x97, 0xd5, 0x33, 0x02, 0x7d, 0xd5, 0x8c, 0xd3, 0x08, 0xfc, 0xba, 0x00, 0x73, 0x9e, 0x4f, - 0x13, 0xb3, 0x7b, 0xfb, 0x01, 0x24, 0x0d, 0xf3, 0xe8, 0x22, 0x9b, 0x81, 0xa4, 0x3c, 0x59, 0x92, - 0xc2, 0x35, 0x8a, 0xb3, 0xbd, 0xff, 0x2e, 0x01, 0x99, 0xa7, 0x95, 0x38, 0x5b, 0xf9, 0x09, 0xdf, - 0x68, 0x66, 0x13, 0x3b, 0x6a, 0x28, 0x7a, 0xcf, 0x5b, 0x7e, 0x5a, 0xd9, 0xc0, 0x27, 0xee, 0x50, - 0x24, 0x5c, 0x68, 0x15, 0x32, 0xce, 0xbe, 0x85, 0xed, 0x7d, 0xb3, 0xad, 0x5d, 0xc4, 0x07, 0xf1, - 0xb9, 0x16, 0x30, 0x4c, 0x52, 0xb9, 0x6e, 0xf0, 0x82, 0x10, 0x11, 0xbc, 0x40, 0x1e, 0xe3, 0xb9, - 0x71, 0x89, 0x8b, 0x3c, 0x26, 0xe0, 0xbf, 0x4d, 0x16, 0xa7, 0xc4, 0x17, 0x00, 0xa4, 0x39, 0x71, - 0x76, 0xc9, 0xdf, 0x48, 0x42, 0x7e, 0xbb, 0x67, 0xef, 0xc7, 0x3c, 0xfa, 0x2a, 0x00, 0xdd, 0x9e, - 0x4d, 0xf1, 0xc1, 0xb1, 0xc1, 0xdb, 0x7c, 0x4e, 0x5c, 0x84, 0xdb, 0x68, 0xc6, 0xd7, 0x3c, 0x36, - 0x50, 0x8d, 0x0b, 0xc1, 0xb2, 0x1f, 0x5c, 0x71, 0x63, 0x14, 0x92, 0x6c, 0x1e, 0x1b, 0x9b, 0xd8, - 0x83, 0x90, 0x4c, 0x12, 0x26, 0x92, 0x3e, 0x81, 0x69, 0x72, 0x21, 0x3b, 0xe6, 0x45, 0xba, 0x79, - 0x8a, 0xf0, 0x34, 0x4d, 0xf4, 0x18, 0x32, 0x8c, 0x9b, 0x2c, 0x4d, 0x53, 0x74, 0x69, 0x8a, 0x6a, - 0x0b, 0x57, 0x23, 0x5d, 0x94, 0xd2, 0x94, 0x95, 0x2c, 0x44, 0xf3, 0x30, 0xb9, 0x67, 0x5a, 0x2a, - 0xa6, 0x11, 0x13, 0x69, 0x89, 0x5d, 0xb0, 0xfe, 0x5c, 0x4f, 0xa5, 0xd3, 0xc5, 0xcc, 0x7a, 0x2a, - 0x9d, 0x29, 0x82, 0xf8, 0x5b, 0x02, 0x14, 0xbc, 0x8e, 0x88, 0xd3, 0x5a, 0x57, 0x42, 0x5a, 0xbc, - 0x78, 0x57, 0x10, 0x05, 0x8a, 0xff, 0x81, 0xba, 0x2b, 0xaa, 0x79, 0x48, 0x7b, 0x26, 0xce, 0x91, - 0xf2, 0x98, 0x85, 0xce, 0x24, 0x2e, 0xda, 0xbb, 0x34, 0x8a, 0xe6, 0x01, 0xcc, 0xeb, 0x1d, 0x62, - 0xc7, 0x75, 0xa7, 0x7d, 0xc2, 0x31, 0x95, 0x83, 0xdd, 0xf7, 0xb3, 0x73, 0xfe, 0xbd, 0x8a, 0x7b, - 0x4b, 0xfc, 0x87, 0x74, 0x77, 0xda, 0x6f, 0x49, 0x9c, 0xaa, 0xae, 0x43, 0xce, 0x62, 0xa2, 0x89, - 0xcf, 0x71, 0x41, 0x6d, 0xcf, 0x78, 0xac, 0x44, 0xe1, 0xbf, 0x93, 0x80, 0xc2, 0x8b, 0x1e, 0xb6, - 0x4e, 0xbe, 0x4e, 0xea, 0xbe, 0x0d, 0x85, 0x23, 0x45, 0x77, 0xe4, 0x3d, 0xd3, 0x92, 0x7b, 0x5d, - 0x4d, 0x71, 0xdc, 0xf8, 0x8d, 0x1c, 0x21, 0x3f, 0x31, 0xad, 0x1d, 0x4a, 0x44, 0x18, 0xd0, 0x81, - 0x61, 0x1e, 0x19, 0x32, 0x21, 0x53, 0x14, 0x7b, 0x6c, 0xf0, 0x2d, 0xe3, 0xf2, 0x87, 0xff, 0xe5, - 0x74, 0xf1, 0xd1, 0x58, 0x51, 0x59, 0x34, 0x02, 0xad, 0xd7, 0xd3, 0xb5, 0xe5, 0x9d, 0x9d, 0xfa, - 0x9a, 0x54, 0xa4, 0x22, 0x5f, 0x31, 0x89, 0xcd, 0x63, 0xc3, 0x16, 0xff, 0x51, 0x02, 0x8a, 0xbe, - 0x8e, 0xe2, 0xec, 0xc8, 0x2a, 0x64, 0x5f, 0xf7, 0xb0, 0xa5, 0xbf, 0x41, 0x37, 0x02, 0x67, 0x24, - 0x66, 0xe7, 0x73, 0x98, 0x09, 0x69, 0x20, 0xf9, 0x93, 0x69, 0x20, 0x7b, 0xe4, 0x37, 0x1e, 0xdd, - 0x87, 0x59, 0xe7, 0xd8, 0x90, 0x59, 0x3c, 0x1e, 0x8b, 0xe1, 0x70, 0xc3, 0x0e, 0x0a, 0x0e, 0xd1, - 0x07, 0xa1, 0xd3, 0xf8, 0x0d, 0x5b, 0xfc, 0x23, 0x01, 0x10, 0x55, 0x54, 0x9d, 0xed, 0xe9, 0x7f, - 0x5d, 0xc6, 0xd3, 0x5d, 0x28, 0xd2, 0x08, 0x47, 0x59, 0xdf, 0x93, 0x3b, 0xba, 0x6d, 0xeb, 0x46, - 0x8b, 0x0f, 0xa8, 0x3c, 0xa5, 0xd7, 0xf7, 0x36, 0x19, 0x55, 0xfc, 0xab, 0x30, 0x17, 0x6a, 0x40, - 0x9c, 0x9d, 0x7d, 0x1d, 0x66, 0xf6, 0xcc, 0x9e, 0xa1, 0xc9, 0xec, 0x8d, 0x07, 0xdf, 0x0e, 0xcc, - 0x52, 0x1a, 0x7b, 0x9e, 0xf8, 0x3f, 0x12, 0x30, 0x2f, 0x61, 0xdb, 0x6c, 0x1f, 0xe2, 0xf8, 0x55, - 0x58, 0x03, 0xfe, 0xae, 0x45, 0x7e, 0x23, 0x4d, 0x66, 0x18, 0x33, 0x5b, 0xe6, 0xc2, 0x7b, 0xea, - 0x37, 0x47, 0x8f, 0xd8, 0xc1, 0x5d, 0x74, 0xbe, 0x27, 0x97, 0x0a, 0xed, 0xc9, 0x99, 0x50, 0xd0, - 0x5b, 0x86, 0x49, 0x6c, 0x9a, 0x8d, 0x5f, 0x1b, 0xbd, 0x8e, 0x8b, 0x54, 0x96, 0x47, 0x55, 0xb2, - 0xce, 0x58, 0x1a, 0xf8, 0xf5, 0x56, 0xaf, 0x43, 0x7d, 0xe7, 0xf2, 0x65, 0x52, 0xdf, 0xb3, 0xd3, - 0xc5, 0x7c, 0xe8, 0x9e, 0x2d, 0xe5, 0x75, 0xef, 0x9a, 0x48, 0x17, 0xbf, 0x0d, 0x97, 0xfa, 0x94, - 0x1d, 0xa7, 0xc7, 0xf3, 0x6f, 0x92, 0x70, 0x35, 0x2c, 0x3e, 0x6e, 0xfc, 0xf1, 0x75, 0xef, 0xd0, - 0x1a, 0xe4, 0x3a, 0xba, 0xf1, 0x66, 0x5b, 0x8b, 0x33, 0x1d, 0xdd, 0xf0, 0xb7, 0x71, 0x23, 0x86, - 0xc6, 0xd4, 0x57, 0x3a, 0x34, 0x14, 0x58, 0x88, 0xea, 0xbb, 0x38, 0xc7, 0xc7, 0xaf, 0x09, 0x30, - 0x13, 0xf7, 0x9e, 0xd9, 0x9b, 0xc5, 0x9b, 0x89, 0x4d, 0xc8, 0x7d, 0x05, 0x9b, 0x6c, 0xbf, 0x23, - 0x00, 0x6a, 0x5a, 0x3d, 0x83, 0x80, 0xda, 0x67, 0x66, 0x2b, 0xce, 0x66, 0xce, 0xc3, 0xa4, 0x6e, - 0x68, 0xf8, 0x98, 0x36, 0x33, 0x25, 0xb1, 0x8b, 0xd0, 0xab, 0xc3, 0xe4, 0x58, 0xaf, 0x0e, 0xc5, - 0xcf, 0x61, 0x2e, 0x54, 0xc5, 0x38, 0xdb, 0xff, 0x8f, 0x13, 0x30, 0xc7, 0x1b, 0x12, 0xfb, 0xf6, - 0xe2, 0x37, 0x61, 0xb2, 0x4d, 0x64, 0x8e, 0xe8, 0x67, 0xfa, 0x4c, 0xb7, 0x9f, 0x69, 0x61, 0xf4, - 0x73, 0x00, 0x5d, 0x0b, 0x1f, 0xca, 0x8c, 0x35, 0x39, 0x16, 0x6b, 0x86, 0x70, 0x50, 0x02, 0xfa, - 0x05, 0x28, 0x90, 0xf9, 0xdc, 0xb5, 0xcc, 0xae, 0x69, 0x13, 0x97, 0xc5, 0x1e, 0x0f, 0xe5, 0xcc, - 0x9e, 0x9d, 0x2e, 0xe6, 0x36, 0x75, 0x63, 0x9b, 0x33, 0x36, 0x1b, 0x12, 0x31, 0x0c, 0xde, 0xa5, - 0x2d, 0xfe, 0x27, 0x01, 0xe6, 0xbf, 0xb2, 0xad, 0xd8, 0xff, 0x1f, 0xba, 0x12, 0x5f, 0x42, 0x91, - 0xfe, 0xa8, 0x1b, 0x7b, 0x66, 0x9c, 0x9b, 0xe2, 0xdf, 0x13, 0x60, 0x36, 0x20, 0x38, 0x4e, 0xff, - 0xe4, 0x8d, 0xf4, 0x24, 0xfe, 0x22, 0xf1, 0x58, 0x82, 0x83, 0x3c, 0xce, 0x29, 0xf4, 0x07, 0x09, - 0xb8, 0x5c, 0x61, 0xaf, 0x90, 0xdd, 0x98, 0x8a, 0x38, 0x47, 0x46, 0x09, 0xa6, 0x0f, 0xb1, 0x65, - 0xeb, 0x26, 0x5b, 0x3d, 0x73, 0x92, 0x7b, 0x89, 0x16, 0x20, 0x6d, 0x1b, 0x4a, 0xd7, 0xde, 0x37, - 0xdd, 0x77, 0x67, 0xde, 0xb5, 0x17, 0xff, 0x31, 0xf9, 0xe6, 0xf1, 0x1f, 0x53, 0xa3, 0xe3, 0x3f, - 0xa6, 0x7f, 0x82, 0xf8, 0x0f, 0xfe, 0xa2, 0xea, 0x3f, 0x0a, 0x70, 0x65, 0x40, 0x73, 0x71, 0x8e, - 0x96, 0xef, 0x42, 0x56, 0xe5, 0x82, 0x89, 0xbd, 0x65, 0x6f, 0xe1, 0xea, 0xa4, 0xd8, 0x1b, 0xc2, - 0x8e, 0xb3, 0xd3, 0x45, 0x70, 0xab, 0x5a, 0x5f, 0xe3, 0xca, 0x21, 0xbf, 0x35, 0xf1, 0x57, 0x72, - 0x50, 0xa8, 0x1e, 0xb3, 0x1d, 0xe8, 0x06, 0x5b, 0xe5, 0xd1, 0x13, 0x48, 0x77, 0x2d, 0xf3, 0x50, - 0x77, 0x9b, 0x91, 0x0f, 0xbd, 0xfc, 0x77, 0x9b, 0xd1, 0xc7, 0xb5, 0xcd, 0x39, 0x24, 0x8f, 0x17, - 0x35, 0x21, 0xf3, 0xcc, 0x54, 0x95, 0xf6, 0x13, 0xbd, 0xed, 0x8e, 0xfc, 0xf7, 0xcf, 0x17, 0xb4, - 0xec, 0xf1, 0x6c, 0x2b, 0xce, 0xbe, 0xdb, 0x09, 0x1e, 0x11, 0xd5, 0x21, 0x5d, 0x73, 0x9c, 0x2e, - 0xb9, 0xc9, 0x6d, 0xc7, 0x9d, 0x31, 0x84, 0x12, 0x16, 0x37, 0x56, 0xd4, 0x65, 0x47, 0x4d, 0x98, - 0x7d, 0x4a, 0x4f, 0x38, 0x55, 0xda, 0x66, 0x4f, 0xab, 0x98, 0xc6, 0x9e, 0xde, 0xe2, 0x76, 0xf7, - 0xf6, 0x18, 0x32, 0x9f, 0x56, 0x1a, 0xd2, 0xa0, 0x00, 0xb4, 0x0a, 0xe9, 0xc6, 0x23, 0x2e, 0x8c, - 0xb9, 0x65, 0xb7, 0xc6, 0x10, 0xd6, 0x78, 0x24, 0x79, 0x6c, 0x68, 0x1d, 0xb2, 0xab, 0x5f, 0xf4, - 0x2c, 0xcc, 0xa5, 0x4c, 0x0d, 0x8d, 0x3c, 0xe8, 0x97, 0x42, 0xb9, 0xa4, 0x20, 0x33, 0x6a, 0x40, - 0xfe, 0x95, 0x69, 0x1d, 0xb4, 0x4d, 0xc5, 0x6d, 0xe1, 0x34, 0x15, 0xf7, 0x8d, 0x31, 0xc4, 0xb9, - 0x8c, 0x52, 0x9f, 0x08, 0xf4, 0x6d, 0x28, 0x90, 0xce, 0x68, 0x2a, 0xbb, 0x6d, 0xb7, 0x92, 0x69, - 0x2a, 0xf5, 0xdd, 0x31, 0xa4, 0x7a, 0x9c, 0xee, 0x2b, 0x90, 0x3e, 0x51, 0x0b, 0x12, 0xe4, 0x42, - 0x83, 0x00, 0x21, 0x48, 0x75, 0x49, 0x7f, 0x0b, 0x34, 0x36, 0x88, 0xfe, 0x46, 0xef, 0xc1, 0xb4, - 0x61, 0x6a, 0xd8, 0x9d, 0x21, 0xb9, 0xf2, 0xfc, 0xd9, 0xe9, 0xe2, 0xd4, 0x96, 0xa9, 0x31, 0x87, - 0x84, 0xff, 0x92, 0xa6, 0x48, 0xa1, 0xba, 0xb6, 0xb0, 0x04, 0x29, 0xd2, 0xef, 0xc4, 0x30, 0xed, - 0x2a, 0x36, 0xde, 0xb1, 0x74, 0x2e, 0xcd, 0xbd, 0x5c, 0xf8, 0x67, 0x09, 0x48, 0x34, 0x1e, 0x11, - 0x97, 0x7b, 0xb7, 0xa7, 0x1e, 0x60, 0x87, 0xdf, 0xe7, 0x57, 0xd4, 0x15, 0xb7, 0xf0, 0x9e, 0xce, - 0x3c, 0xa3, 0x8c, 0xc4, 0xaf, 0xd0, 0x3b, 0x00, 0x8a, 0xaa, 0x62, 0xdb, 0x96, 0xdd, 0x93, 0x6f, - 0x19, 0x29, 0xc3, 0x28, 0x1b, 0xf8, 0x84, 0xb0, 0xd9, 0x58, 0xb5, 0xb0, 0xe3, 0x06, 0x36, 0xb1, - 0x2b, 0xc2, 0xe6, 0xe0, 0x4e, 0x57, 0x76, 0xcc, 0x03, 0x6c, 0xd0, 0x71, 0x92, 0x21, 0xa6, 0xa6, - 0xd3, 0x6d, 0x12, 0x02, 0xb1, 0x92, 0xd8, 0xd0, 0x7c, 0x93, 0x96, 0x91, 0xbc, 0x6b, 0x22, 0xd2, - 0xc2, 0x2d, 0x9d, 0x1f, 0xe1, 0xca, 0x48, 0xfc, 0x8a, 0x68, 0x49, 0xe9, 0x39, 0xfb, 0xb4, 0x27, - 0x32, 0x12, 0xfd, 0x8d, 0x6e, 0x43, 0x81, 0xc5, 0x42, 0xca, 0xd8, 0x50, 0x65, 0x6a, 0x5c, 0x33, - 0xf4, 0x76, 0x8e, 0x91, 0xab, 0x86, 0x4a, 0x4c, 0x29, 0x7a, 0x04, 0x9c, 0x20, 0x1f, 0x74, 0x6c, - 0xa2, 0x53, 0x20, 0xa5, 0xca, 0x85, 0xb3, 0xd3, 0xc5, 0x6c, 0x83, 0xde, 0xd8, 0xd8, 0x6c, 0xd4, - 0xd7, 0xa4, 0x2c, 0x2b, 0xb5, 0xd1, 0xb1, 0xeb, 0xda, 0xc2, 0x6f, 0x08, 0x90, 0x7c, 0x5a, 0x69, - 0x5c, 0x58, 0x65, 0x6e, 0x45, 0x93, 0x81, 0x8a, 0xde, 0x81, 0xc2, 0xae, 0xde, 0x6e, 0xeb, 0x46, - 0x8b, 0x78, 0x41, 0xdf, 0xc5, 0xaa, 0xab, 0xb0, 0x3c, 0x27, 0x6f, 0x33, 0x2a, 0x5a, 0x82, 0xac, - 0x6a, 0x61, 0x0d, 0x1b, 0x8e, 0xae, 0xb4, 0x6d, 0xae, 0xb9, 0x20, 0x69, 0xe1, 0x97, 0x05, 0x98, - 0xa4, 0x33, 0x00, 0xbd, 0x0d, 0x19, 0xd5, 0x34, 0x1c, 0x45, 0x37, 0xb8, 0x29, 0xcb, 0x48, 0x3e, - 0x61, 0x68, 0xf5, 0xae, 0xc3, 0x8c, 0xa2, 0xaa, 0x66, 0xcf, 0x70, 0x64, 0x43, 0xe9, 0x60, 0x5e, - 0xcd, 0x2c, 0xa7, 0x6d, 0x29, 0x1d, 0x8c, 0x16, 0xc1, 0xbd, 0xf4, 0x0e, 0x34, 0x66, 0x24, 0xe0, - 0xa4, 0x0d, 0x7c, 0xb2, 0xf0, 0xc7, 0x02, 0xa4, 0xdd, 0x39, 0x43, 0xaa, 0xd1, 0xc2, 0x06, 0xb6, - 0x14, 0xc7, 0xf4, 0xaa, 0xe1, 0x11, 0xfa, 0x97, 0xca, 0x8c, 0xbf, 0x54, 0xce, 0xc3, 0xa4, 0x43, - 0xa6, 0x05, 0xaf, 0x01, 0xbb, 0xa0, 0xdb, 0xd1, 0x6d, 0xa5, 0xc5, 0x76, 0xe3, 0x32, 0x12, 0xbb, - 0x20, 0x8d, 0xe1, 0x21, 0xb5, 0x4c, 0x23, 0xfc, 0x8a, 0xd4, 0x94, 0x05, 0x7e, 0xee, 0xe2, 0x96, - 0x6e, 0xd0, 0xb1, 0x94, 0x94, 0x80, 0x92, 0xca, 0x84, 0x82, 0xde, 0x82, 0x0c, 0x2b, 0x80, 0x0d, - 0x8d, 0x0e, 0xa8, 0xa4, 0x94, 0xa6, 0x84, 0xaa, 0xa1, 0x2d, 0x60, 0xc8, 0x78, 0x93, 0x93, 0x74, - 0x5b, 0xcf, 0xf6, 0x14, 0x49, 0x7f, 0xa3, 0xf7, 0x61, 0xfe, 0x75, 0x4f, 0x69, 0xeb, 0x7b, 0x74, - 0xa3, 0x8d, 0x14, 0x63, 0x3a, 0x63, 0x2d, 0x41, 0xde, 0x3d, 0x2a, 0x81, 0xaa, 0xce, 0x9d, 0xcb, - 0x49, 0x7f, 0x2e, 0x8b, 0xbf, 0x2f, 0xc0, 0x2c, 0x8b, 0xea, 0x61, 0xc1, 0xa8, 0xf1, 0xf9, 0x21, - 0x1f, 0x43, 0x46, 0x53, 0x1c, 0x85, 0x1d, 0xd1, 0x4c, 0x8c, 0x3c, 0xa2, 0xe9, 0x1d, 0x25, 0x50, - 0x1c, 0x85, 0x1e, 0xd3, 0x44, 0x90, 0x22, 0xbf, 0xd9, 0x69, 0x56, 0x89, 0xfe, 0x16, 0x3f, 0x03, - 0x14, 0xac, 0x68, 0x9c, 0x1e, 0xd9, 0x3d, 0xb8, 0x44, 0x74, 0x5d, 0x35, 0x54, 0xeb, 0xa4, 0xeb, - 0xe8, 0xa6, 0xf1, 0x9c, 0xfe, 0xb5, 0x51, 0x31, 0xf0, 0x5e, 0x8a, 0xbe, 0x8e, 0x12, 0xff, 0x70, - 0x0a, 0x72, 0xd5, 0xe3, 0xae, 0x69, 0xc5, 0xba, 0x8b, 0x55, 0x86, 0x69, 0x0e, 0xf4, 0x47, 0xbc, - 0x17, 0xee, 0x33, 0xe6, 0xee, 0x2b, 0x57, 0xce, 0x88, 0xca, 0x00, 0x2c, 0x60, 0x94, 0x06, 0x05, - 0x25, 0x2f, 0xf0, 0xa6, 0x8c, 0xb2, 0x11, 0x2a, 0xda, 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, 0x91, 0x1a, 0xd9, 0x76, 0x4f, - 0xc9, 0x95, 0x73, 0x67, 0xa7, 0x8b, 0x19, 0x89, 0x52, 0x1b, 0x8d, 0xa6, 0x94, 0x61, 0x05, 0x1a, - 0xb6, 0x83, 0x6e, 0x40, 0xce, 0xec, 0xe8, 0x8e, 0xec, 0x3a, 0x49, 0xdc, 0xa3, 0x9c, 0x21, 0x44, - 0xd7, 0x89, 0x42, 0x4d, 0xb8, 0x83, 0x0d, 0x3a, 0xda, 0x49, 0x3b, 0xe5, 0x5d, 0xb6, 0xf9, 0xe8, - 0xb0, 0x19, 0x2d, 0x9b, 0x5d, 0x47, 0xef, 0xe8, 0x5f, 0xd0, 0x37, 0xd3, 0xfc, 0xa5, 0xd1, 0x0d, - 0x56, 0x9c, 0xb4, 0xaf, 0x4c, 0x77, 0x25, 0x79, 0xd9, 0xe7, 0x81, 0xa2, 0xe8, 0x6f, 0x0a, 0x70, - 0x99, 0x2b, 0x52, 0xde, 0xa5, 0x31, 0xee, 0x4a, 0x5b, 0x77, 0x4e, 0xe4, 0x83, 0xc3, 0x52, 0x9a, - 0xfa, 0xad, 0x3f, 0x1b, 0xd9, 0x21, 0x81, 0x71, 0xb0, 0xec, 0x76, 0xcb, 0xc9, 0x33, 0xce, 0xbc, - 0x71, 0x58, 0x35, 0x1c, 0xeb, 0xa4, 0x7c, 0xe5, 0xec, 0x74, 0x71, 0x6e, 0xf0, 0xee, 0x4b, 0x69, - 0xce, 0x1e, 0x64, 0x41, 0x35, 0x00, 0xec, 0x8d, 0x43, 0xba, 0x62, 0x44, 0xfb, 0x1f, 0x91, 0x03, - 0x56, 0x0a, 0xf0, 0xa2, 0xbb, 0x50, 0xe4, 0x27, 0x5b, 0xf6, 0xf4, 0x36, 0x96, 0x6d, 0xfd, 0x0b, - 0x4c, 0xd7, 0x96, 0xa4, 0x94, 0x67, 0x74, 0x22, 0xa2, 0xa1, 0x7f, 0x81, 0x17, 0xbe, 0x0b, 0xa5, - 0x61, 0xb5, 0x0f, 0x4e, 0x81, 0x0c, 0x7b, 0x23, 0xfb, 0x51, 0x78, 0x3b, 0x66, 0x8c, 0xa1, 0xca, - 0xb7, 0x64, 0x3e, 0x4e, 0x7c, 0x24, 0x88, 0xff, 0x24, 0x01, 0xb9, 0x72, 0xaf, 0x7d, 0xf0, 0xbc, - 0xdb, 0xe8, 0x75, 0x3a, 0x8a, 0x75, 0x42, 0xcc, 0x20, 0x33, 0x14, 0xa4, 0x82, 0x02, 0x33, 0x83, - 0xd4, 0x12, 0xe8, 0x5f, 0x60, 0xb2, 0x38, 0x05, 0x4f, 0x65, 0xb3, 0x18, 0x7e, 0xda, 0x86, 0xc0, - 0x51, 0x6b, 0xf3, 0xc8, 0x46, 0x1f, 0x41, 0x29, 0x50, 0x90, 0xee, 0x9d, 0xc8, 0xd8, 0x70, 0x2c, - 0x1d, 0xb3, 0xfd, 0xbf, 0xa4, 0x14, 0x88, 0x97, 0xa9, 0x93, 0xdb, 0x55, 0x76, 0x17, 0x35, 0x61, - 0x86, 0x14, 0x3c, 0x91, 0xe9, 0x12, 0xe2, 0xee, 0xcf, 0x3e, 0x88, 0x68, 0x56, 0xa8, 0xde, 0xcb, - 0x54, 0x3f, 0x15, 0xca, 0x43, 0x7f, 0x4a, 0x59, 0xec, 0x53, 0x16, 0x3e, 0x85, 0x62, 0x7f, 0x81, - 0xa0, 0x2e, 0x53, 0x4c, 0x97, 0xf3, 0x41, 0x5d, 0x26, 0x03, 0x7a, 0x5a, 0x4f, 0xa5, 0x53, 0xc5, - 0x49, 0xf1, 0x2f, 0x92, 0x90, 0x77, 0x87, 0x59, 0x9c, 0x40, 0xa7, 0x0c, 0x93, 0x64, 0x50, 0xb8, - 0x31, 0x1e, 0xb7, 0x47, 0x8c, 0x6e, 0x1e, 0x35, 0x4e, 0x06, 0x8b, 0x0b, 0x92, 0x29, 0x6b, 0x1c, - 0x06, 0x67, 0xe1, 0x97, 0x13, 0x90, 0xa2, 0xd8, 0xe2, 0x01, 0xa4, 0xe8, 0x42, 0x21, 0x8c, 0xb3, - 0x50, 0xd0, 0xa2, 0xde, 0x72, 0x96, 0x08, 0xb8, 0xa6, 0xc4, 0xe7, 0xdb, 0x57, 0x3e, 0x78, 0xf0, - 0x90, 0x1a, 0x9b, 0x19, 0x89, 0x5f, 0xa1, 0x32, 0x0d, 0x3b, 0x32, 0x2d, 0x07, 0x6b, 0xdc, 0xa7, - 0x5f, 0x3a, 0xaf, 0x7f, 0xdd, 0x45, 0xc9, 0xe5, 0x43, 0x57, 0x21, 0x49, 0xac, 0xd8, 0x34, 0x0b, - 0x52, 0x38, 0x3b, 0x5d, 0x4c, 0x12, 0xfb, 0x45, 0x68, 0x68, 0x05, 0xb2, 0x61, 0x93, 0x41, 0x3c, - 0x38, 0x6a, 0x18, 0x03, 0xd3, 0x1d, 0xda, 0xde, 0xd4, 0x62, 0x78, 0x96, 0xf7, 0xf1, 0x5f, 0xa6, - 0x20, 0x57, 0xef, 0xc4, 0xbd, 0xa4, 0xac, 0x86, 0x7b, 0x38, 0x0a, 0x08, 0x85, 0x1e, 0x1a, 0xd1, - 0xc1, 0xa1, 0x15, 0x3c, 0x79, 0xb1, 0x15, 0xbc, 0x4e, 0x3c, 0x65, 0x9e, 0x66, 0x21, 0x39, 0x04, - 0xf3, 0x84, 0x9f, 0x4f, 0xfd, 0x14, 0x89, 0xf0, 0xf8, 0xe7, 0x28, 0x68, 0xa0, 0xc9, 0xa7, 0xd4, - 0x21, 0x67, 0xa3, 0x6c, 0x6a, 0xfc, 0x51, 0x36, 0x8d, 0x0d, 0x8d, 0x2e, 0x6a, 0x61, 0x8b, 0x3a, - 0xfd, 0xe6, 0x16, 0x75, 0xc1, 0xe1, 0x83, 0xf5, 0x63, 0x48, 0x6a, 0xba, 0xdb, 0x39, 0xe3, 0x2f, - 0xd5, 0x84, 0xe9, 0x9c, 0x51, 0x9b, 0x0a, 0x8e, 0x5a, 0x36, 0x4a, 0x16, 0xea, 0x00, 0xbe, 0x6e, - 0xd0, 0x12, 0x4c, 0x99, 0x6d, 0xcd, 0x3d, 0x48, 0x92, 0x2b, 0x67, 0xce, 0x4e, 0x17, 0x27, 0x9f, - 0xb7, 0xb5, 0xfa, 0x9a, 0x34, 0x69, 0xb6, 0xb5, 0xba, 0x46, 0x73, 0x5c, 0xe0, 0x23, 0xd9, 0x8b, - 0x32, 0x9b, 0x91, 0xa6, 0x0d, 0x7c, 0xb4, 0x86, 0x6d, 0x95, 0x0f, 0xb8, 0xdf, 0x16, 0x20, 0xef, - 0xea, 0x3e, 0x5e, 0xa3, 0x92, 0xd6, 0x3b, 0x7c, 0x92, 0x25, 0x2f, 0x36, 0xc9, 0x5c, 0x3e, 0x7e, - 0xbc, 0xf6, 0x57, 0x05, 0x1e, 0x37, 0xdc, 0x50, 0x15, 0x87, 0x38, 0x15, 0x31, 0x4e, 0x8c, 0x7b, - 0x50, 0xb4, 0x14, 0x43, 0x33, 0x3b, 0xfa, 0x17, 0x98, 0x6d, 0x84, 0xda, 0xfc, 0xad, 0x65, 0xc1, - 0xa3, 0xd3, 0x5d, 0x3f, 0x5b, 0xfc, 0xe3, 0x04, 0x8f, 0x31, 0xf6, 0xaa, 0x11, 0xa7, 0xba, 0xbe, - 0x03, 0xb3, 0xfd, 0x09, 0x47, 0xdc, 0xd9, 0xfa, 0x5e, 0x84, 0xbc, 0xa8, 0x8a, 0xb0, 0x58, 0x41, - 0x37, 0x70, 0xbd, 0x2f, 0xf9, 0x88, 0x8d, 0x2a, 0x90, 0x0d, 0xe6, 0x31, 0x49, 0x8e, 0x9d, 0xc7, - 0x04, 0x2c, 0x2f, 0x7b, 0xc9, 0xc2, 0xcf, 0xc3, 0x24, 0xbd, 0xfd, 0x06, 0x26, 0x9a, 0xf7, 0xe6, - 0x9f, 0x27, 0xe0, 0x26, 0xad, 0xfd, 0x4b, 0x6c, 0xe9, 0x7b, 0x27, 0xdb, 0x96, 0xe9, 0x60, 0xd5, - 0xc1, 0x9a, 0x7f, 0x0a, 0x24, 0x56, 0xbb, 0x97, 0xe9, 0xba, 0x0f, 0xb8, 0x50, 0xbc, 0x98, 0xc7, - 0x85, 0x36, 0xa0, 0xc0, 0x23, 0x03, 0x94, 0xb6, 0x7e, 0x88, 0x65, 0xc5, 0xb9, 0xc8, 0xea, 0x96, - 0x63, 0xbc, 0xab, 0x84, 0x75, 0xd5, 0x41, 0x1a, 0x64, 0xb8, 0x30, 0x5d, 0xe3, 0xe9, 0x77, 0x9e, - 0xfe, 0x64, 0x1b, 0x8a, 0x69, 0x16, 0x9e, 0x50, 0x5f, 0x93, 0xd2, 0x4c, 0x72, 0x5d, 0x13, 0xff, - 0xb3, 0x00, 0xb7, 0xce, 0x51, 0x71, 0x9c, 0x43, 0x77, 0x01, 0xd2, 0x87, 0xe4, 0x41, 0x3a, 0xd7, - 0x71, 0x5a, 0xf2, 0xae, 0xd1, 0x26, 0xe4, 0xf6, 0x14, 0xbd, 0xed, 0x0f, 0xe9, 0xe1, 0xe1, 0x85, - 0xd1, 0x91, 0xae, 0x33, 0x8c, 0x9d, 0x8d, 0x61, 0xf1, 0x37, 0x13, 0x30, 0xbb, 0xaa, 0x69, 0x8d, - 0x06, 0xb7, 0x81, 0xf1, 0x8d, 0x14, 0x17, 0x64, 0x26, 0x7c, 0x90, 0x89, 0xde, 0x03, 0xa4, 0xe9, - 0x36, 0x4b, 0xf3, 0x61, 0xef, 0x2b, 0x9a, 0x79, 0xe4, 0x87, 0x55, 0xcc, 0xba, 0x77, 0x1a, 0xee, - 0x0d, 0xd4, 0x00, 0x8a, 0x76, 0x64, 0xdb, 0x51, 0xbc, 0xf7, 0x46, 0xb7, 0xc6, 0x3a, 0xae, 0xc5, - 0x60, 0x90, 0x77, 0x29, 0x65, 0x88, 0x1c, 0xfa, 0x93, 0xf8, 0xed, 0x3a, 0x69, 0xba, 0x23, 0x2b, - 0xb6, 0x7b, 0x36, 0x87, 0x25, 0x18, 0xc9, 0x33, 0xfa, 0xaa, 0xcd, 0x8e, 0xdc, 0xb0, 0xc3, 0x04, - 0xbe, 0x6a, 0xe2, 0x84, 0xc4, 0xff, 0x40, 0x80, 0xbc, 0x84, 0xf7, 0x2c, 0x6c, 0xc7, 0xba, 0x29, - 0xf0, 0x04, 0x66, 0x2c, 0x26, 0x55, 0xde, 0xb3, 0xcc, 0xce, 0x45, 0xe6, 0x55, 0x96, 0x33, 0x3e, - 0xb1, 0xcc, 0x0e, 0x37, 0x2c, 0x2f, 0xa1, 0xe0, 0xd5, 0x31, 0xce, 0xc6, 0xff, 0x3e, 0x3d, 0x8a, - 0xcc, 0x04, 0xc7, 0x1d, 0xdf, 0x10, 0xaf, 0x06, 0xe8, 0x8b, 0xaa, 0x60, 0x45, 0xe3, 0x54, 0xc3, - 0x7f, 0x17, 0x20, 0xdf, 0xe8, 0xed, 0xb2, 0x3c, 0x52, 0xf1, 0x69, 0xa0, 0x0a, 0x99, 0x36, 0xde, - 0x73, 0xe4, 0x37, 0x0a, 0x83, 0x4f, 0x13, 0x56, 0x7a, 0x08, 0xe0, 0x29, 0x80, 0x45, 0x8f, 0xaf, - 0x51, 0x39, 0xc9, 0x0b, 0xca, 0xc9, 0x50, 0x5e, 0x42, 0x16, 0xff, 0x69, 0x02, 0x0a, 0x5e, 0x33, - 0xe3, 0xb4, 0x92, 0xaf, 0x42, 0xd6, 0x21, 0x79, 0x11, 0xeb, 0x30, 0xcb, 0x43, 0x3a, 0xa2, 0x2d, - 0xc4, 0x32, 0xcc, 0x51, 0xc7, 0x45, 0x56, 0xba, 0xdd, 0xb6, 0xee, 0xc2, 0x5d, 0x6a, 0x7f, 0x52, - 0xd2, 0x2c, 0xbd, 0xb5, 0xca, 0xee, 0x50, 0xa0, 0x4b, 0xc6, 0xdc, 0x9e, 0x85, 0xf1, 0x17, 0x58, - 0xa6, 0xc8, 0xeb, 0x22, 0x21, 0x2b, 0x59, 0xc6, 0xd8, 0x20, 0x7c, 0x7c, 0xcc, 0xbd, 0x82, 0x59, - 0xaa, 0xd3, 0xb8, 0x8f, 0xdd, 0x8a, 0x7f, 0x2f, 0x01, 0x28, 0x28, 0xf9, 0xab, 0xeb, 0x8b, 0x44, - 0x7c, 0x7d, 0xf1, 0x2e, 0x20, 0x16, 0xb4, 0x68, 0xcb, 0x5d, 0x6c, 0xc9, 0x36, 0x56, 0x4d, 0x9e, - 0xe3, 0x48, 0x90, 0x8a, 0xfc, 0xce, 0x36, 0xb6, 0x1a, 0x94, 0x8e, 0x1e, 0x03, 0xf8, 0x1e, 0x19, - 0x5f, 0x30, 0x46, 0x3a, 0x64, 0x52, 0xc6, 0x73, 0xc5, 0xc4, 0xef, 0x2d, 0xc0, 0x0c, 0xd7, 0xe1, - 0x8e, 0xa1, 0x9b, 0x06, 0x7a, 0x00, 0xc9, 0x16, 0xdf, 0xfa, 0xcf, 0x46, 0x6e, 0xd3, 0xf9, 0x49, - 0xdc, 0x6a, 0x13, 0x12, 0x29, 0x4b, 0x58, 0xba, 0x3d, 0x27, 0xc2, 0x31, 0xf2, 0x03, 0xb1, 0x83, - 0x2c, 0xdd, 0x9e, 0x83, 0x1a, 0x50, 0x50, 0xfd, 0x24, 0x55, 0x32, 0x61, 0x4f, 0x0e, 0x05, 0x51, - 0x91, 0x69, 0xc1, 0x6a, 0x13, 0x52, 0x5e, 0x0d, 0xdd, 0x40, 0x95, 0x60, 0x6e, 0xa4, 0xd4, 0x40, - 0x94, 0x97, 0x7f, 0xbe, 0x37, 0x9c, 0x97, 0xa9, 0x36, 0x11, 0x48, 0xa1, 0x84, 0x3e, 0x86, 0x29, - 0x8d, 0x66, 0xe1, 0xe1, 0x23, 0x3a, 0x6a, 0xd0, 0x85, 0x92, 0x1d, 0xd5, 0x26, 0x24, 0xce, 0x81, - 0xd6, 0x61, 0x86, 0xfd, 0x62, 0x6e, 0x0a, 0x47, 0x96, 0xb7, 0x86, 0x4b, 0x08, 0x2c, 0x04, 0xb5, - 0x09, 0x29, 0xab, 0xf9, 0x54, 0xf4, 0x4d, 0x48, 0xd9, 0xaa, 0xe2, 0x62, 0xcb, 0x6b, 0x43, 0x52, - 0x6a, 0xf8, 0xcc, 0xb4, 0x34, 0x7a, 0xcc, 0xd2, 0x32, 0x3a, 0xc7, 0xee, 0x36, 0x5f, 0x54, 0xf5, - 0x43, 0x47, 0xb7, 0x49, 0xf5, 0x31, 0x25, 0xa0, 0xa7, 0x90, 0x55, 0x88, 0xbf, 0x27, 0xd3, 0xc3, - 0x8f, 0x74, 0x5f, 0x2f, 0xfa, 0xe5, 0xfa, 0xc0, 0x61, 0xd5, 0x1a, 0x3d, 0x1f, 0xee, 0x12, 0x7d, - 0x41, 0x1d, 0x6c, 0xb5, 0x70, 0x29, 0x3b, 0x5a, 0x50, 0x30, 0xb6, 0xcb, 0x13, 0x44, 0x89, 0xc4, - 0xef, 0xf3, 0x4e, 0x1e, 0xd3, 0x46, 0xcd, 0x0c, 0x7d, 0x9d, 0x1b, 0x71, 0x6c, 0xa7, 0x36, 0x21, - 0xcd, 0xec, 0x07, 0xc8, 0x68, 0x19, 0x12, 0x2d, 0xb5, 0x94, 0x1b, 0x3a, 0x43, 0xbc, 0xa3, 0x29, - 0xb5, 0x09, 0x29, 0xd1, 0x52, 0xd1, 0xa7, 0x90, 0x66, 0xe7, 0x0c, 0x8e, 0x8d, 0x52, 0x7e, 0xa8, - 0x9d, 0x08, 0x9f, 0xd6, 0xa8, 0x4d, 0x48, 0xf4, 0x68, 0x03, 0x79, 0xde, 0x36, 0xe4, 0x2d, 0x16, - 0x1c, 0xe7, 0x86, 0xb2, 0x16, 0x87, 0xbe, 0xe2, 0x8e, 0x8a, 0x66, 0xad, 0x51, 0xcf, 0x3f, 0x40, - 0x47, 0xdf, 0x81, 0xf9, 0xb0, 0x44, 0x3e, 0xd2, 0x66, 0x87, 0xbe, 0xae, 0x1d, 0x1a, 0x59, 0x59, - 0x9b, 0x90, 0x90, 0x35, 0x70, 0x13, 0x7d, 0x08, 0x93, 0xac, 0xd7, 0x10, 0x15, 0x19, 0x15, 0xb5, - 0xd1, 0xd7, 0x61, 0xac, 0x3c, 0x19, 0xfc, 0x0e, 0x8f, 0x10, 0x93, 0xdb, 0x66, 0xab, 0x34, 0x37, - 0x74, 0xf0, 0x0f, 0xc6, 0xba, 0x91, 0xc1, 0xef, 0xf8, 0x54, 0xd2, 0xef, 0x16, 0xbb, 0xc3, 0xc3, - 0x8a, 0xe6, 0x87, 0xf6, 0x7b, 0x44, 0xe0, 0x58, 0x8d, 0xc6, 0xee, 0xfb, 0x64, 0x52, 0x35, 0x8b, - 0x65, 0x81, 0x91, 0xe9, 0x9c, 0xba, 0x34, 0xb4, 0x6a, 0x83, 0x69, 0x72, 0x6a, 0xd4, 0x47, 0xf2, - 0xa8, 0xe8, 0x25, 0x14, 0x79, 0xae, 0x06, 0xff, 0x9d, 0xc2, 0x65, 0x2a, 0xef, 0x5e, 0xa4, 0xe9, - 0x8a, 0x8a, 0xc9, 0xa9, 0x4d, 0x48, 0x05, 0x35, 0x7c, 0x07, 0x7d, 0x06, 0xb3, 0x54, 0x9e, 0xac, - 0xfa, 0xe9, 0x35, 0x4a, 0xa5, 0x81, 0x64, 0x0d, 0xc3, 0x33, 0x71, 0xb8, 0x92, 0x8b, 0x6a, 0xdf, - 0x2d, 0x32, 0x8c, 0x75, 0x43, 0x77, 0xa8, 0x95, 0x5d, 0x18, 0x3a, 0x8c, 0xc3, 0x49, 0xfd, 0xc8, - 0x30, 0xd6, 0x19, 0x85, 0x0c, 0x63, 0x87, 0xc7, 0x9c, 0xf1, 0xee, 0x78, 0x7b, 0xe8, 0x30, 0x8e, - 0x0a, 0x4e, 0x23, 0xc3, 0xd8, 0x09, 0xd2, 0xc9, 0x30, 0x66, 0x06, 0xa2, 0x4f, 0xee, 0x3b, 0x43, - 0x87, 0xf1, 0xd0, 0x43, 0xc8, 0x64, 0x18, 0x2b, 0x03, 0x37, 0xd1, 0x1a, 0x00, 0x73, 0x67, 0xe8, - 0xa2, 0x78, 0x6d, 0xe8, 0x62, 0xd0, 0x1f, 0x7b, 0x46, 0x16, 0x83, 0xb6, 0x4b, 0x23, 0x86, 0x8c, - 0x82, 0x25, 0x99, 0xbe, 0x42, 0x2d, 0x2d, 0x0e, 0x35, 0x64, 0x03, 0xaf, 0x3b, 0x89, 0x21, 0x3b, - 0xf2, 0x88, 0x64, 0x55, 0x61, 0x7b, 0xbe, 0xa5, 0xa5, 0xe1, 0x66, 0x39, 0xf8, 0xea, 0x87, 0x9a, - 0x65, 0x4a, 0x40, 0xab, 0x90, 0x21, 0x6b, 0xfe, 0x09, 0x35, 0x43, 0xd7, 0x87, 0xfa, 0xa4, 0x7d, - 0x87, 0x53, 0x6a, 0x13, 0x52, 0xfa, 0x35, 0x27, 0x91, 0xc7, 0xb3, 0xdd, 0xb0, 0x92, 0x38, 0xf4, - 0xf1, 0xa1, 0x9d, 0x53, 0xf2, 0x78, 0xc6, 0x81, 0x54, 0xb8, 0xc4, 0xfa, 0x8a, 0x9f, 0x07, 0xb6, - 0xf8, 0x11, 0xd6, 0xd2, 0x0d, 0x2a, 0x6a, 0xe8, 0xb6, 0x52, 0xe4, 0x31, 0xe5, 0xda, 0x84, 0x34, - 0xa7, 0x0c, 0xde, 0x25, 0x13, 0x9e, 0x2f, 0x3d, 0x6c, 0x33, 0xaa, 0x74, 0x73, 0xe8, 0x84, 0x8f, - 0xd8, 0xc3, 0x23, 0x13, 0x5e, 0x09, 0x90, 0xd9, 0x02, 0xa4, 0xc9, 0xb6, 0xcd, 0x5e, 0xb8, 0xdf, - 0x1a, 0xb1, 0x00, 0xf5, 0xed, 0x02, 0xb0, 0x05, 0x48, 0x6b, 0x30, 0x4e, 0x22, 0x48, 0x6d, 0x63, - 0xc5, 0xe2, 0x66, 0xf6, 0xf6, 0x50, 0x41, 0x03, 0xf9, 0xf3, 0x88, 0x20, 0xd5, 0x23, 0x12, 0x87, - 0xc7, 0x72, 0x93, 0xbc, 0x70, 0x87, 0xf1, 0xce, 0x50, 0x87, 0x27, 0x32, 0x0b, 0x0d, 0x71, 0x78, - 0xac, 0xd0, 0x0d, 0xf4, 0x73, 0x30, 0xcd, 0xe1, 0x5b, 0xe9, 0xee, 0x08, 0x37, 0x36, 0x88, 0xb8, - 0xc9, 0xbc, 0xe6, 0x3c, 0xcc, 0xca, 0x32, 0xd8, 0xc8, 0x9a, 0x77, 0x6f, 0x84, 0x95, 0x1d, 0x40, - 0xae, 0xcc, 0xca, 0xfa, 0x64, 0x62, 0x65, 0xd9, 0x38, 0xe5, 0x6b, 0xdd, 0xfd, 0xa1, 0x56, 0x76, - 0xf0, 0xe8, 0x0b, 0xb1, 0xb2, 0xaf, 0x7d, 0x2a, 0x69, 0x99, 0xcd, 0xe0, 0x53, 0xe9, 0x1b, 0x43, - 0x5b, 0x16, 0xc6, 0x91, 0xa4, 0x65, 0x9c, 0x87, 0x74, 0x1b, 0x73, 0x89, 0x99, 0xa6, 0xdf, 0x1d, - 0x7e, 0xe8, 0xbe, 0x1f, 0x74, 0xd4, 0xdc, 0x8d, 0x4a, 0xa6, 0x61, 0xcf, 0x50, 0x59, 0xfc, 0xd0, - 0x31, 0xd7, 0xd4, 0x7b, 0xa3, 0x0d, 0x55, 0xd4, 0x49, 0x6a, 0xcf, 0x50, 0x85, 0x6e, 0xd2, 0xaa, - 0xb2, 0x73, 0x66, 0x74, 0x7e, 0x2f, 0x8f, 0xc8, 0x0f, 0xd0, 0x77, 0xda, 0x8f, 0x56, 0xd5, 0x23, - 0xfa, 0x53, 0xa8, 0xc7, 0x92, 0x57, 0x94, 0x56, 0x46, 0x4f, 0xa1, 0x70, 0xfa, 0x0c, 0x6f, 0x0a, - 0x71, 0xb2, 0xb7, 0x66, 0xba, 0x1e, 0xc6, 0xfb, 0xa3, 0xd7, 0xcc, 0x7e, 0xd7, 0x82, 0xad, 0x99, - 0xdc, 0xa7, 0xf8, 0xeb, 0x02, 0x2c, 0xb1, 0xba, 0xd1, 0x1d, 0xbd, 0x13, 0xd9, 0xdb, 0x17, 0x0d, - 0x9c, 0x79, 0x78, 0x40, 0x1f, 0xf0, 0xe1, 0xb0, 0xea, 0x9e, 0xb3, 0xcf, 0x5b, 0x9b, 0x90, 0xde, - 0x51, 0x46, 0x95, 0x2b, 0x4f, 0xf3, 0xd7, 0xa2, 0xde, 0x21, 0xcf, 0x42, 0xb1, 0xb8, 0x9e, 0x4a, - 0x5f, 0x29, 0x96, 0xd6, 0x53, 0xe9, 0xab, 0xc5, 0x85, 0xf5, 0x54, 0xfa, 0xad, 0xe2, 0xdb, 0xe2, - 0x5f, 0x5e, 0x85, 0x9c, 0x8b, 0xfc, 0x18, 0x22, 0x7a, 0x18, 0x44, 0x44, 0xd7, 0x86, 0x21, 0x22, - 0x8e, 0x15, 0x39, 0x24, 0x7a, 0x18, 0x84, 0x44, 0xd7, 0x86, 0x41, 0x22, 0x9f, 0x87, 0x60, 0xa2, - 0xe6, 0x30, 0x4c, 0x74, 0x6f, 0x0c, 0x4c, 0xe4, 0x89, 0xea, 0x07, 0x45, 0x6b, 0x83, 0xa0, 0xe8, - 0xe6, 0x68, 0x50, 0xe4, 0x89, 0x0a, 0xa0, 0xa2, 0xc7, 0x7d, 0xa8, 0xe8, 0xfa, 0x08, 0x54, 0xe4, - 0xf1, 0xbb, 0xb0, 0x68, 0x23, 0x12, 0x16, 0xdd, 0x3e, 0x0f, 0x16, 0x79, 0x72, 0x42, 0xb8, 0xe8, - 0x83, 0x10, 0x2e, 0x5a, 0x1c, 0x8a, 0x8b, 0x3c, 0x6e, 0x06, 0x8c, 0x3e, 0xe9, 0x07, 0x46, 0xd7, - 0x47, 0x00, 0x23, 0xbf, 0x05, 0x1c, 0x19, 0xd5, 0xa2, 0x90, 0xd1, 0xad, 0x73, 0x90, 0x91, 0x27, - 0x25, 0x08, 0x8d, 0x6a, 0x51, 0xd0, 0xe8, 0xd6, 0x39, 0xd0, 0xa8, 0x4f, 0x12, 0xc3, 0x46, 0x5b, - 0xd1, 0xd8, 0xe8, 0xce, 0xb9, 0xd8, 0xc8, 0x93, 0x16, 0x06, 0x47, 0x2b, 0x01, 0x70, 0xf4, 0xce, - 0x10, 0x70, 0xe4, 0xb1, 0x12, 0x74, 0xf4, 0xad, 0x01, 0x74, 0x24, 0x8e, 0x42, 0x47, 0x1e, 0xaf, - 0x07, 0x8f, 0x5e, 0x0c, 0x81, 0x47, 0x77, 0xcf, 0x87, 0x47, 0x9e, 0xb0, 0x3e, 0x7c, 0xa4, 0x8c, - 0xc4, 0x47, 0xef, 0x8d, 0x89, 0x8f, 0x3c, 0xe9, 0x51, 0x00, 0xe9, 0xa3, 0x30, 0x40, 0x5a, 0x1a, - 0x0e, 0x90, 0x3c, 0x31, 0x1c, 0x21, 0x6d, 0x44, 0x22, 0xa4, 0xdb, 0xe7, 0x21, 0x24, 0x7f, 0x1e, - 0x04, 0x21, 0xd2, 0x56, 0x34, 0x44, 0xba, 0x73, 0x2e, 0x44, 0xf2, 0xbb, 0x3f, 0x84, 0x91, 0x36, - 0x22, 0x31, 0xd2, 0xed, 0xf3, 0x30, 0x92, 0x5f, 0xb9, 0x20, 0x48, 0x7a, 0x35, 0x14, 0x24, 0xdd, - 0x1f, 0x07, 0x24, 0x79, 0x42, 0x07, 0x50, 0xd2, 0xe7, 0xc3, 0x51, 0xd2, 0x37, 0x2e, 0x90, 0xaf, - 0x30, 0x12, 0x26, 0x7d, 0x6b, 0x00, 0x26, 0x89, 0xa3, 0x60, 0x92, 0x3f, 0x9e, 0x5d, 0x9c, 0xa4, - 0x8c, 0x44, 0x35, 0xef, 0x8d, 0x89, 0x6a, 0xfc, 0xc1, 0x17, 0x01, 0x6b, 0xaa, 0x11, 0xb0, 0xe6, - 0xe6, 0x68, 0x58, 0xe3, 0x9b, 0x73, 0x1f, 0xd7, 0xd4, 0xa2, 0x70, 0xcd, 0xad, 0x73, 0x70, 0x8d, - 0x6f, 0x85, 0x02, 0xc0, 0xe6, 0x71, 0x1f, 0xb0, 0xb9, 0x7e, 0x6e, 0xd4, 0x4f, 0x00, 0xd9, 0x94, - 0x07, 0x91, 0xcd, 0x8d, 0x91, 0xc8, 0xc6, 0x93, 0xe0, 0x43, 0x9b, 0xc7, 0x7d, 0xd0, 0xe6, 0xfa, - 0x08, 0x68, 0xe3, 0x57, 0x80, 0x63, 0x1b, 0x6d, 0x34, 0xb6, 0x59, 0x1e, 0x17, 0xdb, 0x78, 0x82, - 0x23, 0xc1, 0xcd, 0x56, 0x34, 0xb8, 0xb9, 0x33, 0xe6, 0x0b, 0xf9, 0x01, 0x74, 0x53, 0x8b, 0x42, - 0x37, 0xb7, 0xce, 0x41, 0x37, 0xc1, 0x35, 0xc4, 0x83, 0x37, 0xb5, 0x28, 0x78, 0x73, 0xeb, 0x1c, - 0x78, 0xe3, 0x4b, 0x0a, 0xe0, 0x9b, 0xe6, 0x30, 0x7c, 0x73, 0x6f, 0x0c, 0x7c, 0xe3, 0x3b, 0x2f, - 0x7d, 0x00, 0xe7, 0xd3, 0x7e, 0x80, 0x23, 0x8e, 0x02, 0x38, 0xfe, 0x8c, 0x74, 0x11, 0xce, 0x56, - 0x34, 0xc2, 0xb9, 0x73, 0x2e, 0xc2, 0x09, 0x1a, 0xc9, 0x00, 0xc4, 0xd9, 0x88, 0x84, 0x38, 0xb7, - 0xcf, 0x83, 0x38, 0xbe, 0x91, 0x0c, 0x62, 0x9c, 0x4f, 0xfb, 0x31, 0x8e, 0x38, 0x0a, 0xe3, 0xf8, - 0x8d, 0x73, 0x41, 0x4e, 0x2d, 0x0a, 0xe4, 0xdc, 0x3a, 0x07, 0xe4, 0xf8, 0x9d, 0x17, 0x40, 0x39, - 0xca, 0x48, 0x94, 0xf3, 0xde, 0x98, 0x28, 0xa7, 0xcf, 0x70, 0x85, 0x61, 0x4e, 0x2d, 0x0a, 0xe6, - 0xdc, 0x3a, 0x07, 0xe6, 0x04, 0x2a, 0xeb, 0xe3, 0x9c, 0xad, 0x68, 0x9c, 0x73, 0xe7, 0x5c, 0x9c, - 0xd3, 0x37, 0x9b, 0x5c, 0xa0, 0xb3, 0x11, 0x09, 0x74, 0x6e, 0x9f, 0x07, 0x74, 0xfa, 0x16, 0x3e, - 0xee, 0x1c, 0xfc, 0xca, 0xf8, 0x48, 0xe7, 0xa3, 0x8b, 0x23, 0x1d, 0xef, 0x99, 0xb1, 0x40, 0x9d, - 0xf5, 0x54, 0xfa, 0xed, 0xe2, 0x3b, 0xe2, 0xdf, 0x9a, 0x86, 0xa9, 0x9a, 0x17, 0xe7, 0xe2, 0xd7, - 0x52, 0x78, 0x93, 0xbc, 0x48, 0x68, 0x8d, 0xcc, 0x58, 0x6a, 0xf7, 0xce, 0x4f, 0x75, 0x37, 0x98, - 0x8c, 0x8d, 0xb3, 0xbe, 0xc1, 0x01, 0x65, 0xf4, 0x01, 0xe4, 0x7a, 0x36, 0xb6, 0xe4, 0xae, 0xa5, - 0x9b, 0x96, 0xee, 0xb0, 0x53, 0x1c, 0x42, 0xb9, 0xf8, 0xe5, 0xe9, 0xe2, 0xcc, 0x8e, 0x8d, 0xad, - 0x6d, 0x4e, 0x97, 0x66, 0x7a, 0x81, 0x2b, 0xf7, 0x43, 0x51, 0x93, 0xe3, 0x7f, 0x28, 0xea, 0x05, - 0x14, 0x2d, 0xac, 0x68, 0x21, 0x0f, 0x84, 0xe5, 0x20, 0x8a, 0x1e, 0x33, 0xf4, 0x94, 0x95, 0x5b, - 0x92, 0xe6, 0x22, 0x2a, 0x58, 0x61, 0x22, 0x7a, 0x00, 0x97, 0x3a, 0xca, 0x31, 0x8d, 0x89, 0x94, - 0x5d, 0xa7, 0x8e, 0xc6, 0x39, 0xb2, 0x6f, 0x30, 0xa1, 0x8e, 0x72, 0x4c, 0xbf, 0x3a, 0xc5, 0x6e, - 0xd1, 0x4f, 0x49, 0xdc, 0x82, 0xbc, 0xa6, 0xdb, 0x8e, 0x6e, 0xa8, 0x0e, 0xcf, 0x3d, 0xcb, 0xf2, - 0xb6, 0xe6, 0x5c, 0x2a, 0x4b, 0x30, 0x7b, 0x1f, 0x66, 0x79, 0xb0, 0x7c, 0xe0, 0x15, 0x21, 0xcf, - 0xdf, 0xca, 0x6e, 0x78, 0x6f, 0x05, 0x51, 0x05, 0x0a, 0x2d, 0xc5, 0xc1, 0x47, 0xca, 0x89, 0xec, - 0x1e, 0xc5, 0xca, 0xd2, 0x94, 0x8c, 0x6f, 0x9d, 0x9d, 0x2e, 0xe6, 0x9e, 0xb2, 0x5b, 0x03, 0x27, - 0xb2, 0x72, 0xad, 0xc0, 0x0d, 0x0d, 0xdd, 0x81, 0x82, 0x62, 0x9f, 0x18, 0x2a, 0x55, 0x0f, 0x36, - 0xec, 0x9e, 0x4d, 0x21, 0x45, 0x5a, 0xca, 0x53, 0x72, 0xc5, 0xa5, 0xa2, 0xeb, 0x30, 0xc3, 0x23, - 0xc9, 0xd9, 0x67, 0x6c, 0x0a, 0xb4, 0xa9, 0xfc, 0xbb, 0x09, 0xf4, 0x4b, 0x36, 0xe8, 0x31, 0x2c, - 0xf0, 0xdc, 0xf2, 0x47, 0x8a, 0xa5, 0xc9, 0x54, 0xeb, 0xfe, 0xf8, 0x2c, 0x52, 0xb1, 0x57, 0x58, - 0x2e, 0x79, 0x52, 0x80, 0xa8, 0xda, 0xcf, 0x84, 0xb0, 0x05, 0xb3, 0x6a, 0x5b, 0xf7, 0x10, 0x00, - 0x6b, 0xf9, 0xec, 0x50, 0x3b, 0x5b, 0xa1, 0x65, 0xfd, 0x57, 0xa4, 0x05, 0x35, 0x4c, 0x40, 0x0d, - 0xa0, 0xd9, 0x5e, 0xe4, 0xae, 0xd9, 0xd6, 0xd5, 0x13, 0xea, 0xfc, 0x87, 0x73, 0x64, 0x8f, 0xcc, - 0x54, 0xff, 0x4a, 0xd1, 0x9d, 0x6d, 0xca, 0x29, 0xc1, 0x91, 0xf7, 0x9b, 0xe5, 0xb7, 0x5d, 0x4f, - 0xa5, 0x67, 0x8a, 0xb9, 0xf5, 0x54, 0x3a, 0x5f, 0x2c, 0x88, 0x7f, 0x5b, 0x80, 0x42, 0x5f, 0x5d, - 0x50, 0x0d, 0x2e, 0x69, 0xde, 0x54, 0x91, 0xf9, 0x51, 0x23, 0xdd, 0x34, 0x78, 0xda, 0xef, 0xb9, - 0x2f, 0x4f, 0x17, 0x0b, 0xb4, 0xf4, 0x53, 0xef, 0x96, 0x34, 0xef, 0x73, 0xf8, 0x54, 0xf4, 0x11, - 0xe4, 0x99, 0xfb, 0xe8, 0x7d, 0x99, 0x8d, 0xc6, 0x88, 0x97, 0x67, 0xbf, 0x3c, 0x5d, 0xcc, 0x51, - 0x9f, 0xd1, 0x4d, 0xcb, 0x2b, 0xe5, 0xda, 0xc1, 0x4b, 0xf1, 0x37, 0x04, 0x98, 0x09, 0x1d, 0xe7, - 0x79, 0xdc, 0xf7, 0x06, 0xfd, 0x6a, 0x34, 0xee, 0x1c, 0x16, 0x50, 0x97, 0xe6, 0xe3, 0xdc, 0x8d, - 0x4e, 0x5c, 0x1c, 0x8e, 0x5b, 0xe8, 0x2e, 0x8c, 0x1b, 0xaa, 0xe1, 0xb2, 0x7d, 0x9c, 0xfa, 0xfe, - 0x0f, 0x16, 0x27, 0xc4, 0x3f, 0x48, 0x41, 0x2e, 0x7c, 0x78, 0xa7, 0xde, 0x57, 0xaf, 0xa8, 0x75, - 0x21, 0xc4, 0xb1, 0x3c, 0x22, 0x6d, 0x61, 0xc6, 0xcf, 0xaf, 0xcf, 0xaa, 0xb9, 0x34, 0x22, 0x4e, - 0x20, 0x58, 0x4f, 0x9f, 0x71, 0xe1, 0x7b, 0x49, 0xcf, 0xbe, 0x2e, 0xc3, 0x24, 0x4d, 0xa3, 0xc3, - 0xab, 0x56, 0xea, 0x1f, 0x3d, 0xc4, 0x57, 0x26, 0xf7, 0x25, 0x56, 0x8c, 0xd8, 0xe3, 0xe6, 0x1b, - 0xe5, 0xa9, 0xf3, 0xa7, 0xc1, 0xc5, 0x3f, 0x84, 0xc7, 0xf3, 0x14, 0x4e, 0x5e, 0x2c, 0x4f, 0x21, - 0xfa, 0x25, 0x28, 0xa8, 0x66, 0xbb, 0xcd, 0xd6, 0x3a, 0x66, 0x91, 0x06, 0x33, 0x8f, 0x50, 0x11, - 0xfc, 0xdb, 0x85, 0xcb, 0xde, 0x37, 0x0c, 0x97, 0x25, 0xfe, 0x0d, 0xc3, 0x40, 0xac, 0x67, 0xde, - 0x13, 0xc6, 0x0c, 0x59, 0x5f, 0xd8, 0xe9, 0xf4, 0x9b, 0x84, 0x9d, 0xb2, 0x50, 0x65, 0x3e, 0x72, - 0xfe, 0x54, 0xe0, 0x81, 0x21, 0xcf, 0x4c, 0xf3, 0xa0, 0xe7, 0x85, 0x8b, 0x2e, 0x04, 0xb3, 0x06, - 0xa6, 0xbf, 0x3c, 0x5d, 0x4c, 0x49, 0x5e, 0xda, 0xc0, 0x28, 0xcb, 0x9f, 0xf8, 0xc9, 0x2c, 0xff, - 0x75, 0x98, 0xe9, 0x5a, 0x78, 0x0f, 0x3b, 0xea, 0xbe, 0x6c, 0xf4, 0x3a, 0xfc, 0x54, 0x49, 0xd6, - 0xa5, 0x6d, 0xf5, 0x3a, 0xe8, 0x1e, 0x14, 0xbd, 0x22, 0x1c, 0x63, 0xbb, 0x69, 0xa5, 0x5c, 0x3a, - 0x47, 0xe4, 0xe2, 0xff, 0x16, 0x60, 0x2e, 0xd4, 0x26, 0x3e, 0x27, 0xd6, 0x21, 0xeb, 0x9b, 0x03, - 0xbb, 0x24, 0x5c, 0x30, 0x78, 0x32, 0xc8, 0x8c, 0x64, 0xb8, 0xec, 0x3e, 0x96, 0xe6, 0x94, 0xf7, - 0xc5, 0x26, 0x2e, 0x28, 0xf6, 0x92, 0x2f, 0x67, 0x2d, 0xf0, 0x00, 0x6f, 0x92, 0x24, 0xc7, 0x9a, - 0x24, 0xe2, 0x6f, 0x0b, 0x50, 0xa4, 0x0f, 0x78, 0x82, 0xb1, 0x16, 0x8b, 0x75, 0x72, 0x83, 0x92, - 0x13, 0xe3, 0x9f, 0x1b, 0x09, 0x7d, 0x07, 0x23, 0x19, 0xfe, 0x0e, 0x86, 0xf8, 0x03, 0x01, 0xf2, - 0x5e, 0x0d, 0xd9, 0xd7, 0xe0, 0x46, 0x24, 0xa7, 0x7c, 0xb3, 0x2f, 0xa1, 0xb9, 0x59, 0x38, 0xc6, - 0xfa, 0x40, 0x5d, 0x30, 0x0b, 0x07, 0xfb, 0x82, 0xd7, 0xdf, 0x75, 0x47, 0x0e, 0xa9, 0x62, 0xc5, - 0xcf, 0xb6, 0xf0, 0x06, 0x47, 0x68, 0x24, 0xfa, 0x21, 0x4d, 0xb3, 0x7d, 0xc8, 0x12, 0x9f, 0x8c, - 0x65, 0xb6, 0x10, 0x0f, 0x87, 0x02, 0xbe, 0x1b, 0xa7, 0x35, 0x1b, 0xf4, 0x13, 0x9b, 0xec, 0xb7, - 0x2d, 0x3e, 0x09, 0x28, 0x90, 0x76, 0x3e, 0xd1, 0xd2, 0x58, 0xa6, 0xd4, 0xd5, 0x12, 0x1b, 0x2b, - 0x7f, 0x12, 0xec, 0x89, 0xea, 0x21, 0x41, 0x61, 0x8f, 0x20, 0x79, 0xa8, 0xb4, 0x47, 0x85, 0x81, - 0x85, 0x7a, 0x4e, 0x22, 0xa5, 0xd1, 0x93, 0x50, 0x92, 0x8a, 0xc4, 0x70, 0xc4, 0x30, 0xa8, 0xd2, - 0x50, 0x32, 0x8b, 0x0f, 0xc3, 0x63, 0x7d, 0xe4, 0xe3, 0x83, 0x83, 0xfe, 0xe3, 0xd4, 0x0f, 0x7f, - 0xb0, 0x28, 0x88, 0x9f, 0x00, 0x92, 0xb0, 0x8d, 0x9d, 0x17, 0x3d, 0xd3, 0xf2, 0x13, 0x7e, 0xdc, - 0xee, 0xfb, 0x30, 0xc8, 0x64, 0x39, 0x7b, 0x16, 0x95, 0xd2, 0xe7, 0x12, 0xcc, 0x85, 0xb8, 0x99, - 0xb1, 0x10, 0x3f, 0x84, 0xab, 0x4f, 0x4d, 0xdb, 0xd6, 0xbb, 0x04, 0x7a, 0xd2, 0x59, 0x49, 0x96, - 0x06, 0xcf, 0x3c, 0xa6, 0xbb, 0x74, 0x13, 0xc2, 0x60, 0x66, 0x24, 0x23, 0x79, 0xd7, 0xe2, 0x1f, - 0x0a, 0x70, 0x65, 0x90, 0x93, 0x69, 0x39, 0xea, 0xc4, 0xdf, 0xb4, 0x6a, 0xfa, 0xf9, 0xdd, 0xce, - 0x1f, 0xad, 0x6e, 0x71, 0xe2, 0x62, 0xf2, 0x67, 0xca, 0x1d, 0x85, 0x9a, 0x0f, 0x7e, 0xfa, 0x38, - 0xcf, 0xc9, 0x9b, 0x8c, 0xea, 0x5b, 0x92, 0xd4, 0x78, 0x96, 0xa4, 0x09, 0x85, 0x75, 0x53, 0x37, - 0x88, 0x27, 0xeb, 0xb6, 0x77, 0x15, 0xf2, 0xbb, 0xba, 0xa1, 0x58, 0x27, 0xb2, 0x7b, 0xa8, 0x9b, - 0x0d, 0x94, 0x85, 0xa8, 0xca, 0xb2, 0x12, 0x52, 0x8e, 0x71, 0xf0, 0x4b, 0xf1, 0x47, 0x02, 0x14, - 0x7d, 0xb1, 0xdc, 0x22, 0xbf, 0x0b, 0xa0, 0xb6, 0x7b, 0xb6, 0x83, 0x2d, 0xb7, 0x97, 0x66, 0x58, - 0xf4, 0x76, 0x85, 0x51, 0xeb, 0x6b, 0x52, 0x86, 0x17, 0xa8, 0x6b, 0xe8, 0x46, 0x38, 0x39, 0xc2, - 0x64, 0x19, 0xce, 0x06, 0x52, 0x22, 0x90, 0x6e, 0xb7, 0x1d, 0xd3, 0xf2, 0x30, 0x13, 0xef, 0x76, - 0x37, 0x95, 0x3a, 0x3d, 0xd3, 0x4b, 0xca, 0xad, 0x42, 0x9e, 0x2c, 0xf7, 0x87, 0xd8, 0x6b, 0x52, - 0xea, 0xfc, 0x26, 0x31, 0x0e, 0xb7, 0x49, 0xbf, 0x47, 0x9c, 0x54, 0xd6, 0x1b, 0x5e, 0x0f, 0x8f, - 0xb0, 0x68, 0x3f, 0x13, 0x4c, 0x49, 0x38, 0x5e, 0xd2, 0x46, 0xea, 0x89, 0x7c, 0x0b, 0xd2, 0xee, - 0x27, 0x8a, 0xf9, 0x04, 0xb9, 0xba, 0xcc, 0xbe, 0x61, 0xbc, 0xec, 0x7e, 0xc3, 0x78, 0x79, 0x8d, - 0x17, 0x60, 0x66, 0xfc, 0xfb, 0xff, 0x6d, 0x51, 0x90, 0x3c, 0xa6, 0xfb, 0x0d, 0x32, 0xc2, 0x07, - 0x56, 0x61, 0x94, 0x07, 0x08, 0x7c, 0x43, 0x86, 0x7f, 0xb5, 0x76, 0x75, 0x4d, 0xde, 0xd9, 0xaa, - 0x3c, 0xdf, 0xdc, 0xac, 0x37, 0x9b, 0xd5, 0xb5, 0xa2, 0x80, 0x8a, 0x30, 0x13, 0xfa, 0x02, 0x4d, - 0x82, 0x7d, 0xc7, 0xf6, 0xfe, 0xcf, 0x00, 0xf8, 0x1f, 0xb3, 0x22, 0xb2, 0x36, 0xaa, 0x9f, 0xc9, - 0x2f, 0x57, 0x9f, 0xed, 0x54, 0x1b, 0xc5, 0x09, 0x84, 0x20, 0x5f, 0x5e, 0x6d, 0x56, 0x6a, 0xb2, - 0x54, 0x6d, 0x6c, 0x3f, 0xdf, 0x6a, 0x54, 0xdd, 0xef, 0xdf, 0xde, 0x5f, 0x83, 0x99, 0x60, 0xba, - 0x1b, 0x34, 0x07, 0x85, 0x4a, 0xad, 0x5a, 0xd9, 0x90, 0x5f, 0xd6, 0x57, 0xe5, 0x17, 0x3b, 0xd5, - 0x9d, 0x6a, 0x71, 0x82, 0x56, 0x8d, 0x12, 0x9f, 0xec, 0x3c, 0x7b, 0x56, 0x14, 0x50, 0x01, 0xb2, - 0xec, 0x9a, 0x7e, 0xad, 0xa6, 0x98, 0xb8, 0xbf, 0x09, 0xd9, 0x40, 0x5a, 0x5b, 0xf2, 0xb8, 0xed, - 0x9d, 0x46, 0x4d, 0x6e, 0xd6, 0x37, 0xab, 0x8d, 0xe6, 0xea, 0xe6, 0x36, 0x93, 0x41, 0x69, 0xab, - 0xe5, 0xe7, 0x52, 0xb3, 0x28, 0x78, 0xd7, 0xcd, 0xe7, 0x3b, 0x95, 0x9a, 0xdb, 0x0c, 0x31, 0x95, - 0x4e, 0x16, 0x93, 0xf7, 0xff, 0x9a, 0x00, 0x57, 0x86, 0xa4, 0x7e, 0x41, 0x59, 0x98, 0xde, 0x31, - 0x68, 0x8e, 0xcf, 0xe2, 0x04, 0xca, 0x05, 0xb2, 0xbf, 0x14, 0x05, 0x94, 0x66, 0xf9, 0x37, 0x8a, - 0x09, 0x34, 0x05, 0x89, 0xc6, 0xa3, 0x62, 0x92, 0xd4, 0x34, 0x90, 0x3c, 0xa5, 0x98, 0x42, 0x19, - 0x9e, 0xb6, 0xa1, 0x38, 0x89, 0x66, 0xfc, 0xec, 0x09, 0xc5, 0x29, 0x22, 0xca, 0xcb, 0x42, 0x50, - 0x9c, 0xbe, 0x7f, 0x1d, 0x02, 0x27, 0xbd, 0x11, 0xc0, 0xd4, 0x33, 0xc5, 0xc1, 0xb6, 0x53, 0x9c, - 0x40, 0xd3, 0x90, 0x5c, 0x6d, 0xb7, 0x8b, 0xc2, 0xc3, 0x7f, 0x9d, 0x82, 0xb4, 0xfb, 0x55, 0x16, - 0xf4, 0x0c, 0x26, 0xd9, 0xd6, 0xf3, 0xe2, 0x70, 0xcf, 0x9e, 0x4e, 0xde, 0x85, 0xa5, 0xf3, 0x5c, - 0x7f, 0x71, 0x02, 0xfd, 0x15, 0xc8, 0x06, 0x3c, 0x26, 0x34, 0x74, 0xfb, 0x2c, 0xe4, 0x25, 0x2e, - 0xdc, 0x3e, 0xaf, 0x98, 0x27, 0xff, 0x15, 0x64, 0x3c, 0x0b, 0x8e, 0x6e, 0x8c, 0xb2, 0xef, 0xae, - 0xec, 0xd1, 0x8b, 0x00, 0x99, 0x6b, 0xe2, 0xc4, 0xfb, 0x02, 0xb2, 0x00, 0x0d, 0x1a, 0x5b, 0x14, - 0x15, 0x91, 0x30, 0xd4, 0x9a, 0x2f, 0xdc, 0x1f, 0xab, 0xb4, 0xff, 0x4c, 0xa2, 0x2c, 0x7f, 0xc5, - 0x88, 0x56, 0xd6, 0xc0, 0x7a, 0x14, 0xad, 0xac, 0x88, 0x85, 0x67, 0x02, 0xbd, 0x80, 0x14, 0xb1, - 0x94, 0x28, 0xca, 0x87, 0xec, 0xb3, 0xcc, 0x0b, 0x37, 0x46, 0x96, 0x71, 0x45, 0x96, 0xef, 0xfd, - 0xf0, 0x2f, 0xae, 0x4d, 0xfc, 0xf0, 0xec, 0x9a, 0xf0, 0xa3, 0xb3, 0x6b, 0xc2, 0x9f, 0x9d, 0x5d, - 0x13, 0xfe, 0xfc, 0xec, 0x9a, 0xf0, 0xeb, 0x3f, 0xbe, 0x36, 0xf1, 0xa3, 0x1f, 0x5f, 0x9b, 0xf8, - 0xb3, 0x1f, 0x5f, 0x9b, 0xf8, 0x7c, 0x9a, 0x73, 0xef, 0x4e, 0x51, 0xa3, 0xf2, 0xe8, 0xff, 0x05, - 0x00, 0x00, 0xff, 0xff, 0x2a, 0xe5, 0xf1, 0xd1, 0x53, 0x7e, 0x00, 0x00, +func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_e319494d56330d3c) } + +var fileDescriptor_api_e319494d56330d3c = []byte{ + // 8186 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x7d, 0x5f, 0x6c, 0x23, 0x49, + 0x7a, 0x9f, 0x9a, 0xa4, 0x24, 0xf2, 0xa3, 0xf8, 0x47, 0x25, 0xcd, 0x0c, 0x87, 0xbb, 0x3b, 0xd2, + 0xf4, 0xfc, 0x9f, 0xdb, 0x95, 0x76, 0x66, 0x6e, 0xbd, 0x9b, 0x9d, 0xf5, 0x9e, 0x45, 0x8a, 0x33, + 0xa4, 0x34, 0xd2, 0x68, 0x9a, 0xd4, 0xcc, 0xed, 0xfa, 0x9c, 0xbe, 0x56, 0x77, 0x89, 0xea, 0x13, + 0xd9, 0xcd, 0xe9, 0x6e, 0xea, 0xcf, 0x02, 0x01, 0x72, 0xf6, 0x19, 0x77, 0x08, 0x82, 0x83, 0x1f, + 0x9c, 0xe4, 0x0c, 0x23, 0xf1, 0x19, 0xb6, 0x63, 0x03, 0x49, 0x90, 0x00, 0x89, 0x93, 0x3c, 0x04, + 0xf1, 0x8b, 0x91, 0x1c, 0x02, 0x07, 0x39, 0x07, 0x08, 0x62, 0xe4, 0x41, 0x4e, 0x74, 0xc9, 0x43, + 0x10, 0xc0, 0x08, 0xf2, 0x12, 0x60, 0x1f, 0x0c, 0xa3, 0xfe, 0xf4, 0x1f, 0x92, 0x4d, 0x8a, 0x9a, + 0xeb, 0xfd, 0x03, 0xdc, 0x8b, 0xc4, 0xfe, 0xaa, 0xbe, 0xaf, 0xab, 0xbe, 0xaa, 0xfa, 0xea, 0xfb, + 0x55, 0x7d, 0x55, 0x0d, 0xb3, 0x96, 0xa9, 0xa8, 0x7b, 0x9d, 0x9d, 0x65, 0xa5, 0xa3, 0x2f, 0x75, + 0x2c, 0xd3, 0x31, 0xd1, 0xac, 0x6a, 0xaa, 0xfb, 0x94, 0xbc, 0xc4, 0x13, 0x8b, 0x77, 0xf7, 0x0f, + 0x96, 0xf7, 0x0f, 0x6c, 0x6c, 0x1d, 0x60, 0x6b, 0x59, 0x35, 0x0d, 0xb5, 0x6b, 0x59, 0xd8, 0x50, + 0x8f, 0x97, 0x5b, 0xa6, 0xba, 0x4f, 0xff, 0xe8, 0x46, 0x93, 0xb1, 0x17, 0x91, 0x2b, 0x51, 0x53, + 0x1c, 0x85, 0xd3, 0xe6, 0x5d, 0x1a, 0xb6, 0x2c, 0xd3, 0xb2, 0x39, 0xf5, 0xa2, 0x4b, 0x6d, 0x63, + 0x47, 0x09, 0xe4, 0x7e, 0xcd, 0x76, 0x4c, 0x4b, 0x69, 0xe2, 0x65, 0x6c, 0x34, 0x75, 0x03, 0x93, + 0x0c, 0x07, 0xaa, 0xca, 0x13, 0x5f, 0x0f, 0x4d, 0x7c, 0xc0, 0x53, 0x0b, 0x5d, 0x47, 0x6f, 0x2d, + 0xef, 0xb5, 0xd4, 0x65, 0x47, 0x6f, 0x63, 0xdb, 0x51, 0xda, 0x1d, 0x9e, 0x72, 0x97, 0xa6, 0x38, + 0x96, 0xa2, 0xea, 0x46, 0xd3, 0xfd, 0xdf, 0xd9, 0x59, 0xb6, 0xb0, 0x6a, 0x5a, 0x1a, 0xd6, 0x64, + 0xbb, 0xa3, 0x18, 0x6e, 0x71, 0x9b, 0x66, 0xd3, 0xa4, 0x3f, 0x97, 0xc9, 0x2f, 0x4e, 0xbd, 0xd2, + 0x34, 0xcd, 0x66, 0x0b, 0x2f, 0xd3, 0xa7, 0x9d, 0xee, 0xee, 0xb2, 0xd6, 0xb5, 0x14, 0x47, 0x37, + 0x39, 0x97, 0xf8, 0x2f, 0x05, 0xc8, 0x48, 0xf8, 0x65, 0x17, 0xdb, 0x4e, 0x15, 0x2b, 0x1a, 0xb6, + 0xd0, 0x65, 0x88, 0xef, 0xe3, 0xe3, 0x42, 0x7c, 0x51, 0xb8, 0x3d, 0x53, 0x9a, 0xfe, 0xf4, 0x64, + 0x21, 0xbe, 0x8e, 0x8f, 0x25, 0x42, 0x43, 0x8b, 0x30, 0x8d, 0x0d, 0x4d, 0x26, 0xc9, 0x89, 0xde, + 0xe4, 0x29, 0x6c, 0x68, 0xeb, 0xf8, 0x18, 0x7d, 0x03, 0x92, 0x36, 0x91, 0x66, 0xa8, 0xb8, 0x30, + 0xb9, 0x28, 0xdc, 0x9e, 0x2c, 0xfd, 0xc2, 0xa7, 0x27, 0x0b, 0x1f, 0x34, 0x75, 0x67, 0xaf, 0xbb, + 0xb3, 0xa4, 0x9a, 0xed, 0x65, 0xaf, 0x9d, 0xb4, 0x1d, 0xff, 0xf7, 0x72, 0x67, 0xbf, 0xb9, 0xdc, + 0xaf, 0xa3, 0xa5, 0xc6, 0x91, 0x51, 0xc7, 0x2f, 0x25, 0x4f, 0xe2, 0x5a, 0x22, 0x29, 0xe4, 0x63, + 0x6b, 0x89, 0x64, 0x2c, 0x1f, 0x17, 0x7f, 0x27, 0x0e, 0x59, 0x09, 0xdb, 0x1d, 0xd3, 0xb0, 0x31, + 0x2f, 0xf9, 0xdb, 0x10, 0x77, 0x8e, 0x0c, 0x5a, 0xf2, 0xf4, 0xfd, 0x2b, 0x4b, 0x03, 0x3d, 0x62, + 0xa9, 0x61, 0x29, 0x86, 0xad, 0xa8, 0xa4, 0xfa, 0x12, 0xc9, 0x8a, 0xde, 0x83, 0xb4, 0x85, 0xed, + 0x6e, 0x1b, 0x53, 0x45, 0xd2, 0x4a, 0xa5, 0xef, 0x5f, 0x0a, 0xe1, 0xac, 0x77, 0x14, 0x43, 0x02, + 0x96, 0x97, 0xfc, 0x46, 0x97, 0x21, 0x69, 0x74, 0xdb, 0x44, 0x15, 0x36, 0xad, 0x68, 0x5c, 0x9a, + 0x36, 0xba, 0xed, 0x75, 0x7c, 0x6c, 0xa3, 0xaf, 0xc3, 0x45, 0x0d, 0x77, 0x2c, 0xac, 0x2a, 0x0e, + 0xd6, 0x64, 0x4b, 0x31, 0x9a, 0x58, 0xd6, 0x8d, 0x5d, 0xd3, 0x2e, 0x4c, 0x2d, 0xc6, 0x6f, 0xa7, + 0xef, 0xbf, 0x1e, 0x22, 0x5f, 0x22, 0xb9, 0x6a, 0xc6, 0xae, 0x59, 0x4a, 0xfc, 0xe8, 0x64, 0x61, + 0x42, 0x9a, 0xf7, 0x25, 0x78, 0x49, 0x36, 0xaa, 0x43, 0x86, 0x17, 0xd7, 0xc2, 0x8a, 0x6d, 0x1a, + 0x85, 0xe9, 0x45, 0xe1, 0x76, 0xf6, 0xfe, 0x52, 0x98, 0xc0, 0x1e, 0xd5, 0x90, 0xc7, 0x6e, 0x1b, + 0x4b, 0x94, 0x4b, 0x9a, 0xb1, 0x02, 0x4f, 0xe8, 0x35, 0x48, 0x91, 0x9a, 0xec, 0x1c, 0x3b, 0xd8, + 0x2e, 0x24, 0x69, 0x55, 0x48, 0xd5, 0x4a, 0xe4, 0x59, 0xfc, 0x10, 0x66, 0x82, 0xac, 0x08, 0x41, + 0x56, 0xaa, 0xd4, 0xb7, 0x37, 0x2a, 0xf2, 0xf6, 0xe6, 0xfa, 0xe6, 0xd3, 0x17, 0x9b, 0xf9, 0x09, + 0x34, 0x0f, 0x79, 0x4e, 0x5b, 0xaf, 0x7c, 0x24, 0x3f, 0xa9, 0x6d, 0xd4, 0x1a, 0x79, 0xa1, 0x98, + 0xf8, 0xde, 0xef, 0x5c, 0x99, 0x10, 0xb7, 0x00, 0x1e, 0x63, 0x87, 0x77, 0x30, 0x54, 0x82, 0xa9, + 0x3d, 0x5a, 0x9e, 0x82, 0x40, 0x35, 0xbd, 0x18, 0x5a, 0xf0, 0x40, 0x67, 0x2c, 0x25, 0x89, 0x36, + 0x7e, 0x7c, 0xb2, 0x20, 0x48, 0x9c, 0x53, 0xfc, 0x23, 0x01, 0xd2, 0x54, 0x24, 0xab, 0x1f, 0x2a, + 0xf7, 0xc9, 0xbc, 0x7a, 0xa6, 0x32, 0x06, 0x85, 0xa2, 0x25, 0x98, 0x3c, 0x50, 0x5a, 0x5d, 0x5c, + 0x88, 0x51, 0x19, 0x85, 0x10, 0x19, 0xcf, 0x49, 0xba, 0xc4, 0xb2, 0xa1, 0x87, 0x30, 0xa3, 0x1b, + 0x0e, 0x36, 0x1c, 0x99, 0xb1, 0xc5, 0xcf, 0x60, 0x4b, 0xb3, 0xdc, 0xf4, 0x41, 0xfc, 0x17, 0x02, + 0xc0, 0x56, 0x37, 0x4a, 0xa5, 0xa0, 0xaf, 0x8e, 0x59, 0x7e, 0xde, 0xbb, 0x78, 0x2d, 0x2e, 0xc2, + 0x94, 0x6e, 0xb4, 0x74, 0x83, 0x95, 0x3f, 0x29, 0xf1, 0x27, 0x34, 0x0f, 0x93, 0x3b, 0x2d, 0xdd, + 0xd0, 0xe8, 0x78, 0x48, 0x4a, 0xec, 0x41, 0x94, 0x20, 0x4d, 0x4b, 0x1d, 0xa1, 0xde, 0xc5, 0x7f, + 0x17, 0x83, 0x0b, 0x65, 0xd3, 0xd0, 0x74, 0x32, 0x24, 0x95, 0xd6, 0x97, 0x42, 0x2b, 0x6b, 0x10, + 0x18, 0x7c, 0x32, 0x3e, 0xea, 0x8c, 0xd9, 0xc6, 0xc8, 0xe7, 0xaa, 0x1c, 0x75, 0x28, 0x2d, 0x5c, + 0x93, 0xe8, 0xab, 0x70, 0x49, 0x69, 0xb5, 0xcc, 0x43, 0x59, 0xdf, 0x95, 0x35, 0x13, 0xdb, 0xb2, + 0x61, 0x3a, 0x32, 0x3e, 0xd2, 0x6d, 0x87, 0x9a, 0x92, 0xa4, 0x34, 0x47, 0x93, 0x6b, 0xbb, 0xab, + 0x26, 0xb6, 0x37, 0x4d, 0xa7, 0x42, 0x92, 0xc8, 0x38, 0x25, 0x85, 0x61, 0xe3, 0x74, 0x8a, 0x98, + 0x5f, 0x29, 0x89, 0x8f, 0x3a, 0x6c, 0x9c, 0xfe, 0x12, 0x5c, 0xec, 0xd7, 0x63, 0x94, 0xed, 0xf4, + 0x27, 0x02, 0x64, 0x6b, 0x86, 0xee, 0x7c, 0x29, 0x1a, 0xc8, 0x53, 0x6a, 0x3c, 0xa8, 0xd4, 0xbb, + 0x90, 0xdf, 0x55, 0xf4, 0xd6, 0x53, 0xa3, 0x61, 0xb6, 0x77, 0x6c, 0xc7, 0x34, 0xb0, 0xcd, 0xb5, + 0x3e, 0x40, 0x17, 0x9f, 0x43, 0xce, 0xab, 0x4d, 0x94, 0x6a, 0x72, 0x20, 0x5f, 0x33, 0x54, 0x0b, + 0xb7, 0xb1, 0x11, 0xa9, 0x9e, 0x5e, 0x87, 0x94, 0xee, 0xca, 0xa5, 0xba, 0x8a, 0x4b, 0x3e, 0x41, + 0xec, 0xc2, 0x6c, 0xe0, 0xad, 0x51, 0x9a, 0x45, 0x32, 0x35, 0xe0, 0x43, 0xd9, 0x6f, 0x23, 0x32, + 0x35, 0xe0, 0x43, 0x66, 0xc6, 0xea, 0x90, 0x59, 0xc5, 0x2d, 0xec, 0xe0, 0x28, 0xad, 0xfb, 0x36, + 0x64, 0x5d, 0xa1, 0x51, 0x36, 0xcc, 0xdf, 0x13, 0x00, 0x71, 0xb9, 0x64, 0x36, 0x8d, 0xb2, 0x6d, + 0x16, 0x88, 0x0b, 0xe1, 0x74, 0x2d, 0x83, 0xf9, 0x02, 0xac, 0x4f, 0x02, 0x23, 0x51, 0x77, 0xc0, + 0xb7, 0xb2, 0x89, 0xa0, 0x95, 0xe5, 0x6e, 0xcc, 0x21, 0xcc, 0xf5, 0x14, 0x2c, 0xda, 0xe6, 0x4b, + 0xd0, 0x32, 0xc5, 0x16, 0xe3, 0x41, 0x5f, 0x8d, 0x12, 0xc5, 0x3f, 0x17, 0x60, 0xb6, 0xdc, 0xc2, + 0x8a, 0x15, 0xb9, 0x46, 0x7e, 0x55, 0x80, 0xa4, 0x86, 0x15, 0x8d, 0xd6, 0x99, 0x8d, 0xec, 0x37, + 0x02, 0x62, 0x88, 0x4b, 0xbb, 0xb4, 0xd7, 0x52, 0x97, 0x1a, 0xae, 0xb3, 0x5b, 0x5a, 0x23, 0x32, + 0x3e, 0x3d, 0x59, 0x28, 0xbd, 0xb2, 0x9f, 0xe8, 0xc9, 0x92, 0xbc, 0x57, 0x8b, 0x1f, 0x01, 0x0a, + 0x56, 0x30, 0xca, 0xfe, 0xf4, 0xbf, 0x04, 0x40, 0x12, 0x3e, 0xc0, 0x96, 0x13, 0xb9, 0xf6, 0xfe, + 0x96, 0x00, 0x69, 0x47, 0xb1, 0x9a, 0xd8, 0x91, 0x09, 0x1a, 0xf8, 0xfc, 0x15, 0x08, 0xec, 0xed, + 0x84, 0x20, 0x7e, 0x0c, 0x73, 0x3d, 0xd5, 0x8c, 0x52, 0x87, 0xff, 0x57, 0x80, 0x74, 0x5d, 0x55, + 0x8c, 0x28, 0x95, 0xf7, 0x21, 0xa4, 0x6d, 0x55, 0x31, 0xe4, 0x5d, 0xd3, 0x6a, 0x2b, 0x0e, 0x1d, + 0x70, 0xd9, 0x1e, 0xdd, 0x79, 0xfe, 0xbc, 0xaa, 0x18, 0x8f, 0x68, 0x26, 0x09, 0x6c, 0xef, 0x37, + 0x7a, 0x06, 0xe9, 0x7d, 0x7c, 0x2c, 0x73, 0x6c, 0x48, 0x67, 0xe3, 0xec, 0xfd, 0xb7, 0x03, 0xfc, + 0xfb, 0x07, 0x4b, 0x2e, 0xa4, 0x5c, 0x0a, 0x40, 0xca, 0x25, 0xc2, 0xb1, 0x54, 0x77, 0x2c, 0x6c, + 0x34, 0x9d, 0x3d, 0x09, 0xf6, 0xf1, 0xf1, 0x13, 0x26, 0x83, 0x0d, 0xf3, 0xb5, 0x44, 0x32, 0x9e, + 0x4f, 0x88, 0xff, 0x5f, 0x80, 0x19, 0x56, 0xe5, 0x28, 0x87, 0xf9, 0x3b, 0x90, 0xb0, 0xcc, 0x43, + 0x36, 0xcc, 0xd3, 0xf7, 0x5f, 0x0b, 0x11, 0xb1, 0x8e, 0x8f, 0x83, 0xf3, 0x28, 0xcd, 0x8e, 0x4a, + 0xc0, 0xbd, 0x52, 0x99, 0x72, 0xc7, 0xc7, 0xe5, 0x06, 0xc6, 0x25, 0x11, 0x19, 0xb7, 0x20, 0xb7, + 0xa3, 0x38, 0xea, 0x9e, 0x6c, 0xf1, 0x42, 0x92, 0x39, 0x37, 0x7e, 0x7b, 0x46, 0xca, 0x52, 0xb2, + 0x5b, 0x74, 0x9b, 0xd4, 0x9c, 0x0d, 0x18, 0x1b, 0xff, 0x8c, 0xb5, 0xf9, 0x5f, 0x0a, 0x7c, 0x0c, + 0xb9, 0x35, 0xff, 0x59, 0x6b, 0xfa, 0xdf, 0x88, 0xc1, 0xa5, 0xf2, 0x1e, 0x56, 0xf7, 0xcb, 0xa6, + 0x61, 0xeb, 0xb6, 0x43, 0x74, 0x17, 0x65, 0xfb, 0xbf, 0x06, 0xa9, 0x43, 0xdd, 0xd9, 0x93, 0x35, + 0x7d, 0x77, 0x97, 0x5a, 0xcb, 0xa4, 0x94, 0x24, 0x84, 0x55, 0x7d, 0x77, 0x17, 0x3d, 0x80, 0x44, + 0xdb, 0xd4, 0x98, 0xf3, 0x9e, 0xbd, 0xbf, 0x10, 0x22, 0x9e, 0x16, 0xcd, 0xee, 0xb6, 0x37, 0x4c, + 0x0d, 0x4b, 0x34, 0x33, 0xba, 0x02, 0xa0, 0x12, 0x6a, 0xc7, 0xd4, 0x0d, 0x87, 0xcf, 0xda, 0x01, + 0x0a, 0xaa, 0x42, 0xca, 0xc1, 0x56, 0x5b, 0x37, 0x14, 0x07, 0x17, 0x26, 0xa9, 0xf2, 0xae, 0x87, + 0x16, 0xbc, 0xd3, 0xd2, 0x55, 0x65, 0x15, 0xdb, 0xaa, 0xa5, 0x77, 0x1c, 0xd3, 0xe2, 0x5a, 0xf4, + 0x99, 0xc5, 0xef, 0x27, 0xa0, 0x30, 0xa8, 0x9b, 0x28, 0x7b, 0xc8, 0x16, 0x4c, 0x11, 0xb4, 0xdf, + 0x72, 0x78, 0x1f, 0xb9, 0x3f, 0x4c, 0x05, 0x21, 0x25, 0xa0, 0xab, 0x06, 0x2d, 0x87, 0x17, 0x9b, + 0xcb, 0x29, 0xfe, 0x1b, 0x01, 0xa6, 0x58, 0x02, 0xba, 0x07, 0x49, 0xbe, 0xbc, 0xa1, 0xd1, 0x32, + 0xc6, 0x4b, 0x17, 0x4f, 0x4f, 0x16, 0xa6, 0xd9, 0x8a, 0xc5, 0xea, 0xa7, 0xfe, 0x4f, 0x69, 0x9a, + 0xe6, 0xab, 0x69, 0xa4, 0xb5, 0x6c, 0x47, 0xb1, 0x1c, 0xba, 0x88, 0x14, 0x63, 0x28, 0x86, 0x12, + 0xd6, 0xf1, 0x31, 0x5a, 0x83, 0x29, 0xdb, 0x51, 0x9c, 0xae, 0xcd, 0xdb, 0xeb, 0x5c, 0x85, 0xad, + 0x53, 0x4e, 0x89, 0x4b, 0x20, 0x6e, 0x97, 0x86, 0x1d, 0x45, 0x6f, 0xd1, 0x06, 0x4c, 0x49, 0xfc, + 0x49, 0xfc, 0x4d, 0x01, 0xa6, 0x58, 0x56, 0x74, 0x09, 0xe6, 0xa4, 0x95, 0xcd, 0xc7, 0x15, 0xb9, + 0xb6, 0xb9, 0x5a, 0x69, 0x54, 0xa4, 0x8d, 0xda, 0xe6, 0x4a, 0xa3, 0x92, 0x9f, 0x40, 0x17, 0x01, + 0xb9, 0x09, 0xe5, 0xa7, 0x9b, 0xf5, 0x5a, 0xbd, 0x51, 0xd9, 0x6c, 0xe4, 0x05, 0xba, 0xd2, 0x41, + 0xe9, 0x01, 0x6a, 0x0c, 0x5d, 0x87, 0xc5, 0x7e, 0xaa, 0x5c, 0x6f, 0xac, 0x34, 0xea, 0x72, 0xa5, + 0xde, 0xa8, 0x6d, 0xac, 0x34, 0x2a, 0xab, 0xf9, 0xf8, 0x88, 0x5c, 0xe4, 0x25, 0x92, 0x54, 0x29, + 0x37, 0xf2, 0x09, 0xd1, 0x81, 0x0b, 0x12, 0x56, 0xcd, 0x76, 0xa7, 0xeb, 0x60, 0x52, 0x4a, 0x3b, + 0xca, 0x91, 0x72, 0x09, 0xa6, 0x35, 0xeb, 0x58, 0xb6, 0xba, 0x06, 0x1f, 0x27, 0x53, 0x9a, 0x75, + 0x2c, 0x75, 0x0d, 0xf1, 0x9f, 0x09, 0x70, 0xb1, 0xff, 0xb5, 0x51, 0x76, 0xc2, 0x67, 0x90, 0x56, + 0x34, 0x0d, 0x6b, 0xb2, 0x86, 0x5b, 0x8e, 0xc2, 0x5d, 0x9a, 0xbb, 0x01, 0x49, 0xdc, 0x23, 0x59, + 0xf2, 0x3c, 0x92, 0x8d, 0xe7, 0xe5, 0x32, 0x2d, 0xc8, 0x2a, 0xe1, 0x70, 0xcd, 0x0f, 0x15, 0x42, + 0x29, 0xe2, 0x1f, 0x4e, 0x42, 0xa6, 0x62, 0x68, 0x8d, 0xa3, 0x48, 0xe7, 0x92, 0x8b, 0x30, 0xa5, + 0x9a, 0xed, 0xb6, 0xee, 0xb8, 0x0a, 0x62, 0x4f, 0xe8, 0xdb, 0x41, 0x97, 0x36, 0x3e, 0x8e, 0x47, + 0xf6, 0x28, 0x6a, 0x77, 0x16, 0x7d, 0x13, 0x2e, 0x11, 0xf3, 0x6b, 0x19, 0x4a, 0x4b, 0x66, 0xc5, + 0x92, 0x1d, 0x4b, 0x6f, 0x36, 0xb1, 0xc5, 0xd7, 0x2d, 0x6f, 0x87, 0x54, 0xb8, 0xc6, 0x39, 0xca, + 0x94, 0xa1, 0xc1, 0xf2, 0x4b, 0x17, 0xf4, 0x30, 0x32, 0xfa, 0x00, 0x80, 0xcc, 0x69, 0x74, 0x2d, + 0xd4, 0xe6, 0x86, 0x6d, 0xd8, 0x62, 0xa8, 0x6b, 0xcb, 0x08, 0x03, 0x79, 0xb6, 0xd1, 0x32, 0x01, + 0x42, 0x2f, 0xbb, 0xba, 0x85, 0xe5, 0x7b, 0x1d, 0x95, 0xae, 0x50, 0x24, 0x4b, 0xd9, 0xd3, 0x93, + 0x05, 0x90, 0x18, 0xf9, 0xde, 0x56, 0x99, 0x00, 0x23, 0xf6, 0xbb, 0xa3, 0xa2, 0x17, 0x70, 0x27, + 0xb0, 0xd0, 0x42, 0xa6, 0x70, 0x5e, 0x2d, 0xc5, 0x91, 0xf7, 0xf4, 0xe6, 0x1e, 0xb6, 0x64, 0x6f, + 0x3d, 0x9c, 0x2e, 0x4c, 0x26, 0xa5, 0xeb, 0x3e, 0x43, 0x59, 0x31, 0x58, 0xe9, 0x57, 0x9c, 0x2a, + 0xcd, 0xec, 0xe9, 0x8c, 0xb4, 0x62, 0xc7, 0xd4, 0x6d, 0xd3, 0x28, 0xa4, 0x58, 0x2b, 0xb2, 0x27, + 0x74, 0x07, 0xf2, 0xce, 0x91, 0x21, 0xef, 0x61, 0xc5, 0x72, 0x76, 0xb0, 0xe2, 0x90, 0xe9, 0x1e, + 0x68, 0x8e, 0x9c, 0x73, 0x64, 0x54, 0x03, 0x64, 0xf4, 0x0c, 0xf2, 0xba, 0x21, 0xef, 0xb6, 0xf4, + 0xe6, 0x9e, 0x23, 0x1f, 0x5a, 0xba, 0x83, 0xed, 0xc2, 0x2c, 0x55, 0x48, 0xd8, 0x00, 0xa8, 0xf3, + 0x05, 0x6a, 0xed, 0x05, 0xc9, 0xc9, 0x55, 0x93, 0xd5, 0x8d, 0x47, 0x94, 0x9f, 0x12, 0xed, 0xb5, + 0x44, 0x72, 0x3a, 0x9f, 0x14, 0x7f, 0x37, 0x06, 0x59, 0xb7, 0xdf, 0x46, 0x39, 0xc4, 0x6e, 0x43, + 0xde, 0x34, 0xb0, 0xdc, 0xd9, 0x53, 0x6c, 0xcc, 0xf5, 0xc8, 0x67, 0xae, 0xac, 0x69, 0xe0, 0x2d, + 0x42, 0x66, 0xea, 0x42, 0x7f, 0x57, 0x80, 0x59, 0xdb, 0x51, 0x9a, 0xba, 0xd1, 0x0c, 0xe8, 0x77, + 0xf2, 0xf3, 0x86, 0x19, 0x79, 0x5e, 0x06, 0x8f, 0xd2, 0xe3, 0x35, 0xfd, 0xa9, 0x00, 0xb3, 0x2b, + 0x5a, 0x5b, 0x37, 0xea, 0x9d, 0x96, 0x1e, 0xe9, 0x5a, 0xca, 0x75, 0x48, 0xd9, 0x44, 0xa6, 0x3f, + 0x01, 0xf9, 0xc8, 0x38, 0x49, 0x53, 0xc8, 0x4c, 0xf4, 0x04, 0x72, 0xf8, 0xa8, 0xa3, 0xb3, 0xad, + 0x12, 0x06, 0xc4, 0x12, 0xe3, 0x68, 0x88, 0x37, 0xbd, 0xcf, 0x4b, 0x92, 0x78, 0x9d, 0x3e, 0x02, + 0x14, 0xac, 0x52, 0x94, 0x58, 0xea, 0x23, 0x98, 0xa3, 0xa2, 0xb7, 0x0d, 0x3b, 0x62, 0x7d, 0x89, + 0xbf, 0x08, 0xf3, 0xbd, 0xa2, 0xa3, 0x2c, 0xf7, 0x0b, 0xde, 0xca, 0x1b, 0xd8, 0x8a, 0x14, 0x45, + 0x7b, 0xba, 0xe6, 0x82, 0xa3, 0x2c, 0xf3, 0x77, 0x04, 0xb8, 0x4c, 0x65, 0xd3, 0xdd, 0xa4, 0x5d, + 0x6c, 0x3d, 0xc1, 0x8a, 0x1d, 0xe9, 0x12, 0xc0, 0x35, 0x98, 0x62, 0x18, 0x9c, 0xf6, 0xcf, 0xc9, + 0x52, 0x9a, 0x78, 0x52, 0x75, 0xc7, 0xb4, 0x88, 0x27, 0xc5, 0x93, 0x44, 0x05, 0x8a, 0x61, 0xa5, + 0x88, 0xb2, 0xa6, 0xff, 0x40, 0x80, 0x59, 0xee, 0xc4, 0x92, 0xae, 0x5c, 0xde, 0x23, 0x3e, 0x1c, + 0xaa, 0x40, 0x5a, 0xa5, 0xbf, 0x64, 0xe7, 0xb8, 0x83, 0xa9, 0xfc, 0xec, 0x28, 0xff, 0x97, 0xb1, + 0x35, 0x8e, 0x3b, 0x98, 0x38, 0xd1, 0xee, 0x6f, 0xa2, 0xa8, 0x40, 0x25, 0x47, 0x7a, 0xd0, 0x74, + 0x1c, 0xd1, 0xbc, 0xae, 0x2b, 0xca, 0x75, 0xf0, 0xcf, 0xe3, 0x5c, 0x09, 0xec, 0x1d, 0x3c, 0x7b, + 0xa4, 0x3e, 0xd3, 0xc7, 0x3d, 0x9b, 0x79, 0xc1, 0x8a, 0xc7, 0xce, 0x51, 0xf1, 0xc0, 0x8e, 0x82, + 0x4f, 0x45, 0x1f, 0x41, 0x60, 0xcf, 0x40, 0x66, 0x75, 0x72, 0xd1, 0xd8, 0x79, 0xd4, 0x31, 0xeb, + 0x4b, 0x61, 0x74, 0x1b, 0x95, 0x21, 0x89, 0x8f, 0x3a, 0xb2, 0x86, 0x6d, 0x95, 0x1b, 0x2e, 0x71, + 0xd8, 0xae, 0xe3, 0x00, 0x3e, 0x99, 0xc6, 0x47, 0x1d, 0x42, 0x44, 0xdb, 0x64, 0x12, 0x74, 0x3d, + 0x0e, 0x5a, 0x6c, 0xfb, 0x6c, 0xb8, 0xe3, 0xf7, 0x14, 0x2e, 0x2e, 0xe7, 0x39, 0x1b, 0x4c, 0x84, + 0xf8, 0x43, 0x01, 0x5e, 0x0b, 0x6d, 0xb5, 0x28, 0xe7, 0xc3, 0x0f, 0x20, 0x41, 0x2b, 0x1f, 0x3b, + 0x67, 0xe5, 0x29, 0x97, 0xf8, 0x7b, 0xee, 0x18, 0x97, 0x70, 0xcb, 0x24, 0x8a, 0x8d, 0x7c, 0x99, + 0x6f, 0x15, 0xa6, 0xdd, 0x06, 0x8f, 0x9d, 0xbb, 0xc1, 0x5d, 0x56, 0xcf, 0x08, 0xf4, 0x15, 0x33, + 0x4a, 0x23, 0xf0, 0x6b, 0x02, 0xcc, 0x79, 0xae, 0x51, 0xc4, 0xee, 0xf6, 0x3b, 0x10, 0x37, 0xcc, + 0xc3, 0xf1, 0x96, 0x38, 0x59, 0xcd, 0x49, 0x7e, 0x32, 0x25, 0xf5, 0x96, 0x28, 0xd2, 0xad, 0xae, + 0x38, 0xa4, 0x1e, 0x97, 0xa3, 0xac, 0xe5, 0x07, 0x7c, 0x19, 0x9e, 0x0d, 0xec, 0xb0, 0xae, 0xe8, + 0xbd, 0x6f, 0xe9, 0x71, 0x79, 0x1d, 0x1f, 0xbb, 0x5d, 0x91, 0x70, 0xa1, 0xef, 0x0a, 0x90, 0x72, + 0xf6, 0x2c, 0x6c, 0xef, 0x99, 0x2d, 0x6d, 0x3c, 0x27, 0x24, 0x4a, 0x37, 0xcd, 0x7f, 0x77, 0xf1, + 0x1f, 0x0b, 0x30, 0x49, 0xcb, 0xe7, 0x86, 0x88, 0x08, 0x21, 0x21, 0x22, 0xb4, 0xb8, 0x9e, 0x57, + 0x19, 0xfb, 0xfc, 0x8b, 0x1b, 0x70, 0x27, 0x27, 0xf3, 0x53, 0xe2, 0x33, 0x00, 0xa2, 0xdd, 0x28, + 0x7b, 0xc8, 0x7f, 0x8c, 0x43, 0x76, 0xab, 0x6b, 0xef, 0x45, 0x3c, 0x18, 0xca, 0x00, 0x9d, 0xae, + 0x4d, 0x51, 0xcf, 0x91, 0xc1, 0x35, 0x77, 0x46, 0x10, 0x8b, 0x0b, 0xc2, 0x18, 0x5f, 0xe3, 0xc8, + 0x40, 0x55, 0x2e, 0x04, 0xcb, 0x7e, 0x24, 0xcc, 0xb5, 0x51, 0x40, 0xbb, 0x71, 0x64, 0x6c, 0x60, + 0x0f, 0x61, 0x33, 0x49, 0x98, 0x48, 0xfa, 0x65, 0x01, 0xa6, 0xc9, 0x93, 0xec, 0x98, 0x9f, 0x7f, + 0xaf, 0x9b, 0x22, 0x6f, 0x6e, 0x98, 0xe8, 0x21, 0xa4, 0x58, 0x19, 0xc8, 0x84, 0x3b, 0x45, 0x27, + 0xdc, 0x30, 0x95, 0xf0, 0xd6, 0xa0, 0x53, 0x6d, 0x92, 0xb2, 0x92, 0xe9, 0x75, 0x1e, 0x26, 0x77, + 0x4d, 0x4b, 0xc5, 0x34, 0x4a, 0x26, 0x29, 0xb1, 0x07, 0xd6, 0x2d, 0xd6, 0x12, 0xc9, 0x64, 0x3e, + 0xb5, 0x96, 0x48, 0xa6, 0xf2, 0x20, 0xfe, 0xa6, 0x00, 0x39, 0xaf, 0x3d, 0xa3, 0x9c, 0x83, 0xca, + 0x3d, 0x8d, 0x71, 0xfe, 0x16, 0x25, 0xed, 0x20, 0xfe, 0x7b, 0xea, 0x84, 0xa9, 0xe6, 0x01, 0x6d, + 0xe0, 0x28, 0x3b, 0xdc, 0x43, 0x16, 0x2e, 0x15, 0x3b, 0x6f, 0x27, 0xa1, 0x91, 0x53, 0xf7, 0x60, + 0x5e, 0x6f, 0x93, 0xd9, 0x49, 0x77, 0x5a, 0xc7, 0x1c, 0x70, 0x3a, 0xd8, 0xdd, 0x93, 0x9f, 0xf3, + 0xd3, 0xca, 0x6e, 0x92, 0xf8, 0xbb, 0x74, 0x0f, 0xc0, 0xaf, 0x49, 0x94, 0xaa, 0xae, 0x41, 0xc6, + 0x62, 0xa2, 0x89, 0x27, 0x75, 0x4e, 0x6d, 0xcf, 0x78, 0xac, 0x44, 0xe1, 0xbf, 0x1d, 0x83, 0xdc, + 0xb3, 0x2e, 0xb6, 0x8e, 0xbf, 0x4c, 0xea, 0xbe, 0x09, 0xb9, 0x43, 0x45, 0x77, 0xe4, 0x5d, 0xd3, + 0x92, 0xbb, 0x1d, 0x4d, 0x71, 0xdc, 0x98, 0x9d, 0x0c, 0x21, 0x3f, 0x32, 0xad, 0x6d, 0x4a, 0x44, + 0x18, 0xd0, 0xbe, 0x61, 0x1e, 0x1a, 0x32, 0x21, 0x53, 0x84, 0x7f, 0x64, 0xf0, 0x85, 0xf9, 0xd2, + 0xbb, 0xff, 0xed, 0x64, 0xe1, 0xc1, 0x58, 0x83, 0x93, 0x46, 0x1d, 0x76, 0xbb, 0xba, 0xb6, 0xb4, + 0xbd, 0x5d, 0x5b, 0x95, 0xf2, 0x54, 0xe4, 0x0b, 0x26, 0xb1, 0x71, 0x64, 0xd8, 0xe2, 0x3f, 0x8c, + 0x41, 0xde, 0xd7, 0x51, 0x94, 0x0d, 0x59, 0x81, 0xf4, 0xcb, 0x2e, 0xb6, 0xf4, 0x57, 0x68, 0x46, + 0xe0, 0x8c, 0xc4, 0x7a, 0x7d, 0x0c, 0x33, 0x3d, 0x1a, 0x88, 0xff, 0x74, 0x1a, 0x48, 0x1f, 0xfa, + 0x95, 0x47, 0x77, 0x61, 0xd6, 0x39, 0x32, 0x64, 0x16, 0x83, 0xc9, 0xe2, 0x76, 0xdc, 0x50, 0x93, + 0x9c, 0x43, 0xf4, 0x41, 0xe8, 0x34, 0x66, 0xc7, 0x16, 0xff, 0x48, 0x00, 0x44, 0x15, 0x55, 0x63, + 0x3b, 0x27, 0x5f, 0x96, 0xfe, 0x74, 0x1b, 0xf2, 0x34, 0xaa, 0x55, 0xd6, 0x77, 0xe5, 0xb6, 0x6e, + 0xdb, 0xba, 0xd1, 0xe4, 0x1d, 0x2a, 0x4b, 0xe9, 0xb5, 0xdd, 0x0d, 0x46, 0x15, 0xff, 0x06, 0xcc, + 0xf5, 0x54, 0x20, 0xca, 0xc6, 0xbe, 0x0a, 0x33, 0xbb, 0x66, 0xd7, 0xd0, 0x64, 0xb6, 0xaf, 0xc4, + 0x17, 0x5d, 0xd3, 0x94, 0xc6, 0xde, 0x27, 0xfe, 0x9f, 0x18, 0xcc, 0x4b, 0xd8, 0x36, 0x5b, 0x07, + 0x38, 0x7a, 0x15, 0x56, 0x81, 0xef, 0x68, 0xc9, 0xaf, 0xa4, 0xc9, 0x14, 0x63, 0x26, 0xfd, 0xed, + 0x83, 0xbe, 0x9d, 0x8b, 0xeb, 0xa3, 0x7b, 0xec, 0xe0, 0x5e, 0x05, 0x5f, 0xb0, 0x4c, 0xf4, 0x2c, + 0x58, 0x9a, 0x90, 0xd3, 0x9b, 0x86, 0x49, 0x6c, 0x9a, 0x8d, 0x5f, 0x1a, 0xdd, 0xb6, 0x8b, 0xbf, + 0x96, 0x46, 0x15, 0xb2, 0xc6, 0x58, 0xea, 0xf8, 0xe5, 0x66, 0xb7, 0x4d, 0x11, 0x41, 0xe9, 0x22, + 0x29, 0xef, 0xe9, 0xc9, 0x42, 0xb6, 0x27, 0xcd, 0x96, 0xb2, 0xba, 0xf7, 0x4c, 0xa4, 0x8b, 0xdf, + 0x80, 0x0b, 0x7d, 0xca, 0x8e, 0xd2, 0x71, 0xfa, 0x76, 0x02, 0x2e, 0xf7, 0x8a, 0x8f, 0x1a, 0x55, + 0x7d, 0xd9, 0x1b, 0xf4, 0xfb, 0x02, 0x64, 0xda, 0xba, 0xf1, 0x45, 0xae, 0xbb, 0xce, 0xb4, 0x75, + 0xff, 0x29, 0xac, 0x87, 0x4d, 0x7d, 0xa6, 0x3d, 0x4c, 0x81, 0x62, 0x58, 0x17, 0x88, 0xb2, 0x9b, + 0x7d, 0x4f, 0x80, 0x99, 0xa8, 0x17, 0x14, 0x5f, 0x2d, 0x54, 0x51, 0x6c, 0x40, 0xe6, 0x33, 0x58, + 0x81, 0xfc, 0x6d, 0x01, 0x50, 0xc3, 0xea, 0x1a, 0x04, 0xf1, 0x3f, 0x31, 0x9b, 0x51, 0x56, 0x73, + 0x1e, 0x26, 0x75, 0x43, 0xc3, 0x47, 0xb4, 0x9a, 0x09, 0x89, 0x3d, 0xf4, 0xec, 0xf3, 0xc6, 0xc7, + 0xda, 0xe7, 0x15, 0x3f, 0x86, 0xb9, 0x9e, 0x22, 0x46, 0x59, 0xff, 0x3f, 0x88, 0xc1, 0x1c, 0xaf, + 0x48, 0xe4, 0x6b, 0xaf, 0x5f, 0x85, 0xc9, 0x16, 0x91, 0x39, 0xa2, 0x9d, 0xe9, 0x3b, 0xdd, 0x76, + 0xa6, 0x99, 0xd1, 0xcf, 0x03, 0x74, 0x2c, 0x7c, 0x20, 0x33, 0xd6, 0xf8, 0x58, 0xac, 0x29, 0xc2, + 0x41, 0x09, 0xe8, 0xeb, 0x90, 0x23, 0x56, 0xa1, 0x63, 0x99, 0x1d, 0xd3, 0x26, 0x9e, 0x8f, 0x3d, + 0x1e, 0xe4, 0x9a, 0x3d, 0x3d, 0x59, 0xc8, 0x6c, 0xe8, 0xc6, 0x16, 0x67, 0x6c, 0xd4, 0x25, 0x62, + 0x5e, 0xbc, 0x47, 0x5b, 0xfc, 0x4f, 0x02, 0xcc, 0x7f, 0x66, 0xeb, 0xd4, 0x5f, 0x84, 0xae, 0xc4, + 0xe7, 0x90, 0xa7, 0x3f, 0x6a, 0xc6, 0xae, 0x19, 0xe5, 0x8e, 0xc1, 0xf7, 0x05, 0x98, 0x0d, 0x08, + 0x8e, 0xd2, 0xcd, 0x79, 0x25, 0x3d, 0x89, 0xbf, 0x48, 0x1c, 0x9f, 0x60, 0x27, 0x8f, 0x72, 0x08, + 0xfd, 0xab, 0x18, 0x5c, 0x2c, 0xb3, 0xfd, 0x7e, 0x37, 0x00, 0x26, 0xca, 0x9e, 0x51, 0x80, 0xe9, + 0x03, 0x6c, 0xd9, 0xba, 0xc9, 0x26, 0xe1, 0x8c, 0xe4, 0x3e, 0xa2, 0x22, 0x24, 0x6d, 0x43, 0xe9, + 0xd8, 0x7b, 0xa6, 0xbb, 0x3f, 0xe9, 0x3d, 0x7b, 0xc1, 0x3a, 0x93, 0xaf, 0x1e, 0xac, 0x33, 0x35, + 0x3a, 0x58, 0x67, 0xfa, 0xa7, 0x08, 0xd6, 0xe1, 0xbb, 0x78, 0xff, 0x41, 0x80, 0x4b, 0x03, 0x9a, + 0x8b, 0xb2, 0xb7, 0x7c, 0x0b, 0xd2, 0x2a, 0x17, 0x4c, 0xec, 0x2d, 0xdb, 0xa2, 0xac, 0x91, 0x6c, + 0xaf, 0x88, 0x5e, 0x4e, 0x4f, 0x16, 0xc0, 0x2d, 0x6a, 0x6d, 0x95, 0x2b, 0x87, 0xfc, 0xd6, 0xc4, + 0x5f, 0xc9, 0x40, 0xae, 0x72, 0xc4, 0x96, 0xe7, 0xeb, 0x6c, 0x96, 0x47, 0x8f, 0x20, 0xd9, 0xb1, + 0xcc, 0x03, 0xdd, 0xad, 0x46, 0xb6, 0x27, 0x52, 0xc3, 0xad, 0x46, 0x1f, 0xd7, 0x16, 0xe7, 0x90, + 0x3c, 0x5e, 0xd4, 0x80, 0xd4, 0x13, 0x53, 0x55, 0x5a, 0x8f, 0xf4, 0x96, 0xdb, 0xf3, 0xdf, 0x3e, + 0x5b, 0xd0, 0x92, 0xc7, 0xb3, 0xa5, 0x38, 0x7b, 0x6e, 0x23, 0x78, 0x44, 0x54, 0x83, 0x64, 0xd5, + 0x71, 0x3a, 0x24, 0x91, 0xdb, 0x8e, 0x5b, 0x63, 0x08, 0x25, 0x2c, 0x5c, 0x96, 0xc7, 0x8e, 0x1a, + 0x30, 0xfb, 0x98, 0x1e, 0x8e, 0x2b, 0xb7, 0xcc, 0xae, 0x56, 0x36, 0x8d, 0x5d, 0xbd, 0xc9, 0xed, + 0xee, 0xcd, 0x31, 0x64, 0x3e, 0x2e, 0xd7, 0xa5, 0x41, 0x01, 0x68, 0x05, 0x92, 0xf5, 0x07, 0x5c, + 0x18, 0x73, 0xee, 0x6e, 0x8c, 0x21, 0xac, 0xfe, 0x40, 0xf2, 0xd8, 0xd0, 0x1a, 0xa4, 0x57, 0x3e, + 0xe9, 0x5a, 0x98, 0x4b, 0x99, 0x1a, 0x1a, 0xdd, 0xd1, 0x2f, 0x85, 0x72, 0x49, 0x41, 0x66, 0x54, + 0x87, 0xec, 0x0b, 0xd3, 0xda, 0x6f, 0x99, 0x8a, 0x5b, 0xc3, 0x69, 0x2a, 0xee, 0x2b, 0x63, 0x88, + 0x73, 0x19, 0xa5, 0x3e, 0x11, 0xe8, 0x1b, 0x90, 0x23, 0x8d, 0xd1, 0x50, 0x76, 0x5a, 0x6e, 0x21, + 0x93, 0x54, 0xea, 0x9b, 0x63, 0x48, 0xf5, 0x38, 0xdd, 0xfd, 0xa1, 0x3e, 0x51, 0x45, 0x09, 0x32, + 0x3d, 0x9d, 0x00, 0x21, 0x48, 0x74, 0x48, 0x7b, 0x0b, 0x34, 0x90, 0x8b, 0xfe, 0x46, 0x6f, 0xc1, + 0xb4, 0x61, 0x6a, 0xd8, 0x1d, 0x21, 0x99, 0xd2, 0xfc, 0xe9, 0xc9, 0xc2, 0xd4, 0xa6, 0xa9, 0x31, + 0x87, 0x84, 0xff, 0x92, 0xa6, 0x48, 0xa6, 0x9a, 0x56, 0x5c, 0x84, 0x04, 0x69, 0x77, 0x62, 0x98, + 0x76, 0x14, 0x1b, 0x6f, 0x5b, 0x3a, 0x97, 0xe6, 0x3e, 0x16, 0xff, 0x69, 0x0c, 0x62, 0xf5, 0x07, + 0xc4, 0x73, 0xdf, 0xe9, 0xaa, 0xfb, 0xd8, 0xe1, 0xe9, 0xfc, 0x89, 0x7a, 0xf4, 0x16, 0xde, 0xd5, + 0x99, 0x67, 0x94, 0x92, 0xf8, 0x13, 0x7a, 0x03, 0x40, 0x51, 0x55, 0x6c, 0xdb, 0xb2, 0x7b, 0x68, + 0x32, 0x25, 0xa5, 0x18, 0x65, 0x1d, 0x1f, 0x13, 0x36, 0x1b, 0xab, 0x16, 0x76, 0xdc, 0x28, 0x34, + 0xf6, 0x44, 0xd8, 0x1c, 0xdc, 0xee, 0xc8, 0x8e, 0xb9, 0x8f, 0x0d, 0xda, 0x4f, 0x52, 0xc4, 0xd4, + 0xb4, 0x3b, 0x0d, 0x42, 0x20, 0x56, 0x12, 0x1b, 0x9a, 0x6f, 0xd2, 0x52, 0x92, 0xf7, 0x4c, 0x44, + 0x5a, 0xb8, 0xa9, 0xf3, 0xd3, 0x7f, 0x29, 0x89, 0x3f, 0x11, 0x2d, 0x29, 0x5d, 0x67, 0x8f, 0xb6, + 0x44, 0x4a, 0xa2, 0xbf, 0xd1, 0x4d, 0xc8, 0xb1, 0xc0, 0x55, 0x19, 0x1b, 0xaa, 0x4c, 0x8d, 0x6b, + 0x8a, 0x26, 0x67, 0x18, 0xb9, 0x62, 0xa8, 0xc4, 0x94, 0xa2, 0x07, 0xc0, 0x09, 0xf2, 0x7e, 0xdb, + 0x26, 0x3a, 0x05, 0x92, 0xab, 0x94, 0x3b, 0x3d, 0x59, 0x48, 0xd7, 0x69, 0xc2, 0xfa, 0x46, 0xbd, + 0xb6, 0x2a, 0xa5, 0x59, 0xae, 0xf5, 0xb6, 0x5d, 0xd3, 0x8a, 0xbf, 0x2e, 0x40, 0xfc, 0x71, 0xb9, + 0x7e, 0x6e, 0x95, 0xb9, 0x05, 0x8d, 0x07, 0x0a, 0x7a, 0x0b, 0x72, 0x3b, 0x7a, 0xab, 0xa5, 0x1b, + 0x4d, 0xe2, 0x05, 0x7d, 0x0b, 0xab, 0xae, 0xc2, 0xb2, 0x9c, 0xbc, 0xc5, 0xa8, 0x68, 0x11, 0xd2, + 0xaa, 0x85, 0x35, 0x6c, 0x38, 0xba, 0xd2, 0xb2, 0xb9, 0xe6, 0x82, 0xa4, 0xe2, 0xb7, 0x05, 0x98, + 0xa4, 0x23, 0x00, 0xbd, 0x0e, 0x29, 0xd5, 0x34, 0x1c, 0x45, 0x37, 0xb8, 0x29, 0x4b, 0x49, 0x3e, + 0x61, 0x68, 0xf1, 0xae, 0xc2, 0x8c, 0xa2, 0xaa, 0x66, 0xd7, 0x70, 0x64, 0x43, 0x69, 0x63, 0x5e, + 0xcc, 0x34, 0xa7, 0x6d, 0x2a, 0x6d, 0x8c, 0x16, 0xc0, 0x7d, 0xf4, 0xce, 0xc2, 0xa6, 0x24, 0xe0, + 0xa4, 0x75, 0x7c, 0x5c, 0xfc, 0x63, 0x01, 0x92, 0xee, 0x98, 0x21, 0xc5, 0x68, 0x62, 0x03, 0x5b, + 0x8a, 0x63, 0x7a, 0xc5, 0xf0, 0x08, 0xfd, 0x53, 0x65, 0xca, 0x9f, 0x2a, 0xe7, 0x61, 0xd2, 0x21, + 0xc3, 0x82, 0x97, 0x80, 0x3d, 0xd0, 0x55, 0xed, 0x96, 0xd2, 0x64, 0x8b, 0x7a, 0x29, 0x89, 0x3d, + 0x90, 0xca, 0xf0, 0xf8, 0x67, 0xa6, 0x11, 0xfe, 0x44, 0x4a, 0xca, 0xa2, 0x74, 0x77, 0x70, 0x53, + 0x37, 0x68, 0x5f, 0x8a, 0x4b, 0x40, 0x49, 0x25, 0x42, 0x41, 0xaf, 0x41, 0x8a, 0x65, 0xc0, 0x86, + 0x46, 0x3b, 0x54, 0x5c, 0x4a, 0x52, 0x42, 0xc5, 0xd0, 0x8a, 0x18, 0x52, 0xde, 0xe0, 0x24, 0xcd, + 0xd6, 0xb5, 0x3d, 0x45, 0xd2, 0xdf, 0xe8, 0x6d, 0x98, 0x7f, 0xd9, 0x55, 0x5a, 0xfa, 0x2e, 0x5d, + 0xaf, 0x23, 0xd9, 0x98, 0xce, 0x58, 0x4d, 0x90, 0x97, 0x46, 0x25, 0x50, 0xd5, 0xb9, 0x63, 0x39, + 0xee, 0x8f, 0x65, 0xf1, 0xf7, 0x05, 0x98, 0x65, 0x91, 0x53, 0x2c, 0x72, 0x38, 0x3a, 0x3f, 0xe4, + 0x7d, 0x48, 0x69, 0x8a, 0xa3, 0xb0, 0xd3, 0xbd, 0xb1, 0x91, 0xa7, 0x7b, 0xdd, 0xe9, 0x81, 0xe4, + 0xa7, 0x27, 0x7c, 0x11, 0x24, 0xc8, 0x6f, 0x76, 0x10, 0x5a, 0xa2, 0xbf, 0xc5, 0x8f, 0x00, 0x05, + 0x0b, 0x1a, 0xa5, 0x47, 0x76, 0x07, 0x2e, 0x10, 0x5d, 0x57, 0x0c, 0xd5, 0x3a, 0xee, 0x38, 0xba, + 0x69, 0x3c, 0xa5, 0x7f, 0x6d, 0x94, 0x0f, 0x6c, 0xb6, 0xd1, 0x3d, 0x36, 0xf1, 0x57, 0xa7, 0x21, + 0x53, 0x39, 0xea, 0x98, 0x56, 0xa4, 0x8b, 0x61, 0x25, 0x98, 0xe6, 0x40, 0x7f, 0xc4, 0xa6, 0x79, + 0x9f, 0x31, 0x77, 0xf7, 0xa3, 0x39, 0x23, 0xfa, 0x9e, 0x00, 0xc0, 0xc2, 0x7b, 0x69, 0xc8, 0x54, + 0xfc, 0x73, 0xdf, 0xfe, 0xa3, 0x2f, 0x27, 0xcf, 0x68, 0x13, 0xd2, 0xed, 0x03, 0x55, 0x95, 0x77, + 0xf5, 0x96, 0xc3, 0x43, 0x24, 0xc3, 0x8f, 0x05, 0x6c, 0x3c, 0x2f, 0x97, 0x1f, 0xd1, 0x4c, 0x2c, + 0x5a, 0xd1, 0x7f, 0x96, 0x80, 0x48, 0x60, 0xbf, 0xd1, 0x9b, 0xc0, 0x0f, 0x75, 0xc9, 0xb6, 0x7b, + 0x4e, 0xb3, 0x94, 0x39, 0x3d, 0x59, 0x48, 0x49, 0x94, 0x5a, 0xaf, 0x37, 0xa4, 0x14, 0xcb, 0x50, + 0xb7, 0x1d, 0x74, 0x0d, 0x32, 0x66, 0x5b, 0x77, 0x64, 0xd7, 0xd7, 0xe2, 0x8e, 0xe9, 0x0c, 0x21, + 0xba, 0xbe, 0x18, 0x6a, 0xc0, 0x2d, 0x6c, 0xd0, 0x41, 0x43, 0xb4, 0x25, 0xef, 0xb0, 0xa5, 0x50, + 0x87, 0x19, 0x06, 0xd9, 0xec, 0x38, 0x7a, 0x5b, 0xff, 0x84, 0xee, 0xfe, 0xf3, 0x2d, 0xac, 0x6b, + 0x2c, 0x3b, 0xa9, 0x5f, 0x89, 0xae, 0x91, 0xf2, 0xbc, 0x4f, 0x03, 0x59, 0xd1, 0x77, 0x05, 0xb8, + 0xc8, 0xb5, 0x24, 0xef, 0xd0, 0x73, 0x0d, 0x4a, 0x4b, 0x77, 0x8e, 0xe5, 0xfd, 0x83, 0x42, 0x92, + 0xba, 0xbf, 0x7f, 0x2d, 0xb4, 0x5d, 0x03, 0xdd, 0x69, 0xc9, 0x6d, 0xdd, 0xe3, 0x27, 0x9c, 0x79, + 0xfd, 0xa0, 0x62, 0x38, 0xd6, 0x71, 0xe9, 0xd2, 0xe9, 0xc9, 0xc2, 0xdc, 0x60, 0xea, 0x73, 0x69, + 0xce, 0x1e, 0x64, 0x41, 0x55, 0x00, 0xec, 0x75, 0x67, 0x3a, 0xf1, 0x84, 0xbb, 0x31, 0xa1, 0xfd, + 0x5e, 0x0a, 0xf0, 0xa2, 0xdb, 0x90, 0xe7, 0x67, 0xa2, 0x76, 0xf5, 0x16, 0x96, 0x6d, 0xfd, 0x13, + 0x4c, 0xa7, 0xa8, 0xb8, 0x94, 0x65, 0x74, 0x22, 0xa2, 0xae, 0x7f, 0x82, 0x8b, 0xdf, 0x82, 0xc2, + 0xb0, 0xd2, 0x07, 0x47, 0x52, 0x8a, 0xed, 0x56, 0xbf, 0xd7, 0xbb, 0xaa, 0x33, 0x46, 0x8f, 0xe7, + 0x2b, 0x3b, 0xef, 0xc7, 0xde, 0x13, 0xc4, 0x7f, 0x14, 0x83, 0x4c, 0xa9, 0xdb, 0xda, 0x7f, 0xda, + 0xa9, 0x77, 0xdb, 0x6d, 0xc5, 0x3a, 0x26, 0xd6, 0x94, 0xd9, 0x1b, 0x52, 0x40, 0x81, 0x59, 0x53, + 0x6a, 0x50, 0xf4, 0x4f, 0x30, 0x99, 0xe3, 0x82, 0xf7, 0x02, 0xb0, 0x73, 0x1b, 0xb4, 0x0e, 0x81, + 0xc3, 0xfe, 0xe6, 0xa1, 0x8d, 0xde, 0x83, 0x42, 0x20, 0x23, 0x5d, 0x82, 0x91, 0xb1, 0xe1, 0x58, + 0x3a, 0x66, 0xab, 0x91, 0x71, 0x29, 0x10, 0x93, 0x54, 0x23, 0xc9, 0x15, 0x96, 0x8a, 0x1a, 0x30, + 0x43, 0x32, 0x1e, 0xcb, 0x74, 0x26, 0x72, 0x57, 0x8b, 0xef, 0x85, 0x54, 0xab, 0xa7, 0xdc, 0x4b, + 0x54, 0x3f, 0x65, 0xca, 0x43, 0x7f, 0x4a, 0x69, 0xec, 0x53, 0x8a, 0x1f, 0x42, 0xbe, 0x3f, 0x43, + 0x50, 0x97, 0x09, 0xa6, 0xcb, 0xf9, 0xa0, 0x2e, 0xe3, 0x01, 0x3d, 0xad, 0x25, 0x92, 0x89, 0xfc, + 0xa4, 0xf8, 0x07, 0x09, 0xc8, 0xba, 0xdd, 0x2c, 0x4a, 0xbc, 0x54, 0x82, 0x49, 0xd2, 0x29, 0xdc, + 0x38, 0x9a, 0x9b, 0x23, 0x7a, 0x37, 0x3f, 0x29, 0x40, 0x3a, 0x8b, 0x8b, 0xb5, 0x29, 0xeb, 0x97, + 0xc8, 0x6e, 0x15, 0xbf, 0x1d, 0x83, 0x04, 0x45, 0x3a, 0xf7, 0x20, 0x41, 0xa7, 0x2d, 0x61, 0x9c, + 0x69, 0x8b, 0x66, 0xf5, 0x26, 0xd7, 0x58, 0xc0, 0x51, 0x26, 0x1e, 0xe8, 0x9e, 0xf2, 0xce, 0xbd, + 0xfb, 0xd4, 0x66, 0xcd, 0x48, 0xfc, 0x09, 0x95, 0x68, 0x84, 0x98, 0x69, 0x39, 0x58, 0xe3, 0x08, + 0x63, 0xf1, 0xac, 0x6e, 0xe2, 0x4e, 0x91, 0x2e, 0x1f, 0xba, 0x0c, 0x71, 0x62, 0x0c, 0xa7, 0x59, + 0x1c, 0xc8, 0xe9, 0xc9, 0x42, 0x9c, 0x98, 0x41, 0x42, 0x43, 0xcb, 0x90, 0xee, 0xb5, 0x3c, 0xc4, + 0x9f, 0xa4, 0xf6, 0x35, 0x60, 0x35, 0xa0, 0xe5, 0x8d, 0x50, 0x86, 0xae, 0x79, 0x57, 0xf9, 0xc3, + 0x49, 0xc8, 0xd4, 0xda, 0x51, 0x4f, 0x70, 0x2b, 0xbd, 0x1d, 0x25, 0x0c, 0x96, 0xf5, 0xbc, 0x34, + 0xa4, 0x9f, 0xf4, 0xf8, 0x13, 0xf1, 0xf3, 0xf9, 0x13, 0x35, 0xe2, 0xb7, 0xf3, 0xfb, 0x42, 0xe2, + 0x43, 0x10, 0x58, 0xef, 0xfb, 0xa9, 0xd7, 0x24, 0x11, 0x1e, 0xff, 0x08, 0x0e, 0x8d, 0x09, 0xfa, + 0x8e, 0x40, 0xf1, 0x01, 0xeb, 0xac, 0x53, 0x9f, 0x77, 0x67, 0x9d, 0xc6, 0x86, 0x46, 0xa7, 0xd8, + 0x5e, 0xfb, 0x3e, 0xfd, 0xea, 0xf6, 0xbd, 0xe8, 0xf0, 0x3e, 0xff, 0x3e, 0xc4, 0x35, 0xdd, 0x6d, + 0xe3, 0xf1, 0xfd, 0x0f, 0xc2, 0x74, 0x46, 0xe7, 0x4f, 0x04, 0x3b, 0x3f, 0xeb, 0x6c, 0xc5, 0x1a, + 0x80, 0xaf, 0x62, 0xb4, 0x08, 0x53, 0x66, 0x4b, 0x73, 0x8f, 0x32, 0x65, 0x4a, 0xa9, 0xd3, 0x93, + 0x85, 0xc9, 0xa7, 0x2d, 0xad, 0xb6, 0x2a, 0x4d, 0x9a, 0x2d, 0xad, 0xa6, 0xd1, 0x3b, 0x5f, 0xf0, + 0xa1, 0xec, 0xc5, 0x15, 0xce, 0x48, 0xd3, 0x06, 0x3e, 0x5c, 0xc5, 0xb6, 0xca, 0xfb, 0xed, 0x6f, + 0x09, 0x90, 0x75, 0x9b, 0x30, 0x5a, 0x13, 0x97, 0xd4, 0xdb, 0x7c, 0xac, 0xc6, 0xcf, 0x37, 0x56, + 0x5d, 0x3e, 0x7e, 0xdc, 0xfc, 0x3b, 0x02, 0x8f, 0x14, 0xaf, 0xab, 0x8a, 0x43, 0x5c, 0x9c, 0x08, + 0xc7, 0xd7, 0x1d, 0xc8, 0x5b, 0x8a, 0xa1, 0x99, 0x6d, 0xfd, 0x13, 0xcc, 0x56, 0x77, 0x6d, 0xbe, + 0xa3, 0x9b, 0xf3, 0xe8, 0x74, 0x29, 0xd3, 0x16, 0xff, 0x38, 0xc6, 0xa3, 0xca, 0xbd, 0x62, 0x44, + 0xa9, 0xae, 0x6f, 0xc2, 0x6c, 0xff, 0x05, 0x3c, 0xee, 0xa0, 0x7f, 0x2b, 0x44, 0x5e, 0x58, 0x41, + 0x58, 0x74, 0x28, 0x57, 0x62, 0xbe, 0xef, 0x32, 0x1e, 0x1b, 0x95, 0x21, 0x1d, 0xbc, 0xd7, 0x27, + 0x3e, 0xf6, 0xbd, 0x3e, 0x60, 0x79, 0xb7, 0xf9, 0x14, 0x7f, 0x01, 0x26, 0x69, 0xf2, 0x2b, 0x58, + 0x7a, 0xde, 0x9a, 0xbf, 0x1f, 0x87, 0xeb, 0xb4, 0xf4, 0xcf, 0xb1, 0xa5, 0xef, 0x1e, 0x6f, 0x59, + 0xa6, 0x83, 0x55, 0x07, 0x6b, 0xfe, 0x58, 0x8d, 0xb0, 0x79, 0xbf, 0x2b, 0x40, 0xaa, 0xe3, 0xbe, + 0xe1, 0x0b, 0x88, 0xec, 0xf3, 0xde, 0x8d, 0xd6, 0x21, 0xc7, 0x83, 0x2f, 0x94, 0x96, 0x7e, 0x80, + 0x65, 0xc5, 0x19, 0x6f, 0xc6, 0x66, 0x0a, 0xcc, 0x30, 0xde, 0x15, 0xc2, 0xba, 0xe2, 0x20, 0x0d, + 0x52, 0x5c, 0x98, 0xae, 0xf1, 0x5b, 0xad, 0x1e, 0xff, 0x74, 0x8b, 0xad, 0x49, 0x16, 0x01, 0x52, + 0x5b, 0x95, 0x92, 0x4c, 0x72, 0x4d, 0x13, 0xff, 0xab, 0x00, 0x37, 0xce, 0x68, 0xa9, 0x28, 0x47, + 0x40, 0x11, 0x92, 0x07, 0xe4, 0x45, 0x3a, 0x6f, 0xa9, 0xa4, 0xe4, 0x3d, 0xa3, 0x0d, 0xc8, 0xec, + 0x2a, 0x7a, 0xcb, 0x1f, 0x19, 0xc3, 0xe3, 0x52, 0xc3, 0x43, 0xa4, 0x67, 0x18, 0x3b, 0x1b, 0x0a, + 0xe2, 0x6f, 0xc4, 0x60, 0x76, 0x45, 0xd3, 0xea, 0x75, 0x6e, 0x4a, 0xa3, 0xeb, 0x70, 0x2e, 0x00, + 0x8f, 0xf9, 0x00, 0x1c, 0xbd, 0x05, 0x48, 0xd3, 0x6d, 0x76, 0x7b, 0x8e, 0xbd, 0xa7, 0x68, 0xe6, + 0xa1, 0x1f, 0xb9, 0x32, 0xeb, 0xa6, 0xd4, 0xdd, 0x04, 0x54, 0x07, 0x0a, 0xe1, 0x64, 0xdb, 0x51, + 0xbc, 0x3d, 0xb5, 0x1b, 0x63, 0x9d, 0x3b, 0x64, 0xd8, 0xce, 0x7b, 0x94, 0x52, 0x44, 0x0e, 0xfd, + 0x49, 0xc0, 0x88, 0x4e, 0xaa, 0xee, 0xc8, 0x8a, 0xed, 0x9e, 0x0d, 0x63, 0xf7, 0xf6, 0x64, 0x19, + 0x7d, 0xc5, 0x66, 0x47, 0xbe, 0xd8, 0x29, 0x14, 0x5f, 0x35, 0x51, 0x2e, 0x17, 0xfc, 0x6f, 0x01, + 0xb2, 0x12, 0xde, 0xb5, 0xb0, 0x1d, 0xe9, 0x82, 0xc9, 0xdf, 0x16, 0x60, 0xc6, 0x62, 0x62, 0xe5, + 0x5d, 0xcb, 0x6c, 0x7f, 0xfe, 0xae, 0x70, 0x9a, 0xbf, 0xfe, 0x91, 0x65, 0xb6, 0xb9, 0x99, 0x7b, + 0x0e, 0x39, 0xaf, 0xaa, 0x51, 0xea, 0xf0, 0x2f, 0xe8, 0xd1, 0x7c, 0x26, 0x38, 0xea, 0x48, 0x94, + 0x2f, 0xa5, 0x22, 0xe9, 0x96, 0x62, 0xb0, 0xbe, 0x51, 0x6a, 0xf3, 0x7f, 0x0a, 0x90, 0xad, 0x77, + 0x77, 0xd8, 0x65, 0x71, 0xd1, 0x29, 0xb2, 0x02, 0xa9, 0x16, 0xde, 0x75, 0xe4, 0x57, 0x3a, 0xcd, + 0x91, 0x24, 0xac, 0xf4, 0x2c, 0xcb, 0x63, 0x00, 0x8b, 0x1e, 0xe6, 0xa4, 0x72, 0xe2, 0xe7, 0x94, + 0x93, 0xa2, 0xbc, 0x84, 0x2c, 0xfe, 0x93, 0x18, 0xe4, 0xbc, 0x6a, 0x46, 0x69, 0xb3, 0x5f, 0xf4, + 0xd8, 0xaa, 0xf8, 0x79, 0x6c, 0xd5, 0x2c, 0x0f, 0xbe, 0x09, 0xb7, 0x57, 0x4b, 0x30, 0x47, 0xbd, + 0x31, 0x59, 0xe9, 0x74, 0x5a, 0xba, 0xbb, 0xa2, 0x40, 0xad, 0x61, 0x42, 0x9a, 0xa5, 0x49, 0x2b, + 0x2c, 0x85, 0xae, 0x25, 0xa0, 0x47, 0x30, 0xb3, 0x6b, 0x61, 0xfc, 0x09, 0x96, 0x29, 0x2a, 0x1d, + 0x2f, 0x44, 0x89, 0xe9, 0x29, 0xcd, 0x18, 0xeb, 0x84, 0x8f, 0xf7, 0xb9, 0x17, 0x30, 0x4b, 0x75, + 0x1a, 0xf5, 0x69, 0x76, 0xf1, 0xef, 0xc7, 0x00, 0x05, 0x25, 0x7f, 0x76, 0x6d, 0x11, 0x8b, 0xae, + 0x2d, 0xde, 0x04, 0xc4, 0xa2, 0x54, 0x6d, 0xb9, 0x83, 0x2d, 0xd9, 0xc6, 0xaa, 0xc9, 0x2f, 0x32, + 0x13, 0xa4, 0x3c, 0x4f, 0xd9, 0xc2, 0x56, 0x9d, 0xd2, 0xd1, 0x43, 0x00, 0xdf, 0xcd, 0xe4, 0xd3, + 0xd7, 0x48, 0x2f, 0x53, 0x4a, 0x79, 0xfe, 0xa5, 0xf8, 0xfd, 0x22, 0xcc, 0x70, 0x1d, 0x6e, 0x1b, + 0xba, 0x69, 0xa0, 0x7b, 0x10, 0x6f, 0xf2, 0x4d, 0x9a, 0x74, 0xe8, 0x4a, 0xa8, 0x7f, 0x53, 0x63, + 0x75, 0x42, 0x22, 0x79, 0x09, 0x4b, 0xa7, 0xeb, 0x84, 0x38, 0x7b, 0x7e, 0xe4, 0x7d, 0x90, 0xa5, + 0xd3, 0x75, 0x50, 0x1d, 0x72, 0xaa, 0x7f, 0x13, 0x9d, 0x4c, 0xd8, 0xe3, 0x43, 0x91, 0x61, 0xe8, + 0xdd, 0x7f, 0xd5, 0x09, 0x29, 0xab, 0xf6, 0x24, 0xa0, 0x72, 0xf0, 0x02, 0xb4, 0xc4, 0x40, 0x58, + 0x9f, 0x7f, 0xda, 0xbd, 0xf7, 0xf2, 0xb5, 0xea, 0x44, 0xe0, 0x9e, 0x34, 0xf4, 0x3e, 0x4c, 0x69, + 0xf4, 0xaa, 0x2d, 0xde, 0xa3, 0xc3, 0x3a, 0x5d, 0xcf, 0x8d, 0x66, 0xd5, 0x09, 0x89, 0x73, 0xa0, + 0x35, 0x98, 0x61, 0xbf, 0x98, 0xd3, 0xc4, 0x41, 0xf7, 0x8d, 0xe1, 0x12, 0x02, 0xf3, 0x49, 0x75, + 0x42, 0x4a, 0x6b, 0x3e, 0x15, 0x7d, 0x15, 0x12, 0xb6, 0xaa, 0xb8, 0x80, 0xf9, 0xca, 0x90, 0x9b, + 0x6a, 0x7c, 0x66, 0x9a, 0x1b, 0x3d, 0x64, 0x77, 0xaf, 0x3a, 0x47, 0xee, 0x4a, 0x6a, 0x58, 0xf1, + 0x7b, 0x6e, 0x44, 0x20, 0xc5, 0xc7, 0x94, 0x80, 0x1e, 0x43, 0x5a, 0x21, 0xde, 0xa7, 0x4c, 0xcf, + 0xf0, 0xd2, 0xa5, 0xd3, 0xf0, 0x30, 0x88, 0x81, 0x33, 0xd7, 0x55, 0x7a, 0xed, 0x82, 0x4b, 0xf4, + 0x05, 0xb5, 0xb1, 0xd5, 0xc4, 0x85, 0xf4, 0x68, 0x41, 0xc1, 0x28, 0x3c, 0x4f, 0x10, 0x25, 0x12, + 0x2f, 0xd4, 0x3b, 0x87, 0x4f, 0x2b, 0x35, 0x33, 0x74, 0xe3, 0x3d, 0xe4, 0xf4, 0x59, 0x75, 0x42, + 0x9a, 0xd9, 0x0b, 0x90, 0xd1, 0x12, 0xc4, 0x9a, 0x6a, 0x21, 0x33, 0x74, 0x84, 0x78, 0x27, 0xac, + 0xaa, 0x13, 0x52, 0xac, 0xa9, 0xa2, 0x0f, 0x21, 0xc9, 0x0e, 0x96, 0x1c, 0x19, 0x85, 0xec, 0x50, + 0x3b, 0xd1, 0x7b, 0xca, 0xa7, 0x3a, 0x21, 0xd1, 0x13, 0x31, 0xe4, 0x7d, 0x5b, 0x90, 0xb5, 0x58, + 0x18, 0xa3, 0x1b, 0xbb, 0x9c, 0x1f, 0x1a, 0x8c, 0x10, 0x16, 0xbe, 0x5c, 0xa5, 0x38, 0x24, 0x40, + 0x47, 0xdf, 0x84, 0xf9, 0x5e, 0x89, 0xbc, 0xa7, 0xcd, 0x0e, 0xdd, 0x58, 0x1f, 0x1a, 0x4a, 0x5b, + 0x9d, 0x90, 0x90, 0x35, 0x90, 0x88, 0xde, 0x85, 0x49, 0xd6, 0x6a, 0x88, 0x8a, 0x0c, 0x8b, 0xaf, + 0xe9, 0x6b, 0x30, 0x96, 0x9f, 0x74, 0x7e, 0x87, 0xc7, 0xf2, 0xc9, 0x2d, 0xb3, 0x59, 0x98, 0x1b, + 0xda, 0xf9, 0x07, 0xa3, 0x12, 0x49, 0xe7, 0x77, 0x7c, 0x2a, 0x69, 0x77, 0x8b, 0xa5, 0xf0, 0x00, + 0xb0, 0xf9, 0xa1, 0xed, 0x1e, 0x12, 0xe2, 0x57, 0xa5, 0x87, 0x35, 0x7c, 0x32, 0x29, 0x9a, 0xc5, + 0x2e, 0x57, 0x92, 0xe9, 0x98, 0xba, 0x30, 0xb4, 0x68, 0x83, 0xb7, 0x4f, 0x91, 0xa2, 0x59, 0x3e, + 0x15, 0x3d, 0x87, 0x3c, 0xbf, 0x02, 0xc5, 0xdf, 0xb6, 0xb9, 0x48, 0xe5, 0xdd, 0x09, 0x35, 0x5d, + 0x61, 0xd1, 0x53, 0xd5, 0x09, 0x29, 0xa7, 0xf6, 0xa6, 0xa0, 0x8f, 0x60, 0x96, 0xca, 0x93, 0x55, + 0xff, 0xd6, 0x9a, 0x42, 0x61, 0xe0, 0x0e, 0x94, 0xe1, 0x17, 0xdc, 0xb8, 0x92, 0xf3, 0x6a, 0x5f, + 0x12, 0xe9, 0xc6, 0xba, 0xa1, 0x3b, 0xd4, 0xca, 0x16, 0x87, 0x76, 0xe3, 0xde, 0x9b, 0x3b, 0x49, + 0x37, 0xd6, 0x19, 0x85, 0x74, 0x63, 0x87, 0x47, 0x07, 0xf2, 0xe6, 0x78, 0x7d, 0x68, 0x37, 0x0e, + 0x0b, 0x23, 0x24, 0xdd, 0xd8, 0x09, 0xd2, 0x49, 0x37, 0x66, 0x06, 0xa2, 0x4f, 0xee, 0x1b, 0x43, + 0xbb, 0xf1, 0xd0, 0xb3, 0xf4, 0xa4, 0x1b, 0x2b, 0x03, 0x89, 0x68, 0x15, 0x80, 0xb9, 0x33, 0x74, + 0x52, 0xbc, 0x32, 0x74, 0x32, 0xe8, 0x8f, 0x12, 0x24, 0x93, 0x41, 0xcb, 0xa5, 0x11, 0x43, 0x46, + 0xa1, 0x9b, 0x4c, 0x37, 0xbb, 0x0b, 0x0b, 0x43, 0x0d, 0xd9, 0xc0, 0xc6, 0x34, 0x31, 0x64, 0x87, + 0x1e, 0x91, 0xcc, 0x2a, 0x6c, 0x3d, 0xbc, 0xb0, 0x38, 0xdc, 0x2c, 0x07, 0x77, 0xd7, 0xa8, 0x59, + 0xa6, 0x04, 0xb4, 0x02, 0x29, 0x32, 0xe7, 0x1f, 0x53, 0x33, 0x74, 0x75, 0xa8, 0x4f, 0xda, 0x77, + 0x1a, 0xa9, 0x3a, 0x21, 0x25, 0x5f, 0x72, 0x12, 0x79, 0x3d, 0x5b, 0xe2, 0x2b, 0x88, 0x43, 0x5f, + 0xdf, 0xb3, 0xaa, 0x4c, 0x5e, 0xcf, 0x38, 0x90, 0x0a, 0x17, 0x58, 0x5b, 0xf1, 0x63, 0xed, 0x16, + 0x3f, 0x89, 0x5d, 0xb8, 0x46, 0x45, 0x0d, 0x5d, 0x2b, 0x0b, 0x3d, 0x6d, 0x5f, 0x9d, 0x90, 0xe6, + 0x94, 0xc1, 0x54, 0x32, 0xe0, 0xf9, 0xd4, 0xc3, 0x56, 0xd8, 0x0a, 0xd7, 0x87, 0x0e, 0xf8, 0x90, + 0x85, 0x49, 0x32, 0xe0, 0x95, 0x00, 0x99, 0x4d, 0x40, 0x9a, 0x6c, 0xdb, 0x2c, 0x34, 0xe2, 0xc6, + 0x88, 0x09, 0xa8, 0x6f, 0x4d, 0x82, 0x4d, 0x40, 0x5a, 0x9d, 0x71, 0x12, 0x41, 0x6a, 0x0b, 0x2b, + 0x16, 0x37, 0xb3, 0x37, 0x87, 0x0a, 0x1a, 0xb8, 0x24, 0x93, 0x08, 0x52, 0x3d, 0x22, 0x71, 0x78, + 0x2c, 0xf7, 0xee, 0x24, 0xee, 0x30, 0xde, 0x1a, 0xea, 0xf0, 0x84, 0x5e, 0xee, 0x44, 0x1c, 0x1e, + 0xab, 0x27, 0x01, 0xfd, 0x3c, 0x4c, 0x73, 0xf8, 0x56, 0xb8, 0x3d, 0xc2, 0x8d, 0x0d, 0xe2, 0x7f, + 0x32, 0xae, 0x39, 0x0f, 0xb3, 0xb2, 0x0c, 0x7c, 0xb2, 0xea, 0xdd, 0x19, 0x61, 0x65, 0x07, 0x00, + 0x30, 0xb3, 0xb2, 0x3e, 0x99, 0x58, 0x59, 0xd6, 0x4f, 0xf9, 0x5c, 0x77, 0x77, 0xa8, 0x95, 0x1d, + 0x3c, 0xeb, 0x44, 0xac, 0xec, 0x4b, 0x9f, 0x4a, 0x6a, 0x66, 0x33, 0xf8, 0x54, 0xf8, 0xca, 0xd0, + 0x9a, 0xf5, 0xe2, 0x48, 0x52, 0x33, 0xce, 0x43, 0x9a, 0x8d, 0xb9, 0xc4, 0x4c, 0xd3, 0x6f, 0x0e, + 0xbf, 0x3b, 0xa2, 0x1f, 0x74, 0x54, 0xdd, 0xd5, 0x57, 0xa6, 0x61, 0xcf, 0x50, 0x59, 0xfc, 0xec, + 0x3c, 0xd7, 0xd4, 0x5b, 0xa3, 0x0d, 0x55, 0xd8, 0x85, 0x00, 0x9e, 0xa1, 0xea, 0x49, 0xa4, 0x45, + 0x65, 0x07, 0x0b, 0xe9, 0xf8, 0x5e, 0x1a, 0x71, 0xcd, 0x45, 0xdf, 0xf1, 0x4e, 0x5a, 0x54, 0x8f, + 0xe8, 0x0f, 0xa1, 0x2e, 0xbb, 0x83, 0xa5, 0xb0, 0x3c, 0x7a, 0x08, 0xf5, 0xde, 0x02, 0xe3, 0x0d, + 0x21, 0x4e, 0xf6, 0xe6, 0x4c, 0xd7, 0xc3, 0x78, 0x7b, 0xf4, 0x9c, 0xd9, 0xef, 0x5a, 0xb0, 0x39, + 0x93, 0xfb, 0x14, 0xbf, 0x2c, 0xc0, 0x22, 0x2b, 0x1b, 0x5d, 0x5f, 0x3c, 0x96, 0xbd, 0x55, 0xda, + 0xc0, 0x19, 0x97, 0x7b, 0xf4, 0x05, 0xef, 0x0e, 0x2b, 0xee, 0x19, 0x8b, 0xd7, 0xd5, 0x09, 0xe9, + 0x0d, 0x65, 0x54, 0xbe, 0xd2, 0x34, 0xdf, 0x79, 0xf6, 0x4e, 0xf5, 0xe6, 0xf2, 0xf9, 0xb5, 0x44, + 0xf2, 0x52, 0xbe, 0xb0, 0x96, 0x48, 0x5e, 0xce, 0x17, 0xd7, 0x12, 0xc9, 0xd7, 0xf2, 0xaf, 0x8b, + 0x7f, 0x71, 0x19, 0x32, 0x2e, 0xf2, 0x63, 0x88, 0xe8, 0x7e, 0x10, 0x11, 0x5d, 0x19, 0x86, 0x88, + 0x38, 0x56, 0xe4, 0x90, 0xe8, 0x7e, 0x10, 0x12, 0x5d, 0x19, 0x06, 0x89, 0x7c, 0x1e, 0x82, 0x89, + 0x1a, 0xc3, 0x30, 0xd1, 0x9d, 0x31, 0x30, 0x91, 0x27, 0xaa, 0x1f, 0x14, 0xad, 0x0e, 0x82, 0xa2, + 0xeb, 0xa3, 0x41, 0x91, 0x27, 0x2a, 0x80, 0x8a, 0x1e, 0xf6, 0xa1, 0xa2, 0xab, 0x23, 0x50, 0x91, + 0xc7, 0xef, 0xc2, 0xa2, 0xf5, 0x50, 0x58, 0x74, 0xf3, 0x2c, 0x58, 0xe4, 0xc9, 0xe9, 0xc1, 0x45, + 0xef, 0xf4, 0xe0, 0xa2, 0x85, 0xa1, 0xb8, 0xc8, 0xe3, 0x66, 0xc0, 0xe8, 0x83, 0x7e, 0x60, 0x74, + 0x75, 0x04, 0x30, 0xf2, 0x6b, 0xc0, 0x91, 0x51, 0x35, 0x0c, 0x19, 0xdd, 0x38, 0x03, 0x19, 0x79, + 0x52, 0x82, 0xd0, 0xa8, 0x1a, 0x06, 0x8d, 0x6e, 0x9c, 0x01, 0x8d, 0xfa, 0x24, 0x31, 0x6c, 0xb4, + 0x19, 0x8e, 0x8d, 0x6e, 0x9d, 0x89, 0x8d, 0x3c, 0x69, 0xbd, 0xe0, 0x68, 0x39, 0x00, 0x8e, 0xde, + 0x18, 0x02, 0x8e, 0x3c, 0x56, 0x82, 0x8e, 0xbe, 0x36, 0x80, 0x8e, 0xc4, 0x51, 0xe8, 0xc8, 0xe3, + 0xf5, 0xe0, 0xd1, 0xb3, 0x21, 0xf0, 0xe8, 0xf6, 0xd9, 0xf0, 0xc8, 0x13, 0xd6, 0x87, 0x8f, 0x94, + 0x91, 0xf8, 0xe8, 0xad, 0x31, 0xf1, 0x91, 0x27, 0x3d, 0x0c, 0x20, 0xbd, 0xd7, 0x0b, 0x90, 0x16, + 0x87, 0x03, 0x24, 0x4f, 0x0c, 0x47, 0x48, 0xeb, 0xa1, 0x08, 0xe9, 0xe6, 0x59, 0x08, 0xc9, 0x1f, + 0x07, 0x41, 0x88, 0xb4, 0x19, 0x0e, 0x91, 0x6e, 0x9d, 0x09, 0x91, 0xfc, 0xe6, 0xef, 0xc1, 0x48, + 0xeb, 0xa1, 0x18, 0xe9, 0xe6, 0x59, 0x18, 0xc9, 0x2f, 0x5c, 0x10, 0x24, 0xbd, 0x18, 0x0a, 0x92, + 0xee, 0x8e, 0x03, 0x92, 0x3c, 0xa1, 0x03, 0x28, 0xe9, 0xe3, 0xe1, 0x28, 0xe9, 0x2b, 0xe7, 0xb8, + 0x06, 0x34, 0x14, 0x26, 0x7d, 0x6d, 0x00, 0x26, 0x89, 0xa3, 0x60, 0x92, 0xdf, 0x9f, 0x5d, 0x9c, + 0xa4, 0x8c, 0x44, 0x35, 0x6f, 0x8d, 0x89, 0x6a, 0xfc, 0xce, 0x17, 0x02, 0x6b, 0x2a, 0x21, 0xb0, + 0xe6, 0xfa, 0x68, 0x58, 0xe3, 0x9b, 0x73, 0x1f, 0xd7, 0x54, 0xc3, 0x70, 0xcd, 0x8d, 0x33, 0x70, + 0x8d, 0x6f, 0x85, 0x02, 0xc0, 0xe6, 0x61, 0x1f, 0xb0, 0xb9, 0x7a, 0x66, 0x60, 0x55, 0x00, 0xd9, + 0x94, 0x06, 0x91, 0xcd, 0xb5, 0x91, 0xc8, 0xc6, 0x93, 0xe0, 0x43, 0x9b, 0x87, 0x7d, 0xd0, 0xe6, + 0xea, 0x08, 0x68, 0xe3, 0x17, 0x80, 0x63, 0x1b, 0x6d, 0x34, 0xb6, 0x59, 0x1a, 0x17, 0xdb, 0x78, + 0x82, 0x43, 0xc1, 0xcd, 0x66, 0x38, 0xb8, 0xb9, 0x35, 0x66, 0x94, 0xc1, 0x00, 0xba, 0xa9, 0x86, + 0xa1, 0x9b, 0x1b, 0x67, 0xa0, 0x9b, 0xe0, 0x1c, 0xe2, 0xc1, 0x9b, 0x6a, 0x18, 0xbc, 0xb9, 0x71, + 0x06, 0xbc, 0xf1, 0x25, 0x05, 0xf0, 0x4d, 0x63, 0x18, 0xbe, 0xb9, 0x33, 0x06, 0xbe, 0xf1, 0x9d, + 0x97, 0x3e, 0x80, 0xf3, 0x61, 0x3f, 0xc0, 0x11, 0x47, 0x01, 0x1c, 0x7f, 0x44, 0xba, 0x08, 0x67, + 0x33, 0x1c, 0xe1, 0xdc, 0x3a, 0x13, 0xe1, 0x04, 0x8d, 0x64, 0x00, 0xe2, 0xac, 0x87, 0x42, 0x9c, + 0x9b, 0x67, 0x41, 0x1c, 0xdf, 0x48, 0x06, 0x31, 0xce, 0x87, 0xfd, 0x18, 0x47, 0x1c, 0x85, 0x71, + 0xfc, 0xca, 0xb9, 0x20, 0xa7, 0x1a, 0x06, 0x72, 0x6e, 0x9c, 0x01, 0x72, 0xfc, 0xc6, 0x0b, 0xa0, + 0x1c, 0x65, 0x24, 0xca, 0x79, 0x6b, 0x4c, 0x94, 0xd3, 0x67, 0xb8, 0x7a, 0x61, 0x4e, 0x35, 0x0c, + 0xe6, 0xdc, 0x38, 0x03, 0xe6, 0x04, 0x0a, 0xeb, 0xe3, 0x9c, 0xcd, 0x70, 0x9c, 0x73, 0xeb, 0x4c, + 0x9c, 0xd3, 0x37, 0x9a, 0x5c, 0xa0, 0xb3, 0x1e, 0x0a, 0x74, 0x6e, 0x9e, 0x05, 0x74, 0xfa, 0x26, + 0x3e, 0xee, 0x1c, 0xfc, 0xca, 0xf8, 0x48, 0xe7, 0xbd, 0xf3, 0x23, 0x1d, 0xef, 0x9d, 0x91, 0x40, + 0x9d, 0xb5, 0x44, 0xf2, 0xf5, 0xfc, 0x1b, 0xe2, 0x7f, 0x9e, 0x86, 0xa9, 0xaa, 0x1f, 0xbc, 0xe3, + 0x17, 0x53, 0xf8, 0xe2, 0xae, 0xe5, 0x42, 0xab, 0x64, 0xe0, 0x53, 0xf3, 0x79, 0xf6, 0xc5, 0x8f, + 0x83, 0x57, 0x13, 0x72, 0xd6, 0x57, 0x38, 0x91, 0x8e, 0xde, 0x81, 0x4c, 0xd7, 0xc6, 0x96, 0xdc, + 0xb1, 0x74, 0xd3, 0xd2, 0x1d, 0x76, 0x6c, 0x47, 0x28, 0xe5, 0x3f, 0x3d, 0x59, 0x98, 0xd9, 0xb6, + 0xb1, 0xb5, 0xc5, 0xe9, 0xd2, 0x4c, 0x37, 0xf0, 0xe4, 0x7e, 0x54, 0x6e, 0x72, 0xfc, 0x8f, 0xca, + 0x3d, 0x83, 0xbc, 0x85, 0x15, 0xad, 0xc7, 0x91, 0x61, 0x77, 0x57, 0x85, 0x77, 0x3d, 0x7a, 0xac, + 0xce, 0xcd, 0x49, 0xef, 0xb0, 0xca, 0x59, 0xbd, 0x44, 0x74, 0x0f, 0x2e, 0xb4, 0x95, 0x23, 0x1a, + 0x76, 0x2a, 0xbb, 0xbe, 0x21, 0x0d, 0x25, 0x65, 0xdf, 0x6b, 0x43, 0x6d, 0xe5, 0x88, 0x7e, 0xa1, + 0x8e, 0x25, 0xd1, 0xcf, 0xce, 0xdc, 0x80, 0xac, 0xa6, 0xdb, 0x8e, 0x6e, 0xa8, 0x0e, 0xbf, 0xd0, + 0x99, 0x5d, 0x86, 0x9c, 0x71, 0xa9, 0xec, 0xd6, 0xe6, 0xbb, 0x30, 0xcb, 0x8f, 0x35, 0x04, 0x76, + 0x1a, 0xf9, 0xa5, 0xc8, 0x2c, 0xc1, 0xdb, 0x5c, 0x44, 0x65, 0xc8, 0x35, 0x15, 0x07, 0x1f, 0x2a, + 0xc7, 0xb2, 0x7b, 0xf6, 0x2e, 0x4d, 0x2f, 0x28, 0x7d, 0xed, 0xf4, 0x64, 0x21, 0xf3, 0x98, 0x25, + 0x0d, 0x1c, 0xc1, 0xcb, 0x34, 0x03, 0x09, 0x1a, 0xba, 0x05, 0x39, 0xc5, 0x3e, 0x36, 0x54, 0xaa, + 0x1e, 0x6c, 0xd8, 0x5d, 0x9b, 0x22, 0x93, 0xa4, 0x94, 0xa5, 0xe4, 0xb2, 0x4b, 0x45, 0x57, 0x61, + 0x86, 0xc7, 0xfc, 0xb3, 0x4f, 0x5e, 0xe5, 0x68, 0x55, 0xf9, 0xb7, 0x51, 0xe8, 0x57, 0xaf, 0xd0, + 0x43, 0x28, 0xf2, 0x2f, 0x3f, 0x1c, 0x2a, 0x96, 0x26, 0x53, 0xad, 0xfb, 0xbd, 0x3c, 0x4f, 0xc5, + 0x5e, 0x62, 0x5f, 0x7a, 0x20, 0x19, 0x88, 0xaa, 0xfd, 0xab, 0x2f, 0x36, 0x61, 0x56, 0x6d, 0xe9, + 0x1e, 0x90, 0x60, 0x35, 0x9f, 0x1d, 0x6a, 0xae, 0xcb, 0x34, 0xaf, 0xbf, 0xd3, 0x9a, 0x53, 0x7b, + 0x09, 0xa8, 0x0e, 0xf4, 0x96, 0x20, 0xb9, 0x63, 0xb6, 0x74, 0xf5, 0x98, 0x62, 0x88, 0xde, 0x1b, + 0xec, 0x47, 0x7e, 0x47, 0xe2, 0x85, 0xa2, 0x3b, 0x5b, 0x94, 0x53, 0x82, 0x43, 0xef, 0x37, 0xbb, + 0x34, 0x7a, 0x2d, 0x91, 0x9c, 0xc9, 0x67, 0xd6, 0x12, 0xc9, 0x6c, 0x3e, 0x27, 0xfe, 0x1d, 0x01, + 0x72, 0x7d, 0x65, 0x41, 0x55, 0xb8, 0xa0, 0x79, 0x43, 0x45, 0xe6, 0x67, 0xcb, 0x74, 0xd3, 0xe0, + 0x97, 0xf2, 0xcf, 0x7d, 0x7a, 0xb2, 0x90, 0xa3, 0xb9, 0x1f, 0x7b, 0x49, 0xd2, 0xbc, 0xcf, 0xe1, + 0x53, 0xd1, 0x7b, 0x90, 0x65, 0x5e, 0xa8, 0xf7, 0x15, 0x47, 0x1a, 0xcd, 0x5f, 0x9a, 0xfd, 0xf4, + 0x64, 0x21, 0x43, 0x5d, 0x4f, 0xf7, 0xae, 0x6b, 0x29, 0xd3, 0x0a, 0x3e, 0x8a, 0xbf, 0x2e, 0xc0, + 0x4c, 0xcf, 0xf9, 0xad, 0x87, 0x7d, 0x1b, 0xf1, 0x97, 0xc3, 0xe1, 0xeb, 0xb0, 0x58, 0xed, 0x24, + 0xef, 0xe7, 0x6e, 0xe4, 0xe6, 0xc2, 0x70, 0xf8, 0x43, 0x17, 0x73, 0xdc, 0x88, 0x0f, 0x97, 0xed, + 0xfd, 0xc4, 0x0f, 0x7e, 0xb8, 0x30, 0x21, 0xfe, 0x65, 0x02, 0x32, 0xbd, 0xa7, 0xb5, 0x6a, 0x7d, + 0xe5, 0x0a, 0x9b, 0x5e, 0x7a, 0x38, 0x96, 0x46, 0x5c, 0xe2, 0x99, 0xf2, 0xbf, 0x7e, 0xc1, 0x8a, + 0xb9, 0x38, 0x22, 0xdc, 0x20, 0x58, 0x4e, 0x9f, 0xb1, 0xf8, 0x5f, 0xe2, 0x9e, 0x99, 0x5e, 0x82, + 0x49, 0x7a, 0xfd, 0x12, 0x2f, 0x5a, 0xa1, 0xbf, 0xf7, 0x10, 0x97, 0x9b, 0xa4, 0x4b, 0x2c, 0x1b, + 0x35, 0xeb, 0x8d, 0x2f, 0xf0, 0xb6, 0x45, 0x7f, 0x34, 0x9d, 0xff, 0xdb, 0x9b, 0xfc, 0xf2, 0xcf, + 0xc9, 0xf3, 0x5d, 0xfe, 0x89, 0x7e, 0x09, 0x72, 0xaa, 0xd9, 0x6a, 0xb1, 0x99, 0x97, 0x19, 0xb6, + 0xc1, 0x1b, 0x6b, 0xa8, 0x08, 0xfe, 0xb9, 0xd4, 0x25, 0xef, 0xb3, 0xa9, 0x4b, 0x12, 0xff, 0x6c, + 0x6a, 0x20, 0x9c, 0x36, 0xeb, 0x09, 0x63, 0xf6, 0xb0, 0x2f, 0xb2, 0x77, 0xfa, 0x55, 0x22, 0x7b, + 0x59, 0x34, 0x38, 0xef, 0x80, 0x7f, 0x2a, 0xf0, 0x30, 0x95, 0x27, 0xa6, 0xb9, 0xdf, 0xf5, 0x22, + 0x72, 0x8b, 0xc1, 0x2b, 0x34, 0x93, 0x9f, 0x9e, 0x2c, 0x24, 0x24, 0xef, 0x0e, 0xcd, 0xb0, 0x09, + 0x24, 0xf6, 0xd3, 0x4d, 0x20, 0x57, 0x61, 0xa6, 0x63, 0xe1, 0x5d, 0xec, 0xa8, 0x7b, 0xb2, 0xd1, + 0x6d, 0xf3, 0x63, 0x44, 0x69, 0x97, 0xb6, 0xd9, 0x6d, 0xa3, 0x3b, 0x90, 0xf7, 0xb2, 0x70, 0xc4, + 0xef, 0xde, 0x6a, 0xe6, 0xd2, 0xf9, 0xfa, 0x80, 0xf8, 0xff, 0x04, 0x98, 0xeb, 0xa9, 0x13, 0x1f, + 0x5a, 0x6b, 0x90, 0xf6, 0xad, 0x8a, 0x5d, 0x10, 0xce, 0x19, 0x58, 0x1a, 0x64, 0x46, 0x32, 0x5c, + 0x74, 0x5f, 0x4b, 0x3f, 0x1c, 0xe1, 0x8b, 0x8d, 0x9d, 0x53, 0xec, 0x05, 0x5f, 0xce, 0x6a, 0xe0, + 0x05, 0xde, 0x58, 0x8b, 0x8f, 0x35, 0xd6, 0xc4, 0xdf, 0x12, 0x20, 0x4f, 0x5f, 0xf0, 0x08, 0x63, + 0x2d, 0x12, 0x23, 0xe7, 0xc6, 0x7d, 0xc7, 0xc6, 0x3f, 0xe1, 0xd3, 0xf3, 0xb1, 0x9b, 0x78, 0xef, + 0xc7, 0x6e, 0xc4, 0x1f, 0x0a, 0x90, 0xf5, 0x4a, 0xc8, 0x3e, 0x40, 0x39, 0xe2, 0xa6, 0xd6, 0x57, + 0xfb, 0xf8, 0xa2, 0x7b, 0x7b, 0xcb, 0x58, 0xdf, 0xc4, 0x0c, 0xde, 0xde, 0xc2, 0x3e, 0x1a, 0xf8, + 0x13, 0xb7, 0xe7, 0x90, 0x22, 0x96, 0xfd, 0x5b, 0x3a, 0x5e, 0xe1, 0xb0, 0xd3, 0x0f, 0x04, 0xfa, + 0xf1, 0x5e, 0xb3, 0x75, 0xc0, 0x6e, 0xcc, 0x19, 0xcb, 0xfa, 0x7d, 0x9d, 0x47, 0x67, 0x01, 0x5f, + 0x1c, 0xd4, 0x1a, 0xf5, 0xa8, 0x3e, 0x9b, 0xe6, 0x96, 0xa5, 0x61, 0x8b, 0x8f, 0x02, 0xed, 0x40, + 0xfb, 0x10, 0x51, 0xf6, 0x58, 0x86, 0xdd, 0x55, 0x36, 0xeb, 0x72, 0x7f, 0x12, 0x6c, 0xd0, 0xca, + 0x01, 0x81, 0x96, 0x0f, 0x20, 0x7e, 0xa0, 0xb4, 0x46, 0xc5, 0xb6, 0xf5, 0x74, 0x00, 0x89, 0xe4, + 0x46, 0x8f, 0x7a, 0xee, 0x48, 0x89, 0x0d, 0x87, 0x41, 0x83, 0x2d, 0xd3, 0x73, 0x97, 0xca, 0xbb, + 0xbd, 0x43, 0x66, 0xe4, 0xeb, 0x83, 0x63, 0xe7, 0xfd, 0xc4, 0x8f, 0x7e, 0xb8, 0x20, 0x88, 0x1f, + 0x00, 0x92, 0xb0, 0x8d, 0x9d, 0x67, 0x5d, 0xd3, 0xf2, 0xef, 0x9b, 0xb9, 0xd9, 0xf7, 0x11, 0xa1, + 0xc9, 0x52, 0xfa, 0x34, 0xec, 0x46, 0xa9, 0x0b, 0x30, 0xd7, 0xc3, 0xcd, 0x6c, 0x8e, 0xf8, 0x2e, + 0x5c, 0x7e, 0x6c, 0xda, 0xb6, 0xde, 0x21, 0x78, 0x9a, 0x0e, 0x6e, 0x32, 0xc3, 0x78, 0x56, 0x36, + 0xd9, 0xa1, 0x2b, 0x2b, 0x06, 0xb3, 0x46, 0x29, 0xc9, 0x7b, 0x16, 0xff, 0xad, 0x00, 0x97, 0x06, + 0x39, 0x99, 0x96, 0xc3, 0x4e, 0x8a, 0x4e, 0xab, 0xa6, 0x7f, 0x4b, 0xe1, 0xd9, 0x9d, 0xde, 0xcd, + 0x4e, 0x1c, 0x5e, 0xfe, 0x4e, 0xb9, 0xad, 0x50, 0x2b, 0xc4, 0x0f, 0xbf, 0x67, 0x39, 0x79, 0x83, + 0x51, 0x7d, 0x83, 0x94, 0x18, 0xcf, 0x20, 0x35, 0x20, 0xb7, 0x66, 0xea, 0x06, 0xf1, 0xab, 0xdd, + 0xfa, 0xae, 0x40, 0x76, 0x47, 0x37, 0x14, 0xeb, 0x58, 0x76, 0xef, 0x14, 0x60, 0x1d, 0xa5, 0x18, + 0x56, 0x58, 0x96, 0x43, 0xca, 0x30, 0x0e, 0xfe, 0x28, 0xfe, 0x58, 0x80, 0xbc, 0x2f, 0x96, 0x1b, + 0xf6, 0x37, 0x01, 0xd4, 0x56, 0xd7, 0x76, 0xb0, 0xe5, 0xb6, 0xd2, 0x0c, 0x0b, 0x90, 0x2f, 0x33, + 0x6a, 0x6d, 0x55, 0x4a, 0xf1, 0x0c, 0x35, 0x0d, 0x5d, 0xeb, 0xbd, 0x9b, 0x63, 0xb2, 0x04, 0xa7, + 0x03, 0x37, 0x72, 0x90, 0x66, 0x27, 0x03, 0xca, 0x43, 0x70, 0xbc, 0xd9, 0xdd, 0xcf, 0x1c, 0xd0, + 0x23, 0xe5, 0x24, 0xdf, 0x0a, 0x64, 0x89, 0xd7, 0x70, 0x80, 0xbd, 0x2a, 0x25, 0xce, 0xae, 0x12, + 0xe3, 0x70, 0xab, 0xf4, 0x7b, 0xc4, 0x65, 0x66, 0xad, 0xe1, 0xb5, 0xf0, 0x08, 0xc3, 0xf8, 0x73, + 0xc1, 0x8b, 0x35, 0xc7, 0xbb, 0x7a, 0x94, 0x3a, 0x34, 0x5f, 0x83, 0xa4, 0xfb, 0x71, 0x75, 0x3e, + 0x40, 0x2e, 0x2f, 0xb1, 0xaf, 0xaf, 0x2f, 0xb9, 0x5f, 0x5f, 0x5f, 0x5a, 0xe5, 0x19, 0xd8, 0x6c, + 0xf0, 0x83, 0x3f, 0x5f, 0x10, 0x24, 0x8f, 0xe9, 0x6e, 0x9d, 0xf4, 0xf0, 0x81, 0xc9, 0x1c, 0x65, + 0x01, 0x02, 0xdf, 0x9b, 0xe2, 0xdf, 0xdb, 0x5e, 0x59, 0x95, 0xb7, 0x37, 0xcb, 0x4f, 0x37, 0x36, + 0x6a, 0x8d, 0x46, 0x65, 0x35, 0x2f, 0xa0, 0x3c, 0xcc, 0xf4, 0x7c, 0xad, 0x2a, 0xc6, 0xbe, 0xc0, + 0x7d, 0xf7, 0xe7, 0x00, 0xfc, 0x0f, 0xdf, 0x11, 0x59, 0xeb, 0x95, 0x8f, 0xe4, 0xe7, 0x2b, 0x4f, + 0xb6, 0x2b, 0xf5, 0xfc, 0x04, 0x42, 0x90, 0x2d, 0xad, 0x34, 0xca, 0x55, 0x59, 0xaa, 0xd4, 0xb7, + 0x9e, 0x6e, 0xd6, 0x2b, 0xee, 0x97, 0xbb, 0xef, 0xae, 0xc2, 0x4c, 0xf0, 0xb6, 0x25, 0x34, 0x07, + 0xb9, 0x72, 0xb5, 0x52, 0x5e, 0x97, 0x9f, 0xd7, 0x56, 0xe4, 0x67, 0xdb, 0x95, 0xed, 0x4a, 0x7e, + 0x82, 0x16, 0x8d, 0x12, 0x1f, 0x6d, 0x3f, 0x79, 0x92, 0x17, 0x50, 0x0e, 0xd2, 0xec, 0x99, 0x7e, + 0xd9, 0x2a, 0x1f, 0xbb, 0xbb, 0x01, 0xe9, 0xc0, 0xe5, 0xcc, 0xe4, 0x75, 0x5b, 0xdb, 0xf5, 0xaa, + 0xdc, 0xa8, 0x6d, 0x54, 0xea, 0x8d, 0x95, 0x8d, 0x2d, 0x26, 0x83, 0xd2, 0x56, 0x4a, 0x4f, 0xa5, + 0x46, 0x5e, 0xf0, 0x9e, 0x1b, 0x4f, 0xb7, 0xcb, 0x55, 0xb7, 0x1a, 0x62, 0x22, 0x19, 0xcf, 0xc7, + 0xef, 0xfe, 0x4d, 0x01, 0x2e, 0x0d, 0xb9, 0x79, 0x08, 0xa5, 0x61, 0x7a, 0xdb, 0xa0, 0x37, 0xd5, + 0xe6, 0x27, 0x50, 0x26, 0x70, 0xf9, 0x50, 0x5e, 0x40, 0x49, 0x76, 0xfd, 0x4b, 0x3e, 0x86, 0xa6, + 0x20, 0x56, 0x7f, 0x90, 0x8f, 0x93, 0x92, 0x06, 0xee, 0xee, 0xc9, 0x27, 0x50, 0x8a, 0xdf, 0x1a, + 0x92, 0x9f, 0x44, 0x33, 0xfe, 0xe5, 0x1d, 0xf9, 0x29, 0x22, 0xca, 0xbb, 0x04, 0x23, 0x3f, 0x7d, + 0xf7, 0x2a, 0x04, 0x6e, 0x08, 0x40, 0x00, 0x53, 0x4f, 0x14, 0x07, 0xdb, 0x4e, 0x7e, 0x02, 0x4d, + 0x43, 0x7c, 0xa5, 0xd5, 0xca, 0x0b, 0xf7, 0xff, 0x75, 0x02, 0x92, 0xee, 0x87, 0x97, 0xd0, 0x13, + 0x98, 0x64, 0xeb, 0xe9, 0x0b, 0xc3, 0x71, 0x06, 0x1d, 0xbc, 0xc5, 0xc5, 0xb3, 0x80, 0x88, 0x38, + 0x81, 0xfe, 0x3a, 0xa4, 0x03, 0x8e, 0x17, 0x1a, 0xba, 0x26, 0xd8, 0xe3, 0x6c, 0x16, 0x6f, 0x9e, + 0x95, 0xcd, 0x93, 0xff, 0x02, 0x52, 0x9e, 0x05, 0x47, 0xd7, 0x46, 0xd9, 0x77, 0x57, 0xf6, 0xe8, + 0x49, 0x80, 0x8c, 0x35, 0x71, 0xe2, 0x6d, 0x01, 0x59, 0x80, 0x06, 0x8d, 0x2d, 0x0a, 0x0b, 0xb3, + 0x18, 0x6a, 0xcd, 0x8b, 0x77, 0xc7, 0xca, 0xed, 0xbf, 0x93, 0x28, 0xcb, 0x9f, 0x31, 0xc2, 0x95, + 0x35, 0x30, 0x1f, 0x85, 0x2b, 0x2b, 0x64, 0xe2, 0x99, 0x40, 0xcf, 0x20, 0x41, 0x2c, 0x25, 0x0a, + 0x73, 0x45, 0xfb, 0x2c, 0x73, 0xf1, 0xda, 0xc8, 0x3c, 0xae, 0xc8, 0xd2, 0x9d, 0x1f, 0xfd, 0x8f, + 0x2b, 0x13, 0x3f, 0x3a, 0xbd, 0x22, 0xfc, 0xf8, 0xf4, 0x8a, 0xf0, 0x67, 0xa7, 0x57, 0x84, 0xff, + 0x7e, 0x7a, 0x45, 0xf8, 0xb5, 0x9f, 0x5c, 0x99, 0xf8, 0xf1, 0x4f, 0xae, 0x4c, 0xfc, 0xd9, 0x4f, + 0xae, 0x4c, 0x7c, 0x3c, 0xcd, 0xb9, 0x77, 0xa6, 0xa8, 0x51, 0x79, 0xf0, 0x57, 0x01, 0x00, 0x00, + 0xff, 0xff, 0xd3, 0x86, 0x42, 0x10, 0x0d, 0x83, 0x00, 0x00, } diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index ab86df7a67b8..ec623f8c92d1 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -316,7 +316,8 @@ message ClearRangeRequest { // that time (plus max offset) to have passed at which point they can reuse // the span they cleared without fear of this request being replayed later and // clearing subsequent writes. - util.hlc.Timestamp deadline = 2 [(gogoproto.nullable) = false]; + util.hlc.Timestamp deadline = 2 [(gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp"]; } // A ClearRangeResponse is the return value from the ClearRange() method. @@ -335,7 +336,8 @@ message RevertRangeRequest { // any MVCC key with a strictly higher timestamp. TargetTime must be higher // than the GC Threshold for the replica - so that it is assured that the keys // for that time are still there — or the request will fail. - util.hlc.Timestamp target_time = 2 [(gogoproto.nullable) = false]; + util.hlc.Timestamp target_time = 2 [(gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp"]; } // A RevertRangeResponse is the return value from the RevertRange() method. @@ -589,7 +591,7 @@ message EndTxnRequest { // If EndTxn(Commit=true) finds that the txn's timestamp has been pushed above // this deadline, an error will be returned and the client is supposed to // rollback the txn. - util.hlc.Timestamp deadline = 3; + util.hlc.Timestamp deadline = 3 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp"]; // commit triggers. Note that commit triggers are for // internal use only and will cause an error if requested through the // external-facing KV API. @@ -676,7 +678,8 @@ message EndTxnResponse { bool one_phase_commit = 4; // The commit timestamp of the STAGING transaction record written // by the request. Only set if the transaction record was staged. - util.hlc.Timestamp staging_timestamp = 5 [(gogoproto.nullable) = false]; + util.hlc.Timestamp staging_timestamp = 5 [(gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp"]; } // An AdminSplitRequest is the argument to the AdminSplit() method. The @@ -847,11 +850,13 @@ message GCRequest { message GCKey { bytes key = 1 [(gogoproto.casttype) = "Key"]; - util.hlc.Timestamp timestamp = 2 [(gogoproto.nullable) = false]; + util.hlc.Timestamp timestamp = 2 [(gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp"]; } repeated GCKey keys = 3 [(gogoproto.nullable) = false]; // Threshold is the expiration timestamp. - util.hlc.Timestamp threshold = 4 [(gogoproto.nullable) = false]; + util.hlc.Timestamp threshold = 4 [(gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp"]; reserved 5; } @@ -904,7 +909,8 @@ message PushTxnRequest { // PushTo is the timestamp which PusheeTxn should be pushed to. During // conflict resolution, it should be set just after the timestamp of the // conflicting read or write. - util.hlc.Timestamp push_to = 4 [(gogoproto.nullable) = false]; + util.hlc.Timestamp push_to = 4 [(gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp"]; // Readers set this to PUSH_TIMESTAMP to move pushee_txn's provisional // commit timestamp forward. Writers set this to PUSH_ABORT to request // that pushee_txn be aborted if possible. Inconsistent readers set @@ -1072,7 +1078,8 @@ message ResolveIntentRangeRequest { // The minimum timestamp for any intents written by this // transaction. If present, this value can be used to optimize the // iteration over the span to find intents to resolve. - util.hlc.Timestamp min_timestamp = 5 [(gogoproto.nullable) = false]; + util.hlc.Timestamp min_timestamp = 5 [(gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp"]; // The list of ignored seqnum ranges as per the Transaction record. repeated storage.enginepb.IgnoredSeqNumRange ignored_seqnums = 6 [ (gogoproto.nullable) = false, @@ -1333,7 +1340,8 @@ message FileEncryptionOptions { message ExportRequest { RequestHeader header = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true]; ExternalStorage storage = 2 [(gogoproto.nullable) = false]; - util.hlc.Timestamp start_time = 3 [(gogoproto.nullable) = false]; + util.hlc.Timestamp start_time = 3 [(gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp"]; MVCCFilter mvcc_filter = 4 [(gogoproto.customname) = "MVCCFilter"]; // Return the exported SST data in the response. @@ -1417,7 +1425,8 @@ message ExportResponse { ResponseHeader header = 1 [(gogoproto.nullable) = false, (gogoproto.embed) = true]; repeated File files = 2 [(gogoproto.nullable) = false]; - util.hlc.Timestamp start_time = 3 [(gogoproto.nullable) = false]; + util.hlc.Timestamp start_time = 3 [(gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp"]; } // ImportRequest is the argument to the Import() method, to bulk load key/value @@ -1438,7 +1447,8 @@ message ImportRequest { Span data_span = 3 [(gogoproto.nullable) = false]; // EndTime, if not the zero value, will cause only entries before it to be // imported. - util.hlc.Timestamp end_time = 6 [(gogoproto.nullable) = false]; + util.hlc.Timestamp end_time = 6 [(gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp"]; reserved 4; message TableRekey { @@ -1502,7 +1512,8 @@ message AdminVerifyProtectedTimestampRequest { ]; // Protected is the timestamp at which the record with RecordID protects. - util.hlc.Timestamp protected = 2 [(gogoproto.nullable) = false]; + util.hlc.Timestamp protected = 2 [(gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp"]; // RecordAliveAt is a an hlc timestamp at which the record being verified is // known to exist. A value for RecordAliveAt is generally determined by @@ -1567,7 +1578,8 @@ message RefreshRequest { // refresh_from specifies the lower-bound of the verification. The request verifies that // there's no write in the range [refresh_from, txn.read_timestamp]. - util.hlc.Timestamp refresh_from = 3 [(gogoproto.nullable) = false]; + util.hlc.Timestamp refresh_from = 3 [(gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp"]; } // RefreshResponse is the response to a Refresh() operation. @@ -1584,7 +1596,8 @@ message RefreshRangeRequest { // refresh_from specifies the lower-bound of the verification. The request verifies that // there's no write in the range [refresh_from, txn.read_timestamp]. - util.hlc.Timestamp refresh_from = 3 [(gogoproto.nullable) = false]; + util.hlc.Timestamp refresh_from = 3 [(gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp"]; } // RefreshRangeResponse is the response to a RefreshRange() operation. @@ -1783,7 +1796,8 @@ message Header { // the server to set it from txn.ReadTimestamp. Also, for transactional // requests, writes are performed at the provisional commit timestamp // (txn.WriteTimestamp). - util.hlc.Timestamp timestamp = 1 [(gogoproto.nullable) = false]; + util.hlc.Timestamp timestamp = 1 [(gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp"]; // replica specifies the destination of the request. ReplicaDescriptor replica = 2 [(gogoproto.nullable) = false]; // range_id specifies the ID of the Raft consensus group which the key @@ -1987,7 +2001,8 @@ message BatchResponse { // which is awkward. We could consider making this field optional and only // populate it for non-transactional requests. The timestamp cache would then // use an accessor on BatchResponse to pick the one that matters. - util.hlc.Timestamp Timestamp = 2 [(gogoproto.nullable) = false]; + util.hlc.Timestamp Timestamp = 2 [(gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp"]; // txn is non-nil if the request specified a non-nil // transaction. The transaction timestamp and/or priority may have // been updated, depending on the outcome of the request. @@ -2070,7 +2085,9 @@ message RangeFeedValue { message RangeFeedCheckpoint { Span span = 1 [(gogoproto.nullable) = false]; util.hlc.Timestamp resolved_ts = 2 [ - (gogoproto.nullable) = false, (gogoproto.customname) = "ResolvedTS"]; + (gogoproto.nullable) = false, + (gogoproto.customname) = "ResolvedTS", + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp"]; } // RangeFeedError is a variant of RangeFeedEvent that indicates that an error diff --git a/pkg/roachpb/batch.go b/pkg/roachpb/batch.go index 08b78cdb61da..75091259401e 100644 --- a/pkg/roachpb/batch.go +++ b/pkg/roachpb/batch.go @@ -48,7 +48,7 @@ func (ba *BatchRequest) SetActiveTimestamp(nowFn func() hlc.Timestamp) error { } else { // When not transactional, allow empty timestamp and use nowFn instead if ba.Timestamp.IsEmpty() { - ba.Timestamp = nowFn() + ba.Timestamp = enginepb.TxnTimestamp(nowFn()) } } return nil @@ -59,7 +59,7 @@ func (ba *BatchRequest) SetActiveTimestamp(nowFn func() hlc.Timestamp) error { // request in the batch operates on a time span such as ExportRequest or // RevertRangeRequest, which both specify the start of that span in their // arguments while using ba.Timestamp to indicate the upper bound of that span. -func (ba BatchRequest) EarliestActiveTimestamp() hlc.Timestamp { +func (ba BatchRequest) EarliestActiveTimestamp() enginepb.TxnTimestamp { ts := ba.Timestamp for _, ru := range ba.Requests { switch t := ru.GetInner().(type) { diff --git a/pkg/roachpb/data.go b/pkg/roachpb/data.go index 605e68560118..66e19c360641 100644 --- a/pkg/roachpb/data.go +++ b/pkg/roachpb/data.go @@ -337,8 +337,8 @@ func MakeValueFromBytes(bs []byte) Value { // MakeValueFromBytesAndTimestamp returns a value with bytes, timestamp and // tag set. -func MakeValueFromBytesAndTimestamp(bs []byte, t hlc.Timestamp) Value { - v := Value{Timestamp: t} +func MakeValueFromBytesAndTimestamp(bs []byte, ts enginepb.TxnTimestamp) Value { + v := Value{Timestamp: ts} v.SetBytes(bs) return v } @@ -884,20 +884,21 @@ func MakeTransaction( name string, baseKey Key, userPriority UserPriority, now hlc.Timestamp, maxOffsetNs int64, ) Transaction { u := uuid.FastMakeV4() - maxTS := now.Add(maxOffsetNs, 0) + txnTS := enginepb.TxnTimestamp(now) + maxTS := enginepb.TxnTimestamp(now.Add(maxOffsetNs, 0)) return Transaction{ TxnMeta: enginepb.TxnMeta{ Key: baseKey, ID: u, - WriteTimestamp: now, - MinTimestamp: now, + WriteTimestamp: txnTS, + MinTimestamp: txnTS, Priority: MakePriority(userPriority), Sequence: 0, // 1-indexed, incremented before each Request }, Name: name, LastHeartbeat: now, - ReadTimestamp: now, + ReadTimestamp: txnTS, MaxTimestamp: maxTS, } } @@ -906,7 +907,8 @@ func MakeTransaction( // occurred, i.e. the maximum of ReadTimestamp and LastHeartbeat. func (t Transaction) LastActive() hlc.Timestamp { ts := t.LastHeartbeat - ts.Forward(t.ReadTimestamp) + // TODO(nvanbenschoten): try to forward by read timestamp? Or just don't try? + // ts.Forward(t.ReadTimestamp) return ts } @@ -1013,12 +1015,10 @@ func MakePriority(userPriority UserPriority) enginepb.TxnPriority { // transaction on restart is set to the maximum of the transaction's // timestamp and the specified timestamp. func (t *Transaction) Restart( - userPriority UserPriority, upgradePriority enginepb.TxnPriority, timestamp hlc.Timestamp, + userPriority UserPriority, upgradePriority enginepb.TxnPriority, timestamp enginepb.TxnTimestamp, ) { t.BumpEpoch() - if t.WriteTimestamp.Less(timestamp) { - t.WriteTimestamp = timestamp - } + t.WriteTimestamp.Forward(timestamp) t.ReadTimestamp = t.WriteTimestamp // Upgrade priority to the maximum of: // - the current transaction priority @@ -1045,7 +1045,7 @@ func (t *Transaction) BumpEpoch() { // Refresh reconfigures a transaction to account for a read refresh up to the // specified timestamp. For details about transaction read refreshes, see the // comment on txnSpanRefresher. -func (t *Transaction) Refresh(timestamp hlc.Timestamp) { +func (t *Transaction) Refresh(timestamp enginepb.TxnTimestamp) { t.WriteTimestamp.Forward(timestamp) t.ReadTimestamp.Forward(t.WriteTimestamp) t.WriteTooOld = false @@ -1425,7 +1425,7 @@ func PrepareTransactionForRetry( // to the specified timestamp to avoid a client-side transaction restart. If // true, returns a cloned, updated Transaction object with the provisional // commit timestamp and read timestamp set appropriately. -func PrepareTransactionForRefresh(txn *Transaction, timestamp hlc.Timestamp) (bool, *Transaction) { +func PrepareTransactionForRefresh(txn *Transaction, timestamp enginepb.TxnTimestamp) (bool, *Transaction) { if txn.CommitTimestampFixed { return false, nil } @@ -1467,23 +1467,25 @@ func CanTransactionRefresh(ctx context.Context, pErr *Error) (bool, *Transaction func readWithinUncertaintyIntervalRetryTimestamp( ctx context.Context, txn *Transaction, err *ReadWithinUncertaintyIntervalError, origin NodeID, -) hlc.Timestamp { +) enginepb.TxnTimestamp { // 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) - if !ok { - log.Fatalf(ctx, - "missing observed timestamp for node %d found on uncertainty restart. "+ - "err: %s. txn: %s. Observed timestamps: %v", - origin, err, txn, txn.ObservedTimestamps) - } - // Also forward by the existing timestamp. - ts.Forward(err.ExistingTimestamp.Next()) - return ts -} - -func writeTooOldRetryTimestamp(err *WriteTooOldError) hlc.Timestamp { + // TODO(nvanbenschoten) + // ts, ok := txn.GetObservedTimestamp(origin) + // if !ok { + // log.Fatalf(ctx, + // "missing observed timestamp for node %d found on uncertainty restart. "+ + // "err: %s. txn: %s. Observed timestamps: %v", + // origin, err, txn, txn.ObservedTimestamps) + // } + // // Also forward by the existing timestamp. + // ts.Forward(err.ExistingTimestamp.Next()) + // return ts + return enginepb.TxnTimestamp{} +} + +func writeTooOldRetryTimestamp(err *WriteTooOldError) enginepb.TxnTimestamp { return err.ActualTimestamp } diff --git a/pkg/roachpb/data.pb.go b/pkg/roachpb/data.pb.go index 113726cb254e..37f36c5a7814 100644 --- a/pkg/roachpb/data.pb.go +++ b/pkg/roachpb/data.pb.go @@ -10,8 +10,8 @@ import lock "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" 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_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 bytes "bytes" @@ -97,7 +97,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_31463b6ee037d927, []int{0} } // ReplicaChangeType is a parameter of ChangeReplicasTrigger. @@ -127,7 +127,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_31463b6ee037d927, []int{1} } // TransactionStatus specifies possible states for a transaction. @@ -179,7 +179,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_31463b6ee037d927, []int{2} } // Span is a key range with an inclusive start Key and an exclusive end Key. @@ -196,7 +196,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_31463b6ee037d927, []int{0} } func (m *Span) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -241,14 +241,14 @@ type Value struct { // Its contents may be modified on the next call to Value.SetFoo. RawBytes []byte `protobuf:"bytes,1,opt,name=raw_bytes,json=rawBytes,proto3" json:"raw_bytes,omitempty"` // Timestamp of value. - Timestamp hlc.Timestamp `protobuf:"bytes,2,opt,name=timestamp,proto3" json:"timestamp"` + Timestamp github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp `protobuf:"bytes,2,opt,name=timestamp,proto3,casttype=github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp" json:"timestamp"` } 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_31463b6ee037d927, []int{1} } func (m *Value) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -284,7 +284,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_31463b6ee037d927, []int{2} } func (m *KeyValue) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -322,7 +322,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_31463b6ee037d927, []int{3} } func (m *StoreIdent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -362,7 +362,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_31463b6ee037d927, []int{4} } func (m *SplitTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -410,7 +410,7 @@ func (m *MergeTrigger) Reset() { *m = MergeTrigger{} } func (m *MergeTrigger) String() string { return proto.CompactTextString(m) } func (*MergeTrigger) ProtoMessage() {} func (*MergeTrigger) Descriptor() ([]byte, []int) { - return fileDescriptor_data_11933f4ce39aae12, []int{5} + return fileDescriptor_data_31463b6ee037d927, []int{5} } func (m *MergeTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -480,7 +480,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_31463b6ee037d927, []int{6} } func (m *ChangeReplicasTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -522,7 +522,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_31463b6ee037d927, []int{7} } func (m *ModifiedSpanTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -568,7 +568,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_31463b6ee037d927, []int{8} } func (m *StickyBitTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -607,7 +607,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_31463b6ee037d927, []int{9} } func (m *InternalCommitTrigger) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -676,7 +676,7 @@ func (m *ObservedTimestamp) Reset() { *m = ObservedTimestamp{} } func (m *ObservedTimestamp) String() string { return proto.CompactTextString(m) } func (*ObservedTimestamp) ProtoMessage() {} func (*ObservedTimestamp) Descriptor() ([]byte, []int) { - return fileDescriptor_data_11933f4ce39aae12, []int{10} + return fileDescriptor_data_31463b6ee037d927, []int{10} } func (m *ObservedTimestamp) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -743,14 +743,14 @@ type Transaction struct { // advance "refreshing the read set". So, the read timestamp advances after a // successful refresh or, if the refresh is unsuccessful, after a transaction // restart. - ReadTimestamp hlc.Timestamp `protobuf:"bytes,15,opt,name=read_timestamp,json=readTimestamp,proto3" json:"read_timestamp"` + ReadTimestamp github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp `protobuf:"bytes,15,opt,name=read_timestamp,json=readTimestamp,proto3,casttype=github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp" json:"read_timestamp"` // Initial Timestamp + clock skew. Reads which encounter values with // timestamps between timestamp and max_timestamp trigger a txn // retry error, unless the node being read is listed in observed_timestamps // (in which case no more read uncertainty can occur). // The case max_timestamp < timestamp is possible for transactions which have // been pushed; in this case, max_timestamp should be ignored. - MaxTimestamp hlc.Timestamp `protobuf:"bytes,7,opt,name=max_timestamp,json=maxTimestamp,proto3" json:"max_timestamp"` + MaxTimestamp github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp `protobuf:"bytes,7,opt,name=max_timestamp,json=maxTimestamp,proto3,casttype=github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp" json:"max_timestamp"` // A list of pairs. The list maps NodeIDs to timestamps // as observed from their local clock during this transaction. The purpose of // this list is to avoid uncertainty related restarts which normally occur @@ -829,7 +829,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_31463b6ee037d927, []int{11} } func (m *Transaction) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -880,7 +880,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_31463b6ee037d927, []int{12} } func (m *TransactionRecord) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -920,7 +920,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_31463b6ee037d927, []int{13} } func (m *Intent) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -957,7 +957,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_31463b6ee037d927, []int{13, 0} } func (m *Intent_SingleKeySpan) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -994,7 +994,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_31463b6ee037d927, []int{14} } func (m *LockAcquisition) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1035,7 +1035,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_31463b6ee037d927, []int{15} } func (m *LockUpdate) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1072,7 +1072,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_31463b6ee037d927, []int{16} } func (m *SequencedWrite) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1137,7 +1137,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_31463b6ee037d927, []int{17} } func (m *Lease) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1173,7 +1173,7 @@ type AbortSpanEntry struct { Key Key `protobuf:"bytes,1,opt,name=key,proto3,casttype=Key" json:"key,omitempty"` // The candidate commit timestamp the transaction record held at the time // it was aborted. - Timestamp hlc.Timestamp `protobuf:"bytes,2,opt,name=timestamp,proto3" json:"timestamp"` + Timestamp github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp `protobuf:"bytes,2,opt,name=timestamp,proto3,casttype=github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp" json:"timestamp"` // The priority of the transaction. Priority github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnPriority `protobuf:"varint,3,opt,name=priority,proto3,casttype=github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnPriority" json:"priority,omitempty"` } @@ -1182,7 +1182,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_31463b6ee037d927, []int{18} } func (m *AbortSpanEntry) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1239,7 +1239,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_31463b6ee037d927, []int{19} } func (m *LeafTxnInputState) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1292,7 +1292,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_31463b6ee037d927, []int{20} } func (m *LeafTxnFinalState) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1327,7 +1327,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_31463b6ee037d927, []int{21} } func (m *RangeInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2628,7 +2628,7 @@ func NewPopulatedTransaction(r randyData, easy bool) *Transaction { this.Status = TransactionStatus([]int32{0, 3, 1, 2}[r.Intn(4)]) v5 := hlc.NewPopulatedTimestamp(r, easy) this.LastHeartbeat = *v5 - v6 := hlc.NewPopulatedTimestamp(r, easy) + v6 := github_com_cockroachdb_cockroach_pkg_storage_enginepb.NewPopulatedTxnTimestamp(r, easy) this.MaxTimestamp = *v6 if r.Intn(10) != 0 { v7 := r.Intn(5) @@ -2647,7 +2647,7 @@ func NewPopulatedTransaction(r randyData, easy bool) *Transaction { } } this.WriteTooOld = bool(bool(r.Intn(2) == 0)) - v11 := hlc.NewPopulatedTimestamp(r, easy) + v11 := github_com_cockroachdb_cockroach_pkg_storage_enginepb.NewPopulatedTxnTimestamp(r, easy) this.ReadTimestamp = *v11 this.CommitTimestampFixed = bool(bool(r.Intn(2) == 0)) if r.Intn(10) != 0 { @@ -2758,7 +2758,7 @@ func NewPopulatedAbortSpanEntry(r randyData, easy bool) *AbortSpanEntry { for i := 0; i < v27; i++ { this.Key[i] = byte(r.Intn(256)) } - v28 := hlc.NewPopulatedTimestamp(r, easy) + v28 := github_com_cockroachdb_cockroach_pkg_storage_enginepb.NewPopulatedTxnTimestamp(r, easy) this.Timestamp = *v28 this.Priority = github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnPriority(r.Int31()) if r.Intn(2) == 0 { @@ -6905,158 +6905,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_31463b6ee037d927) } + +var fileDescriptor_data_31463b6ee037d927 = []byte{ + // 2427 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xdc, 0x59, 0x4f, 0x6c, 0x1b, 0x59, + 0x19, 0xcf, 0xd8, 0x63, 0x7b, 0xfc, 0xf9, 0x4f, 0x26, 0xaf, 0x49, 0xeb, 0xa6, 0x10, 0x17, 0x2f, + 0xb0, 0xa5, 0x62, 0x1d, 0x91, 0x96, 0x15, 0x84, 0x22, 0x61, 0xc7, 0x6e, 0xd7, 0x6e, 0x6c, 0x77, + 0xc7, 0x93, 0x2e, 0xed, 0x22, 0x0d, 0xe3, 0x99, 0x17, 0x67, 0x88, 0x3d, 0xe3, 0xcc, 0x8c, 0xd3, + 0x98, 0x3b, 0x62, 0xb5, 0x48, 0xb0, 0x47, 0x8e, 0x15, 0xdc, 0x40, 0x02, 0x8e, 0x08, 0x89, 0x7b, + 0x8f, 0xbd, 0xed, 0x02, 0x92, 0x05, 0xe9, 0x65, 0xef, 0xdc, 0x22, 0x21, 0xa1, 0xf7, 0xde, 0xfc, + 0x4b, 0xea, 0x86, 0xa4, 0x5d, 0x96, 0x8a, 0x4b, 0x32, 0xef, 0xfb, 0xf3, 0x7b, 0xdf, 0xfb, 0xde, + 0xf7, 0xbe, 0xf7, 0xbd, 0xcf, 0x80, 0x6c, 0x4b, 0xd5, 0x76, 0x46, 0xbd, 0x55, 0x5d, 0x75, 0xd5, + 0xf2, 0xc8, 0xb6, 0x5c, 0x0b, 0x2d, 0x68, 0x96, 0xb6, 0x4b, 0xe9, 0x65, 0x8f, 0xbb, 0x7c, 0x7d, + 0x77, 0x7f, 0x75, 0x77, 0xdf, 0xc1, 0xf6, 0x3e, 0xb6, 0x57, 0x35, 0xcb, 0xd4, 0xc6, 0xb6, 0x8d, + 0x4d, 0x6d, 0xb2, 0x3a, 0xb0, 0xb4, 0x5d, 0xfa, 0xc7, 0x30, 0xfb, 0x4c, 0x7d, 0xf9, 0xa2, 0x0f, + 0x39, 0xc4, 0xae, 0x1a, 0xc2, 0x2e, 0x5f, 0x71, 0x5c, 0xcb, 0x56, 0xfb, 0x78, 0x15, 0x9b, 0x7d, + 0xc3, 0xc4, 0x44, 0x60, 0x5f, 0xd3, 0x3c, 0xe6, 0x17, 0x66, 0x32, 0x6f, 0x78, 0xdc, 0xc2, 0xd8, + 0x35, 0x06, 0xab, 0x3b, 0x03, 0x6d, 0xd5, 0x35, 0x86, 0xd8, 0x71, 0xd5, 0xe1, 0xc8, 0xe3, 0x2c, + 0xf6, 0xad, 0xbe, 0x45, 0x3f, 0x57, 0xc9, 0x17, 0xa3, 0x96, 0x1e, 0x00, 0xdf, 0x1d, 0xa9, 0x26, + 0xba, 0x0c, 0xf1, 0x5d, 0x3c, 0x29, 0xc4, 0xaf, 0x72, 0xd7, 0xb2, 0xd5, 0xd4, 0xd1, 0xb4, 0x18, + 0xbf, 0x8b, 0x27, 0x12, 0xa1, 0xa1, 0xab, 0x90, 0xc2, 0xa6, 0xae, 0x10, 0x36, 0x7f, 0x9c, 0x9d, + 0xc4, 0xa6, 0x7e, 0x17, 0x4f, 0xd6, 0x85, 0x5f, 0x3e, 0x2e, 0xce, 0xfd, 0xf1, 0x71, 0x91, 0x6b, + 0xf2, 0x02, 0x27, 0xc6, 0x9a, 0xbc, 0x10, 0x13, 0xe3, 0xa5, 0xdf, 0x71, 0x90, 0xb8, 0xaf, 0x0e, + 0xc6, 0x18, 0x5d, 0x81, 0xb4, 0xad, 0x3e, 0x52, 0x7a, 0x13, 0x17, 0x3b, 0x05, 0x8e, 0x60, 0x48, + 0x82, 0xad, 0x3e, 0xaa, 0x92, 0x31, 0xfa, 0x29, 0x07, 0xe9, 0xc0, 0xd6, 0x42, 0xec, 0x2a, 0x77, + 0x2d, 0xb3, 0xf6, 0xc5, 0x72, 0xe8, 0x58, 0xb2, 0xa0, 0xf2, 0xce, 0x40, 0x2b, 0xcb, 0xbe, 0x50, + 0xb5, 0xf9, 0x64, 0x5a, 0x9c, 0x3b, 0x9a, 0x16, 0xab, 0x7d, 0xc3, 0xdd, 0x19, 0xf7, 0xca, 0x9a, + 0x35, 0x5c, 0x0d, 0x14, 0xf4, 0x5e, 0xf8, 0xbd, 0x3a, 0xda, 0xed, 0xaf, 0x9e, 0xf4, 0x57, 0x59, + 0x3e, 0x30, 0x03, 0x2c, 0x29, 0x9c, 0xbb, 0xf4, 0x3e, 0x08, 0x77, 0xf1, 0x84, 0x99, 0xec, 0xf9, + 0x83, 0x9b, 0xe1, 0x8f, 0x9b, 0x90, 0xd8, 0x27, 0x32, 0x9e, 0xad, 0x85, 0xf2, 0x73, 0x41, 0x50, + 0xa6, 0x18, 0x55, 0x9e, 0x98, 0x29, 0x31, 0xe1, 0xd2, 0xc7, 0x1c, 0x40, 0xd7, 0xb5, 0x6c, 0xdc, + 0xd0, 0xb1, 0xe9, 0xa2, 0x3e, 0x80, 0x36, 0x18, 0x3b, 0x2e, 0xb6, 0x15, 0x43, 0xf7, 0xa6, 0x79, + 0x87, 0xc8, 0xff, 0x75, 0x5a, 0xbc, 0x71, 0xa6, 0x65, 0xd1, 0x8d, 0x1e, 0x8f, 0x0d, 0xbd, 0xbc, + 0xb5, 0xd5, 0xa8, 0x1d, 0x4e, 0x8b, 0xe9, 0x0d, 0x06, 0xd8, 0xa8, 0x49, 0x69, 0x0f, 0xbb, 0xa1, + 0xa3, 0xb7, 0x20, 0x65, 0x5a, 0x3a, 0x26, 0xb3, 0x10, 0x7b, 0x13, 0xd5, 0xc5, 0xc3, 0x69, 0x31, + 0xd9, 0xb6, 0x74, 0xdc, 0xa8, 0x1d, 0x05, 0x5f, 0x52, 0x92, 0x08, 0x35, 0x74, 0xf4, 0x0d, 0x10, + 0x88, 0xbf, 0xa8, 0x7c, 0x9c, 0xca, 0x5f, 0x3c, 0x9c, 0x16, 0x53, 0xcc, 0x72, 0xa2, 0xe0, 0x7f, + 0x4a, 0x29, 0x87, 0xad, 0xa6, 0xf4, 0x1b, 0x0e, 0xb2, 0xdd, 0xd1, 0xc0, 0x70, 0x65, 0xdb, 0xe8, + 0xf7, 0xb1, 0x8d, 0xea, 0x90, 0x1e, 0xe0, 0x6d, 0x57, 0xd1, 0xb1, 0xa3, 0xd1, 0xa5, 0x65, 0xd6, + 0x4a, 0x33, 0x9c, 0x24, 0xa9, 0x66, 0x1f, 0xd7, 0xb0, 0xa3, 0xd9, 0xc6, 0xc8, 0xb5, 0x6c, 0xcf, + 0x5d, 0x02, 0x51, 0x25, 0x54, 0x74, 0x07, 0xc0, 0x36, 0xfa, 0x3b, 0x1e, 0x4e, 0xec, 0x9c, 0x38, + 0x69, 0xaa, 0x4b, 0xc8, 0xeb, 0xfc, 0xa7, 0x2c, 0x34, 0xe3, 0x22, 0x5f, 0xfa, 0x5b, 0x0c, 0xb2, + 0x2d, 0x6c, 0xf7, 0xf1, 0x6b, 0x6a, 0x2c, 0xea, 0x83, 0xc8, 0x80, 0xc8, 0xa9, 0x56, 0x1c, 0x57, + 0x75, 0x1d, 0x7a, 0xec, 0x32, 0x6b, 0x5f, 0x89, 0xc0, 0x79, 0x31, 0x5d, 0x0e, 0x62, 0xba, 0x75, + 0x7f, 0x63, 0xa3, 0x4b, 0x84, 0xab, 0x17, 0x09, 0xe2, 0xe1, 0xb4, 0x98, 0x97, 0x08, 0x4c, 0x40, + 0x97, 0xf2, 0x14, 0xb6, 0xb5, 0xaf, 0x69, 0x74, 0x8c, 0x6e, 0x43, 0x76, 0xdb, 0xc6, 0xf8, 0xc7, + 0x98, 0x4c, 0x62, 0xbb, 0x85, 0xc4, 0x59, 0x4e, 0x1e, 0x33, 0x37, 0xc3, 0x14, 0xbb, 0x44, 0xef, + 0x98, 0x77, 0x7f, 0x9b, 0x80, 0xa5, 0x8d, 0x1d, 0xb2, 0x44, 0x09, 0x8f, 0x06, 0x86, 0xa6, 0x3a, + 0xbe, 0x9b, 0x1f, 0xc2, 0x45, 0x1d, 0x8f, 0x6c, 0xac, 0xa9, 0x2e, 0xd6, 0x15, 0x8d, 0xca, 0x28, + 0xee, 0x64, 0x84, 0xa9, 0xcf, 0xf3, 0x6b, 0x5f, 0x9e, 0xe5, 0x2b, 0x86, 0xc1, 0x00, 0xe5, 0xc9, + 0x08, 0x4b, 0x8b, 0x21, 0x46, 0x48, 0x45, 0x0f, 0x00, 0x45, 0xb0, 0x6d, 0xa6, 0xe5, 0xed, 0xc1, + 0x29, 0xb8, 0xcf, 0xed, 0xc2, 0x42, 0x88, 0xe2, 0x89, 0xa0, 0x1f, 0xc1, 0x95, 0x08, 0xf4, 0x78, + 0xa4, 0x47, 0xa7, 0x70, 0x0a, 0xf1, 0xab, 0xf1, 0x73, 0xce, 0x71, 0x39, 0x84, 0xdb, 0x62, 0x68, + 0xbe, 0xa7, 0x10, 0x86, 0xe5, 0xc8, 0x5c, 0x26, 0x3e, 0x70, 0xfd, 0x89, 0xc8, 0x61, 0xe4, 0xe9, + 0x61, 0xbc, 0x76, 0x38, 0x2d, 0x5e, 0xaa, 0x05, 0x52, 0x6d, 0x7c, 0xe0, 0x7a, 0xfa, 0xf4, 0x70, + 0xa6, 0x83, 0x81, 0x74, 0x49, 0x9f, 0x29, 0xa5, 0xa3, 0xb7, 0x81, 0xa7, 0x31, 0x9a, 0x38, 0x6b, + 0x8c, 0x4a, 0x54, 0x1e, 0xf5, 0xe0, 0x92, 0x61, 0xba, 0xd8, 0x36, 0xd5, 0x81, 0xa2, 0xea, 0x7a, + 0xd4, 0x0d, 0xc9, 0x73, 0xbb, 0x61, 0xc9, 0x87, 0xaa, 0x10, 0xa4, 0xc0, 0x05, 0xdb, 0x70, 0x39, + 0x98, 0xc3, 0xc6, 0x43, 0x6b, 0x3f, 0x3a, 0x4b, 0xea, 0xdc, 0xb3, 0x04, 0x06, 0x4b, 0x0c, 0xcb, + 0x9f, 0x67, 0x9d, 0x27, 0x17, 0x56, 0xe9, 0x43, 0x0e, 0x2e, 0xb4, 0x2c, 0xdd, 0xd8, 0x36, 0xb0, + 0x4e, 0x2e, 0x41, 0x3f, 0x56, 0xbf, 0x0e, 0xc8, 0x99, 0x38, 0x2e, 0x1e, 0x2a, 0x9a, 0x65, 0x6e, + 0x1b, 0x7d, 0xc5, 0x19, 0xa9, 0x26, 0x8d, 0x53, 0x41, 0x12, 0x19, 0x67, 0x83, 0x32, 0xe8, 0xcd, + 0x59, 0x07, 0x44, 0x13, 0xec, 0xc0, 0xd8, 0xc7, 0x26, 0x76, 0x1c, 0x26, 0xcd, 0xa2, 0xef, 0xd2, + 0x0c, 0x63, 0x89, 0x92, 0x24, 0x12, 0x95, 0x4d, 0x4f, 0x83, 0x50, 0x4a, 0xf7, 0x41, 0xec, 0xba, + 0x86, 0xb6, 0x3b, 0xa9, 0x86, 0x89, 0xb4, 0x0a, 0xe0, 0x50, 0x9a, 0xd2, 0x33, 0x5c, 0x2f, 0x39, + 0x9d, 0xe9, 0x80, 0xa6, 0x1d, 0x1f, 0xaa, 0xf4, 0xa7, 0x38, 0x2c, 0x35, 0x3c, 0x37, 0x6c, 0x58, + 0xc3, 0x61, 0x88, 0x5e, 0x83, 0x9c, 0x43, 0xd2, 0xb6, 0xe2, 0x32, 0x82, 0x37, 0x41, 0x71, 0xa6, + 0xcd, 0x61, 0x7a, 0x97, 0xb2, 0x4e, 0x34, 0xd9, 0xd7, 0x20, 0x37, 0x24, 0xf9, 0x34, 0x40, 0x89, + 0xbd, 0x10, 0x25, 0x9a, 0x77, 0xa5, 0xec, 0x30, 0x9a, 0x85, 0x7f, 0x08, 0x97, 0xbc, 0x9c, 0xe0, + 0x6f, 0x77, 0x80, 0x17, 0xa7, 0x78, 0xd7, 0x66, 0xe0, 0xcd, 0xcc, 0x34, 0xd2, 0x92, 0xf6, 0x82, + 0x04, 0xb4, 0x34, 0xf4, 0xf6, 0x9a, 0xee, 0x50, 0x80, 0xcf, 0x92, 0xeb, 0x57, 0x67, 0xd9, 0xfb, + 0x7c, 0x6c, 0x48, 0x17, 0x86, 0x33, 0x02, 0xe6, 0x5d, 0x40, 0xe1, 0x3e, 0x05, 0xc0, 0xec, 0x80, + 0xbd, 0x31, 0xcb, 0x9d, 0x27, 0x36, 0x5a, 0x12, 0x9d, 0x13, 0x94, 0x75, 0xfe, 0x83, 0xc7, 0x45, + 0xae, 0xf4, 0x33, 0x0e, 0x16, 0x3a, 0x3d, 0x5a, 0x4c, 0xea, 0xc1, 0x1e, 0x47, 0xaf, 0x74, 0xee, + 0x0c, 0x57, 0x7a, 0xe5, 0xdc, 0xf5, 0x95, 0x17, 0x44, 0x81, 0xd6, 0x3a, 0x4f, 0x8a, 0xbb, 0xd2, + 0x3f, 0x53, 0x90, 0x91, 0x6d, 0xd5, 0x74, 0x54, 0xcd, 0x35, 0x2c, 0x13, 0x55, 0x80, 0x27, 0x85, + 0xab, 0x17, 0x37, 0x6f, 0x9c, 0x76, 0x3d, 0xc9, 0x07, 0x66, 0x0b, 0xbb, 0x6a, 0x55, 0x20, 0xc8, + 0x4f, 0xa7, 0x45, 0x4e, 0xa2, 0xaa, 0x08, 0x01, 0x6f, 0xaa, 0x43, 0x56, 0x4a, 0xa5, 0x25, 0xfa, + 0x8d, 0x6e, 0x41, 0x92, 0x5c, 0x7b, 0x63, 0x76, 0xef, 0xcd, 0xbe, 0x1a, 0x22, 0x66, 0x74, 0xa9, + 0xac, 0xe4, 0xe9, 0xa0, 0x26, 0xe4, 0x07, 0xaa, 0xe3, 0x2a, 0x3b, 0x58, 0xb5, 0xdd, 0x1e, 0x56, + 0xcf, 0x75, 0xb1, 0xe5, 0x88, 0xea, 0x3b, 0xbe, 0x26, 0xfa, 0x39, 0x07, 0xb9, 0xa1, 0x7a, 0xa0, + 0x84, 0xee, 0x4b, 0x7d, 0xde, 0xe5, 0x69, 0x76, 0xa8, 0x1e, 0x84, 0x3b, 0xff, 0x3e, 0x5c, 0xb0, + 0xbc, 0x70, 0x08, 0x8d, 0x72, 0x0a, 0xc2, 0x0b, 0x33, 0xe3, 0x73, 0xc1, 0xe3, 0x2d, 0x14, 0x59, + 0x27, 0x19, 0x0e, 0xba, 0x05, 0x40, 0x9e, 0x27, 0xf4, 0x74, 0x38, 0x85, 0x0c, 0xc5, 0x7c, 0x51, + 0x02, 0xf3, 0x43, 0x84, 0x28, 0x90, 0xb1, 0x83, 0x4a, 0x90, 0x7b, 0x64, 0x1b, 0x2e, 0x56, 0x5c, + 0xcb, 0x52, 0xac, 0x81, 0x5e, 0xc8, 0xd2, 0x7c, 0x99, 0xa1, 0x44, 0xd9, 0xb2, 0x3a, 0x03, 0x1d, + 0x7d, 0xc4, 0x41, 0xde, 0xc6, 0x6a, 0xc4, 0xf6, 0xc2, 0xfc, 0xe7, 0xed, 0xd0, 0x1c, 0x31, 0x20, + 0xf4, 0xe8, 0x4d, 0xb8, 0xa8, 0xd1, 0xac, 0x18, 0xda, 0xa4, 0x6c, 0x1b, 0x07, 0x58, 0x2f, 0x88, + 0xd4, 0xfe, 0x45, 0xc6, 0x0d, 0x14, 0x6e, 0x13, 0x1e, 0x7a, 0x17, 0x44, 0xc3, 0x54, 0xb6, 0x07, + 0xb4, 0x50, 0xa3, 0x2b, 0x74, 0x0a, 0x0b, 0xd4, 0x61, 0x5f, 0x9a, 0xe5, 0x30, 0xbc, 0x37, 0xc6, + 0xa6, 0x86, 0xf5, 0xf7, 0x88, 0xa4, 0xe7, 0xba, 0xbc, 0x61, 0xde, 0xa6, 0xfa, 0x94, 0xe8, 0x20, + 0x0b, 0xe6, 0x8d, 0xbe, 0x69, 0xd9, 0x24, 0x3d, 0xe1, 0x3d, 0x73, 0x3c, 0x74, 0x0a, 0x88, 0x22, + 0x96, 0x4f, 0x3b, 0x57, 0x0d, 0xa6, 0xd2, 0xc5, 0x7b, 0xed, 0xf1, 0x90, 0xde, 0xdb, 0x61, 0xfd, + 0x77, 0x8c, 0xe7, 0x48, 0x79, 0x23, 0x18, 0x13, 0xf4, 0x63, 0x8f, 0xb6, 0xb8, 0xc8, 0x37, 0x79, + 0x21, 0x29, 0xa6, 0x9a, 0xbc, 0x90, 0x16, 0xa1, 0xc9, 0x0b, 0x39, 0x31, 0xdf, 0xe4, 0x85, 0xbc, + 0x38, 0x5f, 0xfa, 0x03, 0x0f, 0x0b, 0x91, 0xe3, 0x26, 0x61, 0xcd, 0xb2, 0xf5, 0xcf, 0xe2, 0xec, + 0xbf, 0x3e, 0xe7, 0xfc, 0xd5, 0x22, 0xff, 0xff, 0x21, 0x18, 0x78, 0x2f, 0x10, 0x62, 0x62, 0xfc, + 0x44, 0x38, 0xa4, 0x44, 0xa1, 0xc9, 0x0b, 0x82, 0x98, 0x0e, 0x42, 0x03, 0xc4, 0x4c, 0x93, 0x17, + 0xb2, 0x62, 0x2e, 0x1a, 0x26, 0x4d, 0x5e, 0x98, 0x17, 0xc5, 0x26, 0x2f, 0x88, 0xe2, 0x42, 0x69, + 0xca, 0x41, 0x92, 0xd4, 0x1c, 0xa6, 0x8b, 0x1e, 0xc0, 0xbc, 0x63, 0x98, 0xfd, 0x01, 0x56, 0x76, + 0xf1, 0x24, 0x2c, 0xa4, 0x32, 0x6b, 0x6f, 0xce, 0xf0, 0x0d, 0xd3, 0x29, 0x77, 0xa9, 0xc2, 0x5d, + 0x3c, 0xa1, 0xfe, 0x0e, 0xc3, 0x26, 0xe7, 0x44, 0x19, 0xe8, 0x3b, 0x10, 0x77, 0x0f, 0xfc, 0x4a, + 0xeb, 0x4c, 0x11, 0xc8, 0x9c, 0x4d, 0xb4, 0x96, 0x6f, 0x41, 0xee, 0xd8, 0x34, 0xa7, 0x34, 0x40, + 0xa2, 0x4d, 0x8d, 0x26, 0x2f, 0xf0, 0x62, 0xa2, 0xf4, 0x17, 0x0e, 0xe6, 0x37, 0x2d, 0x6d, 0xb7, + 0xa2, 0xed, 0x8d, 0x0d, 0xc7, 0xa0, 0xb7, 0xe1, 0x37, 0x81, 0x8f, 0x2c, 0xef, 0x85, 0xe1, 0x13, + 0x39, 0x05, 0xce, 0xab, 0xae, 0x02, 0x49, 0x00, 0xfa, 0xd8, 0x56, 0x7b, 0xc6, 0xc0, 0x70, 0x99, + 0xed, 0xf9, 0xb5, 0xb5, 0x08, 0xc6, 0xee, 0x7e, 0xd9, 0xef, 0x45, 0x95, 0x23, 0xbd, 0xa8, 0x32, + 0x09, 0xde, 0x72, 0x2d, 0xd0, 0x94, 0x22, 0x28, 0xa5, 0xdf, 0xc7, 0x00, 0xc8, 0xda, 0xd8, 0xf3, + 0xe4, 0x7f, 0xb2, 0xac, 0x30, 0x33, 0xc4, 0x5f, 0x22, 0x33, 0xcc, 0x38, 0x3c, 0xfc, 0x7f, 0xf3, + 0xf0, 0x94, 0x7e, 0xc1, 0x41, 0xfe, 0xf8, 0xb1, 0x3e, 0xad, 0x7d, 0xf4, 0x03, 0x10, 0x1c, 0x4f, + 0xd8, 0xeb, 0xc8, 0x7c, 0xef, 0x68, 0x5a, 0xbc, 0xf5, 0xd2, 0x57, 0x5b, 0x17, 0xef, 0x49, 0x01, + 0xa2, 0x57, 0xa9, 0xfd, 0x39, 0x0e, 0x89, 0x4d, 0xac, 0x3a, 0x18, 0x7d, 0x1b, 0x12, 0xec, 0x79, + 0x7f, 0x8e, 0xd7, 0x03, 0xd3, 0x40, 0xdf, 0x05, 0xc0, 0x07, 0x23, 0xc3, 0x56, 0x89, 0x8f, 0xcf, + 0x54, 0x38, 0x4a, 0x11, 0x05, 0x54, 0x83, 0x94, 0xff, 0x14, 0x8f, 0x9f, 0xfb, 0x29, 0xee, 0xab, + 0xa2, 0x2d, 0x88, 0x3c, 0x64, 0x59, 0xa7, 0x82, 0xfc, 0x75, 0x0c, 0xbf, 0x2b, 0xf2, 0x1f, 0x2c, + 0x5a, 0x0a, 0xb5, 0x69, 0xbb, 0xa2, 0x4b, 0x75, 0x51, 0x1b, 0x32, 0x23, 0xdb, 0x1a, 0x59, 0x0e, + 0x29, 0xa4, 0x9c, 0xb3, 0x5d, 0x1d, 0xf9, 0xc3, 0x69, 0x11, 0xee, 0x79, 0x5a, 0x72, 0x57, 0x02, + 0x1f, 0x41, 0x76, 0xd0, 0x22, 0x24, 0xf0, 0xc8, 0xd2, 0x76, 0x0a, 0xc9, 0xab, 0xdc, 0xb5, 0xb8, + 0xc4, 0x06, 0xe8, 0xad, 0xc8, 0x56, 0x93, 0xca, 0x31, 0x5e, 0x5d, 0x38, 0x9a, 0x16, 0x73, 0x74, + 0x67, 0xfc, 0x80, 0x89, 0xec, 0x5d, 0x70, 0x23, 0x97, 0x7e, 0x15, 0x83, 0x7c, 0xa5, 0x67, 0xd9, + 0x2e, 0x39, 0x5b, 0x75, 0xd3, 0xb5, 0x27, 0xa7, 0x45, 0xd4, 0x6b, 0xd3, 0x41, 0x45, 0x2a, 0x08, + 0x23, 0xdb, 0xb0, 0x6c, 0x3f, 0x1b, 0x25, 0xaa, 0xf5, 0xa3, 0x69, 0xb1, 0xf2, 0xd2, 0x93, 0xdc, + 0xf3, 0xc0, 0xa4, 0x00, 0x76, 0x5d, 0x20, 0x0e, 0xfa, 0x94, 0x38, 0xe9, 0x5f, 0x31, 0x58, 0xd8, + 0xc4, 0xea, 0xb6, 0x7c, 0x60, 0x36, 0xcc, 0xd1, 0x98, 0x6c, 0xad, 0x8b, 0xd1, 0xdb, 0x2c, 0xf1, + 0xb0, 0x70, 0x5f, 0x39, 0x3d, 0x71, 0x44, 0x73, 0xce, 0x9b, 0x30, 0x6f, 0xe3, 0x6d, 0x1b, 0x3b, + 0x3b, 0x8a, 0x61, 0xee, 0xab, 0x03, 0x43, 0xa7, 0x5b, 0x26, 0x48, 0x79, 0x8f, 0xdc, 0x60, 0xd4, + 0x99, 0xd7, 0xbd, 0xf0, 0x6a, 0xd7, 0xfd, 0x1a, 0x2c, 0x39, 0x2e, 0x1e, 0x8d, 0x0c, 0xb3, 0xaf, + 0x0c, 0xc9, 0xcb, 0x0e, 0x9b, 0x6a, 0x6f, 0x80, 0xf5, 0x42, 0x9a, 0x5a, 0x70, 0xc1, 0x67, 0xb6, + 0x2c, 0x1d, 0xd7, 0x19, 0x0b, 0xf5, 0x20, 0x4b, 0x4b, 0x69, 0x07, 0xef, 0x29, 0xe6, 0x78, 0x58, + 0x80, 0xcf, 0x28, 0x95, 0x00, 0x41, 0x65, 0x99, 0xee, 0x44, 0x3d, 0xc0, 0x8b, 0x89, 0x26, 0x2f, + 0x24, 0xc4, 0x24, 0xab, 0x0d, 0x4a, 0x1f, 0x86, 0xfe, 0xbf, 0x6d, 0x98, 0xea, 0xe0, 0xd5, 0xfc, + 0xff, 0x2d, 0x28, 0x44, 0x1b, 0x84, 0xd6, 0x70, 0xa8, 0x9a, 0xe4, 0xff, 0xd8, 0x74, 0x59, 0x28, + 0x49, 0x91, 0x06, 0xe2, 0x06, 0x63, 0x6f, 0x10, 0x2e, 0xaa, 0x42, 0xce, 0xdf, 0x39, 0x56, 0xc0, + 0xf1, 0x67, 0x29, 0xe0, 0xb2, 0x9e, 0x0e, 0xab, 0xe1, 0xce, 0xba, 0xfb, 0x81, 0x4b, 0x02, 0x37, + 0xb0, 0xe2, 0xa8, 0xf4, 0x13, 0x0e, 0xd2, 0xf4, 0xd6, 0x68, 0x98, 0xdb, 0x16, 0xba, 0xe5, 0xf5, + 0xd8, 0xce, 0xdb, 0x4f, 0x66, 0x9d, 0xb6, 0x9b, 0x90, 0x18, 0x90, 0x14, 0x71, 0xca, 0x0f, 0x0c, + 0x34, 0x85, 0xf8, 0xe9, 0x9a, 0x0a, 0xb3, 0x9e, 0xd6, 0xf5, 0x8f, 0x39, 0x48, 0xd3, 0x5f, 0x1f, + 0x68, 0x67, 0x34, 0x03, 0xa9, 0xad, 0xf6, 0xdd, 0x76, 0xe7, 0xbd, 0xb6, 0x38, 0x87, 0x52, 0x10, + 0x6f, 0xb4, 0x65, 0x91, 0x43, 0x69, 0x48, 0xdc, 0xde, 0xec, 0x54, 0x64, 0x31, 0x46, 0x3e, 0xab, + 0x0f, 0xe4, 0x7a, 0x57, 0x8c, 0xa3, 0x0b, 0x30, 0x5f, 0xab, 0x6f, 0x36, 0x5a, 0x0d, 0xb9, 0x5e, + 0x53, 0x18, 0x51, 0x40, 0x02, 0xf0, 0x72, 0xa3, 0x55, 0x17, 0x79, 0x02, 0x55, 0xab, 0x6f, 0x34, + 0x5a, 0x95, 0x4d, 0x31, 0x81, 0x96, 0x60, 0x21, 0x94, 0xf5, 0xc9, 0x69, 0x94, 0x05, 0xa1, 0xb6, + 0x25, 0x55, 0xe4, 0x46, 0xa7, 0x2d, 0x26, 0x11, 0x40, 0x92, 0xe8, 0xca, 0x0f, 0xc5, 0x2c, 0x99, + 0xfb, 0x4e, 0xbd, 0x23, 0xe6, 0xe8, 0x84, 0x9d, 0xef, 0xaf, 0xd5, 0xc4, 0x3c, 0xf9, 0x94, 0xb7, + 0xee, 0x6d, 0xd6, 0x45, 0x20, 0x8a, 0xd5, 0x86, 0x5c, 0x91, 0xa4, 0xca, 0x03, 0x31, 0x83, 0xf2, + 0x00, 0x44, 0xb1, 0x5b, 0x97, 0x1a, 0xf5, 0xae, 0xa8, 0x97, 0x48, 0x11, 0x9a, 0xba, 0xae, 0xc1, + 0xc2, 0x73, 0x0d, 0x61, 0x94, 0x83, 0x74, 0xa5, 0x56, 0x53, 0xee, 0x77, 0xe4, 0xba, 0x24, 0xce, + 0x21, 0x11, 0xb2, 0x52, 0xbd, 0xd5, 0xb9, 0x5f, 0xf7, 0x28, 0x1c, 0x5a, 0x80, 0x1c, 0x11, 0x68, + 0x77, 0xda, 0x1e, 0x29, 0x86, 0x16, 0x41, 0xf4, 0x84, 0x42, 0x6a, 0x7c, 0x99, 0xff, 0xe0, 0xd7, + 0x2b, 0x73, 0xd7, 0x3b, 0xc7, 0xde, 0x3a, 0xac, 0xb0, 0x20, 0x4b, 0xbf, 0x57, 0x6f, 0xd7, 0x1a, + 0xed, 0x3b, 0xe2, 0x1c, 0x19, 0x74, 0xe5, 0xca, 0x1d, 0x32, 0x88, 0x93, 0xe9, 0x37, 0x3a, 0xad, + 0x56, 0x43, 0x96, 0xeb, 0x35, 0x91, 0x23, 0xbc, 0x4a, 0xb5, 0x23, 0x91, 0x41, 0x8c, 0x01, 0x56, + 0xbf, 0xf6, 0xe4, 0x1f, 0x2b, 0x73, 0x4f, 0x0e, 0x57, 0xb8, 0xa7, 0x87, 0x2b, 0xdc, 0x27, 0x87, + 0x2b, 0xdc, 0xdf, 0x0f, 0x57, 0xb8, 0x8f, 0x9e, 0xad, 0xcc, 0x3d, 0x7d, 0xb6, 0x32, 0xf7, 0xc9, + 0xb3, 0x95, 0xb9, 0x87, 0x29, 0x6f, 0x4f, 0x7b, 0x49, 0xfa, 0x8b, 0xdc, 0x8d, 0x7f, 0x07, 0x00, + 0x00, 0xff, 0xff, 0x70, 0x3f, 0xb2, 0xa6, 0x69, 0x1c, 0x00, 0x00, } diff --git a/pkg/roachpb/data.proto b/pkg/roachpb/data.proto index 09d9208178ec..7bab66015995 100644 --- a/pkg/roachpb/data.proto +++ b/pkg/roachpb/data.proto @@ -86,7 +86,8 @@ message Value { // Its contents may be modified on the next call to Value.SetFoo. bytes raw_bytes = 1; // Timestamp of value. - util.hlc.Timestamp timestamp = 2 [(gogoproto.nullable) = false]; + util.hlc.Timestamp timestamp = 2 [(gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp"]; } // KeyValue is a pair of Key and Value for returned Key/Value pairs @@ -101,8 +102,8 @@ message KeyValue { // store-reserved system key (KeyLocalIdent). message StoreIdent { bytes cluster_id = 1 [(gogoproto.nullable) = false, - (gogoproto.customname) = "ClusterID", - (gogoproto.customtype) = "github.com/cockroachdb/cockroach/pkg/util/uuid.UUID"]; + (gogoproto.customname) = "ClusterID", + (gogoproto.customtype) = "github.com/cockroachdb/cockroach/pkg/util/uuid.UUID"]; int32 node_id = 2 [(gogoproto.customname) = "NodeID", (gogoproto.casttype) = "NodeID"]; int32 store_id = 3 [(gogoproto.customname) = "StoreID", (gogoproto.casttype) = "StoreID"]; } @@ -333,14 +334,16 @@ message Transaction { // advance "refreshing the read set". So, the read timestamp advances after a // successful refresh or, if the refresh is unsuccessful, after a transaction // restart. - util.hlc.Timestamp read_timestamp = 15 [(gogoproto.nullable) = false]; + util.hlc.Timestamp read_timestamp = 15 [(gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp"]; // Initial Timestamp + clock skew. Reads which encounter values with // timestamps between timestamp and max_timestamp trigger a txn // retry error, unless the node being read is listed in observed_timestamps // (in which case no more read uncertainty can occur). // The case max_timestamp < timestamp is possible for transactions which have // been pushed; in this case, max_timestamp should be ignored. - util.hlc.Timestamp max_timestamp = 7 [(gogoproto.nullable) = false]; + util.hlc.Timestamp max_timestamp = 7 [(gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp"]; // A list of pairs. The list maps NodeIDs to timestamps // as observed from their local clock during this transaction. The purpose of // this list is to avoid uncertainty related restarts which normally occur @@ -558,7 +561,8 @@ message AbortSpanEntry { bytes key = 1 [(gogoproto.casttype) = "Key"]; // The candidate commit timestamp the transaction record held at the time // it was aborted. - util.hlc.Timestamp timestamp = 2 [(gogoproto.nullable) = false]; + util.hlc.Timestamp timestamp = 2 [(gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp"]; // The priority of the transaction. int32 priority = 3 [ (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnPriority"]; diff --git a/pkg/roachpb/data_test.go b/pkg/roachpb/data_test.go index d439bc9f2906..632a1236b0f9 100644 --- a/pkg/roachpb/data_test.go +++ b/pkg/roachpb/data_test.go @@ -51,6 +51,14 @@ func makeTSWithFlag(walltime int64, logical int32) hlc.Timestamp { return makeTS(walltime, logical).SetFlag(hlc.TimestampFlag_SYNTHETIC) } +func makeTxnTS(walltime int64, logical int32) enginepb.TxnTimestamp { + return enginepb.ToTxnTimestamp(makeTS(walltime, logical)) +} + +func makeTxnTSWithFlag(walltime int64, logical int32) enginepb.TxnTimestamp { + return enginepb.ToTxnTimestamp(makeTSWithFlag(walltime, logical)) +} + // TestKeyNext tests that the method for creating lexicographic // successors to byte slices works as expected. func TestKeyNext(t *testing.T) { @@ -462,16 +470,16 @@ var nonZeroTxn = Transaction{ Key: Key("foo"), ID: uuid.MakeV4(), Epoch: 2, - WriteTimestamp: makeTSWithFlag(20, 21), - MinTimestamp: makeTSWithFlag(10, 11), + WriteTimestamp: makeTxnTSWithFlag(20, 21), + MinTimestamp: makeTxnTSWithFlag(10, 11), Priority: 957356782, Sequence: 123, }, Name: "name", Status: COMMITTED, LastHeartbeat: makeTSWithFlag(1, 2), - ReadTimestamp: makeTSWithFlag(20, 22), - MaxTimestamp: makeTSWithFlag(40, 41), + ReadTimestamp: makeTxnTSWithFlag(20, 22), + MaxTimestamp: makeTxnTSWithFlag(40, 41), ObservedTimestamps: []ObservedTimestamp{{NodeID: 1, Timestamp: makeTSWithFlag(1, 2)}}, WriteTooOld: true, LockSpans: []Span{{Key: []byte("a"), EndKey: []byte("b")}}, @@ -687,13 +695,13 @@ func TestTransactionClone(t *testing.T) { func TestTransactionRestart(t *testing.T) { txn := nonZeroTxn - txn.Restart(1, 1, makeTS(25, 1)) + txn.Restart(1, 1, makeTxnTS(25, 1)) expTxn := nonZeroTxn expTxn.Epoch++ expTxn.Sequence = 0 - expTxn.WriteTimestamp = makeTS(25, 1) - expTxn.ReadTimestamp = makeTS(25, 1) + expTxn.WriteTimestamp = makeTxnTS(25, 1) + expTxn.ReadTimestamp = makeTxnTS(25, 1) expTxn.WriteTooOld = false expTxn.CommitTimestampFixed = false expTxn.LockSpans = nil @@ -704,11 +712,11 @@ func TestTransactionRestart(t *testing.T) { func TestTransactionRefresh(t *testing.T) { txn := nonZeroTxn - txn.Refresh(makeTS(25, 1)) + txn.Refresh(makeTxnTS(25, 1)) expTxn := nonZeroTxn - expTxn.WriteTimestamp = makeTS(25, 1) - expTxn.ReadTimestamp = makeTS(25, 1) + expTxn.WriteTimestamp = makeTxnTS(25, 1) + expTxn.ReadTimestamp = makeTxnTS(25, 1) expTxn.WriteTooOld = false require.Equal(t, expTxn, txn) } diff --git a/pkg/roachpb/errors.go b/pkg/roachpb/errors.go index 4aee815c3e65..8b10cf5703f7 100644 --- a/pkg/roachpb/errors.go +++ b/pkg/roachpb/errors.go @@ -15,8 +15,8 @@ import ( "fmt" "strings" + enginepb "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/caller" - "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/uuid" @@ -880,7 +880,7 @@ var _ ErrorDetailInterface = &WriteIntentError{} // the timestamp of the operation that hit the error, along with the timestamp // immediately after the existing write which had a higher timestamp and which // caused the error. -func NewWriteTooOldError(operationTS, actualTS hlc.Timestamp) *WriteTooOldError { +func NewWriteTooOldError(operationTS, actualTS enginepb.TxnTimestamp) *WriteTooOldError { return &WriteTooOldError{ Timestamp: operationTS, ActualTimestamp: actualTS, @@ -912,7 +912,7 @@ var _ transactionRestartError = &WriteTooOldError{} // The read and existing timestamps as well as the txn are purely informational // and used for formatting the error message. func NewReadWithinUncertaintyIntervalError( - readTS, existingTS hlc.Timestamp, txn *Transaction, + readTS, existingTS enginepb.TxnTimestamp, txn *Transaction, ) *ReadWithinUncertaintyIntervalError { rwue := &ReadWithinUncertaintyIntervalError{ ReadTimestamp: readTS, diff --git a/pkg/roachpb/errors.pb.go b/pkg/roachpb/errors.pb.go index 1dca723a70d7..f8a481c03ac3 100644 --- a/pkg/roachpb/errors.pb.go +++ b/pkg/roachpb/errors.pb.go @@ -26,6 +26,7 @@ import math "math" import errorspb "github.com/cockroachdb/errors/errorspb" import hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" +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 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_052b1860ae0e9de7, []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_052b1860ae0e9de7, []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_052b1860ae0e9de7, []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_052b1860ae0e9de7, []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_052b1860ae0e9de7, []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_052b1860ae0e9de7, []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_052b1860ae0e9de7, []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_052b1860ae0e9de7, []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_052b1860ae0e9de7, []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_052b1860ae0e9de7, []int{4} } func (m *RangeKeyMismatchError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -558,18 +559,18 @@ var xxx_messageInfo_RangeKeyMismatchError proto.InternalMessageInfo type ReadWithinUncertaintyIntervalError struct { // This data below is purely informational and used to tailor the // error message. - ReadTimestamp hlc.Timestamp `protobuf:"bytes,1,opt,name=read_timestamp,json=readTimestamp" json:"read_timestamp"` - ExistingTimestamp hlc.Timestamp `protobuf:"bytes,2,opt,name=existing_timestamp,json=existingTimestamp" json:"existing_timestamp"` + ReadTimestamp github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp `protobuf:"bytes,1,opt,name=read_timestamp,json=readTimestamp,casttype=github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp" json:"read_timestamp"` + ExistingTimestamp github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp `protobuf:"bytes,2,opt,name=existing_timestamp,json=existingTimestamp,casttype=github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp" json:"existing_timestamp"` // The remaining fields may be missing when running in clusters that have // members at below CockroachDB v2.0. - MaxTimestamp *hlc.Timestamp `protobuf:"bytes,3,opt,name=max_timestamp,json=maxTimestamp" json:"max_timestamp,omitempty"` - ObservedTimestamps []ObservedTimestamp `protobuf:"bytes,4,rep,name=observed_timestamps,json=observedTimestamps" json:"observed_timestamps"` + MaxTimestamp *github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp `protobuf:"bytes,3,opt,name=max_timestamp,json=maxTimestamp,casttype=github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp" json:"max_timestamp,omitempty"` + ObservedTimestamps []ObservedTimestamp `protobuf:"bytes,4,rep,name=observed_timestamps,json=observedTimestamps" json:"observed_timestamps"` } func (m *ReadWithinUncertaintyIntervalError) Reset() { *m = ReadWithinUncertaintyIntervalError{} } func (*ReadWithinUncertaintyIntervalError) ProtoMessage() {} func (*ReadWithinUncertaintyIntervalError) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_06bf79933074ee74, []int{5} + return fileDescriptor_errors_052b1860ae0e9de7, []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_052b1860ae0e9de7, []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_052b1860ae0e9de7, []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_052b1860ae0e9de7, []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_052b1860ae0e9de7, []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_052b1860ae0e9de7, []int{10} } func (m *WriteIntentError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -799,15 +800,15 @@ var xxx_messageInfo_WriteIntentError proto.InternalMessageInfo // by instead bumping the transaction's write timestamp, setting the // transaction's WriteTooOld flag, and dropping the error. type WriteTooOldError struct { - Timestamp hlc.Timestamp `protobuf:"bytes,1,opt,name=timestamp" json:"timestamp"` - ActualTimestamp hlc.Timestamp `protobuf:"bytes,2,opt,name=actual_timestamp,json=actualTimestamp" json:"actual_timestamp"` + Timestamp github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp `protobuf:"bytes,1,opt,name=timestamp,casttype=github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp" json:"timestamp"` + ActualTimestamp github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp `protobuf:"bytes,2,opt,name=actual_timestamp,json=actualTimestamp,casttype=github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp" json:"actual_timestamp"` } 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_052b1860ae0e9de7, []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_052b1860ae0e9de7, []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_052b1860ae0e9de7, []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_052b1860ae0e9de7, []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_052b1860ae0e9de7, []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_052b1860ae0e9de7, []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_052b1860ae0e9de7, []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_052b1860ae0e9de7, []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_052b1860ae0e9de7, []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_052b1860ae0e9de7, []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_052b1860ae0e9de7, []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_052b1860ae0e9de7, []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_052b1860ae0e9de7, []int{23} } func (m *IntegerOverflowError) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1299,15 +1300,15 @@ var xxx_messageInfo_IntegerOverflowError proto.InternalMessageInfo // A BatchTimestampBeforeGCError indicates that a request's timestamp was // before the GC threshold. type BatchTimestampBeforeGCError struct { - Timestamp hlc.Timestamp `protobuf:"bytes,1,opt,name=Timestamp" json:"Timestamp"` - Threshold hlc.Timestamp `protobuf:"bytes,2,opt,name=Threshold" json:"Threshold"` + Timestamp github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp `protobuf:"bytes,1,opt,name=Timestamp,casttype=github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp" json:"Timestamp"` + Threshold github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp `protobuf:"bytes,2,opt,name=Threshold,casttype=github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp" json:"Threshold"` } func (m *BatchTimestampBeforeGCError) Reset() { *m = BatchTimestampBeforeGCError{} } 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_052b1860ae0e9de7, []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_052b1860ae0e9de7, []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_052b1860ae0e9de7, []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_052b1860ae0e9de7, []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_052b1860ae0e9de7, []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_052b1860ae0e9de7, []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_052b1860ae0e9de7, []int{30} } func (m *ErrPosition) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2509,7 +2510,7 @@ type Error struct { func (m *Error) Reset() { *m = Error{} } func (*Error) ProtoMessage() {} func (*Error) Descriptor() ([]byte, []int) { - return fileDescriptor_errors_06bf79933074ee74, []int{31} + return fileDescriptor_errors_052b1860ae0e9de7, []int{31} } func (m *Error) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -5448,7 +5449,7 @@ func (m *ReadWithinUncertaintyIntervalError) Unmarshal(dAtA []byte) error { return io.ErrUnexpectedEOF } if m.MaxTimestamp == nil { - m.MaxTimestamp = &hlc.Timestamp{} + m.MaxTimestamp = &github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp{} } if err := m.MaxTimestamp.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err @@ -8946,194 +8947,197 @@ var ( ErrIntOverflowErrors = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("roachpb/errors.proto", fileDescriptor_errors_06bf79933074ee74) } - -var fileDescriptor_errors_06bf79933074ee74 = []byte{ - // 2971 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, +func init() { proto.RegisterFile("roachpb/errors.proto", fileDescriptor_errors_052b1860ae0e9de7) } + +var fileDescriptor_errors_052b1860ae0e9de7 = []byte{ + // 3022 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x59, 0xcd, 0x6f, 0x1b, 0xc7, + 0xd9, 0xe7, 0x52, 0x94, 0x44, 0x3d, 0xfa, 0xf0, 0x7a, 0x2c, 0xcb, 0x2b, 0x39, 0xa6, 0x94, 0xb5, + 0x9d, 0xd8, 0x0e, 0x42, 0xbd, 0x70, 0x5e, 0x03, 0x6f, 0xf2, 0x26, 0x07, 0x7e, 0xac, 0x44, 0x4a, + 0xfc, 0xd0, 0xbb, 0xa4, 0x62, 0x3b, 0xc6, 0x8b, 0xc9, 0x8a, 0x3b, 0xa2, 0xb6, 0x26, 0x77, 0x99, + 0xd9, 0xa5, 0x3e, 0x80, 0x1e, 0x8a, 0xf6, 0xd0, 0x22, 0x05, 0x8a, 0x1c, 0x0a, 0x34, 0xc7, 0x02, + 0x3d, 0x15, 0x28, 0x8a, 0x16, 0xbd, 0xf7, 0xd0, 0x93, 0x8f, 0x39, 0x06, 0x45, 0x61, 0xb4, 0x4e, + 0xff, 0x8a, 0x9c, 0x8a, 0xf9, 0x58, 0x72, 0x29, 0x2e, 0x65, 0x15, 0x28, 0x92, 0x1b, 0xf7, 0xf9, + 0x9a, 0x67, 0x66, 0x9e, 0xf9, 0xcd, 0xef, 0x19, 0xc2, 0x32, 0xf5, 0xac, 0xd6, 0x51, 0xef, 0x60, + 0x93, 0x50, 0xea, 0x51, 0x3f, 0xdb, 0xa3, 0x5e, 0xe0, 0xa1, 0xab, 0x2d, 0xaf, 0xf5, 0x9c, 0x6b, + 0xb2, 0xcf, 0x8f, 0xb3, 0xcf, 0x8f, 0x7b, 0x07, 0x6b, 0xd7, 0x85, 0xc1, 0x39, 0xcb, 0x35, 0x14, + 0xfa, 0xdb, 0x56, 0x60, 0x49, 0xd9, 0x4a, 0x28, 0xeb, 0x92, 0xc0, 0x8a, 0xc8, 0xb5, 0x7e, 0xe0, + 0x74, 0x36, 0x8f, 0x3a, 0xad, 0xcd, 0xc0, 0xe9, 0x12, 0x3f, 0xb0, 0xba, 0x3d, 0xa9, 0x59, 0x6e, + 0x7b, 0x6d, 0x8f, 0xff, 0xdc, 0x64, 0xbf, 0x84, 0x54, 0xff, 0x43, 0x12, 0xae, 0xd5, 0xbc, 0xa0, + 0x42, 0x2c, 0x9f, 0x94, 0xbc, 0x8e, 0x4d, 0xa8, 0xc1, 0x86, 0x46, 0x45, 0x98, 0xa5, 0xa4, 0xd7, + 0x71, 0x5a, 0x96, 0xa6, 0x6c, 0x28, 0xf7, 0xe6, 0x1f, 0xde, 0xc9, 0x0e, 0xf3, 0x95, 0x63, 0x67, + 0x4d, 0x61, 0x51, 0x24, 0x7e, 0x8b, 0x3a, 0xbd, 0xc0, 0xa3, 0xf9, 0xd4, 0x8b, 0x97, 0xeb, 0x09, + 0x33, 0x74, 0x45, 0xdb, 0xb0, 0xd0, 0x61, 0x91, 0xf1, 0x11, 0x0f, 0xad, 0x25, 0x2f, 0x1f, 0xca, + 0x9c, 0xef, 0x0c, 0x73, 0x42, 0x8f, 0x20, 0x4d, 0x2d, 0xb7, 0x4d, 0xb0, 0x63, 0x6b, 0x53, 0x1b, + 0xca, 0xbd, 0xa9, 0xfc, 0x1a, 0x1b, 0xe9, 0xd5, 0xcb, 0xf5, 0x59, 0x93, 0xc9, 0xcb, 0xc5, 0x6f, + 0x87, 0x3f, 0xcd, 0x59, 0x6e, 0x5b, 0xb6, 0x51, 0x16, 0xa6, 0x79, 0x14, 0x2d, 0xc5, 0x07, 0xd6, + 0x62, 0x06, 0xe6, 0x33, 0x37, 0x85, 0x19, 0xba, 0x0d, 0xd0, 0xea, 0xfb, 0x81, 0xd7, 0xc5, 0x5d, + 0xbf, 0xad, 0x4d, 0x6f, 0x28, 0xf7, 0xe6, 0xe4, 0x94, 0xe6, 0x84, 0xbc, 0xea, 0xb7, 0xf5, 0x15, + 0x58, 0xae, 0x79, 0x36, 0xd9, 0x77, 0xad, 0x63, 0xcb, 0xe9, 0x58, 0x07, 0x1d, 0xc2, 0x97, 0x4c, + 0x5f, 0x85, 0x1b, 0xfb, 0xae, 0xdf, 0xef, 0xf5, 0x3c, 0x1a, 0x10, 0xdb, 0x24, 0x9f, 0xf5, 0x89, + 0x1f, 0x08, 0xd5, 0x8f, 0x15, 0x40, 0x3c, 0xb9, 0x9a, 0x17, 0x6c, 0x79, 0x7d, 0xd7, 0x16, 0x8b, + 0x1c, 0x9d, 0x95, 0x72, 0xf9, 0x59, 0x3d, 0x82, 0xb4, 0x1f, 0x78, 0x94, 0xbb, 0x25, 0x47, 0xdd, + 0x1a, 0x4c, 0x2e, 0xdc, 0xe4, 0x4f, 0x73, 0x96, 0xdb, 0x96, 0x6d, 0xfd, 0x57, 0x53, 0x70, 0x9d, + 0xc7, 0xda, 0x25, 0x67, 0x55, 0xc7, 0xef, 0x5a, 0x41, 0xeb, 0x48, 0xe4, 0xf1, 0x1e, 0x5c, 0xa5, + 0x22, 0x5d, 0xec, 0x07, 0x16, 0x0d, 0xf0, 0x73, 0x72, 0xc6, 0x13, 0x5a, 0xc8, 0xcf, 0x7e, 0xfb, + 0x72, 0x7d, 0x6a, 0x97, 0x9c, 0x99, 0x57, 0xa4, 0x45, 0x83, 0x19, 0xec, 0x92, 0x33, 0xb4, 0x09, + 0xa1, 0x08, 0x13, 0xd7, 0xe6, 0x2e, 0xc9, 0x51, 0x97, 0x45, 0xa9, 0x37, 0x5c, 0x9b, 0x39, 0x1c, + 0xc1, 0x4d, 0x9b, 0xf4, 0x28, 0x69, 0x59, 0x01, 0xb1, 0x71, 0x57, 0x66, 0x40, 0x6c, 0xcc, 0xe7, + 0xc5, 0xb7, 0x75, 0xfe, 0xa1, 0x1e, 0x57, 0x1b, 0x4c, 0x3f, 0x56, 0x64, 0xab, 0xc3, 0x60, 0xd5, + 0x41, 0x2c, 0x6e, 0x8a, 0x3e, 0x85, 0xb5, 0xc8, 0x48, 0x7e, 0xbf, 0xdd, 0x26, 0x7e, 0x30, 0x18, + 0x28, 0x75, 0xd9, 0x81, 0x4c, 0x6d, 0x18, 0xa5, 0x11, 0x06, 0x11, 0x23, 0x54, 0x60, 0x86, 0x07, + 0xf3, 0xb5, 0xe9, 0x8d, 0xa9, 0x7b, 0xf3, 0x0f, 0xdf, 0x98, 0x14, 0xad, 0xec, 0x1e, 0x7a, 0xf9, + 0x15, 0xb9, 0x3d, 0x4b, 0xc2, 0xa7, 0xec, 0x06, 0x84, 0xba, 0x56, 0xc7, 0x94, 0x31, 0xf4, 0xbf, + 0xa4, 0x40, 0x37, 0x89, 0x65, 0x3f, 0x76, 0x82, 0x23, 0xc7, 0xdd, 0x77, 0x5b, 0x84, 0x06, 0x96, + 0xe3, 0x06, 0x67, 0xdc, 0xf2, 0xd8, 0xea, 0x88, 0x6d, 0xfa, 0x42, 0x81, 0x25, 0x4a, 0x2c, 0x1b, + 0x0f, 0x8e, 0xb6, 0x3c, 0x9b, 0xb7, 0x22, 0xa3, 0xb3, 0xf3, 0x9f, 0x3d, 0xea, 0xb4, 0xb2, 0xcd, + 0xd0, 0x28, 0xbf, 0xc3, 0x86, 0xff, 0xf6, 0xe5, 0x7a, 0xbe, 0xed, 0x04, 0x47, 0xfd, 0x83, 0x6c, + 0xcb, 0xeb, 0x6e, 0x0e, 0x1c, 0xec, 0x83, 0xe1, 0xef, 0xcd, 0xde, 0xf3, 0xf6, 0x26, 0x2b, 0x17, + 0xab, 0x4d, 0x36, 0x89, 0xdb, 0x76, 0x5c, 0xd2, 0x3b, 0xc8, 0x36, 0x4f, 0xdd, 0x41, 0x2c, 0xb6, + 0xa7, 0x96, 0x3d, 0xf8, 0x44, 0x5f, 0x2a, 0x80, 0xc8, 0xa9, 0xe3, 0x07, 0x8e, 0xdb, 0x8e, 0xa4, + 0x95, 0xfc, 0xae, 0xd3, 0xba, 0x1a, 0x26, 0x31, 0x4c, 0xed, 0x73, 0x05, 0x16, 0xbb, 0xd6, 0x69, + 0x24, 0xab, 0xa9, 0xcb, 0x64, 0xb5, 0xf5, 0x1f, 0xca, 0x68, 0xa1, 0x6b, 0x9d, 0x0e, 0x93, 0x79, + 0x06, 0xd7, 0xbc, 0x03, 0x9f, 0xd0, 0x63, 0x12, 0xd9, 0x3d, 0x5f, 0x4b, 0xf1, 0xe2, 0x89, 0xc3, + 0xc3, 0xba, 0xb4, 0x1e, 0x26, 0x26, 0xaa, 0x1e, 0x79, 0xe7, 0x15, 0xfe, 0x07, 0xa9, 0x2f, 0x7f, + 0xbd, 0x9e, 0xd0, 0x6d, 0xb8, 0xd1, 0xa4, 0x96, 0xeb, 0x5b, 0xad, 0xc0, 0xf1, 0xdc, 0xdc, 0x01, + 0x47, 0x21, 0x51, 0x38, 0x65, 0x98, 0xa1, 0xc4, 0xf2, 0x3d, 0x97, 0xd7, 0xcb, 0xd2, 0xc3, 0x77, + 0xb2, 0x63, 0x77, 0x4f, 0x76, 0xdc, 0xd7, 0xe4, 0x2e, 0x72, 0x5c, 0x19, 0x40, 0x7f, 0x06, 0xcb, + 0x11, 0xcb, 0xbd, 0xbe, 0x2f, 0x21, 0xa4, 0x00, 0xd0, 0xeb, 0xfb, 0x47, 0x84, 0xe0, 0xe0, 0xd4, + 0x95, 0x65, 0x99, 0x89, 0x99, 0x57, 0xc4, 0x39, 0x44, 0x56, 0xe1, 0xd7, 0x3c, 0x75, 0xf5, 0x9f, + 0x28, 0x70, 0x3d, 0x62, 0x60, 0x92, 0x80, 0x9e, 0x89, 0xf0, 0xdb, 0xe7, 0x66, 0x70, 0xff, 0xe2, + 0x19, 0x70, 0xcf, 0xb8, 0xfc, 0xd1, 0x9b, 0x30, 0x47, 0x4e, 0x03, 0x6a, 0x71, 0x80, 0x4f, 0x46, + 0x00, 0x3e, 0xcd, 0xc5, 0x0c, 0xdf, 0xff, 0xac, 0xc0, 0x4a, 0x24, 0x56, 0x23, 0xb0, 0x82, 0xbe, + 0x2f, 0xd2, 0x58, 0x81, 0x29, 0xe6, 0xa7, 0x44, 0xfc, 0x98, 0x00, 0xd5, 0x06, 0xe9, 0x25, 0x79, + 0x7a, 0xff, 0x75, 0x71, 0x7a, 0x91, 0x90, 0xd9, 0xd8, 0x55, 0xfe, 0x10, 0x66, 0x84, 0x1c, 0x21, + 0x58, 0x32, 0x8d, 0x5c, 0xa3, 0x5e, 0xc3, 0xfb, 0xb5, 0xdd, 0x5a, 0xfd, 0x71, 0x4d, 0x4d, 0x20, + 0x0d, 0x96, 0xa5, 0xac, 0xf9, 0xa4, 0x86, 0x0b, 0xf5, 0x6a, 0xb5, 0xdc, 0x6c, 0x1a, 0x45, 0x35, + 0xa9, 0xa7, 0xd2, 0x8a, 0xaa, 0xe8, 0x0d, 0x50, 0x1f, 0x53, 0x27, 0x20, 0x0c, 0x3d, 0x5c, 0x71, + 0x03, 0xa1, 0xf7, 0x61, 0xd6, 0xe1, 0x9f, 0xbe, 0xa6, 0xf0, 0xa2, 0x5b, 0x8d, 0xd9, 0x1c, 0xe1, + 0x10, 0x5e, 0xe2, 0xd2, 0x7e, 0x27, 0x95, 0x4e, 0xaa, 0x53, 0xfa, 0x9f, 0x92, 0x32, 0x6a, 0xd3, + 0xf3, 0xea, 0x1d, 0x59, 0x58, 0x3f, 0x55, 0x60, 0xee, 0x7b, 0x04, 0xa3, 0xe1, 0xd8, 0xe8, 0x97, + 0x0a, 0xa8, 0x56, 0x2b, 0xe8, 0x5b, 0x9d, 0xef, 0x13, 0x86, 0xae, 0x88, 0x14, 0x06, 0x02, 0x7d, + 0x19, 0x50, 0xbd, 0xc7, 0xa8, 0x80, 0x43, 0x89, 0xdf, 0x3c, 0x75, 0x05, 0x1d, 0x68, 0xc0, 0x72, + 0xc1, 0x73, 0x6d, 0x87, 0xd5, 0xc2, 0x96, 0xe5, 0x74, 0xc2, 0x73, 0xfa, 0xbf, 0xb0, 0x20, 0xe7, + 0x70, 0x6c, 0x75, 0xfa, 0x44, 0x2e, 0x68, 0x1c, 0x6b, 0xf9, 0x98, 0xe9, 0xcd, 0x79, 0x61, 0xcd, + 0x3f, 0xf4, 0xdf, 0x2b, 0x80, 0x04, 0x99, 0x21, 0x3f, 0x20, 0xad, 0xc1, 0xd9, 0xcf, 0xc0, 0x6c, + 0x97, 0xf8, 0xbe, 0xd5, 0x26, 0x23, 0x65, 0x1b, 0x0a, 0xd1, 0x87, 0x30, 0x27, 0xaf, 0x69, 0x62, + 0xcb, 0x05, 0x9b, 0x48, 0x93, 0xc2, 0x13, 0x3b, 0x70, 0x40, 0x1f, 0x40, 0x3a, 0x44, 0x5e, 0x09, + 0xaf, 0xaf, 0x73, 0x1e, 0xd8, 0xeb, 0x9f, 0xc1, 0x72, 0xae, 0x7b, 0xe0, 0xb4, 0xfb, 0x5e, 0xdf, + 0x37, 0x89, 0xdf, 0xef, 0x04, 0x97, 0xcb, 0xf8, 0x7d, 0x98, 0x3f, 0xa1, 0x56, 0xaf, 0x47, 0x6c, + 0x4c, 0x28, 0x8d, 0xc9, 0x39, 0x3c, 0x71, 0x3c, 0x9c, 0x09, 0xd2, 0xd8, 0xa0, 0x54, 0xbf, 0xc1, + 0x18, 0xd0, 0x61, 0xb0, 0x4d, 0xbd, 0x7e, 0xaf, 0x48, 0x3a, 0x24, 0x5c, 0x25, 0x1d, 0xc3, 0x8a, + 0x64, 0xa0, 0x05, 0x8f, 0xd2, 0x7e, 0x8f, 0xed, 0x8c, 0xc8, 0x86, 0x01, 0x06, 0xfb, 0x81, 0xcf, + 0x1f, 0xfc, 0x34, 0x17, 0x57, 0xfd, 0x36, 0xd2, 0x61, 0xae, 0x47, 0xbd, 0x16, 0xf1, 0x7d, 0xb9, + 0x84, 0xe9, 0x01, 0xb4, 0x85, 0x62, 0xbd, 0x01, 0x48, 0x0e, 0x10, 0x3d, 0x3f, 0x1f, 0x01, 0x48, + 0xaa, 0x1c, 0x52, 0xc0, 0xe9, 0x7c, 0x46, 0x92, 0x85, 0x39, 0x69, 0xcf, 0xd9, 0xdc, 0xf0, 0x83, + 0xad, 0xbe, 0xf8, 0x69, 0xeb, 0xbb, 0x80, 0x38, 0xcb, 0x1b, 0x63, 0x95, 0x03, 0x7a, 0xa8, 0x5c, + 0x9e, 0x1e, 0x36, 0x18, 0x7d, 0x3d, 0xb2, 0x5c, 0xbb, 0xc3, 0xa0, 0x3f, 0xa0, 0x67, 0x03, 0x66, + 0x8b, 0x1e, 0x42, 0xaa, 0x67, 0x50, 0x1a, 0x53, 0x8f, 0x23, 0x4b, 0x2d, 0x67, 0xcd, 0x6d, 0xe5, + 0xa5, 0xf4, 0x4f, 0x05, 0xee, 0x9e, 0xc7, 0x65, 0xc6, 0x72, 0xf6, 0x58, 0xf3, 0x61, 0x92, 0x43, + 0x4a, 0xc2, 0x0b, 0x64, 0x12, 0xb4, 0x3e, 0x83, 0x99, 0xe0, 0xd4, 0x0d, 0xa9, 0xee, 0x42, 0xbe, + 0xc8, 0x54, 0x7f, 0x7d, 0xb9, 0xfe, 0xde, 0xa5, 0x8e, 0x2b, 0xef, 0x86, 0xfa, 0x7d, 0xc7, 0xce, + 0xee, 0xef, 0x97, 0x8b, 0xaf, 0x5e, 0xae, 0x4f, 0x37, 0x4f, 0xdd, 0x72, 0xd1, 0x9c, 0x0e, 0x4e, + 0xdd, 0xb2, 0x8d, 0xb6, 0x60, 0x3e, 0x18, 0x66, 0x27, 0x2b, 0xf8, 0x72, 0xd7, 0x56, 0xd4, 0x51, + 0xdf, 0x82, 0xf5, 0xe6, 0xa9, 0x9b, 0xeb, 0x30, 0x72, 0x74, 0x66, 0xb8, 0x2d, 0xaf, 0xcf, 0x88, + 0x9b, 0x2c, 0x2e, 0x31, 0xbf, 0xdb, 0x00, 0x3d, 0x4a, 0x8e, 0x31, 0xaf, 0x9a, 0x91, 0x69, 0xce, + 0x31, 0xb9, 0x28, 0xc3, 0x9f, 0x2b, 0xb0, 0xcc, 0x40, 0xb8, 0x4d, 0x68, 0xfd, 0x98, 0xd0, 0xc3, + 0x8e, 0x77, 0x22, 0xbc, 0x57, 0x61, 0x2a, 0x86, 0x93, 0x33, 0x19, 0xba, 0x0f, 0x8b, 0xad, 0x3e, + 0xa5, 0xc4, 0x0d, 0x24, 0x6a, 0x88, 0x96, 0x40, 0xc4, 0x5e, 0x90, 0x2a, 0x0e, 0x11, 0xe8, 0x5d, + 0xb8, 0xe2, 0xb8, 0x2d, 0x4a, 0xba, 0x43, 0xe3, 0xa9, 0x88, 0xf1, 0xd2, 0x40, 0x29, 0x10, 0xe5, + 0x8f, 0x49, 0xb8, 0x99, 0x67, 0xbc, 0x7a, 0x08, 0x96, 0xe4, 0xd0, 0xa3, 0x64, 0xbb, 0x30, 0x44, + 0xff, 0xe6, 0xf7, 0x88, 0xfe, 0x43, 0x7a, 0xc5, 0x33, 0x39, 0x62, 0xc5, 0xe4, 0x75, 0xec, 0xef, + 0x1e, 0xf6, 0x87, 0x63, 0xeb, 0x5d, 0x40, 0xe2, 0x16, 0xad, 0x3a, 0xbe, 0xef, 0xb8, 0x6d, 0xb1, + 0x52, 0x1f, 0xc2, 0xc2, 0x09, 0xf5, 0xdc, 0x36, 0x16, 0x77, 0xaa, 0x5c, 0xab, 0xc9, 0x57, 0xb0, + 0x39, 0xcf, 0xcd, 0xc5, 0x47, 0xb8, 0xf9, 0xc9, 0xf1, 0xcd, 0x67, 0xbd, 0x68, 0x95, 0x50, 0xd6, + 0x66, 0xec, 0x51, 0xaf, 0x4d, 0x89, 0x2f, 0x58, 0x85, 0xfe, 0x8b, 0x24, 0x5c, 0xe3, 0xfd, 0xc7, + 0x16, 0x91, 0xa7, 0x59, 0x24, 0xb2, 0x7b, 0x8e, 0x47, 0xbd, 0x1b, 0x73, 0x96, 0x63, 0xfc, 0xe2, + 0x59, 0xca, 0x6f, 0x95, 0x01, 0x4d, 0x59, 0x83, 0x15, 0x49, 0x49, 0x4c, 0x63, 0xaf, 0x52, 0x2e, + 0xe4, 0xb0, 0x69, 0x54, 0xeb, 0x1f, 0x1b, 0x45, 0x35, 0x81, 0x56, 0x00, 0x85, 0xba, 0x5c, 0x6d, + 0xdb, 0xc0, 0x8d, 0xbd, 0x4a, 0xb9, 0xa9, 0x2a, 0xe8, 0x06, 0x5c, 0x1b, 0x91, 0x57, 0x0d, 0x73, + 0x9b, 0xb1, 0x98, 0x08, 0xbf, 0x31, 0x73, 0x5b, 0x4d, 0xdc, 0xa8, 0xe5, 0xf6, 0x1a, 0xa5, 0x7a, + 0x53, 0x9d, 0x42, 0x19, 0x58, 0x93, 0x9a, 0x4a, 0x7d, 0xbb, 0x5c, 0xc8, 0x55, 0x70, 0x7d, 0xaf, + 0x81, 0xab, 0xe5, 0x46, 0xa3, 0x5c, 0xdb, 0x56, 0x53, 0x11, 0xcf, 0x46, 0xa5, 0xfe, 0x18, 0x17, + 0xea, 0xb5, 0xc6, 0x7e, 0xd5, 0x30, 0xd5, 0x69, 0xdd, 0x02, 0xad, 0xec, 0xda, 0x24, 0x20, 0xb4, + 0xeb, 0xb8, 0x56, 0x40, 0x0a, 0x5e, 0xb7, 0xeb, 0xc8, 0x0b, 0xc7, 0x80, 0x79, 0x3f, 0xb0, 0xda, + 0xbc, 0x85, 0xf9, 0x37, 0xc9, 0x2b, 0x48, 0x47, 0xc6, 0x5e, 0x5f, 0x2c, 0xc3, 0x3c, 0x0f, 0x58, + 0x24, 0x81, 0xe5, 0x74, 0x90, 0x09, 0xaa, 0xeb, 0x05, 0x78, 0xe4, 0x01, 0x44, 0xc4, 0x7e, 0x2b, + 0x66, 0xd5, 0x63, 0x1e, 0x61, 0x4a, 0x09, 0x73, 0xc9, 0x1d, 0x11, 0xa3, 0x3a, 0x5c, 0x11, 0x2f, + 0x06, 0x2c, 0xf2, 0x21, 0xc3, 0x7c, 0x59, 0xed, 0x77, 0x27, 0x6d, 0xe4, 0xc8, 0xdd, 0x50, 0x4a, + 0x98, 0x8b, 0x34, 0x2a, 0x45, 0x4f, 0x00, 0x89, 0x80, 0xcf, 0xc9, 0xd9, 0xa0, 0x27, 0x97, 0x40, + 0x78, 0x6f, 0x52, 0xcc, 0xf3, 0x0f, 0x08, 0xa5, 0x84, 0xa9, 0xd2, 0x73, 0x0a, 0xf4, 0x23, 0x05, + 0x36, 0x78, 0xb7, 0x7a, 0xc2, 0xbb, 0x5a, 0xdc, 0x1f, 0xb6, 0xb5, 0xfc, 0x18, 0xb0, 0xbe, 0x56, + 0xf6, 0xe2, 0x8f, 0xe2, 0x06, 0x7a, 0x6d, 0x3f, 0x5c, 0x4a, 0x98, 0xb7, 0xe8, 0x45, 0x56, 0xe8, + 0xff, 0xe1, 0x5a, 0x04, 0xa5, 0xb1, 0x25, 0xfa, 0x1a, 0xfe, 0xae, 0x33, 0xff, 0xf0, 0xc1, 0xa5, + 0x9a, 0xa0, 0x70, 0x24, 0x14, 0x8c, 0xa9, 0x50, 0x13, 0xd4, 0x68, 0x78, 0xd6, 0xc7, 0x68, 0x33, + 0x3c, 0xf6, 0xdb, 0x17, 0xc7, 0x1e, 0xb4, 0x4d, 0xa5, 0x84, 0x79, 0x25, 0x18, 0x95, 0xa3, 0xc7, + 0x70, 0x35, 0x1a, 0x95, 0xb2, 0x43, 0xa8, 0xcd, 0x4e, 0xdc, 0x90, 0xd8, 0x7e, 0x89, 0x6d, 0x48, + 0x70, 0x4e, 0x81, 0x3e, 0x81, 0xe8, 0x24, 0xb0, 0xcf, 0x9b, 0x10, 0x2d, 0xcd, 0x23, 0xdf, 0xbf, + 0x74, 0xc3, 0x52, 0x4a, 0x98, 0xd1, 0xfc, 0x84, 0x06, 0x95, 0x18, 0xc0, 0x39, 0x01, 0x09, 0x01, + 0x6e, 0x8e, 0x47, 0xbd, 0x1d, 0x13, 0xf5, 0x7c, 0x67, 0x52, 0x4a, 0x30, 0xb0, 0x1b, 0xc8, 0x50, + 0x19, 0x16, 0x45, 0xa4, 0xc0, 0xf3, 0x30, 0x43, 0x73, 0xb8, 0x38, 0x54, 0x84, 0x4e, 0x0d, 0x42, + 0x09, 0x19, 0x3b, 0x2c, 0x5e, 0x0f, 0x53, 0xc9, 0xbe, 0xf9, 0xd9, 0x9e, 0x9f, 0x78, 0x58, 0xc6, + 0x69, 0x3a, 0x3b, 0x2c, 0x5e, 0x54, 0xca, 0x36, 0xbc, 0x15, 0xf2, 0x76, 0x7c, 0xc8, 0x89, 0xbb, + 0xb6, 0x30, 0x71, 0xc3, 0xe3, 0x28, 0x3e, 0xdb, 0xf0, 0xd6, 0xa8, 0x1c, 0xd5, 0x60, 0x49, 0x60, + 0x04, 0x95, 0xc4, 0x5d, 0x5b, 0x9c, 0x98, 0xe5, 0x38, 0xc1, 0x67, 0x59, 0x76, 0xa2, 0x52, 0x96, + 0xa5, 0xeb, 0xd9, 0x04, 0xf7, 0x87, 0x0f, 0x94, 0xda, 0xd2, 0xc4, 0x2c, 0xe3, 0x9e, 0x32, 0x59, + 0x96, 0xee, 0xa8, 0x5c, 0x00, 0xc5, 0x61, 0x80, 0xdb, 0x8c, 0x3b, 0x63, 0x5b, 0x90, 0x67, 0x4d, + 0xbd, 0x00, 0x28, 0x62, 0x78, 0xb6, 0x00, 0x8a, 0x51, 0x05, 0xab, 0xcb, 0x90, 0x04, 0xb7, 0x06, + 0xe4, 0x5b, 0xbb, 0x3a, 0xb1, 0x2e, 0xe3, 0x89, 0x3a, 0xab, 0x4b, 0x7a, 0x5e, 0xc3, 0xf1, 0x52, + 0xc6, 0x0e, 0xeb, 0x09, 0x4d, 0xc6, 0xcb, 0x31, 0x82, 0xce, 0xf1, 0x32, 0x2a, 0x65, 0x8b, 0x6b, + 0x85, 0x4d, 0x0b, 0xa6, 0xbc, 0x6b, 0xd1, 0xd6, 0x26, 0x2e, 0x6e, 0x5c, 0x7f, 0xc3, 0x16, 0xd7, + 0x1a, 0x95, 0xb3, 0x34, 0x05, 0x65, 0x1f, 0xc2, 0xfa, 0xcd, 0x89, 0x69, 0x8e, 0x53, 0x7e, 0x96, + 0xa6, 0x1f, 0x95, 0xa2, 0xcf, 0x15, 0xb8, 0x33, 0x86, 0x22, 0x1c, 0x89, 0x31, 0x7f, 0xf7, 0xc7, + 0x54, 0x70, 0x6f, 0xed, 0x0d, 0x3e, 0xcc, 0xff, 0x5c, 0x02, 0x58, 0x62, 0x69, 0x7b, 0x29, 0x61, + 0x6e, 0x04, 0xaf, 0x31, 0x64, 0x6b, 0xe6, 0x08, 0x52, 0x8b, 0x3d, 0xc9, 0x6a, 0xb5, 0xf5, 0x89, + 0x6b, 0x16, 0xc7, 0x7f, 0xd9, 0x9a, 0x39, 0xa3, 0x72, 0x06, 0xee, 0xfd, 0xe1, 0x73, 0x3b, 0x96, + 0x3d, 0xa9, 0xb6, 0x31, 0x11, 0xdc, 0x27, 0x3c, 0xce, 0x33, 0x70, 0xef, 0x8f, 0xa9, 0xd0, 0x33, + 0x50, 0x07, 0x0f, 0x09, 0xf8, 0x80, 0xf3, 0x5e, 0x4d, 0xe7, 0xb1, 0xb3, 0x31, 0xb1, 0x2f, 0xa0, + 0xc9, 0x1c, 0xe3, 0x47, 0x35, 0xe8, 0x04, 0x6e, 0xb1, 0xa6, 0xc6, 0x12, 0x0d, 0x03, 0x26, 0xc3, + 0x8e, 0x41, 0xf6, 0x07, 0xb7, 0xf9, 0x48, 0x0f, 0xe3, 0xb6, 0xe5, 0xe2, 0x3e, 0xa3, 0x94, 0x30, + 0xd7, 0x82, 0x89, 0x26, 0x0c, 0x6b, 0x04, 0x42, 0xb3, 0xbb, 0x9e, 0xf1, 0x53, 0xed, 0xce, 0xc4, + 0x3a, 0x1b, 0xe7, 0xb1, 0xac, 0xce, 0x9c, 0xa8, 0x14, 0xed, 0xc3, 0xd5, 0x2e, 0xe3, 0x9f, 0xd8, + 0x71, 0x59, 0x61, 0x71, 0x06, 0xaa, 0xdd, 0x9d, 0xb8, 0xb7, 0x71, 0x5c, 0x95, 0xad, 0x4f, 0x77, + 0x54, 0x8e, 0xfe, 0x4f, 0xd2, 0x9c, 0x43, 0xc2, 0x77, 0x96, 0xdd, 0x80, 0x6f, 0x4d, 0x64, 0x4e, + 0x31, 0x7c, 0x95, 0x31, 0xa7, 0x41, 0x00, 0x71, 0xfb, 0x7d, 0x0a, 0xcb, 0x4e, 0x94, 0x00, 0xe2, + 0x16, 0x67, 0x80, 0xda, 0xdb, 0x3c, 0xee, 0x3b, 0xb1, 0xf3, 0x8f, 0xe7, 0x8b, 0xa5, 0x84, 0x79, + 0xcd, 0x19, 0xd7, 0xe5, 0x67, 0x61, 0x9a, 0xf7, 0x54, 0x3b, 0xa9, 0xf4, 0x15, 0x55, 0xdd, 0x49, + 0xa5, 0xaf, 0xa9, 0xcb, 0x3b, 0xa9, 0xf4, 0xb2, 0x7a, 0x7d, 0x27, 0x95, 0xbe, 0xae, 0xae, 0xec, + 0xa4, 0xd2, 0x2b, 0xea, 0x8d, 0x9d, 0x54, 0xfa, 0x86, 0xaa, 0xed, 0xa4, 0xd2, 0x9a, 0xba, 0xba, + 0x93, 0x4a, 0xaf, 0xaa, 0x6b, 0x3b, 0xa9, 0xf4, 0x2d, 0x35, 0xb3, 0x93, 0x4a, 0x67, 0xd4, 0xf5, + 0x9d, 0x54, 0xfa, 0x4d, 0x55, 0xd7, 0xef, 0x73, 0x26, 0xb9, 0xe7, 0xf9, 0xfc, 0x9e, 0x40, 0x6b, + 0x30, 0xcd, 0x06, 0x3c, 0x95, 0x2f, 0x04, 0x82, 0x7a, 0x0a, 0x91, 0xfe, 0x22, 0x05, 0xd3, 0xe1, + 0xbf, 0x38, 0xe7, 0xde, 0x4d, 0x56, 0x65, 0xdb, 0x7f, 0x35, 0xf2, 0x4f, 0x89, 0x30, 0x18, 0x3e, + 0xa6, 0xfc, 0x70, 0x94, 0x22, 0x51, 0xc2, 0xff, 0x00, 0xe2, 0x04, 0x70, 0x29, 0xb6, 0x2a, 0x46, + 0x60, 0x81, 0x1b, 0xe7, 0xef, 0xc8, 0x71, 0xde, 0x18, 0x8e, 0x33, 0x6e, 0x35, 0xc2, 0xa0, 0xa4, + 0x0c, 0x15, 0x60, 0xb1, 0xef, 0x92, 0xd3, 0x9e, 0xe7, 0x13, 0x9b, 0xdf, 0xcf, 0xa9, 0xcb, 0x70, + 0x6f, 0x73, 0x61, 0xe0, 0xc4, 0x6e, 0xe5, 0x4d, 0x98, 0xf7, 0xa8, 0xd3, 0x76, 0x5c, 0xcc, 0xee, + 0x2c, 0xce, 0xee, 0xa6, 0xf3, 0x4b, 0xb2, 0xbd, 0x9b, 0x61, 0xf7, 0x5b, 0xb9, 0x68, 0x82, 0x30, + 0x61, 0x5f, 0x68, 0x0f, 0x66, 0x6c, 0x4e, 0xd1, 0x25, 0x5b, 0xcb, 0x4c, 0x7a, 0xd0, 0x10, 0x44, + 0x3e, 0xaf, 0xc9, 0xf9, 0xa9, 0xc3, 0xf9, 0x09, 0x8d, 0x29, 0xe3, 0xa0, 0xff, 0x0e, 0x37, 0x68, + 0xf6, 0xa2, 0x80, 0xe1, 0x7e, 0xca, 0xad, 0x43, 0x8f, 0x60, 0xca, 0xf5, 0x4e, 0x24, 0x03, 0x7b, + 0x4d, 0xbb, 0x2a, 0x5f, 0x44, 0x5c, 0xef, 0x04, 0xed, 0xc2, 0x22, 0x03, 0x0c, 0x7b, 0x00, 0x16, + 0x82, 0x6c, 0x6d, 0x44, 0x02, 0x84, 0xff, 0x23, 0x67, 0x0d, 0x61, 0x18, 0x7d, 0x9e, 0x59, 0x20, + 0x11, 0x99, 0x78, 0xa6, 0x11, 0x4f, 0xbc, 0x0f, 0xfe, 0x96, 0x04, 0x6d, 0xd2, 0xdf, 0x00, 0xac, + 0xb5, 0xca, 0xe5, 0xeb, 0x66, 0x13, 0x8f, 0x3d, 0x47, 0xdf, 0x85, 0x37, 0x47, 0x34, 0xfc, 0xc3, + 0x28, 0x62, 0xd3, 0x28, 0xd4, 0xcd, 0x22, 0xde, 0xaa, 0xef, 0xd7, 0x8a, 0xaa, 0xc2, 0x7a, 0xb7, + 0x11, 0xb3, 0x42, 0xa5, 0x6c, 0xd4, 0xd8, 0xd7, 0x8e, 0x51, 0x60, 0xbd, 0xdd, 0x3a, 0xdc, 0x1c, + 0xd1, 0xef, 0xed, 0x37, 0x4a, 0x86, 0x19, 0x46, 0x53, 0x53, 0xe8, 0x26, 0xdc, 0x18, 0x1f, 0x07, + 0x37, 0xf6, 0x72, 0x35, 0x75, 0x1a, 0xe5, 0xe0, 0xa3, 0x51, 0x65, 0xc5, 0x34, 0x72, 0xc5, 0xa7, + 0xc3, 0xd7, 0x71, 0x5c, 0x37, 0xb1, 0x59, 0xaf, 0x54, 0x8c, 0x22, 0xce, 0xe7, 0x0a, 0xbb, 0x78, + 0xaf, 0xde, 0x68, 0x94, 0xf3, 0x15, 0x83, 0x37, 0xac, 0xb9, 0xa7, 0xea, 0x0c, 0x7a, 0x1b, 0x6e, + 0x8f, 0x84, 0xa8, 0x19, 0x8f, 0x71, 0xc5, 0xc8, 0x35, 0x0c, 0xbc, 0x67, 0x1a, 0x1f, 0x1b, 0xb5, + 0x66, 0x03, 0x37, 0x9f, 0xd4, 0xd4, 0x34, 0xba, 0x0f, 0x77, 0x47, 0x0c, 0x9b, 0xe5, 0xaa, 0xd1, + 0x68, 0xe6, 0xaa, 0x7b, 0xb8, 0x90, 0x2b, 0x94, 0x0c, 0x39, 0x25, 0xa3, 0xa8, 0xce, 0xae, 0xa5, + 0x7e, 0xf6, 0x9b, 0x4c, 0x42, 0x67, 0xcb, 0x9b, 0x7c, 0xf0, 0xbb, 0xd1, 0xff, 0x15, 0x22, 0xff, + 0x51, 0x88, 0xbe, 0xb5, 0x69, 0x3e, 0x1d, 0x5f, 0x5c, 0xde, 0x24, 0x33, 0xcd, 0x63, 0xb3, 0xdc, + 0x34, 0x70, 0xb3, 0x5e, 0xc7, 0xf5, 0x0a, 0x5b, 0x4e, 0xde, 0x55, 0x33, 0x45, 0xc3, 0x30, 0xcb, + 0xb9, 0x4a, 0xf9, 0x93, 0x5c, 0xbe, 0x62, 0xa8, 0x53, 0xe8, 0x16, 0xac, 0x0a, 0x79, 0xae, 0xf1, + 0xb4, 0x56, 0x90, 0x6e, 0x5b, 0xb9, 0x72, 0x65, 0xdf, 0x34, 0xd4, 0x69, 0xa4, 0x43, 0x46, 0xa8, + 0xc5, 0xc2, 0xe0, 0xa2, 0x91, 0x2b, 0x56, 0xca, 0x35, 0x03, 0x1b, 0x4f, 0x0a, 0x86, 0x51, 0x34, + 0x8a, 0xea, 0x8c, 0x48, 0xfa, 0xc1, 0x07, 0x80, 0xc6, 0x4f, 0x31, 0x4a, 0x43, 0xaa, 0x56, 0xaf, + 0x19, 0x6a, 0x02, 0xcd, 0xc3, 0x2c, 0x5b, 0xc8, 0xfa, 0xd6, 0x96, 0xaa, 0xa0, 0x45, 0x98, 0x2b, + 0x57, 0xab, 0x46, 0xb1, 0x9c, 0x6b, 0x1a, 0x6a, 0x32, 0x7f, 0xff, 0xc5, 0x3f, 0x32, 0x89, 0x17, + 0xaf, 0x32, 0xca, 0x57, 0xaf, 0x32, 0xca, 0xd7, 0xaf, 0x32, 0xca, 0xdf, 0x5f, 0x65, 0x94, 0x2f, + 0xbe, 0xc9, 0x24, 0xbe, 0xfa, 0x26, 0x93, 0xf8, 0xfa, 0x9b, 0x4c, 0xe2, 0x93, 0x59, 0x79, 0xae, + 0xff, 0x15, 0x00, 0x00, 0xff, 0xff, 0x1e, 0x69, 0x90, 0xf2, 0x1d, 0x21, 0x00, 0x00, } diff --git a/pkg/roachpb/errors.proto b/pkg/roachpb/errors.proto index 35f462c48244..f261bf3c65dd 100644 --- a/pkg/roachpb/errors.proto +++ b/pkg/roachpb/errors.proto @@ -121,11 +121,14 @@ message ReadWithinUncertaintyIntervalError { // This data below is purely informational and used to tailor the // error message. - optional util.hlc.Timestamp read_timestamp = 1 [(gogoproto.nullable) = false]; - optional util.hlc.Timestamp existing_timestamp = 2 [(gogoproto.nullable) = false]; + optional util.hlc.Timestamp read_timestamp = 1 [(gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp"]; + optional util.hlc.Timestamp existing_timestamp = 2 [(gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp"]; // The remaining fields may be missing when running in clusters that have // members at below CockroachDB v2.0. - optional util.hlc.Timestamp max_timestamp = 3; + optional util.hlc.Timestamp max_timestamp = 3 [ + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp"]; repeated roachpb.ObservedTimestamp observed_timestamps = 4 [(gogoproto.nullable) = false]; } @@ -285,8 +288,10 @@ message WriteIntentError { // by instead bumping the transaction's write timestamp, setting the // transaction's WriteTooOld flag, and dropping the error. message WriteTooOldError { - optional util.hlc.Timestamp timestamp = 1 [(gogoproto.nullable) = false]; - optional util.hlc.Timestamp actual_timestamp = 2 [(gogoproto.nullable) = false]; + optional util.hlc.Timestamp timestamp = 1 [(gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp"]; + optional util.hlc.Timestamp actual_timestamp = 2 [(gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp"]; } // An OpRequiresTxnError indicates that a command required to be @@ -418,8 +423,10 @@ message IntegerOverflowError { // A BatchTimestampBeforeGCError indicates that a request's timestamp was // before the GC threshold. message BatchTimestampBeforeGCError { - optional util.hlc.Timestamp Timestamp = 1 [(gogoproto.nullable) = false]; - optional util.hlc.Timestamp Threshold = 2 [(gogoproto.nullable) = false]; + optional util.hlc.Timestamp Timestamp = 1 [(gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp"]; + optional util.hlc.Timestamp Threshold = 2 [(gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp"]; } // An IntentMissingError indicates that a QueryIntent request expected diff --git a/pkg/roachpb/errors_test.go b/pkg/roachpb/errors_test.go index fd1f8e4b6ac0..164a6f0391a6 100644 --- a/pkg/roachpb/errors_test.go +++ b/pkg/roachpb/errors_test.go @@ -98,10 +98,10 @@ func TestErrorTxn(t *testing.T) { func TestReadWithinUncertaintyIntervalError(t *testing.T) { { rwueNew := NewReadWithinUncertaintyIntervalError( - hlc.Timestamp{WallTime: 1}, hlc.Timestamp{WallTime: 2}, + makeTxnTS(1, 0), makeTxnTS(2, 0), &Transaction{ - MaxTimestamp: hlc.Timestamp{WallTime: 3}, - ObservedTimestamps: []ObservedTimestamp{{NodeID: 12, Timestamp: hlc.Timestamp{WallTime: 4}}}, + MaxTimestamp: makeTxnTS(3, 0), + ObservedTimestamps: []ObservedTimestamp{{NodeID: 12, Timestamp: makeTS(4, 0)}}, }) expNew := "ReadWithinUncertaintyIntervalError: read at time 0.000000001,0 encountered " + "previous write with future timestamp 0.000000002,0 within uncertainty interval " + @@ -112,8 +112,7 @@ func TestReadWithinUncertaintyIntervalError(t *testing.T) { } { - rwueOld := NewReadWithinUncertaintyIntervalError( - hlc.Timestamp{WallTime: 1}, hlc.Timestamp{WallTime: 2}, nil) + rwueOld := NewReadWithinUncertaintyIntervalError(makeTxnTS(1, 0), makeTxnTS(2, 0), nil) expOld := "ReadWithinUncertaintyIntervalError: read at time 0.000000001,0 encountered " + "previous write with future timestamp 0.000000002,0 within uncertainty interval " + @@ -136,12 +135,12 @@ func TestErrorRedaction(t *testing.T) { t.Run("uncertainty-restart", func(t *testing.T) { // NB: most other errors don't redact properly. More elbow grease is needed. wrappedPErr := NewError(NewReadWithinUncertaintyIntervalError( - hlc.Timestamp{WallTime: 1}, hlc.Timestamp{WallTime: 2}, + makeTxnTS(1, 0), makeTxnTS(2, 0), &Transaction{ - MaxTimestamp: hlc.Timestamp{WallTime: 3}, - ObservedTimestamps: []ObservedTimestamp{{NodeID: 12, Timestamp: hlc.Timestamp{WallTime: 4}}}, + MaxTimestamp: makeTxnTS(3, 0), + ObservedTimestamps: []ObservedTimestamp{{NodeID: 12, Timestamp: makeTS(4, 0)}}, })) - txn := MakeTransaction("foo", Key("bar"), 1, hlc.Timestamp{WallTime: 1}, 1) + txn := MakeTransaction("foo", Key("bar"), 1, makeTS(1, 0), 1) txn.ID = uuid.Nil txn.Priority = 1234 wrappedPErr.UnexposedTxn = &txn @@ -171,13 +170,13 @@ func TestErrorDeprecatedFields(t *testing.T) { require.Equal(t, TransactionRestart_NONE, pErr.deprecatedTransactionRestart) require.Nil(t, pErr.deprecatedDetail.Value) }) - txn := MakeTransaction("foo", Key("k"), 0, hlc.Timestamp{WallTime: 1}, 50000) + txn := MakeTransaction("foo", Key("k"), 0, makeTS(1, 0), 50000) t.Run("structured-wrapped", func(t *testing.T) { // For extra spice, wrap the structured error. This ensures // that we populate the deprecated fields even when // the error detail is not the head of the error chain. - err := NewReadWithinUncertaintyIntervalError(hlc.Timestamp{WallTime: 1}, hlc.Timestamp{WallTime: 2}, &txn) + err := NewReadWithinUncertaintyIntervalError(makeTxnTS(1, 0), makeTxnTS(2, 0), &txn) pErr := NewError(errors.Wrap(err, "foo")) // Quick check that the detail round-trips when EncodedError is still there. diff --git a/pkg/roachpb/internal_raft.pb.go b/pkg/roachpb/internal_raft.pb.go index eacf6b22b73b..00fb3ef7ff8a 100644 --- a/pkg/roachpb/internal_raft.pb.go +++ b/pkg/roachpb/internal_raft.pb.go @@ -6,7 +6,8 @@ package roachpb import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" -import hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" + +import github_com_cockroachdb_cockroach_pkg_storage_enginepb "github.com/cockroachdb/cockroach/pkg/storage/enginepb" import io "io" @@ -35,7 +36,7 @@ func (m *RaftTruncatedState) Reset() { *m = RaftTruncatedState{} } func (m *RaftTruncatedState) String() string { return proto.CompactTextString(m) } func (*RaftTruncatedState) ProtoMessage() {} func (*RaftTruncatedState) Descriptor() ([]byte, []int) { - return fileDescriptor_internal_raft_eb225e3f0789b512, []int{0} + return fileDescriptor_internal_raft_57b43f385af18f11, []int{0} } func (m *RaftTruncatedState) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -69,7 +70,7 @@ func (m *RangeTombstone) Reset() { *m = RangeTombstone{} } func (m *RangeTombstone) String() string { return proto.CompactTextString(m) } func (*RangeTombstone) ProtoMessage() {} func (*RangeTombstone) Descriptor() ([]byte, []int) { - return fileDescriptor_internal_raft_eb225e3f0789b512, []int{1} + return fileDescriptor_internal_raft_57b43f385af18f11, []int{1} } func (m *RangeTombstone) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -107,7 +108,7 @@ func (m *RaftSnapshotData) Reset() { *m = RaftSnapshotData{} } func (m *RaftSnapshotData) String() string { return proto.CompactTextString(m) } func (*RaftSnapshotData) ProtoMessage() {} func (*RaftSnapshotData) Descriptor() ([]byte, []int) { - return fileDescriptor_internal_raft_eb225e3f0789b512, []int{2} + return fileDescriptor_internal_raft_57b43f385af18f11, []int{2} } func (m *RaftSnapshotData) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -133,16 +134,16 @@ func (m *RaftSnapshotData) XXX_DiscardUnknown() { var xxx_messageInfo_RaftSnapshotData proto.InternalMessageInfo type RaftSnapshotData_KeyValue struct { - Key []byte `protobuf:"bytes,1,opt,name=key" json:"key,omitempty"` - Value []byte `protobuf:"bytes,2,opt,name=value" json:"value,omitempty"` - Timestamp hlc.Timestamp `protobuf:"bytes,3,opt,name=timestamp" json:"timestamp"` + Key []byte `protobuf:"bytes,1,opt,name=key" json:"key,omitempty"` + Value []byte `protobuf:"bytes,2,opt,name=value" json:"value,omitempty"` + Timestamp github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp `protobuf:"bytes,3,opt,name=timestamp,casttype=github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp" json:"timestamp"` } func (m *RaftSnapshotData_KeyValue) Reset() { *m = RaftSnapshotData_KeyValue{} } func (m *RaftSnapshotData_KeyValue) String() string { return proto.CompactTextString(m) } func (*RaftSnapshotData_KeyValue) ProtoMessage() {} func (*RaftSnapshotData_KeyValue) Descriptor() ([]byte, []int) { - return fileDescriptor_internal_raft_eb225e3f0789b512, []int{2, 0} + return fileDescriptor_internal_raft_57b43f385af18f11, []int{2, 0} } func (m *RaftSnapshotData_KeyValue) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1000,36 +1001,38 @@ var ( ) func init() { - proto.RegisterFile("roachpb/internal_raft.proto", fileDescriptor_internal_raft_eb225e3f0789b512) + proto.RegisterFile("roachpb/internal_raft.proto", fileDescriptor_internal_raft_57b43f385af18f11) } -var fileDescriptor_internal_raft_eb225e3f0789b512 = []byte{ - // 422 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x64, 0x51, 0xcd, 0x6a, 0xdb, 0x40, - 0x10, 0xd6, 0x4a, 0x36, 0x75, 0xd6, 0x4e, 0xeb, 0x2e, 0x39, 0x08, 0x97, 0xae, 0x8c, 0x4f, 0x2e, - 0x14, 0x19, 0x72, 0xec, 0xad, 0x26, 0x85, 0x36, 0x86, 0x1e, 0x36, 0xc6, 0x87, 0x52, 0x30, 0x1b, - 0x79, 0x22, 0x8b, 0xac, 0x77, 0xc5, 0x7a, 0x54, 0x9c, 0xb7, 0xe8, 0x23, 0xe4, 0x31, 0xfa, 0x08, - 0x3e, 0xe6, 0x98, 0x93, 0x69, 0xe5, 0x4b, 0x9f, 0x21, 0xa7, 0xa2, 0x9f, 0xa4, 0xa6, 0xbd, 0xcd, - 0x7c, 0xdf, 0x37, 0xc3, 0x37, 0xdf, 0xd0, 0x57, 0xd6, 0xc8, 0x68, 0x99, 0x5e, 0x8e, 0x12, 0x8d, - 0x60, 0xb5, 0x54, 0x73, 0x2b, 0xaf, 0x30, 0x4c, 0xad, 0x41, 0xc3, 0x5e, 0x46, 0x26, 0xba, 0x2e, - 0x05, 0x61, 0x2d, 0xeb, 0xf9, 0x19, 0x26, 0x6a, 0xb4, 0x54, 0xd1, 0x08, 0x93, 0x15, 0xac, 0x51, - 0xae, 0xd2, 0x4a, 0xdc, 0x3b, 0x89, 0x4d, 0x6c, 0xca, 0x72, 0x54, 0x54, 0x15, 0x3a, 0x98, 0x52, - 0x26, 0xe4, 0x15, 0x4e, 0x6d, 0xa6, 0x23, 0x89, 0xb0, 0xb8, 0x40, 0x89, 0xc0, 0x7a, 0xb4, 0x99, - 0xe8, 0x05, 0x6c, 0x7c, 0xd2, 0x27, 0xc3, 0xc6, 0xb8, 0xb1, 0xdd, 0x05, 0x8e, 0xa8, 0x20, 0xe6, - 0xd3, 0x06, 0x82, 0x5d, 0xf9, 0xee, 0x01, 0x55, 0x22, 0xef, 0x5a, 0x3f, 0x6e, 0x03, 0xf2, 0xfb, - 0x36, 0x20, 0x83, 0xaf, 0xf4, 0xb9, 0x90, 0x3a, 0x86, 0xa9, 0x59, 0x5d, 0xae, 0xd1, 0x68, 0x60, - 0xe7, 0xf4, 0x85, 0x86, 0x0d, 0xce, 0x2d, 0xa4, 0x2a, 0x89, 0xe4, 0x3c, 0x59, 0x94, 0xbb, 0x9b, - 0xe3, 0x41, 0xb1, 0x20, 0xdf, 0x05, 0xc7, 0x9f, 0x61, 0x83, 0xa2, 0x62, 0x3f, 0x9d, 0x3d, 0xec, - 0x82, 0xa3, 0xa7, 0x46, 0x1c, 0xeb, 0x03, 0x6e, 0x31, 0x78, 0x20, 0xb4, 0x5b, 0x98, 0xbe, 0xd0, - 0x32, 0x5d, 0x2f, 0x0d, 0x9e, 0x49, 0x94, 0xec, 0x23, 0x75, 0x27, 0x33, 0xdf, 0xed, 0x7b, 0xc3, - 0xf6, 0xe9, 0xdb, 0xf0, 0xbf, 0x60, 0xc2, 0x7f, 0x07, 0xc2, 0x09, 0xdc, 0xcc, 0xa4, 0xca, 0x60, - 0x4c, 0x6b, 0x07, 0xee, 0x64, 0x26, 0xdc, 0xc9, 0x8c, 0x05, 0xb4, 0xad, 0x4c, 0x3c, 0x07, 0x8d, - 0x36, 0x81, 0xb5, 0xef, 0xf5, 0xbd, 0x61, 0x47, 0x50, 0x65, 0xe2, 0x0f, 0x15, 0xd2, 0xcb, 0x68, - 0xeb, 0x71, 0x98, 0x75, 0xa9, 0x77, 0x0d, 0x37, 0xe5, 0x2d, 0x1d, 0x51, 0x94, 0xec, 0x84, 0x36, - 0xbf, 0x15, 0x54, 0x19, 0x50, 0x47, 0x54, 0x0d, 0x7b, 0x4f, 0x8f, 0x9e, 0x1e, 0xe2, 0x7b, 0x7d, - 0x32, 0x6c, 0x9f, 0xbe, 0x3e, 0x70, 0x59, 0x7c, 0x2d, 0x5c, 0xaa, 0x28, 0x9c, 0x3e, 0x8a, 0xea, - 0x64, 0xff, 0x4e, 0x9d, 0x37, 0x5a, 0xa4, 0xeb, 0x8e, 0xdf, 0x6c, 0x7f, 0x71, 0x67, 0x9b, 0x73, - 0x72, 0x97, 0x73, 0x72, 0x9f, 0x73, 0xf2, 0x33, 0xe7, 0xe4, 0xfb, 0x9e, 0x3b, 0x77, 0x7b, 0xee, - 0xdc, 0xef, 0xb9, 0xf3, 0xe5, 0x59, 0x7d, 0xf2, 0x9f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x3b, 0xde, - 0x36, 0x90, 0x3c, 0x02, 0x00, 0x00, +var fileDescriptor_internal_raft_57b43f385af18f11 = []byte{ + // 464 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x64, 0x51, 0xb1, 0x8e, 0xd3, 0x40, + 0x10, 0xcd, 0x3a, 0x89, 0xc8, 0x6d, 0x72, 0x10, 0x56, 0x57, 0x58, 0x41, 0xd8, 0x51, 0xaa, 0x20, + 0x21, 0x5b, 0xba, 0x92, 0xd2, 0x3a, 0x24, 0x48, 0x24, 0x8a, 0xbd, 0x28, 0x05, 0x42, 0x8a, 0xd6, + 0xf6, 0x9c, 0x6d, 0xc5, 0xde, 0xb5, 0xec, 0x09, 0xca, 0x7d, 0x01, 0x2d, 0x9f, 0x70, 0x9f, 0x41, + 0x41, 0x45, 0x95, 0xf2, 0xca, 0xab, 0x22, 0x70, 0x1a, 0xbe, 0xe1, 0x2a, 0x64, 0x3b, 0x97, 0x8b, + 0xb8, 0xee, 0xcd, 0x7b, 0x6f, 0x66, 0xdf, 0xce, 0xd0, 0x57, 0x99, 0x12, 0x5e, 0x98, 0xba, 0x76, + 0x24, 0x11, 0x32, 0x29, 0xe2, 0x45, 0x26, 0xae, 0xd0, 0x4a, 0x33, 0x85, 0x8a, 0xbd, 0xf4, 0x94, + 0xb7, 0xac, 0x0c, 0xd6, 0xde, 0x36, 0xd0, 0x57, 0x18, 0xc5, 0x76, 0x18, 0x7b, 0x36, 0x46, 0x09, + 0xe4, 0x28, 0x92, 0xb4, 0x36, 0x0f, 0xce, 0x02, 0x15, 0xa8, 0x0a, 0xda, 0x25, 0xaa, 0xd9, 0xd1, + 0x8c, 0x32, 0x2e, 0xae, 0x70, 0x96, 0xad, 0xa4, 0x27, 0x10, 0xfc, 0x4b, 0x14, 0x08, 0x6c, 0x40, + 0xdb, 0x91, 0xf4, 0x61, 0xad, 0x93, 0x21, 0x19, 0xb7, 0x9c, 0xd6, 0x66, 0x6b, 0x36, 0x78, 0x4d, + 0x31, 0x9d, 0xb6, 0x10, 0xb2, 0x44, 0xd7, 0x8e, 0xa4, 0x8a, 0x79, 0xd7, 0xf9, 0x71, 0x63, 0x92, + 0xbf, 0x37, 0x26, 0x19, 0x7d, 0xa1, 0xcf, 0xb9, 0x90, 0x01, 0xcc, 0x54, 0xe2, 0xe6, 0xa8, 0x24, + 0xb0, 0x09, 0x7d, 0x21, 0x61, 0x8d, 0x8b, 0x0c, 0xd2, 0x38, 0xf2, 0xc4, 0x22, 0xf2, 0xab, 0xd9, + 0x6d, 0x67, 0x54, 0x0e, 0x28, 0xb6, 0xe6, 0xe9, 0x27, 0x58, 0x23, 0xaf, 0xd5, 0x8f, 0x17, 0xf7, + 0x5b, 0xf3, 0xe4, 0x50, 0xf0, 0x53, 0x79, 0xa4, 0xf9, 0xa3, 0x5f, 0x1a, 0xed, 0x97, 0xa1, 0x2f, + 0xa5, 0x48, 0xf3, 0x50, 0xe1, 0x85, 0x40, 0xc1, 0x3e, 0x50, 0x6d, 0x3a, 0xd7, 0xb5, 0x61, 0x73, + 0xdc, 0x3d, 0x7f, 0x6b, 0x3d, 0x59, 0x8c, 0xf5, 0x7f, 0x83, 0x35, 0x85, 0xeb, 0xb9, 0x88, 0x57, + 0xe0, 0xd0, 0x7d, 0x02, 0x6d, 0x3a, 0xe7, 0xda, 0x74, 0xce, 0x4c, 0xda, 0x8d, 0x55, 0xb0, 0x00, + 0x89, 0x59, 0x04, 0xb9, 0xde, 0x1c, 0x36, 0xc7, 0x3d, 0x4e, 0x63, 0x15, 0xbc, 0xaf, 0x99, 0xc1, + 0x4f, 0x42, 0x3b, 0x0f, 0xdd, 0xac, 0x4f, 0x9b, 0x4b, 0xb8, 0xae, 0x3e, 0xd3, 0xe3, 0x25, 0x64, + 0x67, 0xb4, 0xfd, 0xb5, 0x94, 0xaa, 0x0d, 0xf5, 0x78, 0x5d, 0xb0, 0x6f, 0x84, 0x9e, 0x1c, 0x4e, + 0xa2, 0x37, 0x87, 0x64, 0xdc, 0x3d, 0x7f, 0x7d, 0x94, 0xb3, 0xbc, 0x9b, 0x15, 0xc6, 0x9e, 0x35, + 0x7b, 0x30, 0x39, 0x93, 0x32, 0xd8, 0xfd, 0xd6, 0x74, 0x82, 0x08, 0xc3, 0x95, 0x6b, 0x79, 0x2a, + 0xb1, 0x0f, 0x0d, 0xbe, 0xfb, 0x88, 0xed, 0x74, 0x19, 0xd8, 0x39, 0xaa, 0x4c, 0x04, 0x60, 0x83, + 0x0c, 0x22, 0x09, 0xa9, 0x6b, 0xcd, 0xd6, 0xf2, 0x30, 0x8b, 0x3f, 0xbe, 0x3d, 0x69, 0x75, 0x48, + 0x5f, 0x73, 0xde, 0x6c, 0xfe, 0x18, 0x8d, 0x4d, 0x61, 0x90, 0xdb, 0xc2, 0x20, 0x77, 0x85, 0x41, + 0x7e, 0x17, 0x06, 0xf9, 0xbe, 0x33, 0x1a, 0xb7, 0x3b, 0xa3, 0x71, 0xb7, 0x33, 0x1a, 0x9f, 0x9f, + 0xed, 0x57, 0xf7, 0x2f, 0x00, 0x00, 0xff, 0xff, 0xf1, 0x52, 0x11, 0x06, 0x84, 0x02, 0x00, 0x00, } diff --git a/pkg/roachpb/internal_raft.proto b/pkg/roachpb/internal_raft.proto index 08d7a2322efc..3e0b19f624a6 100644 --- a/pkg/roachpb/internal_raft.proto +++ b/pkg/roachpb/internal_raft.proto @@ -40,7 +40,8 @@ message RaftSnapshotData { message KeyValue { optional bytes key = 1; optional bytes value = 2; - optional util.hlc.Timestamp timestamp = 3 [(gogoproto.nullable) = false]; + optional util.hlc.Timestamp timestamp = 3 [(gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp"]; } repeated KeyValue KV = 2 [(gogoproto.nullable) = false, (gogoproto.customname) = "KV"]; diff --git a/pkg/roachpb/string_test.go b/pkg/roachpb/string_test.go index 6f7305104279..31ceb8f5813b 100644 --- a/pkg/roachpb/string_test.go +++ b/pkg/roachpb/string_test.go @@ -34,16 +34,16 @@ func TestTransactionString(t *testing.T) { Key: roachpb.Key("foo"), ID: txnID, Epoch: 2, - WriteTimestamp: hlc.Timestamp{WallTime: 20, Logical: 21}, - MinTimestamp: hlc.Timestamp{WallTime: 10, Logical: 11}, + WriteTimestamp: enginepb.ToTxnTimestamp(hlc.Timestamp{WallTime: 20, Logical: 21}), + MinTimestamp: enginepb.ToTxnTimestamp(hlc.Timestamp{WallTime: 10, Logical: 11}), Priority: 957356782, Sequence: 15, }, Name: "name", Status: roachpb.COMMITTED, LastHeartbeat: hlc.Timestamp{WallTime: 10, Logical: 11}, - ReadTimestamp: hlc.Timestamp{WallTime: 30, Logical: 31}, - MaxTimestamp: hlc.Timestamp{WallTime: 40, Logical: 41}, + ReadTimestamp: enginepb.ToTxnTimestamp(hlc.Timestamp{WallTime: 30, Logical: 31}), + MaxTimestamp: enginepb.ToTxnTimestamp(hlc.Timestamp{WallTime: 40, Logical: 41}), } 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` diff --git a/pkg/server/node_tombstone_storage.go b/pkg/server/node_tombstone_storage.go index caa517d6a472..b4fd11770f02 100644 --- a/pkg/server/node_tombstone_storage.go +++ b/pkg/server/node_tombstone_storage.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil" @@ -59,7 +60,7 @@ func (s *nodeTombstoneStorage) IsDecommissioned( // No cache hit. k := s.key(nodeID) for _, eng := range s.engs { - v, _, err := storage.MVCCGet(ctx, eng, k, hlc.Timestamp{}, storage.MVCCGetOptions{}) + v, _, err := storage.MVCCGet(ctx, eng, k, enginepb.TxnTimestamp{}, storage.MVCCGetOptions{}) if err != nil { return time.Time{}, err } @@ -136,7 +137,7 @@ func (s *nodeTombstoneStorage) SetDecommissioned( } if err := storage.MVCCPut( - ctx, eng, nil /* MVCCStats */, k, hlc.Timestamp{}, v, nil, /* txn */ + ctx, eng, nil /* MVCCStats */, k, enginepb.TxnTimestamp{}, v, nil, /* txn */ ); err != nil { return err } diff --git a/pkg/server/server_sql.go b/pkg/server/server_sql.go index a2806b666723..b4528804554d 100644 --- a/pkg/server/server_sql.go +++ b/pkg/server/server_sql.go @@ -61,6 +61,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sqlmigrations" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/cloud" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/envutil" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -378,7 +379,7 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*sqlServer, error) { ParentMemoryMonitor: rootSQLMemoryMonitor, BulkAdder: func( - ctx context.Context, db *kv.DB, ts hlc.Timestamp, opts kvserverbase.BulkAdderOptions, + ctx context.Context, db *kv.DB, ts enginepb.TxnTimestamp, opts kvserverbase.BulkAdderOptions, ) (kvserverbase.BulkAdder, error) { // Attach a child memory monitor to enable control over the BulkAdder's // memory usage. diff --git a/pkg/server/settings_cache.go b/pkg/server/settings_cache.go index 43803bd3d68b..6f5552d1a4dc 100644 --- a/pkg/server/settings_cache.go +++ b/pkg/server/settings_cache.go @@ -17,7 +17,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" - "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/protoutil" ) @@ -27,9 +26,9 @@ func storeCachedSettingsKVs(ctx context.Context, eng storage.Engine, kvs []roach batch := eng.NewBatch() defer batch.Close() for _, kv := range kvs { - kv.Value.Timestamp = hlc.Timestamp{} // nb: Timestamp is not part of checksum + kv.Value.Timestamp = enginepb.TxnTimestamp{} // nb: Timestamp is not part of checksum if err := storage.MVCCPut( - ctx, batch, nil, keys.StoreCachedSettingsKey(kv.Key), hlc.Timestamp{}, kv.Value, nil, + ctx, batch, nil, keys.StoreCachedSettingsKey(kv.Key), enginepb.TxnTimestamp{}, kv.Value, nil, ); err != nil { return err } diff --git a/pkg/server/testserver.go b/pkg/server/testserver.go index b502a9e1a7fd..e6c532e55b42 100644 --- a/pkg/server/testserver.go +++ b/pkg/server/testserver.go @@ -53,6 +53,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sqlmigrations" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/cloud" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/ts" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -1265,7 +1266,7 @@ func (ts *TestServer) ForceTableGC( Key: tblKey, EndKey: tblKey.PrefixEnd(), }, - Threshold: timestamp, + Threshold: enginepb.TxnTimestamp(timestamp), } _, pErr := kv.SendWrapped(ctx, ts.distSender, &gcr) return pErr.GoError() diff --git a/pkg/sql/backfill.go b/pkg/sql/backfill.go index 5053698d425e..4ead84e0f9f3 100644 --- a/pkg/sql/backfill.go +++ b/pkg/sql/backfill.go @@ -40,8 +40,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/rowexec" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" - "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/mon" "github.com/cockroachdb/cockroach/pkg/util/timeutil" @@ -124,7 +124,7 @@ type scTxnFn func(ctx context.Context, txn *kv.Txn, evalCtx *extendedEvalContext type historicalTxnRunner func(ctx context.Context, fn scTxnFn) error // makeFixedTimestampRunner creates a historicalTxnRunner suitable for use by the helpers. -func (sc *SchemaChanger) makeFixedTimestampRunner(readAsOf hlc.Timestamp) historicalTxnRunner { +func (sc *SchemaChanger) makeFixedTimestampRunner(readAsOf enginepb.TxnTimestamp) historicalTxnRunner { runner := func(ctx context.Context, retryable scTxnFn) error { return sc.fixedTimestampTxn(ctx, readAsOf, func(ctx context.Context, txn *kv.Txn) error { // We need to re-create the evalCtx since the txn may retry. @@ -137,7 +137,7 @@ func (sc *SchemaChanger) makeFixedTimestampRunner(readAsOf hlc.Timestamp) histor func (sc *SchemaChanger) fixedTimestampTxn( ctx context.Context, - readAsOf hlc.Timestamp, + readAsOf enginepb.TxnTimestamp, retryable func(ctx context.Context, txn *kv.Txn) error, ) error { return sc.db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { @@ -618,7 +618,7 @@ func (sc *SchemaChanger) validateConstraints( } } - readAsOf := sc.clock.Now() + readAsOf := enginepb.TxnTimestamp(sc.clock.Now()) var tableDesc *tabledesc.Immutable if err := sc.fixedTimestampTxn(ctx, readAsOf, func(ctx context.Context, txn *kv.Txn) error { @@ -916,7 +916,7 @@ func (sc *SchemaChanger) distBackfill( origNRanges := -1 origFractionCompleted := sc.job.FractionCompleted() fractionLeft := 1 - origFractionCompleted - readAsOf := sc.clock.Now() + readAsOf := enginepb.TxnTimestamp(sc.clock.Now()) // Index backfilling ingests SSTs that don't play nicely with running txns // since they just add their keys blindly. Running a Scan of the target // spans at the time the SSTs' keys will be written will calcify history up @@ -1118,7 +1118,7 @@ func (sc *SchemaChanger) validateIndexes(ctx context.Context) error { } } - readAsOf := sc.clock.Now() + readAsOf := enginepb.TxnTimestamp(sc.clock.Now()) var tableDesc *tabledesc.Immutable if err := sc.fixedTimestampTxn(ctx, readAsOf, func(ctx context.Context, txn *kv.Txn) (err error) { tableDesc, err = catalogkv.MustGetTableDescByID(ctx, txn, sc.execCfg.Codec, sc.descID) diff --git a/pkg/sql/catalog/catalogkv/catalogkv.go b/pkg/sql/catalog/catalogkv/catalogkv.go index 1e4428041948..fe9070919a93 100644 --- a/pkg/sql/catalog/catalogkv/catalogkv.go +++ b/pkg/sql/catalog/catalogkv/catalogkv.go @@ -31,7 +31,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" - "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" ) @@ -303,12 +303,12 @@ func validateDescriptor(ctx context.Context, dg catalog.DescGetter, desc catalog func unwrapDescriptor( ctx context.Context, dg catalog.DescGetter, - ts hlc.Timestamp, + ts enginepb.TxnTimestamp, desc *descpb.Descriptor, validate bool, ) (catalog.Descriptor, error) { descpb.MaybeSetDescriptorModificationTimeFromMVCCTimestamp(ctx, desc, ts) - table, database, typ, schema := descpb.TableFromDescriptor(desc, hlc.Timestamp{}), + table, database, typ, schema := descpb.TableFromDescriptor(desc, enginepb.TxnTimestamp{}), desc.GetDatabase(), desc.GetType(), desc.GetSchema() var unwrapped catalog.Descriptor switch { @@ -339,11 +339,11 @@ func unwrapDescriptor( // unwraps it into an implementation of catalog.MutableDescriptor. It ensures // that the ModificationTime is set properly. func unwrapDescriptorMutable( - ctx context.Context, dg catalog.DescGetter, ts hlc.Timestamp, desc *descpb.Descriptor, + ctx context.Context, dg catalog.DescGetter, ts enginepb.TxnTimestamp, desc *descpb.Descriptor, ) (catalog.MutableDescriptor, error) { descpb.MaybeSetDescriptorModificationTimeFromMVCCTimestamp(ctx, desc, ts) table, database, typ, schema := - descpb.TableFromDescriptor(desc, hlc.Timestamp{}), + descpb.TableFromDescriptor(desc, enginepb.TxnTimestamp{}), desc.GetDatabase(), desc.GetType(), desc.GetSchema() switch { case table != nil: @@ -701,8 +701,8 @@ func GetSchemaDescriptorsFromIDs( // // TODO(ajwerner): unify this with the other unwrapping logic. func UnwrapDescriptorRaw(ctx context.Context, desc *descpb.Descriptor) catalog.MutableDescriptor { - descpb.MaybeSetDescriptorModificationTimeFromMVCCTimestamp(ctx, desc, hlc.Timestamp{}) - table, database, typ, schema := descpb.TableFromDescriptor(desc, hlc.Timestamp{}), + descpb.MaybeSetDescriptorModificationTimeFromMVCCTimestamp(ctx, desc, enginepb.TxnTimestamp{}) + table, database, typ, schema := descpb.TableFromDescriptor(desc, enginepb.TxnTimestamp{}), desc.GetDatabase(), desc.GetType(), desc.GetSchema() switch { case table != nil: diff --git a/pkg/sql/catalog/dbdesc/database_desc.go b/pkg/sql/catalog/dbdesc/database_desc.go index 8bec84cffcc3..1e97f22107b2 100644 --- a/pkg/sql/catalog/dbdesc/database_desc.go +++ b/pkg/sql/catalog/dbdesc/database_desc.go @@ -20,7 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/privilege" - "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/errors" "github.com/cockroachdb/redact" @@ -303,7 +303,7 @@ func (desc *Mutable) MaybeIncrementVersion() { return } desc.Version++ - desc.ModificationTime = hlc.Timestamp{} + desc.ModificationTime = enginepb.TxnTimestamp{} } // OriginalName implements the MutableDescriptor interface. diff --git a/pkg/sql/catalog/descpb/descriptor.go b/pkg/sql/catalog/descpb/descriptor.go index a8d3d34dce49..79400adb9994 100644 --- a/pkg/sql/catalog/descpb/descriptor.go +++ b/pkg/sql/catalog/descpb/descriptor.go @@ -14,7 +14,7 @@ import ( "context" "runtime/debug" - "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" ) @@ -78,7 +78,7 @@ func GetDescriptorVersion(desc *Descriptor) DescriptorVersion { } // GetDescriptorModificationTime returns the ModificationTime of the descriptor. -func GetDescriptorModificationTime(desc *Descriptor) hlc.Timestamp { +func GetDescriptorModificationTime(desc *Descriptor) enginepb.TxnTimestamp { switch t := desc.Union.(type) { case *Descriptor_Table: return t.Table.ModificationTime @@ -112,7 +112,7 @@ func GetDescriptorState(desc *Descriptor) DescriptorState { } // setDescriptorModificationTime sets the ModificationTime of the descriptor. -func setDescriptorModificationTime(desc *Descriptor, ts hlc.Timestamp) { +func setDescriptorModificationTime(desc *Descriptor, ts enginepb.TxnTimestamp) { switch t := desc.Union.(type) { case *Descriptor_Table: t.Table.ModificationTime = ts @@ -145,7 +145,7 @@ func setDescriptorModificationTime(desc *Descriptor, ts hlc.Timestamp) { // It is vital that users which read table descriptor values from the KV store // call this method. func MaybeSetDescriptorModificationTimeFromMVCCTimestamp( - ctx context.Context, desc *Descriptor, ts hlc.Timestamp, + ctx context.Context, desc *Descriptor, ts enginepb.TxnTimestamp, ) { switch t := desc.Union.(type) { case nil: @@ -208,7 +208,7 @@ func MaybeSetDescriptorModificationTimeFromMVCCTimestamp( // TODO(ajwerner): Now that all descriptors have their modification time set // this way, this function should be retired and similar or better safeguards // for all descriptors should be pursued. -func TableFromDescriptor(desc *Descriptor, ts hlc.Timestamp) *TableDescriptor { +func TableFromDescriptor(desc *Descriptor, ts enginepb.TxnTimestamp) *TableDescriptor { //nolint:descriptormarshal t := desc.GetTable() if t != nil { @@ -218,7 +218,7 @@ func TableFromDescriptor(desc *Descriptor, ts hlc.Timestamp) *TableDescriptor { } // TypeFromDescriptor is the same thing as TableFromDescriptor, but for types. -func TypeFromDescriptor(desc *Descriptor, ts hlc.Timestamp) *TypeDescriptor { +func TypeFromDescriptor(desc *Descriptor, ts enginepb.TxnTimestamp) *TypeDescriptor { t := desc.GetType() if t != nil { MaybeSetDescriptorModificationTimeFromMVCCTimestamp(context.TODO(), desc, ts) diff --git a/pkg/sql/catalog/descpb/structured.pb.go b/pkg/sql/catalog/descpb/structured.pb.go index d880eded7ec1..8a391dca64bf 100644 --- a/pkg/sql/catalog/descpb/structured.pb.go +++ b/pkg/sql/catalog/descpb/structured.pb.go @@ -8,8 +8,8 @@ import fmt "fmt" import math "math" import geoindex "github.com/cockroachdb/cockroach/pkg/geo/geoindex" import types "github.com/cockroachdb/cockroach/pkg/sql/types" -import hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" +import github_com_cockroachdb_cockroach_pkg_storage_enginepb "github.com/cockroachdb/cockroach/pkg/storage/enginepb" import github_com_cockroachdb_cockroach_pkg_roachpb "github.com/cockroachdb/cockroach/pkg/roachpb" import bytes "bytes" @@ -76,7 +76,7 @@ func (x *ConstraintValidity) UnmarshalJSON(data []byte) error { return nil } func (ConstraintValidity) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_c6d9278eb5f43f16, []int{0} + return fileDescriptor_structured_bf44202de9726669, []int{0} } // SystemColumnKind is an enum representing the different kind of system @@ -121,7 +121,7 @@ func (x *SystemColumnKind) UnmarshalJSON(data []byte) error { return nil } func (SystemColumnKind) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_c6d9278eb5f43f16, []int{1} + return fileDescriptor_structured_bf44202de9726669, []int{1} } // State indicates whether a descriptor is public (i.e., normally visible, @@ -173,7 +173,7 @@ func (x *DescriptorState) UnmarshalJSON(data []byte) error { return nil } func (DescriptorState) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_c6d9278eb5f43f16, []int{2} + return fileDescriptor_structured_bf44202de9726669, []int{2} } // SurvivalGoal is the survival goal for a database. @@ -212,7 +212,7 @@ func (x *SurvivalGoal) UnmarshalJSON(data []byte) error { return nil } func (SurvivalGoal) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_c6d9278eb5f43f16, []int{3} + return fileDescriptor_structured_bf44202de9726669, []int{3} } type ForeignKeyReference_Action int32 @@ -257,7 +257,7 @@ func (x *ForeignKeyReference_Action) UnmarshalJSON(data []byte) error { return nil } func (ForeignKeyReference_Action) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_c6d9278eb5f43f16, []int{0, 0} + return fileDescriptor_structured_bf44202de9726669, []int{0, 0} } // Match is the algorithm used to compare composite keys. @@ -297,7 +297,7 @@ func (x *ForeignKeyReference_Match) UnmarshalJSON(data []byte) error { return nil } func (ForeignKeyReference_Match) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_c6d9278eb5f43f16, []int{0, 1} + return fileDescriptor_structured_bf44202de9726669, []int{0, 1} } // The direction of a column in the index. @@ -334,7 +334,7 @@ func (x *IndexDescriptor_Direction) UnmarshalJSON(data []byte) error { return nil } func (IndexDescriptor_Direction) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_c6d9278eb5f43f16, []int{8, 0} + return fileDescriptor_structured_bf44202de9726669, []int{8, 0} } // The type of the index. @@ -371,7 +371,7 @@ func (x *IndexDescriptor_Type) UnmarshalJSON(data []byte) error { return nil } func (IndexDescriptor_Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_c6d9278eb5f43f16, []int{8, 1} + return fileDescriptor_structured_bf44202de9726669, []int{8, 1} } type ConstraintToUpdate_ConstraintType int32 @@ -417,7 +417,7 @@ func (x *ConstraintToUpdate_ConstraintType) UnmarshalJSON(data []byte) error { return nil } func (ConstraintToUpdate_ConstraintType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_c6d9278eb5f43f16, []int{9, 0} + return fileDescriptor_structured_bf44202de9726669, []int{9, 0} } // A descriptor within a mutation is unavailable for reads, writes @@ -482,7 +482,7 @@ func (x *DescriptorMutation_State) UnmarshalJSON(data []byte) error { return nil } func (DescriptorMutation_State) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_c6d9278eb5f43f16, []int{13, 0} + return fileDescriptor_structured_bf44202de9726669, []int{13, 0} } // Direction of mutation. @@ -525,7 +525,7 @@ func (x *DescriptorMutation_Direction) UnmarshalJSON(data []byte) error { return nil } func (DescriptorMutation_Direction) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_c6d9278eb5f43f16, []int{13, 1} + return fileDescriptor_structured_bf44202de9726669, []int{13, 1} } // AuditMode indicates which auditing actions to take when this table is used. @@ -562,7 +562,7 @@ func (x *TableDescriptor_AuditMode) UnmarshalJSON(data []byte) error { return nil } func (TableDescriptor_AuditMode) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_c6d9278eb5f43f16, []int{15, 0} + return fileDescriptor_structured_bf44202de9726669, []int{15, 0} } // Represents the kind of type that this type descriptor represents. @@ -607,7 +607,7 @@ func (x *TypeDescriptor_Kind) UnmarshalJSON(data []byte) error { return nil } func (TypeDescriptor_Kind) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_c6d9278eb5f43f16, []int{17, 0} + return fileDescriptor_structured_bf44202de9726669, []int{17, 0} } // Represents what operations are allowed on this ENUM member. @@ -648,7 +648,7 @@ func (x *TypeDescriptor_EnumMember_Capability) UnmarshalJSON(data []byte) error return nil } func (TypeDescriptor_EnumMember_Capability) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_structured_c6d9278eb5f43f16, []int{17, 0, 0} + return fileDescriptor_structured_bf44202de9726669, []int{17, 0, 0} } // ForeignKeyReference is deprecated, replaced by ForeignKeyConstraint in v19.2 @@ -678,7 +678,7 @@ func (m *ForeignKeyReference) Reset() { *m = ForeignKeyReference{} } func (m *ForeignKeyReference) String() string { return proto.CompactTextString(m) } func (*ForeignKeyReference) ProtoMessage() {} func (*ForeignKeyReference) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_c6d9278eb5f43f16, []int{0} + return fileDescriptor_structured_bf44202de9726669, []int{0} } func (m *ForeignKeyReference) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -726,7 +726,7 @@ func (m *ForeignKeyConstraint) Reset() { *m = ForeignKeyConstraint{} } func (m *ForeignKeyConstraint) String() string { return proto.CompactTextString(m) } func (*ForeignKeyConstraint) ProtoMessage() {} func (*ForeignKeyConstraint) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_c6d9278eb5f43f16, []int{1} + return fileDescriptor_structured_bf44202de9726669, []int{1} } func (m *ForeignKeyConstraint) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -764,7 +764,7 @@ func (m *UniqueWithoutIndexConstraint) Reset() { *m = UniqueWithoutIndex func (m *UniqueWithoutIndexConstraint) String() string { return proto.CompactTextString(m) } func (*UniqueWithoutIndexConstraint) ProtoMessage() {} func (*UniqueWithoutIndexConstraint) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_c6d9278eb5f43f16, []int{2} + return fileDescriptor_structured_bf44202de9726669, []int{2} } func (m *UniqueWithoutIndexConstraint) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -832,7 +832,7 @@ func (m *ColumnDescriptor) Reset() { *m = ColumnDescriptor{} } func (m *ColumnDescriptor) String() string { return proto.CompactTextString(m) } func (*ColumnDescriptor) ProtoMessage() {} func (*ColumnDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_c6d9278eb5f43f16, []int{3} + return fileDescriptor_structured_bf44202de9726669, []int{3} } func (m *ColumnDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -888,7 +888,7 @@ func (m *ColumnFamilyDescriptor) Reset() { *m = ColumnFamilyDescriptor{} func (m *ColumnFamilyDescriptor) String() string { return proto.CompactTextString(m) } func (*ColumnFamilyDescriptor) ProtoMessage() {} func (*ColumnFamilyDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_c6d9278eb5f43f16, []int{4} + return fileDescriptor_structured_bf44202de9726669, []int{4} } func (m *ColumnFamilyDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -934,7 +934,7 @@ func (m *InterleaveDescriptor) Reset() { *m = InterleaveDescriptor{} } func (m *InterleaveDescriptor) String() string { return proto.CompactTextString(m) } func (*InterleaveDescriptor) ProtoMessage() {} func (*InterleaveDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_c6d9278eb5f43f16, []int{5} + return fileDescriptor_structured_bf44202de9726669, []int{5} } func (m *InterleaveDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -978,7 +978,7 @@ func (m *InterleaveDescriptor_Ancestor) Reset() { *m = InterleaveDescrip func (m *InterleaveDescriptor_Ancestor) String() string { return proto.CompactTextString(m) } func (*InterleaveDescriptor_Ancestor) ProtoMessage() {} func (*InterleaveDescriptor_Ancestor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_c6d9278eb5f43f16, []int{5, 0} + return fileDescriptor_structured_bf44202de9726669, []int{5, 0} } func (m *InterleaveDescriptor_Ancestor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1033,7 +1033,7 @@ func (m *ShardedDescriptor) Reset() { *m = ShardedDescriptor{} } func (m *ShardedDescriptor) String() string { return proto.CompactTextString(m) } func (*ShardedDescriptor) ProtoMessage() {} func (*ShardedDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_c6d9278eb5f43f16, []int{6} + return fileDescriptor_structured_bf44202de9726669, []int{6} } func (m *ShardedDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1078,7 +1078,7 @@ func (m *PartitioningDescriptor) Reset() { *m = PartitioningDescriptor{} func (m *PartitioningDescriptor) String() string { return proto.CompactTextString(m) } func (*PartitioningDescriptor) ProtoMessage() {} func (*PartitioningDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_c6d9278eb5f43f16, []int{7} + return fileDescriptor_structured_bf44202de9726669, []int{7} } func (m *PartitioningDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1121,7 +1121,7 @@ func (m *PartitioningDescriptor_List) Reset() { *m = PartitioningDescrip func (m *PartitioningDescriptor_List) String() string { return proto.CompactTextString(m) } func (*PartitioningDescriptor_List) ProtoMessage() {} func (*PartitioningDescriptor_List) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_c6d9278eb5f43f16, []int{7, 0} + return fileDescriptor_structured_bf44202de9726669, []int{7, 0} } func (m *PartitioningDescriptor_List) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1166,7 +1166,7 @@ func (m *PartitioningDescriptor_Range) Reset() { *m = PartitioningDescri func (m *PartitioningDescriptor_Range) String() string { return proto.CompactTextString(m) } func (*PartitioningDescriptor_Range) ProtoMessage() {} func (*PartitioningDescriptor_Range) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_c6d9278eb5f43f16, []int{7, 1} + return fileDescriptor_structured_bf44202de9726669, []int{7, 1} } func (m *PartitioningDescriptor_Range) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1333,7 +1333,7 @@ func (m *IndexDescriptor) Reset() { *m = IndexDescriptor{} } func (m *IndexDescriptor) String() string { return proto.CompactTextString(m) } func (*IndexDescriptor) ProtoMessage() {} func (*IndexDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_c6d9278eb5f43f16, []int{8} + return fileDescriptor_structured_bf44202de9726669, []int{8} } func (m *IndexDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1385,7 +1385,7 @@ func (m *ConstraintToUpdate) Reset() { *m = ConstraintToUpdate{} } func (m *ConstraintToUpdate) String() string { return proto.CompactTextString(m) } func (*ConstraintToUpdate) ProtoMessage() {} func (*ConstraintToUpdate) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_c6d9278eb5f43f16, []int{9} + return fileDescriptor_structured_bf44202de9726669, []int{9} } func (m *ConstraintToUpdate) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1432,7 +1432,7 @@ func (m *PrimaryKeySwap) Reset() { *m = PrimaryKeySwap{} } func (m *PrimaryKeySwap) String() string { return proto.CompactTextString(m) } func (*PrimaryKeySwap) ProtoMessage() {} func (*PrimaryKeySwap) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_c6d9278eb5f43f16, []int{10} + return fileDescriptor_structured_bf44202de9726669, []int{10} } func (m *PrimaryKeySwap) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1472,7 +1472,7 @@ func (m *ComputedColumnSwap) Reset() { *m = ComputedColumnSwap{} } func (m *ComputedColumnSwap) String() string { return proto.CompactTextString(m) } func (*ComputedColumnSwap) ProtoMessage() {} func (*ComputedColumnSwap) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_c6d9278eb5f43f16, []int{11} + return fileDescriptor_structured_bf44202de9726669, []int{11} } func (m *ComputedColumnSwap) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1508,7 +1508,7 @@ type MaterializedViewRefresh struct { // NewIndexes are the new set of indexes to backfill the view into. NewIndexes []IndexDescriptor `protobuf:"bytes,2,rep,name=new_indexes,json=newIndexes" json:"new_indexes"` // AsOf is the timestamp to perform the view query at. - AsOf hlc.Timestamp `protobuf:"bytes,3,opt,name=as_of,json=asOf" json:"as_of"` + AsOf github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp `protobuf:"bytes,3,opt,name=as_of,json=asOf,casttype=github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp" json:"as_of"` // ShouldBackfill indicates whether or not the schema changer should backfill // the query into the new indexes. This can be false if the `WITH NO DATA` flag // was specified for the `REFRESH MATERIALIZED VIEW` statement. `WITH NO DATA` @@ -1520,7 +1520,7 @@ func (m *MaterializedViewRefresh) Reset() { *m = MaterializedViewRefresh func (m *MaterializedViewRefresh) String() string { return proto.CompactTextString(m) } func (*MaterializedViewRefresh) ProtoMessage() {} func (*MaterializedViewRefresh) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_c6d9278eb5f43f16, []int{12} + return fileDescriptor_structured_bf44202de9726669, []int{12} } func (m *MaterializedViewRefresh) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1577,7 +1577,7 @@ func (m *DescriptorMutation) Reset() { *m = DescriptorMutation{} } func (m *DescriptorMutation) String() string { return proto.CompactTextString(m) } func (*DescriptorMutation) ProtoMessage() {} func (*DescriptorMutation) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_c6d9278eb5f43f16, []int{13} + return fileDescriptor_structured_bf44202de9726669, []int{13} } func (m *DescriptorMutation) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1915,7 +1915,7 @@ func (m *NameInfo) Reset() { *m = NameInfo{} } func (m *NameInfo) String() string { return proto.CompactTextString(m) } func (*NameInfo) ProtoMessage() {} func (*NameInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_c6d9278eb5f43f16, []int{14} + return fileDescriptor_structured_bf44202de9726669, []int{14} } func (m *NameInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1974,7 +1974,7 @@ type TableDescriptor struct { // should live inside of a Descriptor. The Descriptor.Table() method takes an // hlc timestamp to ensure that this field is set properly when extracted from // a Descriptor. - ModificationTime hlc.Timestamp `protobuf:"bytes,7,opt,name=modification_time,json=modificationTime" json:"modification_time"` + ModificationTime github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp `protobuf:"bytes,7,opt,name=modification_time,json=modificationTime,casttype=github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp" json:"modification_time"` // A list of draining names. The draining name entries are drained from // the cluster wide name caches by incrementing the version for this // descriptor and ensuring that there are no leases on prior @@ -2079,7 +2079,7 @@ type TableDescriptor struct { // like ModificationTime. See Descriptor.Table(). // CreateAsOfSystemTime is used for CREATE TABLE ... AS ... and was // added in 19.1. - CreateAsOfTime hlc.Timestamp `protobuf:"bytes,35,opt,name=create_as_of_time,json=createAsOfTime" json:"create_as_of_time"` + CreateAsOfTime github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp `protobuf:"bytes,35,opt,name=create_as_of_time,json=createAsOfTime,casttype=github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp" json:"create_as_of_time"` // outbound_fks contains all foreign key constraints that have this table as // the origin table. OutboundFKs []ForeignKeyConstraint `protobuf:"bytes,36,rep,name=outbound_fks,json=outboundFks" json:"outbound_fks"` @@ -2101,7 +2101,7 @@ func (m *TableDescriptor) Reset() { *m = TableDescriptor{} } func (m *TableDescriptor) String() string { return proto.CompactTextString(m) } func (*TableDescriptor) ProtoMessage() {} func (*TableDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_c6d9278eb5f43f16, []int{15} + return fileDescriptor_structured_bf44202de9726669, []int{15} } func (m *TableDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2147,11 +2147,11 @@ func (m *TableDescriptor) GetVersion() DescriptorVersion { return 0 } -func (m *TableDescriptor) GetModificationTime() hlc.Timestamp { +func (m *TableDescriptor) GetModificationTime() github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp { if m != nil { return m.ModificationTime } - return hlc.Timestamp{} + return github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp{} } func (m *TableDescriptor) GetDrainingNames() []NameInfo { @@ -2365,11 +2365,11 @@ func (m *TableDescriptor) GetCreateQuery() string { return "" } -func (m *TableDescriptor) GetCreateAsOfTime() hlc.Timestamp { +func (m *TableDescriptor) GetCreateAsOfTime() github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp { if m != nil { return m.CreateAsOfTime } - return hlc.Timestamp{} + return github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp{} } func (m *TableDescriptor) GetOutboundFKs() []ForeignKeyConstraint { @@ -2422,7 +2422,7 @@ func (m *TableDescriptor_SchemaChangeLease) Reset() { *m = TableDescript func (m *TableDescriptor_SchemaChangeLease) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_SchemaChangeLease) ProtoMessage() {} func (*TableDescriptor_SchemaChangeLease) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_c6d9278eb5f43f16, []int{15, 0} + return fileDescriptor_structured_bf44202de9726669, []int{15, 0} } func (m *TableDescriptor_SchemaChangeLease) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2468,7 +2468,7 @@ func (m *TableDescriptor_CheckConstraint) Reset() { *m = TableDescriptor func (m *TableDescriptor_CheckConstraint) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_CheckConstraint) ProtoMessage() {} func (*TableDescriptor_CheckConstraint) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_c6d9278eb5f43f16, []int{15, 1} + return fileDescriptor_structured_bf44202de9726669, []int{15, 1} } func (m *TableDescriptor_CheckConstraint) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2508,7 +2508,7 @@ func (m *TableDescriptor_Reference) Reset() { *m = TableDescriptor_Refer func (m *TableDescriptor_Reference) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_Reference) ProtoMessage() {} func (*TableDescriptor_Reference) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_c6d9278eb5f43f16, []int{15, 2} + return fileDescriptor_structured_bf44202de9726669, []int{15, 2} } func (m *TableDescriptor_Reference) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2545,7 +2545,7 @@ func (m *TableDescriptor_MutationJob) Reset() { *m = TableDescriptor_Mut func (m *TableDescriptor_MutationJob) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_MutationJob) ProtoMessage() {} func (*TableDescriptor_MutationJob) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_c6d9278eb5f43f16, []int{15, 3} + return fileDescriptor_structured_bf44202de9726669, []int{15, 3} } func (m *TableDescriptor_MutationJob) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2588,7 +2588,7 @@ func (m *TableDescriptor_SequenceOpts) Reset() { *m = TableDescriptor_Se func (m *TableDescriptor_SequenceOpts) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_SequenceOpts) ProtoMessage() {} func (*TableDescriptor_SequenceOpts) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_c6d9278eb5f43f16, []int{15, 4} + return fileDescriptor_structured_bf44202de9726669, []int{15, 4} } func (m *TableDescriptor_SequenceOpts) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2628,7 +2628,7 @@ func (m *TableDescriptor_SequenceOpts_SequenceOwner) String() string { } func (*TableDescriptor_SequenceOpts_SequenceOwner) ProtoMessage() {} func (*TableDescriptor_SequenceOpts_SequenceOwner) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_c6d9278eb5f43f16, []int{15, 4, 0} + return fileDescriptor_structured_bf44202de9726669, []int{15, 4, 0} } func (m *TableDescriptor_SequenceOpts_SequenceOwner) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2661,14 +2661,14 @@ type TableDescriptor_Replacement struct { // transaction which created this replacement. In 20.1 and after it is // populated with the read timestamp at which the descriptor being // replaced was read. - Time hlc.Timestamp `protobuf:"bytes,2,opt,name=time" json:"time"` + Time github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp `protobuf:"bytes,2,opt,name=time,casttype=github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp" json:"time"` } func (m *TableDescriptor_Replacement) Reset() { *m = TableDescriptor_Replacement{} } func (m *TableDescriptor_Replacement) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_Replacement) ProtoMessage() {} func (*TableDescriptor_Replacement) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_c6d9278eb5f43f16, []int{15, 5} + return fileDescriptor_structured_bf44202de9726669, []int{15, 5} } func (m *TableDescriptor_Replacement) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2705,7 +2705,7 @@ func (m *TableDescriptor_GCDescriptorMutation) Reset() { *m = TableDescr func (m *TableDescriptor_GCDescriptorMutation) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_GCDescriptorMutation) ProtoMessage() {} func (*TableDescriptor_GCDescriptorMutation) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_c6d9278eb5f43f16, []int{15, 6} + return fileDescriptor_structured_bf44202de9726669, []int{15, 6} } func (m *TableDescriptor_GCDescriptorMutation) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2742,7 +2742,7 @@ func (m *TableDescriptor_LocalityConfig) Reset() { *m = TableDescriptor_ func (m *TableDescriptor_LocalityConfig) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_LocalityConfig) ProtoMessage() {} func (*TableDescriptor_LocalityConfig) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_c6d9278eb5f43f16, []int{15, 7} + return fileDescriptor_structured_bf44202de9726669, []int{15, 7} } func (m *TableDescriptor_LocalityConfig) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2922,7 +2922,7 @@ func (m *TableDescriptor_LocalityConfig_RegionalByTable) String() string { } func (*TableDescriptor_LocalityConfig_RegionalByTable) ProtoMessage() {} func (*TableDescriptor_LocalityConfig_RegionalByTable) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_c6d9278eb5f43f16, []int{15, 7, 0} + return fileDescriptor_structured_bf44202de9726669, []int{15, 7, 0} } func (m *TableDescriptor_LocalityConfig_RegionalByTable) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2958,7 +2958,7 @@ func (m *TableDescriptor_LocalityConfig_RegionalByRow) String() string { } func (*TableDescriptor_LocalityConfig_RegionalByRow) ProtoMessage() {} func (*TableDescriptor_LocalityConfig_RegionalByRow) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_c6d9278eb5f43f16, []int{15, 7, 1} + return fileDescriptor_structured_bf44202de9726669, []int{15, 7, 1} } func (m *TableDescriptor_LocalityConfig_RegionalByRow) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -2990,7 +2990,7 @@ func (m *TableDescriptor_LocalityConfig_Global) Reset() { *m = TableDesc func (m *TableDescriptor_LocalityConfig_Global) String() string { return proto.CompactTextString(m) } func (*TableDescriptor_LocalityConfig_Global) ProtoMessage() {} func (*TableDescriptor_LocalityConfig_Global) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_c6d9278eb5f43f16, []int{15, 7, 2} + return fileDescriptor_structured_bf44202de9726669, []int{15, 7, 2} } func (m *TableDescriptor_LocalityConfig_Global) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3023,10 +3023,10 @@ type DatabaseDescriptor struct { Name string `protobuf:"bytes,1,opt,name=name" json:"name"` ID ID `protobuf:"varint,2,opt,name=id,casttype=ID" json:"id"` // Last modification time of the descriptor. - ModificationTime hlc.Timestamp `protobuf:"bytes,4,opt,name=modification_time,json=modificationTime" json:"modification_time"` - Version DescriptorVersion `protobuf:"varint,5,opt,name=version,casttype=DescriptorVersion" json:"version"` - DrainingNames []NameInfo `protobuf:"bytes,6,rep,name=draining_names,json=drainingNames" json:"draining_names"` - Privileges *PrivilegeDescriptor `protobuf:"bytes,3,opt,name=privileges" json:"privileges,omitempty"` + ModificationTime github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp `protobuf:"bytes,4,opt,name=modification_time,json=modificationTime,casttype=github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp" json:"modification_time"` + Version DescriptorVersion `protobuf:"varint,5,opt,name=version,casttype=DescriptorVersion" json:"version"` + DrainingNames []NameInfo `protobuf:"bytes,6,rep,name=draining_names,json=drainingNames" json:"draining_names"` + Privileges *PrivilegeDescriptor `protobuf:"bytes,3,opt,name=privileges" json:"privileges,omitempty"` // schemas is a mapping from child schema name to ID. It is used during // name resolution to know without a KV lookup whether a database has a // child schema with a target name. Temporary schemas are not stored here. @@ -3041,7 +3041,7 @@ func (m *DatabaseDescriptor) Reset() { *m = DatabaseDescriptor{} } func (m *DatabaseDescriptor) String() string { return proto.CompactTextString(m) } func (*DatabaseDescriptor) ProtoMessage() {} func (*DatabaseDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_c6d9278eb5f43f16, []int{16} + return fileDescriptor_structured_bf44202de9726669, []int{16} } func (m *DatabaseDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3080,11 +3080,11 @@ func (m *DatabaseDescriptor) GetID() ID { return 0 } -func (m *DatabaseDescriptor) GetModificationTime() hlc.Timestamp { +func (m *DatabaseDescriptor) GetModificationTime() github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp { if m != nil { return m.ModificationTime } - return hlc.Timestamp{} + return github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp{} } func (m *DatabaseDescriptor) GetVersion() DescriptorVersion { @@ -3149,7 +3149,7 @@ func (m *DatabaseDescriptor_SchemaInfo) Reset() { *m = DatabaseDescripto func (m *DatabaseDescriptor_SchemaInfo) String() string { return proto.CompactTextString(m) } func (*DatabaseDescriptor_SchemaInfo) ProtoMessage() {} func (*DatabaseDescriptor_SchemaInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_c6d9278eb5f43f16, []int{16, 0} + return fileDescriptor_structured_bf44202de9726669, []int{16, 0} } func (m *DatabaseDescriptor_SchemaInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3189,7 +3189,7 @@ func (m *DatabaseDescriptor_RegionConfig) Reset() { *m = DatabaseDescrip func (m *DatabaseDescriptor_RegionConfig) String() string { return proto.CompactTextString(m) } func (*DatabaseDescriptor_RegionConfig) ProtoMessage() {} func (*DatabaseDescriptor_RegionConfig) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_c6d9278eb5f43f16, []int{16, 2} + return fileDescriptor_structured_bf44202de9726669, []int{16, 2} } func (m *DatabaseDescriptor_RegionConfig) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3224,8 +3224,8 @@ type TypeDescriptor struct { ID ID `protobuf:"varint,4,opt,name=id,casttype=ID" json:"id"` Version DescriptorVersion `protobuf:"varint,9,opt,name=version,casttype=DescriptorVersion" json:"version"` // Last modification time of the descriptor. - ModificationTime hlc.Timestamp `protobuf:"bytes,10,opt,name=modification_time,json=modificationTime" json:"modification_time"` - DrainingNames []NameInfo `protobuf:"bytes,11,rep,name=draining_names,json=drainingNames" json:"draining_names"` + ModificationTime github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp `protobuf:"bytes,10,opt,name=modification_time,json=modificationTime,casttype=github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp" json:"modification_time"` + DrainingNames []NameInfo `protobuf:"bytes,11,rep,name=draining_names,json=drainingNames" json:"draining_names"` // privileges contains the privileges for the type. Privileges *PrivilegeDescriptor `protobuf:"bytes,14,opt,name=privileges" json:"privileges,omitempty"` // parent_id represents the ID of the database that this type resides in. @@ -3252,7 +3252,7 @@ func (m *TypeDescriptor) Reset() { *m = TypeDescriptor{} } func (m *TypeDescriptor) String() string { return proto.CompactTextString(m) } func (*TypeDescriptor) ProtoMessage() {} func (*TypeDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_c6d9278eb5f43f16, []int{17} + return fileDescriptor_structured_bf44202de9726669, []int{17} } func (m *TypeDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3298,11 +3298,11 @@ func (m *TypeDescriptor) GetVersion() DescriptorVersion { return 0 } -func (m *TypeDescriptor) GetModificationTime() hlc.Timestamp { +func (m *TypeDescriptor) GetModificationTime() github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp { if m != nil { return m.ModificationTime } - return hlc.Timestamp{} + return github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp{} } func (m *TypeDescriptor) GetDrainingNames() []NameInfo { @@ -3400,7 +3400,7 @@ func (m *TypeDescriptor_EnumMember) Reset() { *m = TypeDescriptor_EnumMe func (m *TypeDescriptor_EnumMember) String() string { return proto.CompactTextString(m) } func (*TypeDescriptor_EnumMember) ProtoMessage() {} func (*TypeDescriptor_EnumMember) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_c6d9278eb5f43f16, []int{17, 0} + return fileDescriptor_structured_bf44202de9726669, []int{17, 0} } func (m *TypeDescriptor_EnumMember) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3436,7 +3436,7 @@ func (m *TypeDescriptor_RegionConfig) Reset() { *m = TypeDescriptor_Regi func (m *TypeDescriptor_RegionConfig) String() string { return proto.CompactTextString(m) } func (*TypeDescriptor_RegionConfig) ProtoMessage() {} func (*TypeDescriptor_RegionConfig) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_c6d9278eb5f43f16, []int{17, 1} + return fileDescriptor_structured_bf44202de9726669, []int{17, 1} } func (m *TypeDescriptor_RegionConfig) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3471,9 +3471,9 @@ type SchemaDescriptor struct { State DescriptorState `protobuf:"varint,8,opt,name=state,enum=cockroach.sql.sqlbase.DescriptorState" json:"state"` OfflineReason string `protobuf:"bytes,9,opt,name=offline_reason,json=offlineReason" json:"offline_reason"` // Last modification time of the descriptor. - ModificationTime hlc.Timestamp `protobuf:"bytes,5,opt,name=modification_time,json=modificationTime" json:"modification_time"` - Version DescriptorVersion `protobuf:"varint,6,opt,name=version,casttype=DescriptorVersion" json:"version"` - DrainingNames []NameInfo `protobuf:"bytes,7,rep,name=draining_names,json=drainingNames" json:"draining_names"` + ModificationTime github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp `protobuf:"bytes,5,opt,name=modification_time,json=modificationTime,casttype=github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp" json:"modification_time"` + Version DescriptorVersion `protobuf:"varint,6,opt,name=version,casttype=DescriptorVersion" json:"version"` + DrainingNames []NameInfo `protobuf:"bytes,7,rep,name=draining_names,json=drainingNames" json:"draining_names"` // parent_id refers to the database the schema is in. ParentID ID `protobuf:"varint,1,opt,name=parent_id,json=parentId,casttype=ID" json:"parent_id"` // privileges contains the privileges for the schema. @@ -3484,7 +3484,7 @@ func (m *SchemaDescriptor) Reset() { *m = SchemaDescriptor{} } func (m *SchemaDescriptor) String() string { return proto.CompactTextString(m) } func (*SchemaDescriptor) ProtoMessage() {} func (*SchemaDescriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_c6d9278eb5f43f16, []int{18} + return fileDescriptor_structured_bf44202de9726669, []int{18} } func (m *SchemaDescriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3537,11 +3537,11 @@ func (m *SchemaDescriptor) GetOfflineReason() string { return "" } -func (m *SchemaDescriptor) GetModificationTime() hlc.Timestamp { +func (m *SchemaDescriptor) GetModificationTime() github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp { if m != nil { return m.ModificationTime } - return hlc.Timestamp{} + return github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp{} } func (m *SchemaDescriptor) GetVersion() DescriptorVersion { @@ -3587,7 +3587,7 @@ func (m *Descriptor) Reset() { *m = Descriptor{} } func (m *Descriptor) String() string { return proto.CompactTextString(m) } func (*Descriptor) ProtoMessage() {} func (*Descriptor) Descriptor() ([]byte, []int) { - return fileDescriptor_structured_c6d9278eb5f43f16, []int{19} + return fileDescriptor_structured_bf44202de9726669, []int{19} } func (m *Descriptor) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -17930,327 +17930,330 @@ var ( ) func init() { - proto.RegisterFile("sql/catalog/descpb/structured.proto", fileDescriptor_structured_c6d9278eb5f43f16) -} - -var fileDescriptor_structured_c6d9278eb5f43f16 = []byte{ - // 5087 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x3c, 0x4b, 0x70, 0x1b, 0xe7, - 0x79, 0xc4, 0x1b, 0xf8, 0xf0, 0x5a, 0xfe, 0xa2, 0x24, 0x98, 0xb1, 0x49, 0x0a, 0xb2, 0x6c, 0xda, - 0x8e, 0x49, 0x99, 0xca, 0x43, 0x89, 0x93, 0x8c, 0x41, 0x00, 0x14, 0xc1, 0x07, 0x40, 0x2f, 0x49, - 0xc9, 0x49, 0xda, 0x6c, 0x96, 0xd8, 0x1f, 0xe0, 0x5a, 0x8b, 0x5d, 0x68, 0x77, 0x21, 0x11, 0x9d, - 0x1e, 0x3a, 0xe9, 0xa5, 0xa7, 0xb6, 0x39, 0xf4, 0xd8, 0x69, 0xa6, 0x93, 0x99, 0xe6, 0xd6, 0xc9, - 0xa5, 0xbd, 0xf5, 0xd0, 0x43, 0x27, 0xb7, 0xa6, 0xb7, 0x9c, 0x38, 0x2d, 0x7d, 0xe9, 0xad, 0xbd, - 0x65, 0xc6, 0xa7, 0xce, 0xff, 0xda, 0x07, 0x1e, 0x14, 0x48, 0xba, 0x39, 0xd8, 0xc3, 0xfd, 0x5e, - 0xff, 0xeb, 0x7b, 0xff, 0x3f, 0x04, 0xf7, 0x9d, 0x17, 0xc6, 0x7a, 0x5b, 0x75, 0x55, 0xc3, 0xea, - 0xae, 0x6b, 0xd8, 0x69, 0xf7, 0x4f, 0xd6, 0x1d, 0xd7, 0x1e, 0xb4, 0xdd, 0x81, 0x8d, 0xb5, 0xb5, - 0xbe, 0x6d, 0xb9, 0x16, 0xba, 0xdd, 0xb6, 0xda, 0xcf, 0x6d, 0x4b, 0x6d, 0x9f, 0xae, 0x39, 0x2f, - 0x0c, 0xf2, 0xdf, 0x89, 0xea, 0xe0, 0xc5, 0xd2, 0xc0, 0xd5, 0x8d, 0xf5, 0x53, 0xa3, 0xbd, 0xee, - 0xea, 0x3d, 0xec, 0xb8, 0x6a, 0xaf, 0xcf, 0x18, 0x16, 0xcb, 0x13, 0xa4, 0xf6, 0x6d, 0xfd, 0xa5, - 0x6e, 0xe0, 0x2e, 0xe6, 0x34, 0xb7, 0x09, 0x8d, 0x3b, 0xec, 0x63, 0x87, 0xfd, 0x9f, 0x83, 0xdf, - 0xe8, 0x62, 0x6b, 0xbd, 0x8b, 0x2d, 0xdd, 0xd4, 0xf0, 0xd9, 0x7a, 0xdb, 0x32, 0x3b, 0x7a, 0x97, - 0xa3, 0x16, 0xba, 0x56, 0xd7, 0xa2, 0x7f, 0xae, 0x93, 0xbf, 0x18, 0xb4, 0xfc, 0xb3, 0x04, 0xdc, - 0xda, 0xb2, 0x6c, 0xac, 0x77, 0xcd, 0x5d, 0x3c, 0x94, 0x71, 0x07, 0xdb, 0xd8, 0x6c, 0x63, 0xb4, - 0x02, 0x09, 0x57, 0x3d, 0x31, 0x70, 0x29, 0xb2, 0x12, 0x59, 0xcd, 0x6f, 0xc2, 0x6f, 0xce, 0x97, - 0xe7, 0xbe, 0x3c, 0x5f, 0x8e, 0x36, 0x6a, 0x32, 0x43, 0xa0, 0x07, 0x90, 0xa0, 0xa3, 0x94, 0xa2, - 0x94, 0xa2, 0xc8, 0x29, 0x52, 0x0d, 0x02, 0x24, 0x64, 0x14, 0x8b, 0x4a, 0x10, 0x37, 0xd5, 0x1e, - 0x2e, 0xc5, 0x56, 0x22, 0xab, 0x99, 0xcd, 0x38, 0xa1, 0x92, 0x29, 0x04, 0xed, 0x42, 0xfa, 0xa5, - 0x6a, 0xe8, 0x9a, 0xee, 0x0e, 0x4b, 0xf1, 0x95, 0xc8, 0x6a, 0x61, 0xe3, 0xbd, 0xb5, 0x89, 0x5b, - 0xb5, 0x56, 0xb5, 0x4c, 0xc7, 0xb5, 0x55, 0xdd, 0x74, 0x9f, 0x72, 0x06, 0x2e, 0xc8, 0x13, 0x80, - 0x1e, 0xc2, 0xbc, 0x73, 0xaa, 0xda, 0x58, 0x53, 0xfa, 0x36, 0xee, 0xe8, 0x67, 0x8a, 0x81, 0xcd, - 0x52, 0x62, 0x25, 0xb2, 0x9a, 0xe0, 0xa4, 0x45, 0x86, 0x3e, 0xa0, 0xd8, 0x3d, 0x6c, 0xa2, 0x23, - 0xc8, 0x58, 0xa6, 0xa2, 0x61, 0x03, 0xbb, 0xb8, 0x94, 0xa4, 0xe3, 0x7f, 0x34, 0x65, 0xfc, 0x09, - 0x1b, 0xb4, 0x56, 0x69, 0xbb, 0xba, 0x65, 0x8a, 0x79, 0x58, 0x66, 0x8d, 0x0a, 0xe2, 0x52, 0x07, - 0x7d, 0x4d, 0x75, 0x71, 0x29, 0x75, 0x63, 0xa9, 0xc7, 0x54, 0x10, 0xda, 0x83, 0x44, 0x4f, 0x75, - 0xdb, 0xa7, 0xa5, 0x34, 0x95, 0xf8, 0xf0, 0x0a, 0x12, 0xf7, 0x09, 0x1f, 0x17, 0xc8, 0x84, 0x94, - 0x9f, 0x41, 0x92, 0x8d, 0x83, 0xf2, 0x90, 0x69, 0xb6, 0x94, 0x4a, 0xf5, 0xa8, 0xd1, 0x6a, 0x4a, - 0x73, 0x28, 0x07, 0x69, 0xb9, 0x7e, 0x78, 0x24, 0x37, 0xaa, 0x47, 0x52, 0x84, 0x7c, 0x1d, 0xd6, - 0x8f, 0x94, 0xe6, 0xf1, 0xde, 0x9e, 0x14, 0x45, 0x45, 0xc8, 0x92, 0xaf, 0x5a, 0x7d, 0xab, 0x72, - 0xbc, 0x77, 0x24, 0xc5, 0x50, 0x16, 0x52, 0xd5, 0xca, 0x61, 0xb5, 0x52, 0xab, 0x4b, 0xf1, 0xc5, - 0xf8, 0xaf, 0x7e, 0xb9, 0x34, 0x57, 0x7e, 0x08, 0x09, 0x3a, 0x1c, 0x02, 0x48, 0x1e, 0x36, 0xf6, - 0x0f, 0xf6, 0xea, 0xd2, 0x1c, 0x4a, 0x43, 0x7c, 0x8b, 0x88, 0x88, 0x10, 0x8e, 0x83, 0x8a, 0x7c, - 0xd4, 0xa8, 0xec, 0x49, 0x51, 0xc6, 0xf1, 0xdd, 0xf8, 0x7f, 0xff, 0x62, 0x39, 0x52, 0xfe, 0x8f, - 0x04, 0x2c, 0xf8, 0x73, 0xf7, 0x4f, 0x1b, 0x55, 0xa1, 0x68, 0xd9, 0x7a, 0x57, 0x37, 0x15, 0xaa, - 0x73, 0x8a, 0xae, 0x71, 0x7d, 0xfc, 0x1a, 0x59, 0xcf, 0xc5, 0xf9, 0x72, 0xbe, 0x45, 0xd1, 0x47, - 0x04, 0xdb, 0xa8, 0x71, 0x05, 0xcd, 0x5b, 0x01, 0xa0, 0x86, 0x76, 0x61, 0x9e, 0x0b, 0x69, 0x5b, - 0xc6, 0xa0, 0x67, 0x2a, 0xba, 0xe6, 0x94, 0xa2, 0x2b, 0xb1, 0xd5, 0xfc, 0xe6, 0xf2, 0xc5, 0xf9, - 0x72, 0x91, 0x89, 0xa8, 0x52, 0x5c, 0xa3, 0xe6, 0x7c, 0x79, 0xbe, 0x9c, 0x16, 0x1f, 0x32, 0x1f, - 0x9e, 0x7f, 0x6b, 0x0e, 0x7a, 0x06, 0xb7, 0x6d, 0xb1, 0xb7, 0x5a, 0x50, 0x60, 0x8c, 0x0a, 0xbc, - 0x7f, 0x71, 0xbe, 0x7c, 0xcb, 0xdb, 0x7c, 0x6d, 0xb2, 0xd0, 0x5b, 0xf6, 0x28, 0x81, 0xe6, 0xa0, - 0x16, 0x04, 0xc0, 0xfe, 0x72, 0xe3, 0x74, 0xb9, 0xcb, 0x7c, 0xb9, 0xf3, 0xbe, 0xe8, 0xf0, 0x92, - 0xe7, 0xed, 0x11, 0x84, 0xe6, 0x19, 0x5e, 0xe2, 0x52, 0xc3, 0x4b, 0xde, 0xd4, 0xf0, 0x42, 0x66, - 0x94, 0xfa, 0x7f, 0x31, 0xa3, 0xf4, 0x57, 0x6e, 0x46, 0x99, 0xaf, 0xc0, 0x8c, 0x98, 0xee, 0xee, - 0xc4, 0xd3, 0x20, 0x65, 0x77, 0xe2, 0xe9, 0xac, 0x94, 0xdb, 0x89, 0xa7, 0x73, 0x52, 0x7e, 0x27, - 0x9e, 0xce, 0x4b, 0x85, 0xf2, 0xef, 0x23, 0xf0, 0xe6, 0xb1, 0xa9, 0xbf, 0x18, 0xe0, 0x67, 0xba, - 0x7b, 0x6a, 0x0d, 0x5c, 0xea, 0x17, 0x03, 0xba, 0xfd, 0x10, 0xd2, 0x23, 0x4a, 0x7d, 0x9b, 0x9f, - 0x72, 0x2a, 0x7c, 0xb6, 0x29, 0x97, 0x9f, 0xe8, 0x63, 0x80, 0x31, 0x0d, 0x7e, 0xe3, 0xe2, 0x7c, - 0x39, 0x33, 0x59, 0xcd, 0x32, 0x6d, 0x4f, 0xb9, 0xfe, 0x30, 0x4e, 0x98, 0x5b, 0xf3, 0xcf, 0x13, - 0x20, 0xb1, 0x49, 0xd4, 0xb0, 0xd3, 0xb6, 0xf5, 0xbe, 0x6b, 0xd9, 0xde, 0x0c, 0x22, 0x63, 0x33, - 0x78, 0x07, 0xa2, 0xba, 0xc6, 0x83, 0xc8, 0x1d, 0xbe, 0x03, 0x51, 0xba, 0x78, 0x7f, 0x29, 0x51, - 0x5d, 0x43, 0x6b, 0x10, 0x27, 0x91, 0x8e, 0xae, 0x21, 0xbb, 0xb1, 0x38, 0x3a, 0x4b, 0xdc, 0x5b, - 0x63, 0x81, 0xf0, 0x48, 0xa6, 0x74, 0x68, 0x05, 0xd2, 0xe6, 0xc0, 0x30, 0x68, 0x10, 0x23, 0x2b, - 0x4b, 0x8b, 0xe9, 0x0a, 0x28, 0xba, 0x07, 0x39, 0x0d, 0x77, 0xd4, 0x81, 0xe1, 0x2a, 0xf8, 0xac, - 0x6f, 0x33, 0x4b, 0x91, 0xb3, 0x1c, 0x56, 0x3f, 0xeb, 0xdb, 0xe8, 0x4d, 0x48, 0x9e, 0xea, 0x9a, - 0x86, 0x4d, 0x6a, 0x28, 0x42, 0x04, 0x87, 0xa1, 0x0d, 0x98, 0x1f, 0x38, 0xd8, 0x51, 0x1c, 0xfc, - 0x62, 0x40, 0xb4, 0x84, 0x9e, 0x0b, 0xd0, 0x73, 0x49, 0xf2, 0xc3, 0x2b, 0x12, 0x82, 0x43, 0x8e, - 0x27, 0x47, 0x71, 0x0f, 0x72, 0x6d, 0xab, 0xd7, 0x1f, 0xb8, 0x98, 0x0d, 0x9a, 0x65, 0x83, 0x72, - 0x18, 0x1d, 0x74, 0x03, 0xe6, 0xad, 0x57, 0xe6, 0x88, 0xd8, 0x5c, 0x58, 0x2c, 0x21, 0x08, 0x8a, - 0xfd, 0x04, 0xa4, 0x7e, 0x57, 0x51, 0x5d, 0xd7, 0xd6, 0x4f, 0x88, 0x6c, 0x73, 0xd0, 0x2b, 0xe5, - 0x43, 0x7b, 0x5a, 0x38, 0x78, 0x52, 0x11, 0xe8, 0xe6, 0xa0, 0x27, 0x17, 0xfa, 0xdd, 0xe0, 0x37, - 0xda, 0x82, 0xb7, 0x54, 0xc3, 0xc5, 0xb6, 0x70, 0x6a, 0x64, 0x13, 0x15, 0xdd, 0x54, 0xfa, 0xb6, - 0xd5, 0xb5, 0xb1, 0xe3, 0x94, 0x0a, 0x81, 0x1d, 0x78, 0x83, 0x92, 0xb2, 0xf3, 0x39, 0x1a, 0xf6, - 0x71, 0xc3, 0x3c, 0xe0, 0x64, 0xe8, 0xc7, 0x80, 0x9c, 0xa1, 0xe3, 0xe2, 0x9e, 0x10, 0xf4, 0x5c, - 0x37, 0xb5, 0x52, 0x91, 0xea, 0xd6, 0xbb, 0x53, 0x74, 0xeb, 0x90, 0x32, 0x30, 0x71, 0xbb, 0xba, - 0xa9, 0xf1, 0x51, 0x24, 0x67, 0x04, 0x8e, 0x96, 0x20, 0xf5, 0x52, 0xb7, 0xdd, 0x81, 0x6a, 0x94, - 0xa4, 0xc0, 0x74, 0x04, 0xd0, 0xb3, 0xc9, 0xb4, 0x94, 0xd9, 0x89, 0xa7, 0x33, 0x12, 0xec, 0xc4, - 0xd3, 0x29, 0x29, 0x5d, 0xfe, 0xcb, 0x28, 0xdc, 0x61, 0x62, 0xb6, 0xd4, 0x9e, 0x6e, 0x0c, 0x6f, - 0xaa, 0x99, 0x4c, 0x0a, 0xd7, 0x4c, 0x7a, 0xa4, 0x74, 0xa9, 0x84, 0x8d, 0x85, 0x02, 0x7a, 0xa4, - 0x04, 0xd6, 0x24, 0xa0, 0x11, 0xd3, 0x8d, 0x5f, 0xc1, 0x74, 0x5b, 0x30, 0x2f, 0x94, 0xd4, 0x93, - 0x40, 0x35, 0x35, 0xbf, 0x79, 0x9f, 0xcf, 0xa9, 0x58, 0x63, 0x04, 0x82, 0x3d, 0x1c, 0xc1, 0xb4, - 0x10, 0x52, 0xe3, 0x46, 0xfa, 0xcf, 0x51, 0x58, 0x68, 0x98, 0x2e, 0xb6, 0x0d, 0xac, 0xbe, 0xc4, - 0x81, 0xed, 0xf8, 0x0c, 0x32, 0xaa, 0xd9, 0xc6, 0x8e, 0x6b, 0xd9, 0x4e, 0x29, 0xb2, 0x12, 0x5b, - 0xcd, 0x6e, 0x7c, 0x63, 0xca, 0xa9, 0x4d, 0xe2, 0x5f, 0xab, 0x70, 0x66, 0xbe, 0x93, 0xbe, 0xb0, - 0xc5, 0x7f, 0x89, 0x40, 0x5a, 0x60, 0xaf, 0xe1, 0xfd, 0xbe, 0x09, 0x69, 0x9a, 0x51, 0x2a, 0xde, - 0x99, 0x2c, 0x0a, 0x0e, 0x9e, 0x72, 0x06, 0xb3, 0xcf, 0x14, 0xa5, 0x6d, 0x68, 0xa8, 0x3a, 0x29, - 0x31, 0x8c, 0x51, 0xfe, 0xbb, 0x62, 0xff, 0x0e, 0xc3, 0xa9, 0xe1, 0x58, 0xae, 0xc8, 0xf6, 0x8c, - 0xef, 0xdc, 0x3f, 0x45, 0x60, 0x9e, 0x30, 0x68, 0x58, 0x0b, 0x6c, 0xdb, 0x7d, 0x00, 0xdd, 0x51, - 0x1c, 0x06, 0xa7, 0x2b, 0x12, 0xba, 0x99, 0xd1, 0x1d, 0x4e, 0xee, 0xa9, 0x5a, 0x74, 0x4c, 0xd5, - 0xbe, 0x03, 0x79, 0xca, 0xab, 0x9c, 0x0c, 0xda, 0xcf, 0xb1, 0xeb, 0xd0, 0x19, 0x26, 0x36, 0x17, - 0xf8, 0x0c, 0x73, 0x54, 0xc2, 0x26, 0xc3, 0xc9, 0x39, 0x27, 0xf0, 0x35, 0xa6, 0x7d, 0xf1, 0x31, - 0xed, 0xe3, 0x13, 0xff, 0x7d, 0x0c, 0xee, 0x1c, 0xa8, 0xb6, 0xab, 0x93, 0xd8, 0xa8, 0x9b, 0xdd, - 0xc0, 0xec, 0x1f, 0x40, 0xd6, 0x1c, 0x08, 0x83, 0x75, 0xf8, 0x81, 0xb0, 0xf9, 0x81, 0x39, 0xe0, - 0x06, 0xe8, 0xa0, 0x3d, 0x88, 0x1b, 0xba, 0xe3, 0xd2, 0xd0, 0x93, 0xdd, 0xd8, 0x98, 0xa2, 0x16, - 0x93, 0xc7, 0x58, 0xdb, 0xd3, 0x1d, 0x57, 0xac, 0x99, 0x48, 0x41, 0x2d, 0x48, 0xd8, 0xaa, 0xd9, - 0xc5, 0xd4, 0x5e, 0xb2, 0x1b, 0x8f, 0xae, 0x26, 0x4e, 0x26, 0xac, 0x22, 0x20, 0x53, 0x39, 0x8b, - 0x7f, 0x1b, 0x81, 0x38, 0x19, 0xe5, 0x12, 0x93, 0xbe, 0x03, 0xc9, 0x97, 0xaa, 0x31, 0xc0, 0x2c, - 0x7c, 0xe6, 0x64, 0xfe, 0x85, 0xfe, 0x18, 0x8a, 0xce, 0xe0, 0xa4, 0x1f, 0x18, 0x8a, 0xc7, 0x99, - 0x0f, 0xaf, 0x34, 0x2b, 0xaf, 0xd6, 0x08, 0xcb, 0x62, 0x07, 0xb0, 0xf8, 0x02, 0x12, 0x74, 0xd6, - 0x97, 0xcc, 0xef, 0x1e, 0xe4, 0x5c, 0x4b, 0xc1, 0x67, 0x6d, 0x63, 0xe0, 0xe8, 0x2f, 0x99, 0xa6, - 0xe4, 0xe4, 0xac, 0x6b, 0xd5, 0x05, 0x08, 0x3d, 0x80, 0x42, 0xc7, 0xb6, 0x7a, 0x8a, 0x6e, 0x0a, - 0xa2, 0x18, 0x25, 0xca, 0x13, 0x68, 0x43, 0x00, 0x43, 0x2a, 0xfb, 0x37, 0x39, 0x28, 0x52, 0xc3, - 0x98, 0xc9, 0xed, 0x3d, 0x08, 0xb8, 0xbd, 0xdb, 0x21, 0xb7, 0xe7, 0x59, 0x17, 0xf1, 0x7a, 0x6f, - 0x42, 0x72, 0x40, 0xf3, 0x1b, 0x3a, 0xbe, 0x17, 0x1a, 0x19, 0x6c, 0x06, 0xad, 0x44, 0x5f, 0x07, - 0x44, 0x5c, 0x01, 0x56, 0x42, 0x84, 0x09, 0x4a, 0x28, 0x51, 0x4c, 0x75, 0xaa, 0x07, 0x4d, 0x5e, - 0xc1, 0x83, 0x6e, 0x83, 0x84, 0xcf, 0x5c, 0x5b, 0x0d, 0x66, 0xeb, 0x29, 0xca, 0xbf, 0x44, 0xc2, - 0x62, 0x9d, 0xe0, 0x26, 0x0b, 0x29, 0xe0, 0x00, 0x4e, 0x23, 0x5a, 0x32, 0xcf, 0x65, 0x68, 0xba, - 0x8d, 0x69, 0x8e, 0xe9, 0x94, 0xd2, 0x2b, 0xb1, 0x4b, 0x72, 0xc9, 0x91, 0x6d, 0x5f, 0xab, 0x09, - 0x46, 0x59, 0x62, 0xa2, 0x3c, 0x80, 0x83, 0x0e, 0x21, 0xdb, 0x61, 0xa9, 0xa7, 0xf2, 0x1c, 0x0f, - 0x69, 0x92, 0x9a, 0xdd, 0x78, 0x7f, 0xf6, 0x24, 0x75, 0x33, 0x49, 0x8e, 0xa0, 0x14, 0x91, 0xa1, - 0xe3, 0x21, 0xd1, 0x33, 0xc8, 0x07, 0xea, 0x8a, 0x93, 0x21, 0xcd, 0x4f, 0xae, 0x27, 0x36, 0xe7, - 0x0b, 0xda, 0x1c, 0xa2, 0x4f, 0x01, 0x74, 0x2f, 0x00, 0xd0, 0x34, 0x26, 0xbb, 0xf1, 0xc1, 0x15, - 0x22, 0x85, 0xf0, 0x2f, 0xbe, 0x10, 0xf4, 0x0c, 0x0a, 0xfe, 0x17, 0x9d, 0x6c, 0xee, 0xca, 0x93, - 0x65, 0x52, 0xf3, 0x01, 0x39, 0x9b, 0xa4, 0x48, 0x59, 0x20, 0x09, 0x96, 0xe5, 0xe8, 0x2e, 0x0e, - 0xaa, 0x41, 0x9e, 0xaa, 0x41, 0xf9, 0xe2, 0x7c, 0x19, 0x55, 0x05, 0x7e, 0xb2, 0x2a, 0xa0, 0xf6, - 0x08, 0x9e, 0x29, 0x56, 0x48, 0x81, 0x89, 0xc4, 0x82, 0xaf, 0x58, 0x87, 0xbe, 0x0a, 0x8f, 0x29, - 0x56, 0x40, 0xbd, 0x59, 0x55, 0x99, 0x0b, 0xf9, 0x9e, 0xe2, 0xf5, 0x7d, 0x4f, 0x48, 0x10, 0xaa, - 0xf3, 0xa4, 0x59, 0xa2, 0xe9, 0xd7, 0x07, 0x33, 0x2a, 0x29, 0xc9, 0xe8, 0x84, 0x4b, 0xa0, 0xb9, - 0xf4, 0x23, 0x40, 0x6d, 0x1b, 0xab, 0x2e, 0xd6, 0x48, 0xd2, 0x6a, 0xe8, 0x6d, 0xdd, 0x35, 0x86, - 0xa5, 0xf9, 0x80, 0xdd, 0xcf, 0x73, 0x7c, 0xdd, 0x43, 0xa3, 0xc7, 0x90, 0x7a, 0x89, 0x6d, 0x47, - 0xb7, 0xcc, 0x12, 0xa2, 0xce, 0x64, 0x89, 0xb7, 0x88, 0xee, 0x8c, 0x8c, 0xf7, 0x94, 0x51, 0xc9, - 0x82, 0x1c, 0x6d, 0x43, 0x1e, 0x9b, 0x6d, 0x4b, 0xd3, 0xcd, 0x2e, 0x4d, 0x43, 0x4b, 0xb7, 0xfc, - 0x7c, 0xe7, 0xcb, 0xf3, 0xe5, 0xaf, 0x8d, 0xf0, 0xd7, 0x39, 0x2d, 0x99, 0xb6, 0x9c, 0xc3, 0x81, - 0x2f, 0xb4, 0x0d, 0x29, 0x11, 0x93, 0x17, 0xe8, 0x9e, 0xae, 0x4e, 0xcb, 0x40, 0x47, 0x23, 0xba, - 0xc8, 0x2c, 0x39, 0x3b, 0x29, 0x27, 0x34, 0xdd, 0x21, 0xb9, 0x88, 0x56, 0xba, 0x1d, 0x2c, 0x27, - 0x04, 0x14, 0x55, 0x01, 0xba, 0xd8, 0x52, 0x58, 0xd3, 0xad, 0x74, 0x87, 0x0e, 0xb7, 0x14, 0x18, - 0xae, 0x8b, 0xad, 0x35, 0xd1, 0x9a, 0x23, 0xd5, 0x54, 0x47, 0xef, 0x8a, 0x14, 0xa1, 0x8b, 0x2d, - 0x06, 0x40, 0x65, 0xc8, 0xf4, 0x6d, 0xac, 0xe9, 0x6d, 0x52, 0xf8, 0xde, 0x0d, 0xf8, 0x66, 0x1f, - 0x5c, 0x5e, 0x82, 0x8c, 0xe7, 0x35, 0x50, 0x0a, 0x62, 0x95, 0xc3, 0x2a, 0xeb, 0xb3, 0xd4, 0xea, - 0x87, 0x55, 0x29, 0x52, 0xbe, 0x07, 0x71, 0xba, 0xf8, 0x2c, 0xa4, 0xb6, 0x5a, 0xf2, 0xb3, 0x8a, - 0x5c, 0x63, 0xbd, 0x9d, 0x46, 0xf3, 0x69, 0x5d, 0x3e, 0xaa, 0xd7, 0x24, 0x11, 0x17, 0xce, 0xe3, - 0x80, 0xfc, 0xb2, 0xee, 0xc8, 0xe2, 0x65, 0x72, 0x17, 0x8a, 0x6d, 0x0f, 0xca, 0x0e, 0x20, 0xb2, - 0x12, 0x5d, 0x2d, 0x6c, 0x3c, 0x7e, 0x6d, 0x69, 0x28, 0x64, 0x04, 0x41, 0xbe, 0x32, 0x15, 0xda, - 0x21, 0x68, 0x20, 0x1f, 0x8a, 0x8e, 0xc4, 0x20, 0x19, 0x12, 0xed, 0x53, 0xdc, 0x7e, 0xce, 0xa3, - 0xf0, 0xb7, 0xa6, 0x0c, 0x4c, 0x53, 0xc5, 0x80, 0xe2, 0x56, 0x09, 0x8f, 0x3f, 0xb4, 0x48, 0x0f, - 0xa8, 0x28, 0x24, 0x87, 0xdd, 0x6b, 0xfc, 0x52, 0x8f, 0x35, 0xa9, 0x1d, 0x25, 0x3c, 0x56, 0xc0, - 0xbb, 0x3e, 0x86, 0xa2, 0x69, 0xb9, 0x0a, 0x29, 0x29, 0xb9, 0x17, 0xa0, 0x85, 0x62, 0x7e, 0x53, - 0xe2, 0xba, 0xea, 0xdb, 0x7c, 0xde, 0xb4, 0xdc, 0xe6, 0xc0, 0x30, 0x18, 0x00, 0xfd, 0x59, 0x04, - 0x96, 0x59, 0xac, 0x54, 0x5e, 0xb1, 0x06, 0x81, 0xc2, 0xd2, 0x5b, 0x7f, 0x8f, 0x68, 0x3b, 0x65, - 0x7a, 0x62, 0x74, 0x59, 0x77, 0x81, 0x4f, 0xf5, 0xcd, 0xc1, 0x25, 0x34, 0xe5, 0x23, 0x28, 0x84, - 0x8f, 0x09, 0x65, 0x20, 0x51, 0xdd, 0xae, 0x57, 0x77, 0xa5, 0x39, 0x54, 0x84, 0xec, 0x56, 0x4b, - 0xae, 0x37, 0x9e, 0x34, 0x95, 0xdd, 0xfa, 0x0f, 0x59, 0x3b, 0xb0, 0xd9, 0xf2, 0xda, 0x81, 0x25, - 0x58, 0x38, 0x6e, 0x36, 0x3e, 0x3d, 0xae, 0x2b, 0xcf, 0x1a, 0x47, 0xdb, 0xad, 0xe3, 0x23, 0xa5, - 0xd1, 0xac, 0xd5, 0x3f, 0x93, 0x62, 0x5e, 0x09, 0x96, 0x90, 0x92, 0xe5, 0x5f, 0x47, 0xa1, 0x70, - 0x60, 0xeb, 0x3d, 0xd5, 0x1e, 0xee, 0xe2, 0xe1, 0xe1, 0x2b, 0xb5, 0x8f, 0x3e, 0x81, 0x05, 0x13, - 0xbf, 0x52, 0xfa, 0x0c, 0xaa, 0x78, 0x29, 0x7d, 0x64, 0x72, 0x17, 0x79, 0xde, 0xc4, 0xaf, 0xb8, - 0x84, 0x06, 0xcf, 0xe8, 0xbf, 0x0e, 0x59, 0xcb, 0xd0, 0x18, 0x27, 0x16, 0x7d, 0x90, 0x6c, 0x90, - 0x09, 0x2c, 0x43, 0x6b, 0x30, 0x34, 0xa1, 0x26, 0xe3, 0x09, 0xea, 0xd8, 0x04, 0x6a, 0x13, 0xbf, - 0x12, 0xd4, 0x9f, 0xc0, 0x02, 0x91, 0x3d, 0x36, 0xbb, 0xf8, 0x94, 0xd9, 0x59, 0x86, 0x36, 0x32, - 0xbb, 0xef, 0xc0, 0x9d, 0xf1, 0xf5, 0x8d, 0x35, 0xe2, 0x6e, 0x8d, 0x2c, 0x8b, 0xe4, 0x38, 0xdc, - 0x28, 0xff, 0x31, 0x02, 0x34, 0x18, 0x0d, 0x5c, 0xd1, 0x1e, 0xa4, 0xfb, 0xf6, 0x0d, 0xc8, 0x13, - 0xb9, 0x7e, 0x0d, 0x18, 0x99, 0xa2, 0x67, 0x64, 0xb9, 0x22, 0xb2, 0x10, 0x2e, 0xb2, 0x1e, 0x9f, - 0x2b, 0x3a, 0x8d, 0xcb, 0x32, 0xbc, 0x66, 0x24, 0x7a, 0x17, 0x72, 0xba, 0x49, 0x9c, 0x31, 0xef, - 0x51, 0x04, 0xdb, 0x46, 0x59, 0x8e, 0xa9, 0x9f, 0xf5, 0x6d, 0x3e, 0xe3, 0x5f, 0x47, 0xe1, 0xee, - 0xbe, 0xea, 0x62, 0x5b, 0x57, 0x0d, 0xfd, 0x4f, 0xb0, 0xf6, 0x54, 0xc7, 0xaf, 0x64, 0xdc, 0xb1, - 0xb1, 0x73, 0x8a, 0x3e, 0x83, 0xf9, 0xb1, 0xed, 0xa0, 0x53, 0xcf, 0x6e, 0xbc, 0x33, 0x5b, 0x34, - 0x12, 0x39, 0xf5, 0xc8, 0x8e, 0xa1, 0xfd, 0xf0, 0xc1, 0xb2, 0x9a, 0xe4, 0x6a, 0x32, 0x83, 0x27, - 0xff, 0x18, 0x12, 0xaa, 0xa3, 0x58, 0x1d, 0xee, 0x71, 0xde, 0x0a, 0x08, 0x1a, 0xb8, 0xba, 0xb1, - 0x76, 0x6a, 0xb4, 0xd7, 0x8e, 0xc4, 0x45, 0x8d, 0xf0, 0x55, 0xaa, 0xd3, 0xea, 0xa0, 0x0f, 0xa1, - 0xe8, 0x9c, 0x5a, 0x03, 0x43, 0x53, 0x4e, 0xd4, 0xf6, 0xf3, 0x8e, 0x6e, 0x18, 0xa1, 0x7e, 0x53, - 0x81, 0x21, 0x37, 0x39, 0x8e, 0xef, 0xd9, 0x5f, 0xa5, 0x00, 0xf9, 0xf3, 0xd9, 0x1f, 0xb8, 0x2a, - 0xf5, 0xe6, 0x15, 0x48, 0x72, 0x37, 0xc2, 0xf6, 0xe8, 0xdd, 0xa9, 0x1e, 0x37, 0xdc, 0x5f, 0xdb, - 0x9e, 0x93, 0x39, 0x23, 0xfa, 0x41, 0xf0, 0x5e, 0x66, 0xe6, 0x1d, 0xd9, 0x9e, 0x13, 0x17, 0x36, - 0xbb, 0x90, 0x70, 0x5c, 0x12, 0x7d, 0x62, 0x34, 0x67, 0x58, 0x9f, 0xc2, 0x3f, 0x3e, 0xf9, 0xb5, - 0x43, 0xc2, 0x26, 0x7c, 0x2e, 0x95, 0x81, 0x9e, 0x41, 0xc6, 0x4b, 0x95, 0x79, 0x7f, 0xf1, 0xd1, - 0xec, 0x02, 0xbd, 0x28, 0x27, 0x62, 0xa0, 0x27, 0x0b, 0x55, 0x20, 0xdb, 0xe3, 0x64, 0x7e, 0x43, - 0x64, 0x85, 0x57, 0x2b, 0x20, 0x24, 0xd0, 0xaa, 0x25, 0xf0, 0x25, 0x83, 0x60, 0x6a, 0xd0, 0x88, - 0x6e, 0x5b, 0x86, 0x41, 0x0e, 0x8d, 0x7a, 0x5a, 0x2f, 0xa2, 0x0b, 0x28, 0xda, 0x25, 0x35, 0x87, - 0xe7, 0x8d, 0xd3, 0x74, 0x3f, 0xdf, 0x9b, 0x39, 0x06, 0x6e, 0xcf, 0xc9, 0x01, 0x76, 0xd4, 0x82, - 0x42, 0x3f, 0xe4, 0x0a, 0x79, 0x82, 0xff, 0x60, 0x5a, 0x96, 0x17, 0x22, 0xde, 0x9e, 0x93, 0x47, - 0xd8, 0xd1, 0x8f, 0x01, 0xb5, 0xc7, 0xfc, 0x44, 0x09, 0x5e, 0x33, 0xcb, 0x51, 0x86, 0xed, 0x39, - 0x79, 0x82, 0x18, 0xf4, 0x39, 0xdc, 0xed, 0x4d, 0x36, 0x69, 0x9e, 0xea, 0xaf, 0x4d, 0x19, 0x61, - 0x8a, 0x23, 0xd8, 0x9e, 0x93, 0xa7, 0x09, 0x2c, 0x7f, 0x02, 0x09, 0xaa, 0x3a, 0x24, 0x61, 0x39, - 0x6e, 0xee, 0x36, 0x5b, 0xcf, 0x9a, 0x2c, 0x00, 0xd5, 0xea, 0x7b, 0xf5, 0xa3, 0xba, 0xd2, 0x6a, - 0xee, 0x91, 0x00, 0xf4, 0x06, 0xdc, 0xe6, 0x80, 0x4a, 0xb3, 0xa6, 0x3c, 0x93, 0x1b, 0x02, 0x15, - 0x2d, 0xaf, 0x06, 0x33, 0xa2, 0x34, 0xc4, 0x9b, 0xad, 0x66, 0x5d, 0x9a, 0xa3, 0xb9, 0x51, 0xad, - 0x26, 0x45, 0x68, 0x6e, 0x24, 0xb7, 0x0e, 0xa4, 0x28, 0xb3, 0xbe, 0xcd, 0x1c, 0x80, 0xe6, 0xa9, - 0xdb, 0x4e, 0x3c, 0x9d, 0x94, 0x52, 0xe5, 0x7f, 0x88, 0x40, 0x9a, 0xb8, 0xe1, 0x86, 0xd9, 0xb1, - 0xd0, 0x23, 0xc8, 0xf4, 0x55, 0x1b, 0x9b, 0xae, 0xef, 0x69, 0x45, 0x07, 0x30, 0x7d, 0x40, 0x11, - 0x5e, 0x83, 0x2a, 0xcd, 0x08, 0x1b, 0x97, 0xb5, 0x77, 0xb6, 0x40, 0xe2, 0xe2, 0x9c, 0xf6, 0x29, - 0xee, 0xa9, 0x44, 0x2a, 0xeb, 0x41, 0xbd, 0xe9, 0x75, 0x67, 0x29, 0xfe, 0x90, 0xa2, 0x3d, 0xd9, - 0x85, 0x7e, 0x10, 0x2a, 0x7a, 0x77, 0xff, 0xfb, 0x0e, 0x14, 0x47, 0x32, 0x9f, 0x4b, 0xca, 0xf9, - 0x15, 0x5a, 0xce, 0xc7, 0x7c, 0xbf, 0xef, 0x95, 0xf3, 0x51, 0x5e, 0xc9, 0x87, 0x16, 0x1b, 0x9f, - 0x71, 0xb1, 0x8f, 0xfc, 0xec, 0x9e, 0x19, 0xdf, 0x1b, 0x3c, 0xa6, 0xcc, 0x5f, 0x92, 0xd8, 0x1f, - 0xc0, 0x7c, 0xcf, 0xd2, 0xf4, 0x0e, 0xc9, 0x63, 0x89, 0xe5, 0xba, 0x7a, 0x0f, 0xf3, 0x2c, 0x67, - 0x26, 0x87, 0x2b, 0x05, 0xb9, 0x09, 0x12, 0x3d, 0x81, 0x94, 0xe8, 0x5a, 0xa5, 0x69, 0x04, 0x98, - 0xd5, 0x63, 0x8a, 0xfc, 0x9e, 0x73, 0xa3, 0x2d, 0x28, 0x98, 0xf8, 0x2c, 0xd8, 0x64, 0xcd, 0x84, - 0x7c, 0x4a, 0xae, 0x89, 0xcf, 0x26, 0x77, 0x58, 0x73, 0xa6, 0x8f, 0xd1, 0xd0, 0xa7, 0x90, 0x0f, - 0x07, 0x3b, 0xb8, 0x46, 0xb0, 0xcb, 0xf5, 0x83, 0x91, 0x6e, 0x0b, 0x52, 0x22, 0xca, 0x65, 0xaf, - 0x11, 0xe5, 0x04, 0x33, 0xda, 0x24, 0x29, 0xc4, 0x99, 0xeb, 0x67, 0x35, 0x39, 0xbf, 0x2c, 0xbb, - 0x38, 0x5f, 0xce, 0x92, 0x15, 0x4e, 0x68, 0xa5, 0x66, 0x4d, 0x0f, 0xae, 0xa1, 0x1d, 0x00, 0xef, - 0x29, 0x82, 0x43, 0x6f, 0x18, 0xa6, 0x97, 0xe7, 0x07, 0x82, 0xd0, 0x9f, 0x92, 0x1c, 0xe0, 0x46, - 0xfb, 0x90, 0x11, 0xee, 0x98, 0x15, 0xce, 0xd3, 0xfd, 0xd6, 0x78, 0x70, 0x10, 0x21, 0xc1, 0x93, - 0x40, 0x6a, 0x06, 0x03, 0xab, 0x0e, 0xe6, 0xd5, 0xf3, 0xe3, 0x19, 0x6b, 0x06, 0x66, 0x5c, 0xd5, - 0x53, 0xd5, 0xec, 0xe2, 0x3d, 0xc2, 0xbf, 0x19, 0x2d, 0x45, 0x64, 0x26, 0x0a, 0x35, 0x41, 0xa2, - 0x5b, 0x16, 0x8c, 0x35, 0x12, 0xdd, 0xb5, 0xb7, 0x85, 0xe1, 0x92, 0x5d, 0x9b, 0x1a, 0x6f, 0xa8, - 0x4e, 0xed, 0xfb, 0x31, 0xe7, 0x7b, 0x50, 0xe8, 0x58, 0x76, 0x4f, 0x75, 0x15, 0x61, 0x3c, 0xf3, - 0x7e, 0x9f, 0xed, 0xcb, 0xf3, 0xe5, 0xfc, 0x16, 0xc5, 0x0a, 0xc3, 0xc9, 0x77, 0x82, 0x9f, 0x68, - 0x53, 0x84, 0xe6, 0x5b, 0x34, 0x92, 0xbe, 0xf3, 0xda, 0xcd, 0x9a, 0x10, 0x91, 0x9b, 0x90, 0xa4, - 0xe5, 0x90, 0x53, 0x5a, 0xa0, 0x3b, 0x7e, 0xcd, 0xd2, 0x4a, 0xe6, 0x52, 0xd0, 0x1e, 0x14, 0x34, - 0x02, 0x21, 0xb5, 0x3a, 0xeb, 0xe0, 0xdd, 0xa6, 0x72, 0x97, 0xa7, 0xc8, 0x15, 0x2e, 0x56, 0x34, - 0x6a, 0x04, 0x33, 0xeb, 0xf2, 0xb5, 0x20, 0xdd, 0x51, 0x7b, 0xba, 0xa1, 0x63, 0xa7, 0x74, 0x87, - 0xca, 0xf9, 0xf0, 0x52, 0x7b, 0x1e, 0xbd, 0xcd, 0x11, 0x21, 0x5c, 0x08, 0xf1, 0xcc, 0x9a, 0x02, - 0x86, 0xe4, 0xf8, 0xee, 0x8e, 0x9b, 0xb5, 0xb8, 0xcd, 0x09, 0xdd, 0xec, 0x50, 0xb3, 0xe6, 0x5f, - 0x1a, 0xba, 0x0f, 0xf0, 0x52, 0xc7, 0xaf, 0x94, 0x17, 0x03, 0x6c, 0x0f, 0x4b, 0xa5, 0x60, 0x61, - 0x4e, 0xe0, 0x9f, 0x12, 0x30, 0xfa, 0x08, 0x32, 0x1a, 0xee, 0x63, 0x53, 0x73, 0x5a, 0x66, 0xe9, - 0x0d, 0x5a, 0x69, 0xdc, 0xba, 0x38, 0x5f, 0xce, 0xd4, 0x04, 0x90, 0x7b, 0x51, 0x9f, 0x0a, 0x7d, - 0x0e, 0x39, 0xf6, 0x81, 0xb5, 0x96, 0xb9, 0x39, 0x2c, 0x2d, 0xd2, 0x45, 0x3f, 0x9c, 0xf1, 0x50, - 0xfc, 0xb6, 0x97, 0x77, 0x53, 0x50, 0x0b, 0x48, 0x93, 0x43, 0xb2, 0xd1, 0x1f, 0x41, 0x4e, 0xe8, - 0xf1, 0x8e, 0x75, 0xe2, 0x94, 0xbe, 0x76, 0x69, 0x1b, 0x7f, 0x74, 0xac, 0x7d, 0x9f, 0x55, 0x78, - 0xa9, 0xa0, 0x34, 0xf4, 0x19, 0xe4, 0xbd, 0x0b, 0x4b, 0xab, 0xef, 0x3a, 0xa5, 0x37, 0x2f, 0xad, - 0x5e, 0xc7, 0xcc, 0x90, 0xf3, 0xb6, 0xfa, 0xf4, 0x86, 0x23, 0xf0, 0x85, 0xee, 0x41, 0x46, 0xb3, - 0xad, 0x3e, 0x8b, 0x16, 0x6f, 0xad, 0x44, 0x56, 0x63, 0x5e, 0xef, 0xc5, 0xb6, 0xfa, 0x34, 0x0c, - 0x28, 0x50, 0xb0, 0x71, 0xdf, 0x50, 0xdb, 0xb8, 0x47, 0xe2, 0x98, 0xd5, 0x29, 0x2d, 0xd1, 0xd1, - 0x37, 0x66, 0xde, 0x48, 0x8f, 0x59, 0x28, 0x66, 0x40, 0x5e, 0xab, 0x83, 0x8e, 0x01, 0xd4, 0x81, - 0xa6, 0xbb, 0x4a, 0xcf, 0xd2, 0x70, 0x69, 0xf9, 0xd2, 0xf7, 0x03, 0xa3, 0xc2, 0x2b, 0x84, 0x71, - 0xdf, 0xd2, 0xb0, 0x77, 0x27, 0x26, 0x00, 0xe8, 0x23, 0xc8, 0xd2, 0xa5, 0x7d, 0x6e, 0x9d, 0x10, - 0xdd, 0x5c, 0xa1, 0x8b, 0x9b, 0xe7, 0x67, 0x99, 0xa9, 0xd9, 0x56, 0x7f, 0xc7, 0x3a, 0xa1, 0x1a, - 0xc3, 0xff, 0xd4, 0x90, 0x03, 0xb9, 0x6e, 0x5b, 0xf1, 0x1d, 0xe7, 0x3d, 0x7a, 0x8a, 0x1f, 0xcf, - 0x38, 0x97, 0x27, 0xd5, 0x09, 0xae, 0xf4, 0x96, 0x88, 0x00, 0x4f, 0xaa, 0x02, 0xe6, 0xc8, 0xd9, - 0x6e, 0xdb, 0xfb, 0x20, 0x15, 0x21, 0x6b, 0xf0, 0x71, 0x03, 0x28, 0x07, 0x2b, 0x42, 0x86, 0x61, - 0x26, 0xd0, 0x04, 0xde, 0x09, 0x54, 0x68, 0x35, 0xc5, 0xce, 0xec, 0xfe, 0xec, 0x11, 0xbe, 0xc0, - 0xb8, 0x2b, 0x4e, 0xab, 0x43, 0x0f, 0xb6, 0x0d, 0x39, 0x6b, 0xe0, 0x9e, 0x58, 0x03, 0x53, 0x53, - 0x3a, 0xcf, 0x9d, 0xd2, 0xdb, 0x74, 0xb5, 0x57, 0xea, 0xda, 0x78, 0xab, 0x6b, 0x71, 0x41, 0x5b, - 0xbb, 0x8e, 0x9c, 0x15, 0x52, 0xb7, 0x9e, 0x3b, 0xe8, 0xa7, 0x90, 0xd5, 0x4d, 0x7f, 0x8c, 0x07, - 0x57, 0x1f, 0x03, 0x89, 0xca, 0xa3, 0x61, 0x7a, 0x43, 0x00, 0x97, 0x49, 0x46, 0xf8, 0x00, 0x0a, - 0x56, 0xa7, 0x63, 0xe8, 0x26, 0x56, 0x6c, 0xac, 0x3a, 0x96, 0x59, 0x7a, 0x27, 0xb0, 0x83, 0x79, - 0x8e, 0x93, 0x29, 0x0a, 0x95, 0x21, 0xe3, 0xe2, 0x5e, 0xdf, 0xb2, 0x55, 0x7b, 0x58, 0x7a, 0x37, - 0x78, 0x95, 0xe8, 0x81, 0xd1, 0x09, 0x2c, 0x0e, 0x4c, 0x7c, 0xd6, 0xb7, 0x1c, 0xac, 0x29, 0x63, - 0xb9, 0xe5, 0x2a, 0xf5, 0x71, 0x0f, 0xf8, 0xa4, 0xee, 0x1e, 0x0b, 0xca, 0x89, 0x49, 0xe6, 0xdd, - 0xc1, 0x44, 0xb4, 0x86, 0xbe, 0x05, 0x0b, 0xba, 0xa3, 0x04, 0xb3, 0x76, 0x85, 0xf8, 0xba, 0xd2, - 0x7b, 0x81, 0x29, 0x21, 0xdd, 0x19, 0xcd, 0xf8, 0xd1, 0x4f, 0xa0, 0x68, 0x58, 0x6d, 0xd5, 0xd0, - 0xdd, 0xa1, 0xe8, 0x86, 0xbe, 0x4f, 0x35, 0xe0, 0x9b, 0x33, 0x2a, 0xe9, 0x1e, 0xe7, 0x66, 0x3d, - 0x51, 0xb9, 0x60, 0x84, 0xbe, 0xd1, 0xcf, 0x22, 0xb0, 0xf2, 0x9a, 0xd6, 0x99, 0x53, 0xfa, 0xe0, - 0xd2, 0x4b, 0xc5, 0x19, 0x7a, 0x67, 0x6f, 0x5d, 0xd6, 0x3b, 0x73, 0x16, 0x7f, 0x15, 0x81, 0xf9, - 0xb1, 0xd4, 0x01, 0xfd, 0x04, 0x52, 0xa6, 0xa5, 0x05, 0x6e, 0xb5, 0xeb, 0xfc, 0x0c, 0x92, 0x4d, - 0x4b, 0x63, 0x97, 0xda, 0x8f, 0xba, 0xba, 0x7b, 0x3a, 0x38, 0x59, 0x6b, 0x5b, 0xbd, 0x75, 0x6f, - 0x72, 0xda, 0x89, 0xff, 0xf7, 0x7a, 0xff, 0x79, 0x77, 0x9d, 0xfe, 0xd5, 0x3f, 0x59, 0x63, 0x6c, - 0x72, 0x92, 0x48, 0x6d, 0x68, 0xe8, 0x43, 0x28, 0xe2, 0xb3, 0xbe, 0x6e, 0x07, 0xd2, 0xe7, 0x68, - 0xc0, 0x21, 0x16, 0x7c, 0x24, 0xb1, 0x1e, 0x7e, 0xef, 0xf8, 0xeb, 0x28, 0x14, 0x47, 0xc2, 0x37, - 0xa9, 0x17, 0x68, 0x6b, 0x27, 0x54, 0x2f, 0x10, 0xc8, 0x25, 0x55, 0x4c, 0xf0, 0xad, 0x50, 0xec, - 0xa6, 0xef, 0xc6, 0xc2, 0xf7, 0x79, 0x89, 0x2b, 0xdc, 0xe7, 0x7d, 0x07, 0xee, 0xe8, 0x8e, 0x62, - 0x5a, 0xa6, 0x68, 0xbb, 0x7a, 0x15, 0x7a, 0xf0, 0x8d, 0xce, 0x2d, 0xdd, 0x69, 0x5a, 0x26, 0x6b, - 0xb8, 0x7a, 0xab, 0xf6, 0x9f, 0xf3, 0xa4, 0xc6, 0x9f, 0xf3, 0x78, 0x9d, 0xcb, 0xb8, 0x94, 0x58, - 0xfc, 0x65, 0x04, 0x32, 0xc1, 0xd7, 0xb0, 0xd1, 0x70, 0xc7, 0x6d, 0xac, 0x86, 0xba, 0xe6, 0xeb, - 0x84, 0xf0, 0x2e, 0xc4, 0x66, 0xdf, 0x05, 0x7e, 0xb4, 0x7f, 0x0a, 0xd9, 0x40, 0x5c, 0x1e, 0xed, - 0x8a, 0x44, 0xae, 0xd1, 0x15, 0x79, 0x1b, 0x92, 0x3c, 0x18, 0x31, 0xc5, 0xca, 0x73, 0xee, 0x04, - 0x0b, 0x44, 0x89, 0xcf, 0x49, 0x10, 0xe2, 0xa3, 0xff, 0x7b, 0x0c, 0x72, 0xc1, 0xb8, 0x4d, 0x3c, - 0x97, 0x6e, 0xb6, 0x6d, 0x1a, 0x34, 0xe9, 0xe8, 0x31, 0xef, 0x11, 0x84, 0x00, 0x93, 0x68, 0xde, - 0xd3, 0x4d, 0x85, 0x5e, 0xbc, 0x87, 0x94, 0x37, 0xdd, 0xd3, 0xcd, 0xa7, 0x04, 0x4a, 0x49, 0xd4, - 0x33, 0x4e, 0x12, 0x0b, 0x91, 0xa8, 0x67, 0x8c, 0x64, 0x91, 0xa6, 0xc2, 0xb6, 0x4b, 0xeb, 0xd5, - 0x58, 0x20, 0xc5, 0xb5, 0xdd, 0xe0, 0x23, 0xa1, 0xc4, 0x84, 0x47, 0x42, 0xc8, 0x84, 0x82, 0x9f, - 0xa9, 0xbc, 0x32, 0xb1, 0x4d, 0x15, 0x27, 0xbb, 0x51, 0xb9, 0x46, 0xaa, 0xe2, 0x7f, 0x10, 0x41, - 0xc2, 0x9f, 0x3b, 0x41, 0xe0, 0xe2, 0xdf, 0x47, 0x20, 0x1f, 0x22, 0x43, 0x0d, 0x28, 0xd2, 0x81, - 0xc7, 0xda, 0xb9, 0xf7, 0xbc, 0x77, 0xad, 0x04, 0x3d, 0xb1, 0xdc, 0xcc, 0x5b, 0x01, 0x94, 0x86, - 0x3e, 0x81, 0x02, 0x13, 0xe5, 0x3d, 0xa7, 0x09, 0xab, 0x5f, 0x8e, 0x4a, 0x0a, 0xbf, 0xa9, 0xc9, - 0x59, 0x3e, 0x4c, 0x0b, 0xbe, 0x14, 0x58, 0x34, 0x21, 0x1b, 0x48, 0x85, 0x66, 0xd0, 0xfb, 0x6f, - 0x43, 0xdc, 0xf3, 0x42, 0xb3, 0x76, 0x4d, 0x5d, 0xdf, 0x35, 0xfd, 0x22, 0x02, 0x0b, 0x93, 0x52, - 0x92, 0x90, 0x3d, 0x31, 0x45, 0x9a, 0xc9, 0x9e, 0xee, 0x07, 0x53, 0x45, 0xa6, 0x5c, 0xe2, 0xf6, - 0xda, 0x4f, 0x16, 0xdf, 0xf1, 0x54, 0x9c, 0xe9, 0x56, 0x31, 0xa4, 0xe2, 0xa4, 0xf8, 0x0b, 0x2a, - 0xf9, 0xbf, 0xc6, 0xa0, 0x10, 0x0e, 0x48, 0xe8, 0x29, 0x24, 0xbb, 0x86, 0x75, 0xa2, 0x1a, 0xbc, - 0x4b, 0xfb, 0xbd, 0x6b, 0xc5, 0xb5, 0xb5, 0x27, 0x54, 0xc6, 0xf6, 0x9c, 0xcc, 0xa5, 0x21, 0x07, - 0xe6, 0x6d, 0xdc, 0xd5, 0x2d, 0x53, 0x35, 0x94, 0x93, 0x21, 0x3b, 0x51, 0xbe, 0xb3, 0xf5, 0xeb, - 0x0d, 0x21, 0x73, 0x71, 0x9b, 0x43, 0x4a, 0xb8, 0x3d, 0x27, 0x17, 0xed, 0x30, 0x08, 0xf5, 0xa0, - 0x18, 0x1c, 0xd4, 0xb6, 0x5e, 0xf1, 0x16, 0x78, 0xf5, 0xa6, 0x43, 0xca, 0xd6, 0xab, 0x6d, 0x9a, - 0x48, 0x07, 0x00, 0x8b, 0x1f, 0x43, 0x71, 0x64, 0x52, 0xa8, 0x0c, 0x49, 0x46, 0xc3, 0xa3, 0x11, - 0x7c, 0x79, 0xbe, 0x9c, 0x64, 0x44, 0x32, 0xc7, 0xf0, 0xb3, 0xb8, 0x0d, 0xf9, 0x90, 0x78, 0x0e, - 0x2e, 0x40, 0x92, 0xed, 0x65, 0x50, 0x97, 0x37, 0x01, 0xd2, 0x22, 0x71, 0x28, 0xaf, 0x42, 0xc6, - 0xcb, 0xc2, 0x51, 0x0e, 0xd2, 0xb5, 0xc6, 0x61, 0x65, 0x73, 0xaf, 0x5e, 0x93, 0xe6, 0x50, 0x1e, - 0x32, 0x72, 0xbd, 0x52, 0xa3, 0xfd, 0x44, 0x29, 0xf2, 0xdd, 0xf4, 0x5f, 0xfc, 0x62, 0x39, 0xc2, - 0x03, 0x41, 0x52, 0x4a, 0xed, 0xc4, 0xd3, 0x48, 0xba, 0x55, 0xfe, 0x9f, 0x34, 0xa0, 0x9a, 0xea, - 0xaa, 0x64, 0x03, 0xae, 0xd0, 0x75, 0x8b, 0x5e, 0x62, 0x39, 0xe1, 0x4e, 0x4a, 0xec, 0x46, 0x9d, - 0x94, 0x89, 0x7d, 0xb5, 0xf8, 0x4d, 0xfa, 0x6a, 0xd7, 0x6a, 0xef, 0x8d, 0xf7, 0x02, 0x92, 0x37, - 0xe8, 0x05, 0x3c, 0x85, 0x14, 0xcb, 0x68, 0xd9, 0x73, 0x9d, 0xe9, 0xad, 0x8a, 0xf1, 0x83, 0xe1, - 0x4d, 0x1d, 0xa7, 0x6e, 0xba, 0xf6, 0xd0, 0xbb, 0xc9, 0x67, 0x30, 0xbf, 0x8b, 0x92, 0xbe, 0x7e, - 0x17, 0x65, 0x3c, 0x9f, 0xcf, 0x4c, 0xcf, 0xe7, 0x7f, 0x0c, 0xdc, 0x06, 0x44, 0x36, 0x0c, 0x97, - 0x5e, 0x6a, 0x4f, 0x58, 0x0e, 0x53, 0x7a, 0x9e, 0x0e, 0xe7, 0xec, 0xc0, 0xd7, 0xe2, 0x11, 0x00, - 0x4f, 0xd8, 0xcd, 0x8e, 0x35, 0x83, 0xc3, 0x5e, 0x82, 0x14, 0x71, 0x84, 0x7d, 0xcc, 0xb4, 0xd3, - 0x0b, 0x8e, 0x1c, 0xc8, 0x2d, 0xaa, 0x0f, 0xb9, 0xe0, 0x16, 0x22, 0x09, 0x62, 0xcf, 0xf1, 0x90, - 0xe9, 0xb9, 0x4c, 0xfe, 0x44, 0x3b, 0x90, 0xf0, 0x43, 0xf8, 0xf4, 0x37, 0xa2, 0x53, 0xcf, 0x86, - 0x4c, 0x57, 0x66, 0x22, 0xbe, 0x1b, 0x7d, 0x1c, 0x59, 0xfc, 0xf3, 0x28, 0xe4, 0x82, 0xcb, 0x44, - 0x6f, 0x43, 0x8a, 0x2d, 0x94, 0x3d, 0x43, 0x0d, 0xbb, 0x05, 0x81, 0x42, 0x4d, 0xc8, 0x3b, 0x03, - 0xfb, 0xa5, 0xfe, 0x52, 0x35, 0x94, 0xae, 0xa5, 0x1a, 0x74, 0x3a, 0x85, 0x8d, 0xfb, 0xd3, 0x9e, - 0x79, 0x70, 0xda, 0x27, 0x96, 0x6a, 0x88, 0x2e, 0x86, 0x13, 0x80, 0xa1, 0x6f, 0x7a, 0xb7, 0x34, - 0x0a, 0xf7, 0x49, 0xec, 0xf2, 0xb3, 0xc0, 0xd5, 0x5f, 0x4c, 0x40, 0x34, 0x79, 0xd9, 0x27, 0x89, - 0xc2, 0xfc, 0x88, 0xb1, 0x39, 0xe8, 0xf9, 0x7d, 0x74, 0x2f, 0x0a, 0x33, 0xba, 0xba, 0x39, 0xe8, - 0xf9, 0x51, 0xd8, 0xf6, 0x61, 0x22, 0x0a, 0x7b, 0xde, 0xa7, 0xfc, 0x6f, 0x59, 0x28, 0x1c, 0x0d, - 0xfb, 0x41, 0x6f, 0x73, 0xad, 0x4b, 0x89, 0x49, 0x57, 0x0f, 0xd1, 0xab, 0x5f, 0x3d, 0x5c, 0xf2, - 0x13, 0x02, 0xa6, 0x73, 0xf1, 0x4b, 0x74, 0xae, 0x06, 0x71, 0xfa, 0x08, 0x3c, 0x41, 0xcf, 0x66, - 0x9a, 0x93, 0x0b, 0xaf, 0x76, 0x2d, 0xf0, 0x0e, 0x9c, 0x72, 0xa3, 0x1f, 0x42, 0x8e, 0x6e, 0x6e, - 0x0f, 0xf7, 0x4e, 0xb0, 0x2d, 0x7c, 0xcb, 0xc3, 0xd9, 0xa4, 0x91, 0x5d, 0xde, 0xa7, 0x8c, 0xa2, - 0x6b, 0x81, 0x3d, 0x88, 0x83, 0x1e, 0x42, 0x42, 0x35, 0x74, 0xea, 0x68, 0x5e, 0xf7, 0xe3, 0x02, - 0x46, 0x88, 0xbe, 0x0f, 0x79, 0xd5, 0xb6, 0xd5, 0x21, 0x7f, 0x26, 0xaf, 0x51, 0x67, 0xc2, 0xbd, - 0xe4, 0xc5, 0xf9, 0x72, 0xb6, 0x42, 0x90, 0xf4, 0x65, 0xbc, 0xd8, 0x88, 0xac, 0xea, 0x81, 0x42, - 0xb7, 0x27, 0x99, 0x9b, 0xdd, 0x9e, 0xc0, 0x4d, 0xbc, 0xfc, 0xb8, 0xc3, 0xce, 0xde, 0xc0, 0x61, - 0xff, 0x14, 0x16, 0xc5, 0x0b, 0x41, 0x22, 0xd0, 0xbf, 0x61, 0x0b, 0xfc, 0x80, 0xa1, 0x7c, 0x71, - 0xbe, 0x5c, 0x92, 0x7d, 0x2a, 0x7f, 0xb9, 0xac, 0xd6, 0x21, 0x3b, 0x55, 0xb2, 0x27, 0xe2, 0xb5, - 0x80, 0xeb, 0xce, 0x5f, 0xdf, 0x75, 0x87, 0xe3, 0x6e, 0xe1, 0x46, 0x71, 0x77, 0x3c, 0x0c, 0x14, - 0xa7, 0x87, 0x81, 0x67, 0xa3, 0x61, 0x40, 0xba, 0xbc, 0x45, 0x19, 0x56, 0xe0, 0x4b, 0x42, 0xc0, - 0xcf, 0xa3, 0x00, 0xbe, 0x7e, 0xa3, 0x6f, 0xc3, 0xdd, 0xfe, 0xe9, 0xd0, 0xd1, 0xdb, 0xaa, 0xa1, - 0xd8, 0xb8, 0x6f, 0x63, 0x07, 0x9b, 0x2c, 0xab, 0xa6, 0x4e, 0x23, 0x27, 0xdf, 0x11, 0x68, 0x39, - 0x84, 0x45, 0x1f, 0xc3, 0x1d, 0xc3, 0xea, 0x4e, 0xe2, 0x0b, 0xf6, 0x02, 0x6e, 0x73, 0x9a, 0x11, - 0x66, 0x15, 0xa0, 0xad, 0xf6, 0xd5, 0x13, 0xdd, 0xf0, 0xdb, 0x03, 0x1f, 0x5f, 0xd5, 0x36, 0xd7, - 0xaa, 0x9e, 0x08, 0xf1, 0x44, 0xc3, 0x17, 0x5a, 0x7e, 0x1b, 0xc0, 0xc7, 0xd3, 0xcb, 0xdd, 0xbd, - 0x3d, 0x3f, 0x81, 0xe3, 0xd7, 0xc4, 0x3c, 0x80, 0xed, 0x8e, 0x44, 0x93, 0x71, 0xbf, 0x1e, 0x99, - 0xc1, 0xaf, 0xf3, 0x0b, 0xd7, 0x8f, 0x20, 0x4e, 0x7f, 0x7d, 0x92, 0x86, 0x78, 0xbd, 0x79, 0xbc, - 0x2f, 0xcd, 0xa1, 0x0c, 0x24, 0x2a, 0x7b, 0x8d, 0xca, 0xa1, 0x14, 0x41, 0x0b, 0x20, 0xed, 0x1f, - 0xef, 0x1d, 0x35, 0xe4, 0xfa, 0x93, 0x46, 0xab, 0xa9, 0x50, 0x82, 0x68, 0xc0, 0x91, 0xff, 0x5d, - 0x1c, 0x24, 0xe6, 0x45, 0x6f, 0xea, 0xca, 0xa7, 0x77, 0x66, 0x5e, 0x7f, 0xc7, 0x1b, 0xd6, 0xfa, - 0xf8, 0x57, 0x9f, 0x6d, 0x26, 0xbe, 0xa2, 0x6c, 0x33, 0x79, 0x83, 0x6c, 0x33, 0x75, 0x03, 0xe7, - 0xf5, 0x87, 0xce, 0x0a, 0x03, 0x1a, 0xf2, 0xcb, 0x28, 0x40, 0x40, 0x37, 0x7e, 0x10, 0xfc, 0xe9, - 0xf5, 0xf4, 0xcb, 0xde, 0x91, 0x32, 0x6c, 0x7b, 0x4e, 0xfc, 0x30, 0xfb, 0x09, 0xa4, 0x35, 0x9e, - 0x75, 0xf1, 0xe4, 0xec, 0xbd, 0x99, 0x93, 0xb3, 0xed, 0x39, 0xd9, 0x63, 0x46, 0x1f, 0x87, 0x7e, - 0x71, 0xf7, 0x60, 0x26, 0x63, 0xde, 0x16, 0x4f, 0x86, 0x2b, 0x90, 0x64, 0x09, 0x07, 0x57, 0xb6, - 0xa9, 0x3f, 0xfd, 0x1a, 0x31, 0x0d, 0x52, 0x0e, 0x33, 0x46, 0x5e, 0xc6, 0xa5, 0x20, 0x31, 0x30, - 0x75, 0xcb, 0x7c, 0x5f, 0x0e, 0x3e, 0x56, 0x15, 0x7d, 0x45, 0x62, 0xfd, 0xf4, 0x6f, 0xd5, 0xc5, - 0x1a, 0x7b, 0x35, 0x72, 0x6c, 0xbe, 0xf4, 0x00, 0x11, 0x54, 0x00, 0xe0, 0x78, 0xdd, 0xec, 0x4a, - 0x51, 0x5a, 0xfc, 0x91, 0x54, 0x97, 0x7c, 0xc5, 0xde, 0xff, 0x3e, 0x48, 0xa3, 0xbf, 0x3d, 0x0b, - 0xbc, 0x1f, 0x99, 0x87, 0xfc, 0xfe, 0xd3, 0x6a, 0xf5, 0xa8, 0xb1, 0x5f, 0x3f, 0x3c, 0xaa, 0xec, - 0x1f, 0xb0, 0x57, 0x90, 0x47, 0xa4, 0x72, 0x6c, 0x35, 0x6a, 0x52, 0xf4, 0xfd, 0xef, 0x43, 0x71, - 0x44, 0x21, 0x10, 0x40, 0xf2, 0xe0, 0x78, 0x73, 0xaf, 0x51, 0x9d, 0xf8, 0xfe, 0x04, 0x65, 0x21, - 0xd5, 0xda, 0xda, 0xda, 0x6b, 0x34, 0xeb, 0x52, 0xec, 0xfd, 0x6f, 0x40, 0x2e, 0x98, 0x90, 0x22, - 0x09, 0x72, 0x3f, 0x6a, 0x35, 0xeb, 0xca, 0x56, 0xa5, 0xb1, 0x77, 0x2c, 0x93, 0x19, 0x20, 0x28, - 0x70, 0xbf, 0x22, 0x60, 0x91, 0xcd, 0xd5, 0xdf, 0xfc, 0xd7, 0xd2, 0xdc, 0x6f, 0x2e, 0x96, 0x22, - 0xbf, 0xbd, 0x58, 0x8a, 0xfc, 0xee, 0x62, 0x29, 0xf2, 0x9f, 0x17, 0x4b, 0x91, 0xbf, 0xfe, 0x62, - 0x69, 0xee, 0xb7, 0x5f, 0x2c, 0xcd, 0xfd, 0xee, 0x8b, 0xa5, 0xb9, 0x1f, 0x25, 0xd9, 0x3f, 0x1a, - 0xf0, 0x7f, 0x01, 0x00, 0x00, 0xff, 0xff, 0x4a, 0xa9, 0x99, 0xd2, 0x9f, 0x40, 0x00, 0x00, + proto.RegisterFile("sql/catalog/descpb/structured.proto", fileDescriptor_structured_bf44202de9726669) +} + +var fileDescriptor_structured_bf44202de9726669 = []byte{ + // 5128 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xcc, 0x3c, 0xcb, 0x6f, 0x23, 0xe7, + 0x7d, 0xe2, 0x9b, 0xfc, 0xf1, 0x35, 0xfa, 0x56, 0xbb, 0x4b, 0x2b, 0xb6, 0xa4, 0xe5, 0x7a, 0x6d, + 0xd9, 0x8e, 0xa5, 0xb5, 0x9c, 0xa4, 0x4e, 0x9c, 0x04, 0xa6, 0x48, 0x6a, 0x45, 0xad, 0x44, 0xca, + 0x23, 0x6a, 0xd7, 0x49, 0xda, 0x4c, 0x46, 0x9c, 0x8f, 0xd4, 0x78, 0x87, 0x33, 0xf4, 0xcc, 0x70, + 0x57, 0x2c, 0xda, 0xa2, 0x48, 0x2f, 0xbd, 0xf4, 0x11, 0xa0, 0xed, 0xad, 0x40, 0x50, 0x04, 0x68, + 0x6e, 0x45, 0x80, 0x22, 0xbd, 0xf5, 0x50, 0xa0, 0x40, 0x6e, 0x4d, 0x6f, 0x39, 0x09, 0xad, 0x0c, + 0x14, 0xfd, 0x0f, 0x02, 0xf8, 0x54, 0x7c, 0xaf, 0x79, 0xf0, 0x21, 0x53, 0x52, 0xe2, 0xf4, 0x60, + 0x43, 0xf3, 0x7b, 0x7d, 0xaf, 0xdf, 0xfb, 0xfb, 0xb8, 0x70, 0xdf, 0xf9, 0xc4, 0xd8, 0xec, 0xa8, + 0xae, 0x6a, 0x58, 0xbd, 0x4d, 0x0d, 0x3b, 0x9d, 0xc1, 0xc9, 0xa6, 0xe3, 0xda, 0xc3, 0x8e, 0x3b, + 0xb4, 0xb1, 0xb6, 0x31, 0xb0, 0x2d, 0xd7, 0x42, 0xb7, 0x3b, 0x56, 0xe7, 0x99, 0x6d, 0xa9, 0x9d, + 0xd3, 0x0d, 0xe7, 0x13, 0x83, 0xfc, 0x77, 0xa2, 0x3a, 0x78, 0xb9, 0x34, 0x74, 0x75, 0x63, 0xf3, + 0xd4, 0xe8, 0x6c, 0xba, 0x7a, 0x1f, 0x3b, 0xae, 0xda, 0x1f, 0x30, 0x86, 0xe5, 0xf2, 0x14, 0xa9, + 0x03, 0x5b, 0x7f, 0xae, 0x1b, 0xb8, 0x87, 0x39, 0xcd, 0x6d, 0x42, 0xe3, 0x8e, 0x06, 0xd8, 0x61, + 0xff, 0xe7, 0xe0, 0x97, 0x7a, 0xd8, 0xda, 0xec, 0x61, 0x4b, 0x37, 0x35, 0x7c, 0xb6, 0xd9, 0xb1, + 0xcc, 0xae, 0xde, 0xe3, 0xa8, 0xa5, 0x9e, 0xd5, 0xb3, 0xe8, 0x9f, 0x9b, 0xe4, 0x2f, 0x06, 0x2d, + 0xff, 0x30, 0x01, 0xb7, 0x76, 0x2c, 0x1b, 0xeb, 0x3d, 0xf3, 0x31, 0x1e, 0xc9, 0xb8, 0x8b, 0x6d, + 0x6c, 0x76, 0x30, 0x5a, 0x83, 0x84, 0xab, 0x9e, 0x18, 0xb8, 0x14, 0x59, 0x8b, 0xac, 0xe7, 0xb7, + 0xe1, 0x17, 0xe7, 0xab, 0x0b, 0x9f, 0x9d, 0xaf, 0x46, 0x1b, 0x35, 0x99, 0x21, 0xd0, 0x03, 0x48, + 0xd0, 0x51, 0x4a, 0x51, 0x4a, 0x51, 0xe4, 0x14, 0xa9, 0x06, 0x01, 0x12, 0x32, 0x8a, 0x45, 0x25, + 0x88, 0x9b, 0x6a, 0x1f, 0x97, 0x62, 0x6b, 0x91, 0xf5, 0xcc, 0x76, 0x9c, 0x50, 0xc9, 0x14, 0x82, + 0x1e, 0x43, 0xfa, 0xb9, 0x6a, 0xe8, 0x9a, 0xee, 0x8e, 0x4a, 0xf1, 0xb5, 0xc8, 0x7a, 0x61, 0xeb, + 0x8d, 0x8d, 0xa9, 0x5b, 0xb5, 0x51, 0xb5, 0x4c, 0xc7, 0xb5, 0x55, 0xdd, 0x74, 0x9f, 0x70, 0x06, + 0x2e, 0xc8, 0x13, 0x80, 0x1e, 0xc2, 0xa2, 0x73, 0xaa, 0xda, 0x58, 0x53, 0x06, 0x36, 0xee, 0xea, + 0x67, 0x8a, 0x81, 0xcd, 0x52, 0x62, 0x2d, 0xb2, 0x9e, 0xe0, 0xa4, 0x45, 0x86, 0x3e, 0xa4, 0xd8, + 0x7d, 0x6c, 0xa2, 0x36, 0x64, 0x2c, 0x53, 0xd1, 0xb0, 0x81, 0x5d, 0x5c, 0x4a, 0xd2, 0xf1, 0xdf, + 0x99, 0x31, 0xfe, 0x94, 0x0d, 0xda, 0xa8, 0x74, 0x5c, 0xdd, 0x32, 0xc5, 0x3c, 0x2c, 0xb3, 0x46, + 0x05, 0x71, 0xa9, 0xc3, 0x81, 0xa6, 0xba, 0xb8, 0x94, 0xba, 0xb1, 0xd4, 0x63, 0x2a, 0x08, 0xed, + 0x43, 0xa2, 0xaf, 0xba, 0x9d, 0xd3, 0x52, 0x9a, 0x4a, 0x7c, 0x78, 0x05, 0x89, 0x07, 0x84, 0x8f, + 0x0b, 0x64, 0x42, 0xca, 0x4f, 0x21, 0xc9, 0xc6, 0x41, 0x79, 0xc8, 0x34, 0x5b, 0x4a, 0xa5, 0xda, + 0x6e, 0xb4, 0x9a, 0xd2, 0x02, 0xca, 0x41, 0x5a, 0xae, 0x1f, 0xb5, 0xe5, 0x46, 0xb5, 0x2d, 0x45, + 0xc8, 0xd7, 0x51, 0xbd, 0xad, 0x34, 0x8f, 0xf7, 0xf7, 0xa5, 0x28, 0x2a, 0x42, 0x96, 0x7c, 0xd5, + 0xea, 0x3b, 0x95, 0xe3, 0xfd, 0xb6, 0x14, 0x43, 0x59, 0x48, 0x55, 0x2b, 0x47, 0xd5, 0x4a, 0xad, + 0x2e, 0xc5, 0x97, 0xe3, 0x3f, 0xfd, 0xc9, 0xca, 0x42, 0xf9, 0x21, 0x24, 0xe8, 0x70, 0x08, 0x20, + 0x79, 0xd4, 0x38, 0x38, 0xdc, 0xaf, 0x4b, 0x0b, 0x28, 0x0d, 0xf1, 0x1d, 0x22, 0x22, 0x42, 0x38, + 0x0e, 0x2b, 0x72, 0xbb, 0x51, 0xd9, 0x97, 0xa2, 0x8c, 0xe3, 0x1b, 0xf1, 0xff, 0xfd, 0xf1, 0x6a, + 0xa4, 0xfc, 0x9f, 0x09, 0x58, 0xf2, 0xe7, 0xee, 0x9f, 0x36, 0xaa, 0x42, 0xd1, 0xb2, 0xf5, 0x9e, + 0x6e, 0x2a, 0x54, 0xe7, 0x14, 0x5d, 0xe3, 0xfa, 0xf8, 0x25, 0xb2, 0x9e, 0x8b, 0xf3, 0xd5, 0x7c, + 0x8b, 0xa2, 0xdb, 0x04, 0xdb, 0xa8, 0x71, 0x05, 0xcd, 0x5b, 0x01, 0xa0, 0x86, 0x1e, 0xc3, 0x22, + 0x17, 0xd2, 0xb1, 0x8c, 0x61, 0xdf, 0x54, 0x74, 0xcd, 0x29, 0x45, 0xd7, 0x62, 0xeb, 0xf9, 0xed, + 0xd5, 0x8b, 0xf3, 0xd5, 0x22, 0x13, 0x51, 0xa5, 0xb8, 0x46, 0xcd, 0xf9, 0xec, 0x7c, 0x35, 0x2d, + 0x3e, 0x64, 0x3e, 0x3c, 0xff, 0xd6, 0x1c, 0xf4, 0x14, 0x6e, 0xdb, 0x62, 0x6f, 0xb5, 0xa0, 0xc0, + 0x18, 0x15, 0x78, 0xff, 0xe2, 0x7c, 0xf5, 0x96, 0xb7, 0xf9, 0xda, 0x74, 0xa1, 0xb7, 0xec, 0x71, + 0x02, 0xcd, 0x41, 0x2d, 0x08, 0x80, 0xfd, 0xe5, 0xc6, 0xe9, 0x72, 0x57, 0xf9, 0x72, 0x17, 0x7d, + 0xd1, 0xe1, 0x25, 0x2f, 0xda, 0x63, 0x08, 0xcd, 0x33, 0xbc, 0xc4, 0xa5, 0x86, 0x97, 0xbc, 0xa9, + 0xe1, 0x85, 0xcc, 0x28, 0xf5, 0x5b, 0x31, 0xa3, 0xf4, 0x6f, 0xdc, 0x8c, 0x32, 0xbf, 0x01, 0x33, + 0x62, 0xba, 0xbb, 0x17, 0x4f, 0x83, 0x94, 0xdd, 0x8b, 0xa7, 0xb3, 0x52, 0x6e, 0x2f, 0x9e, 0xce, + 0x49, 0xf9, 0xbd, 0x78, 0x3a, 0x2f, 0x15, 0xca, 0xbf, 0x8e, 0xc0, 0xcb, 0xc7, 0xa6, 0xfe, 0xc9, + 0x10, 0x3f, 0xd5, 0xdd, 0x53, 0x6b, 0xe8, 0x52, 0xbf, 0x18, 0xd0, 0xed, 0x87, 0x90, 0x1e, 0x53, + 0xea, 0xdb, 0xfc, 0x94, 0x53, 0xe1, 0xb3, 0x4d, 0xb9, 0xfc, 0x44, 0xdf, 0x03, 0x98, 0xd0, 0xe0, + 0x97, 0x2e, 0xce, 0x57, 0x33, 0xd3, 0xd5, 0x2c, 0xd3, 0xf1, 0x94, 0xeb, 0x8b, 0x71, 0xc2, 0xdc, + 0x9a, 0x7f, 0x94, 0x00, 0x89, 0x4d, 0xa2, 0x86, 0x9d, 0x8e, 0xad, 0x0f, 0x5c, 0xcb, 0xf6, 0x66, + 0x10, 0x99, 0x98, 0xc1, 0x6b, 0x10, 0xd5, 0x35, 0x1e, 0x44, 0xee, 0xf0, 0x1d, 0x88, 0xd2, 0xc5, + 0xfb, 0x4b, 0x89, 0xea, 0x1a, 0xda, 0x80, 0x38, 0x89, 0x74, 0x74, 0x0d, 0xd9, 0xad, 0xe5, 0xf1, + 0x59, 0xe2, 0xfe, 0x06, 0x0b, 0x84, 0x6d, 0x99, 0xd2, 0xa1, 0x35, 0x48, 0x9b, 0x43, 0xc3, 0xa0, + 0x41, 0x8c, 0xac, 0x2c, 0x2d, 0xa6, 0x2b, 0xa0, 0xe8, 0x1e, 0xe4, 0x34, 0xdc, 0x55, 0x87, 0x86, + 0xab, 0xe0, 0xb3, 0x81, 0xcd, 0x2c, 0x45, 0xce, 0x72, 0x58, 0xfd, 0x6c, 0x60, 0xa3, 0x97, 0x21, + 0x79, 0xaa, 0x6b, 0x1a, 0x36, 0xa9, 0xa1, 0x08, 0x11, 0x1c, 0x86, 0xb6, 0x60, 0x71, 0xe8, 0x60, + 0x47, 0x71, 0xf0, 0x27, 0x43, 0xa2, 0x25, 0xf4, 0x5c, 0x80, 0x9e, 0x4b, 0x92, 0x1f, 0x5e, 0x91, + 0x10, 0x1c, 0x71, 0x3c, 0x39, 0x8a, 0x7b, 0x90, 0xeb, 0x58, 0xfd, 0xc1, 0xd0, 0xc5, 0x6c, 0xd0, + 0x2c, 0x1b, 0x94, 0xc3, 0xe8, 0xa0, 0x5b, 0xb0, 0x68, 0xbd, 0x30, 0xc7, 0xc4, 0xe6, 0xc2, 0x62, + 0x09, 0x41, 0x50, 0xec, 0x07, 0x20, 0x0d, 0x7a, 0x8a, 0xea, 0xba, 0xb6, 0x7e, 0x42, 0x64, 0x9b, + 0xc3, 0x7e, 0x29, 0x1f, 0xda, 0xd3, 0xc2, 0xe1, 0xa3, 0x8a, 0x40, 0x37, 0x87, 0x7d, 0xb9, 0x30, + 0xe8, 0x05, 0xbf, 0xd1, 0x0e, 0xbc, 0xa2, 0x1a, 0x2e, 0xb6, 0x85, 0x53, 0x23, 0x9b, 0xa8, 0xe8, + 0xa6, 0x32, 0xb0, 0xad, 0x9e, 0x8d, 0x1d, 0xa7, 0x54, 0x08, 0xec, 0xc0, 0x4b, 0x94, 0x94, 0x9d, + 0x4f, 0x7b, 0x34, 0xc0, 0x0d, 0xf3, 0x90, 0x93, 0xa1, 0xef, 0x01, 0x72, 0x46, 0x8e, 0x8b, 0xfb, + 0x42, 0xd0, 0x33, 0xdd, 0xd4, 0x4a, 0x45, 0xaa, 0x5b, 0xaf, 0xcf, 0xd0, 0xad, 0x23, 0xca, 0xc0, + 0xc4, 0x3d, 0xd6, 0x4d, 0x8d, 0x8f, 0x22, 0x39, 0x63, 0x70, 0xb4, 0x02, 0xa9, 0xe7, 0xba, 0xed, + 0x0e, 0x55, 0xa3, 0x24, 0x05, 0xa6, 0x23, 0x80, 0x9e, 0x4d, 0xa6, 0xa5, 0xcc, 0x5e, 0x3c, 0x9d, + 0x91, 0x60, 0x2f, 0x9e, 0x4e, 0x49, 0xe9, 0xf2, 0x5f, 0x46, 0xe1, 0x0e, 0x13, 0xb3, 0xa3, 0xf6, + 0x75, 0x63, 0x74, 0x53, 0xcd, 0x64, 0x52, 0xb8, 0x66, 0xd2, 0x23, 0xa5, 0x4b, 0x25, 0x6c, 0x2c, + 0x14, 0xd0, 0x23, 0x25, 0xb0, 0x26, 0x01, 0x8d, 0x99, 0x6e, 0xfc, 0x0a, 0xa6, 0xdb, 0x82, 0x45, + 0xa1, 0xa4, 0x9e, 0x04, 0xaa, 0xa9, 0xf9, 0xed, 0xfb, 0x7c, 0x4e, 0xc5, 0x1a, 0x23, 0x10, 0xec, + 0xe1, 0x08, 0xa6, 0x85, 0x90, 0x1a, 0x37, 0xd2, 0x7f, 0x89, 0xc2, 0x52, 0xc3, 0x74, 0xb1, 0x6d, + 0x60, 0xf5, 0x39, 0x0e, 0x6c, 0xc7, 0x47, 0x90, 0x51, 0xcd, 0x0e, 0x76, 0x5c, 0xcb, 0x76, 0x4a, + 0x91, 0xb5, 0xd8, 0x7a, 0x76, 0xeb, 0x2b, 0x33, 0x4e, 0x6d, 0x1a, 0xff, 0x46, 0x85, 0x33, 0xf3, + 0x9d, 0xf4, 0x85, 0x2d, 0xff, 0x6b, 0x04, 0xd2, 0x02, 0x7b, 0x0d, 0xef, 0xf7, 0x55, 0x48, 0xd3, + 0x8c, 0x52, 0xf1, 0xce, 0x64, 0x59, 0x70, 0xf0, 0x94, 0x33, 0x98, 0x7d, 0xa6, 0x28, 0x6d, 0x43, + 0x43, 0xd5, 0x69, 0x89, 0x61, 0x8c, 0xf2, 0xdf, 0x15, 0xfb, 0x77, 0x14, 0x4e, 0x0d, 0x27, 0x72, + 0x45, 0xb6, 0x67, 0x7c, 0xe7, 0x7e, 0x1e, 0x81, 0x45, 0xc2, 0xa0, 0x61, 0x2d, 0xb0, 0x6d, 0xf7, + 0x01, 0x74, 0x47, 0x71, 0x18, 0x9c, 0xae, 0x48, 0xe8, 0x66, 0x46, 0x77, 0x38, 0xb9, 0xa7, 0x6a, + 0xd1, 0x09, 0x55, 0xfb, 0x3a, 0xe4, 0x29, 0xaf, 0x72, 0x32, 0xec, 0x3c, 0xc3, 0xae, 0x43, 0x67, + 0x98, 0xd8, 0x5e, 0xe2, 0x33, 0xcc, 0x51, 0x09, 0xdb, 0x0c, 0x27, 0xe7, 0x9c, 0xc0, 0xd7, 0x84, + 0xf6, 0xc5, 0x27, 0xb4, 0x8f, 0x4f, 0xfc, 0xd7, 0x31, 0xb8, 0x73, 0xa8, 0xda, 0xae, 0x4e, 0x62, + 0xa3, 0x6e, 0xf6, 0x02, 0xb3, 0x7f, 0x00, 0x59, 0x73, 0x28, 0x0c, 0xd6, 0xe1, 0x07, 0xc2, 0xe6, + 0x07, 0xe6, 0x90, 0x1b, 0xa0, 0x83, 0xf6, 0x21, 0x6e, 0xe8, 0x8e, 0x4b, 0x43, 0x4f, 0x76, 0x6b, + 0x6b, 0x86, 0x5a, 0x4c, 0x1f, 0x63, 0x63, 0x5f, 0x77, 0x5c, 0xb1, 0x66, 0x22, 0x05, 0xb5, 0x20, + 0x61, 0xab, 0x66, 0x0f, 0x53, 0x7b, 0xc9, 0x6e, 0xbd, 0x7b, 0x35, 0x71, 0x32, 0x61, 0x15, 0x01, + 0x99, 0xca, 0x59, 0xfe, 0xfb, 0x08, 0xc4, 0xc9, 0x28, 0x97, 0x98, 0xf4, 0x1d, 0x48, 0x3e, 0x57, + 0x8d, 0x21, 0x66, 0xe1, 0x33, 0x27, 0xf3, 0x2f, 0xf4, 0x07, 0x50, 0x74, 0x86, 0x27, 0x83, 0xc0, + 0x50, 0x3c, 0xce, 0xbc, 0x7d, 0xa5, 0x59, 0x79, 0xb5, 0x46, 0x58, 0x16, 0x3b, 0x80, 0xe5, 0x4f, + 0x20, 0x41, 0x67, 0x7d, 0xc9, 0xfc, 0xee, 0x41, 0xce, 0xb5, 0x14, 0x7c, 0xd6, 0x31, 0x86, 0x8e, + 0xfe, 0x9c, 0x69, 0x4a, 0x4e, 0xce, 0xba, 0x56, 0x5d, 0x80, 0xd0, 0x03, 0x28, 0x74, 0x6d, 0xab, + 0xaf, 0xe8, 0xa6, 0x20, 0x8a, 0x51, 0xa2, 0x3c, 0x81, 0x36, 0x04, 0x30, 0xa4, 0xb2, 0x7f, 0x9b, + 0x83, 0x22, 0x35, 0x8c, 0xb9, 0xdc, 0xde, 0x83, 0x80, 0xdb, 0xbb, 0x1d, 0x72, 0x7b, 0x9e, 0x75, + 0x11, 0xaf, 0xf7, 0x32, 0x24, 0x87, 0x34, 0xbf, 0xa1, 0xe3, 0x7b, 0xa1, 0x91, 0xc1, 0xe6, 0xd0, + 0x4a, 0xf4, 0x65, 0x40, 0xc4, 0x15, 0x60, 0x25, 0x44, 0x98, 0xa0, 0x84, 0x12, 0xc5, 0x54, 0x67, + 0x7a, 0xd0, 0xe4, 0x15, 0x3c, 0xe8, 0x2e, 0x48, 0xf8, 0xcc, 0xb5, 0xd5, 0x60, 0xb6, 0x9e, 0xa2, + 0xfc, 0x2b, 0x24, 0x2c, 0xd6, 0x09, 0x6e, 0xba, 0x90, 0x02, 0x0e, 0xe0, 0x34, 0xa2, 0x25, 0x8b, + 0x5c, 0x86, 0xa6, 0xdb, 0x98, 0xe6, 0x98, 0x4e, 0x29, 0xbd, 0x16, 0xbb, 0x24, 0x97, 0x1c, 0xdb, + 0xf6, 0x8d, 0x9a, 0x60, 0x94, 0x25, 0x26, 0xca, 0x03, 0x38, 0xe8, 0x08, 0xb2, 0x5d, 0x96, 0x7a, + 0x2a, 0xcf, 0xf0, 0x88, 0x26, 0xa9, 0xd9, 0xad, 0x37, 0xe7, 0x4f, 0x52, 0xb7, 0x93, 0xe4, 0x08, + 0x4a, 0x11, 0x19, 0xba, 0x1e, 0x12, 0x3d, 0x85, 0x7c, 0xa0, 0xae, 0x38, 0x19, 0xd1, 0xfc, 0xe4, + 0x7a, 0x62, 0x73, 0xbe, 0xa0, 0xed, 0x11, 0xfa, 0x10, 0x40, 0xf7, 0x02, 0x00, 0x4d, 0x63, 0xb2, + 0x5b, 0x6f, 0x5d, 0x21, 0x52, 0x08, 0xff, 0xe2, 0x0b, 0x41, 0x4f, 0xa1, 0xe0, 0x7f, 0xd1, 0xc9, + 0xe6, 0xae, 0x3c, 0x59, 0x26, 0x35, 0x1f, 0x90, 0xb3, 0x4d, 0x8a, 0x94, 0x25, 0x92, 0x60, 0x59, + 0x8e, 0xee, 0xe2, 0xa0, 0x1a, 0xe4, 0xa9, 0x1a, 0x94, 0x2f, 0xce, 0x57, 0x51, 0x55, 0xe0, 0xa7, + 0xab, 0x02, 0xea, 0x8c, 0xe1, 0x99, 0x62, 0x85, 0x14, 0x98, 0x48, 0x2c, 0xf8, 0x8a, 0x75, 0xe4, + 0xab, 0xf0, 0x84, 0x62, 0x05, 0xd4, 0x9b, 0x55, 0x95, 0xb9, 0x90, 0xef, 0x29, 0x5e, 0xdf, 0xf7, + 0x84, 0x04, 0xa1, 0x3a, 0x4f, 0x9a, 0x25, 0x9a, 0x7e, 0xbd, 0x35, 0xa7, 0x92, 0x92, 0x8c, 0x4e, + 0xb8, 0x04, 0x9a, 0x4b, 0xbf, 0x0b, 0xa8, 0x63, 0x63, 0xd5, 0xc5, 0x1a, 0x49, 0x5a, 0x0d, 0xbd, + 0xa3, 0xbb, 0xc6, 0xa8, 0xb4, 0x18, 0xb0, 0xfb, 0x45, 0x8e, 0xaf, 0x7b, 0x68, 0xf4, 0x1e, 0xa4, + 0x9e, 0x63, 0xdb, 0xd1, 0x2d, 0xb3, 0x84, 0xa8, 0x33, 0x59, 0xe1, 0x2d, 0xa2, 0x3b, 0x63, 0xe3, + 0x3d, 0x61, 0x54, 0xb2, 0x20, 0x47, 0xbb, 0x90, 0xc7, 0x66, 0xc7, 0xd2, 0x74, 0xb3, 0x47, 0xd3, + 0xd0, 0xd2, 0x2d, 0x3f, 0xdf, 0xf9, 0xec, 0x7c, 0xf5, 0x4b, 0x63, 0xfc, 0x75, 0x4e, 0x4b, 0xa6, + 0x2d, 0xe7, 0x70, 0xe0, 0x0b, 0xed, 0x42, 0x4a, 0xc4, 0xe4, 0x25, 0xba, 0xa7, 0xeb, 0xb3, 0x32, + 0xd0, 0xf1, 0x88, 0x2e, 0x32, 0x4b, 0xce, 0x4e, 0xca, 0x09, 0x4d, 0x77, 0x48, 0x2e, 0xa2, 0x95, + 0x6e, 0x07, 0xcb, 0x09, 0x01, 0x45, 0x55, 0x80, 0x1e, 0xb6, 0x14, 0xd6, 0x74, 0x2b, 0xdd, 0xa1, + 0xc3, 0xad, 0x04, 0x86, 0xeb, 0x61, 0x6b, 0x43, 0xb4, 0xe6, 0x48, 0x35, 0xd5, 0xd5, 0x7b, 0x22, + 0x45, 0xe8, 0x61, 0x8b, 0x01, 0x50, 0x19, 0x32, 0x03, 0x1b, 0x6b, 0x7a, 0x87, 0x14, 0xbe, 0x77, + 0x03, 0xbe, 0xd9, 0x07, 0x97, 0x57, 0x20, 0xe3, 0x79, 0x0d, 0x94, 0x82, 0x58, 0xe5, 0xa8, 0xca, + 0xfa, 0x2c, 0xb5, 0xfa, 0x51, 0x55, 0x8a, 0x94, 0xef, 0x41, 0x9c, 0x2e, 0x3e, 0x0b, 0xa9, 0x9d, + 0x96, 0xfc, 0xb4, 0x22, 0xd7, 0x58, 0x6f, 0xa7, 0xd1, 0x7c, 0x52, 0x97, 0xdb, 0xf5, 0x9a, 0x24, + 0xe2, 0xc2, 0x79, 0x1c, 0x90, 0x5f, 0xd6, 0xb5, 0x2d, 0x5e, 0x26, 0xf7, 0xa0, 0xd8, 0xf1, 0xa0, + 0xec, 0x00, 0x22, 0x6b, 0xd1, 0xf5, 0xc2, 0xd6, 0x7b, 0x9f, 0x5b, 0x1a, 0x0a, 0x19, 0x41, 0x90, + 0xaf, 0x4c, 0x85, 0x4e, 0x08, 0x1a, 0xc8, 0x87, 0xa2, 0x63, 0x31, 0x48, 0x86, 0x44, 0xe7, 0x14, + 0x77, 0x9e, 0xf1, 0x28, 0xfc, 0xb5, 0x19, 0x03, 0xd3, 0x54, 0x31, 0xa0, 0xb8, 0x55, 0xc2, 0xe3, + 0x0f, 0x2d, 0xd2, 0x03, 0x2a, 0x0a, 0xc9, 0x61, 0xf7, 0x1a, 0xbf, 0xd4, 0x63, 0x4d, 0x6b, 0x47, + 0x09, 0x8f, 0x15, 0xf0, 0xae, 0xef, 0x41, 0xd1, 0xb4, 0x5c, 0x85, 0x94, 0x94, 0xdc, 0x0b, 0xd0, + 0x42, 0x31, 0xbf, 0x2d, 0x71, 0x5d, 0xf5, 0x6d, 0x3e, 0x6f, 0x5a, 0x6e, 0x73, 0x68, 0x18, 0x0c, + 0x80, 0xfe, 0x34, 0x02, 0xab, 0x2c, 0x56, 0x2a, 0x2f, 0x58, 0x83, 0x40, 0x61, 0xe9, 0xad, 0xbf, + 0x47, 0xb4, 0x9d, 0x32, 0x3b, 0x31, 0xba, 0xac, 0xbb, 0xc0, 0xa7, 0xfa, 0xf2, 0xf0, 0x12, 0x9a, + 0x72, 0x1b, 0x0a, 0xe1, 0x63, 0x42, 0x19, 0x48, 0x54, 0x77, 0xeb, 0xd5, 0xc7, 0xd2, 0x02, 0x2a, + 0x42, 0x76, 0xa7, 0x25, 0xd7, 0x1b, 0x8f, 0x9a, 0xca, 0xe3, 0xfa, 0x77, 0x58, 0x3b, 0xb0, 0xd9, + 0xf2, 0xda, 0x81, 0x25, 0x58, 0x3a, 0x6e, 0x36, 0x3e, 0x3c, 0xae, 0x2b, 0x4f, 0x1b, 0xed, 0xdd, + 0xd6, 0x71, 0x5b, 0x69, 0x34, 0x6b, 0xf5, 0x8f, 0xa4, 0x98, 0x57, 0x82, 0x25, 0xa4, 0x64, 0xf9, + 0x67, 0x51, 0x28, 0x1c, 0xda, 0x7a, 0x5f, 0xb5, 0x47, 0x8f, 0xf1, 0xe8, 0xe8, 0x85, 0x3a, 0x40, + 0x1f, 0xc0, 0x92, 0x89, 0x5f, 0x28, 0x03, 0x06, 0x55, 0xbc, 0x94, 0x3e, 0x32, 0xbd, 0x8b, 0xbc, + 0x68, 0xe2, 0x17, 0x5c, 0x42, 0x83, 0x67, 0xf4, 0x5f, 0x86, 0xac, 0x65, 0x68, 0x8c, 0x13, 0x8b, + 0x3e, 0x48, 0x36, 0xc8, 0x04, 0x96, 0xa1, 0x35, 0x18, 0x9a, 0x50, 0x93, 0xf1, 0x04, 0x75, 0x6c, + 0x0a, 0xb5, 0x89, 0x5f, 0x08, 0xea, 0x0f, 0x60, 0x89, 0xc8, 0x9e, 0x98, 0x5d, 0x7c, 0xc6, 0xec, + 0x2c, 0x43, 0x1b, 0x9b, 0xdd, 0xd7, 0xe1, 0xce, 0xe4, 0xfa, 0x26, 0x1a, 0x71, 0xb7, 0xc6, 0x96, + 0x45, 0x72, 0x1c, 0x6e, 0x94, 0xff, 0x14, 0x01, 0x1a, 0x8c, 0x86, 0xae, 0x68, 0x0f, 0xd2, 0x7d, + 0xfb, 0x0a, 0xe4, 0x89, 0x5c, 0xbf, 0x06, 0x8c, 0xcc, 0xd0, 0x33, 0xb2, 0x5c, 0x11, 0x59, 0x08, + 0x17, 0x59, 0x8f, 0xcf, 0x15, 0x9d, 0xc5, 0x65, 0x19, 0x5e, 0x33, 0x12, 0xbd, 0x0e, 0x39, 0xdd, + 0x24, 0xce, 0x98, 0xf7, 0x28, 0x82, 0x6d, 0xa3, 0x2c, 0xc7, 0xd4, 0xcf, 0x06, 0xb6, 0x28, 0x2c, + 0xa2, 0x70, 0xf7, 0x40, 0x75, 0xb1, 0xad, 0xab, 0x86, 0xfe, 0x87, 0x58, 0x7b, 0xa2, 0xe3, 0x17, + 0x32, 0xee, 0xda, 0xd8, 0x39, 0x45, 0x1f, 0xc1, 0xe2, 0xc4, 0x76, 0xd0, 0xa9, 0x67, 0xb7, 0x5e, + 0x9b, 0x2f, 0x1a, 0x89, 0x9c, 0x7a, 0x6c, 0xc7, 0xd0, 0x41, 0xf8, 0x60, 0x59, 0x4d, 0x72, 0x35, + 0x99, 0xc1, 0x93, 0xff, 0x13, 0x48, 0xa8, 0x8e, 0x62, 0x75, 0xb9, 0xc7, 0x79, 0x25, 0x20, 0x68, + 0xe8, 0xea, 0xc6, 0xc6, 0xa9, 0xd1, 0xd9, 0x68, 0x8b, 0x8b, 0x9a, 0xed, 0x3d, 0xbe, 0x81, 0xdb, + 0x3d, 0xdd, 0x3d, 0x1d, 0x9e, 0x6c, 0x74, 0xac, 0xfe, 0xa6, 0xc7, 0xa0, 0x9d, 0xf8, 0x7f, 0x6f, + 0x0e, 0x9e, 0xf5, 0x36, 0x49, 0xb4, 0x57, 0x7b, 0x78, 0x13, 0x9b, 0x3d, 0xdd, 0xc4, 0x83, 0x93, + 0x8d, 0xf6, 0x99, 0xe9, 0xc9, 0x92, 0xe3, 0xaa, 0xd3, 0xea, 0xa2, 0xb7, 0xa1, 0xe8, 0x9c, 0x5a, + 0x43, 0x43, 0x53, 0x4e, 0xd4, 0xce, 0xb3, 0xae, 0x6e, 0x18, 0xa1, 0xae, 0x55, 0x81, 0x21, 0xb7, + 0x39, 0x8e, 0xef, 0xfc, 0x5f, 0xa5, 0x00, 0xf9, 0xab, 0x3a, 0x18, 0xba, 0x2a, 0x8d, 0x09, 0x15, + 0x48, 0x72, 0x67, 0xc4, 0x76, 0xfa, 0xf5, 0x99, 0x7e, 0x3b, 0xdc, 0xa5, 0xdb, 0x5d, 0x90, 0x39, + 0x23, 0xfa, 0x76, 0xf0, 0x76, 0x67, 0xee, 0x7d, 0xdd, 0x5d, 0x10, 0xd7, 0x3e, 0x8f, 0x21, 0xe1, + 0xb8, 0x24, 0x86, 0xc5, 0x68, 0xe6, 0xb1, 0x39, 0x83, 0x7f, 0x72, 0xf2, 0x1b, 0x47, 0x84, 0x4d, + 0x78, 0x6e, 0x2a, 0x03, 0x3d, 0x85, 0x8c, 0x97, 0x70, 0xf3, 0x2e, 0xe5, 0xbb, 0xf3, 0x0b, 0xf4, + 0x62, 0xa5, 0x88, 0xa4, 0x9e, 0x2c, 0x54, 0x81, 0x6c, 0x9f, 0x93, 0xf9, 0x6d, 0x95, 0x35, 0x5e, + 0xf3, 0x80, 0x90, 0x40, 0x6b, 0x9f, 0xc0, 0x97, 0x0c, 0x82, 0xa9, 0x41, 0xf3, 0x02, 0xdb, 0x32, + 0x0c, 0x72, 0x68, 0xd4, 0x5f, 0x7b, 0x79, 0x81, 0x80, 0xa2, 0xc7, 0xa4, 0x72, 0xf1, 0x7c, 0x7a, + 0x9a, 0xee, 0xe7, 0x1b, 0x73, 0x47, 0xd2, 0xdd, 0x05, 0x39, 0xc0, 0x8e, 0x5a, 0x50, 0x18, 0x84, + 0x1c, 0x2a, 0x2f, 0x13, 0x1e, 0xcc, 0xca, 0x15, 0x43, 0xc4, 0xbb, 0x0b, 0xf2, 0x18, 0x3b, 0xfa, + 0x1e, 0xa0, 0xce, 0x84, 0xb7, 0x29, 0xc1, 0xe7, 0xcc, 0x72, 0x9c, 0x61, 0x77, 0x41, 0x9e, 0x22, + 0x06, 0x7d, 0x0c, 0x77, 0xfb, 0xd3, 0x1d, 0x03, 0x2f, 0x18, 0x36, 0x66, 0x8c, 0x30, 0xc3, 0x9d, + 0xec, 0x2e, 0xc8, 0xb3, 0x04, 0x96, 0x3f, 0x80, 0x04, 0x55, 0x1d, 0x92, 0xf6, 0x1c, 0x37, 0x1f, + 0x37, 0x5b, 0x4f, 0x9b, 0x2c, 0x8c, 0xd5, 0xea, 0xfb, 0xf5, 0x76, 0x5d, 0x69, 0x35, 0xf7, 0x49, + 0x18, 0x7b, 0x09, 0x6e, 0x73, 0x40, 0xa5, 0x59, 0x53, 0x9e, 0xca, 0x0d, 0x81, 0x8a, 0x96, 0xd7, + 0x83, 0x79, 0x55, 0x1a, 0xe2, 0xcd, 0x56, 0xb3, 0x2e, 0x2d, 0xd0, 0x0c, 0xab, 0x56, 0x93, 0x22, + 0x34, 0xc3, 0x92, 0x5b, 0x87, 0x52, 0x94, 0x59, 0xdf, 0x76, 0x0e, 0x40, 0xf3, 0xd4, 0x6d, 0x2f, + 0x9e, 0x4e, 0x4a, 0xa9, 0xf2, 0x3f, 0x46, 0x20, 0x4d, 0x9c, 0x79, 0xc3, 0xec, 0x5a, 0xe8, 0x5d, + 0xc8, 0x0c, 0x54, 0x1b, 0x9b, 0xae, 0xef, 0xaf, 0x45, 0x1f, 0x31, 0x7d, 0x48, 0x11, 0x5e, 0x9b, + 0x2b, 0xcd, 0x08, 0x1b, 0x97, 0x35, 0x89, 0x76, 0x40, 0xe2, 0xe2, 0x9c, 0xce, 0x29, 0xee, 0xab, + 0x44, 0x2a, 0xeb, 0x64, 0xbd, 0xec, 0xf5, 0x78, 0x29, 0xfe, 0x88, 0xa2, 0x3d, 0xd9, 0x85, 0x41, + 0x10, 0x2a, 0x3a, 0x80, 0xff, 0xbe, 0x0e, 0xc5, 0xb1, 0xfc, 0xe9, 0x92, 0xa6, 0xc0, 0x1a, 0x6d, + 0x0a, 0xc4, 0xfc, 0xe8, 0xe1, 0x35, 0x05, 0xa2, 0xbc, 0x1f, 0x10, 0x5a, 0x6c, 0x7c, 0xce, 0xc5, + 0xbe, 0xeb, 0xd7, 0x08, 0xcc, 0xf8, 0x5e, 0xe2, 0x8e, 0x75, 0xf1, 0x92, 0xf2, 0xe0, 0xef, 0x22, + 0xb0, 0xd8, 0xb7, 0x34, 0xbd, 0x4b, 0xd2, 0x61, 0x62, 0xba, 0xae, 0xde, 0xc7, 0x3c, 0x59, 0xfa, + 0x02, 0xfd, 0xb6, 0x14, 0x9c, 0x03, 0x01, 0xa3, 0x47, 0x90, 0x12, 0x2d, 0xb4, 0x34, 0x0d, 0x47, + 0xf3, 0x3a, 0x5e, 0x51, 0x6c, 0x70, 0x6e, 0xb4, 0x03, 0x05, 0x13, 0x9f, 0x05, 0x3b, 0xbe, 0x99, + 0x90, 0x6b, 0xca, 0x35, 0xf1, 0xd9, 0xf4, 0x76, 0x6f, 0xce, 0xf4, 0x31, 0x1a, 0xfa, 0x10, 0xf2, + 0xe1, 0xc8, 0x0b, 0xd7, 0x88, 0xbc, 0xb9, 0x41, 0x30, 0xec, 0xee, 0x40, 0x4a, 0x84, 0xdc, 0xec, + 0x35, 0x42, 0xae, 0x60, 0x46, 0xdb, 0x24, 0x9f, 0x39, 0x73, 0xfd, 0x14, 0x2b, 0xe7, 0xd7, 0x88, + 0x17, 0xe7, 0xab, 0x59, 0xb2, 0xc2, 0x29, 0x7d, 0xdd, 0xac, 0xe9, 0xc1, 0x35, 0xb4, 0x07, 0xe0, + 0xbd, 0x8b, 0x70, 0xe8, 0x75, 0xc7, 0xec, 0x5e, 0xc1, 0xa1, 0x20, 0xf4, 0xa7, 0x24, 0x07, 0xb8, + 0xd1, 0x01, 0x64, 0x84, 0x57, 0x67, 0x55, 0xfc, 0x6c, 0xf7, 0x37, 0x19, 0x63, 0x44, 0x64, 0xf1, + 0x24, 0x90, 0x02, 0xc6, 0xc0, 0xaa, 0x83, 0x79, 0x29, 0xff, 0xde, 0x9c, 0x05, 0x0c, 0xb3, 0xd1, + 0xea, 0xa9, 0x6a, 0xf6, 0xf0, 0x3e, 0xe1, 0xdf, 0x8e, 0x96, 0x22, 0x32, 0x13, 0x85, 0x9a, 0x20, + 0xd1, 0x2d, 0x0b, 0x86, 0x2c, 0x89, 0xee, 0xda, 0xab, 0xc2, 0xfe, 0xc9, 0xae, 0xcd, 0x0c, 0x5b, + 0x54, 0xa7, 0x0e, 0xfc, 0xd0, 0xf5, 0x4d, 0x28, 0x74, 0x2d, 0xbb, 0xaf, 0xba, 0x8a, 0xb0, 0xc1, + 0x45, 0xbf, 0xe9, 0xf7, 0xd9, 0xf9, 0x6a, 0x7e, 0x87, 0x62, 0x85, 0xfd, 0xe5, 0xbb, 0xc1, 0x4f, + 0xb4, 0x2d, 0x22, 0xfc, 0x2d, 0x1a, 0x90, 0x5f, 0xfb, 0xdc, 0xcd, 0x9a, 0x12, 0xd8, 0x9b, 0x90, + 0xa4, 0xb5, 0x99, 0x53, 0x5a, 0xa2, 0x3b, 0x7e, 0xcd, 0x3a, 0x4f, 0xe6, 0x52, 0xd0, 0x3e, 0x14, + 0x34, 0x02, 0xd1, 0xcd, 0x1e, 0x6f, 0x27, 0xde, 0xa6, 0x72, 0x57, 0x67, 0xc8, 0x15, 0x9e, 0x5a, + 0x74, 0x8d, 0x04, 0x33, 0x6b, 0x39, 0xb6, 0x20, 0xdd, 0x55, 0xfb, 0xba, 0xa1, 0x63, 0xa7, 0x74, + 0x87, 0xca, 0x79, 0xfb, 0x52, 0x7b, 0x1e, 0xbf, 0x5a, 0x12, 0x99, 0x80, 0x10, 0xe2, 0x99, 0x35, + 0x05, 0x8c, 0xc8, 0xf1, 0xdd, 0x9d, 0x34, 0x6b, 0x71, 0xb5, 0x14, 0xba, 0x66, 0xa2, 0x66, 0xcd, + 0xbf, 0x34, 0x74, 0x1f, 0xe0, 0xb9, 0x8e, 0x5f, 0x28, 0x9f, 0x0c, 0xb1, 0x3d, 0x2a, 0x95, 0x82, + 0x5d, 0x02, 0x02, 0xff, 0x90, 0x80, 0xd1, 0x3b, 0x90, 0xd1, 0xf0, 0x00, 0x9b, 0x9a, 0xd3, 0x32, + 0x4b, 0x2f, 0xd1, 0xb2, 0xe7, 0xd6, 0xc5, 0xf9, 0x6a, 0xa6, 0x26, 0x80, 0xdc, 0x19, 0xfb, 0x54, + 0xe8, 0x63, 0xc8, 0xb1, 0x0f, 0xac, 0xb5, 0xcc, 0xed, 0x51, 0x69, 0x99, 0x2e, 0xfa, 0xe1, 0x9c, + 0x87, 0xe2, 0xf7, 0xe0, 0xbc, 0x6b, 0x8b, 0x5a, 0x40, 0x9a, 0x1c, 0x92, 0x8d, 0x7e, 0x1f, 0x72, + 0x42, 0x8f, 0xf7, 0xac, 0x13, 0xa7, 0xf4, 0xa5, 0x4b, 0xef, 0x14, 0xc6, 0xc7, 0x3a, 0xf0, 0x59, + 0x85, 0x97, 0x0a, 0x4a, 0x43, 0x1f, 0x41, 0xde, 0xbb, 0x3d, 0xb5, 0x06, 0xae, 0x53, 0x7a, 0xf9, + 0xd2, 0x52, 0x7a, 0xc2, 0x0c, 0x39, 0x6f, 0x6b, 0x40, 0xaf, 0x5b, 0x02, 0x5f, 0xe8, 0x1e, 0x64, + 0x34, 0xdb, 0x1a, 0xb0, 0x98, 0xf3, 0xca, 0x5a, 0x64, 0x3d, 0xe6, 0x35, 0x82, 0x6c, 0x6b, 0x40, + 0xc3, 0x80, 0x02, 0x05, 0x1b, 0x0f, 0x0c, 0xb5, 0x83, 0xfb, 0x24, 0x1c, 0x5a, 0xdd, 0xd2, 0x0a, + 0x1d, 0x7d, 0x6b, 0xee, 0x8d, 0xf4, 0x98, 0x85, 0x62, 0x06, 0xe4, 0xb5, 0xba, 0xe8, 0x18, 0x40, + 0x1d, 0x6a, 0xba, 0xab, 0xf4, 0x2d, 0x0d, 0x97, 0x56, 0x2f, 0x7d, 0xcc, 0x30, 0x2e, 0xbc, 0x42, + 0x18, 0x0f, 0x2c, 0x0d, 0x7b, 0x17, 0x74, 0x02, 0x80, 0xde, 0x81, 0x2c, 0x5d, 0xda, 0xc7, 0xd6, + 0x09, 0xd1, 0xcd, 0x35, 0xba, 0xb8, 0x45, 0x7e, 0x96, 0x99, 0x9a, 0x6d, 0x0d, 0xf6, 0xac, 0x13, + 0xaa, 0x31, 0xfc, 0x4f, 0x0d, 0x39, 0x90, 0xeb, 0x75, 0x14, 0xdf, 0x71, 0xde, 0xa3, 0xa7, 0xf8, + 0xfe, 0x9c, 0x73, 0x79, 0x54, 0x9d, 0xe2, 0x4a, 0x6f, 0x89, 0x08, 0xf0, 0xa8, 0x2a, 0x60, 0x8e, + 0x9c, 0xed, 0x75, 0xbc, 0x0f, 0x52, 0x9e, 0xb2, 0x6e, 0x23, 0x37, 0x80, 0x72, 0xb0, 0x3c, 0x65, + 0x18, 0x66, 0x02, 0x7f, 0x13, 0x01, 0xde, 0x97, 0x54, 0x68, 0x6d, 0xc7, 0x0e, 0xed, 0xfe, 0x17, + 0x9d, 0x28, 0x14, 0xd8, 0x1c, 0x2a, 0x4e, 0xab, 0x4b, 0xf5, 0xa3, 0x03, 0x39, 0x6b, 0xe8, 0x9e, + 0x58, 0x43, 0x53, 0x53, 0xba, 0xcf, 0x9c, 0xd2, 0xab, 0x74, 0xd3, 0xae, 0xd4, 0x89, 0xf2, 0x36, + 0xa9, 0xc5, 0x05, 0xed, 0x3c, 0x76, 0xe4, 0xac, 0x90, 0xba, 0xf3, 0xcc, 0x41, 0x3f, 0x80, 0xac, + 0x6e, 0xfa, 0x63, 0x3c, 0xb8, 0xfa, 0x18, 0x48, 0xd4, 0x41, 0x0d, 0xd3, 0x1b, 0x02, 0xb8, 0x4c, + 0x32, 0xc2, 0x5b, 0x50, 0xb0, 0xba, 0x5d, 0x43, 0x37, 0xb1, 0x62, 0x63, 0xd5, 0xb1, 0xcc, 0xd2, + 0x6b, 0x81, 0x83, 0xc8, 0x73, 0x9c, 0x4c, 0x51, 0xa8, 0x0c, 0x19, 0x17, 0xf7, 0x07, 0x96, 0xad, + 0xda, 0xa3, 0xd2, 0xeb, 0xc1, 0xeb, 0x51, 0x0f, 0x8c, 0x4e, 0x60, 0x79, 0x68, 0xe2, 0xb3, 0x81, + 0xe5, 0x60, 0x4d, 0x99, 0xc8, 0x74, 0xd7, 0xa9, 0xab, 0x7c, 0xc0, 0x27, 0x75, 0xf7, 0x58, 0x50, + 0x4e, 0x4d, 0x79, 0xef, 0x0e, 0xa7, 0xa2, 0x35, 0xf4, 0x35, 0x58, 0xd2, 0x1d, 0x25, 0x58, 0x43, + 0x28, 0xc4, 0x65, 0x96, 0xde, 0x08, 0x4c, 0x09, 0xe9, 0xce, 0x78, 0xfd, 0x81, 0xbe, 0x0f, 0x45, + 0xc3, 0xea, 0xa8, 0x86, 0xee, 0x8e, 0x44, 0x87, 0xf7, 0x4d, 0xaa, 0x47, 0x5f, 0x9d, 0x53, 0xd7, + 0xf7, 0x39, 0x37, 0xeb, 0xf3, 0xca, 0x05, 0x23, 0xf4, 0x8d, 0x7e, 0x18, 0x81, 0xb5, 0xcf, 0x69, + 0x07, 0x3a, 0xa5, 0xb7, 0x2e, 0xbd, 0x28, 0x9d, 0xa3, 0x1f, 0xf8, 0xca, 0x65, 0xfd, 0x40, 0x67, + 0xf9, 0xa7, 0x11, 0x58, 0x9c, 0xc8, 0x40, 0xd0, 0xf7, 0x21, 0x65, 0x5a, 0x5a, 0xe0, 0xa6, 0xbe, + 0xce, 0xcf, 0x20, 0xd9, 0xb4, 0x34, 0x76, 0x51, 0xff, 0xee, 0x5c, 0x56, 0x42, 0xff, 0x1a, 0x9c, + 0x6c, 0x30, 0x36, 0x39, 0x49, 0xa4, 0x36, 0x34, 0xf4, 0x36, 0x14, 0xf1, 0xd9, 0x40, 0xb7, 0x03, + 0xb9, 0x7c, 0x34, 0xe0, 0x57, 0x0b, 0x3e, 0x92, 0x58, 0x0f, 0xbf, 0x4b, 0xfd, 0x59, 0x14, 0x8a, + 0x63, 0x59, 0x00, 0xa9, 0x5e, 0x68, 0xbb, 0x2a, 0x54, 0xbd, 0x10, 0xc8, 0x25, 0x35, 0x55, 0xf0, + 0xfd, 0x53, 0xec, 0xa6, 0x6f, 0xe1, 0xc2, 0x77, 0x94, 0x89, 0x2b, 0xdc, 0x51, 0x7e, 0x1d, 0xee, + 0xe8, 0x8e, 0x62, 0x5a, 0xa6, 0x68, 0x25, 0x7b, 0xfd, 0x82, 0xe0, 0xbb, 0xa3, 0x5b, 0xba, 0xd3, + 0xb4, 0x4c, 0xd6, 0x44, 0xf6, 0x56, 0xed, 0x3f, 0x51, 0x4a, 0x4d, 0x3e, 0x51, 0xf2, 0xba, 0xb1, + 0x71, 0x29, 0xb1, 0xfc, 0x93, 0x08, 0x64, 0x82, 0x2f, 0x7c, 0xa3, 0xe1, 0x2e, 0xe2, 0x44, 0x45, + 0x77, 0xcd, 0x17, 0x17, 0xe1, 0x5d, 0x88, 0xcd, 0xbf, 0x0b, 0xfc, 0x68, 0xff, 0x08, 0xb2, 0x81, + 0xf0, 0x3e, 0xde, 0xa3, 0x89, 0x5c, 0xa3, 0x47, 0xf3, 0x2a, 0x24, 0x79, 0x4c, 0x63, 0x8a, 0x95, + 0xe7, 0xdc, 0x09, 0x16, 0xcf, 0x12, 0x1f, 0x93, 0x58, 0xc6, 0x47, 0xff, 0x8f, 0x18, 0xe4, 0x82, + 0xe1, 0x9f, 0x78, 0x2e, 0xdd, 0xec, 0xd8, 0x34, 0xf6, 0xd2, 0xd1, 0x63, 0xde, 0xc3, 0x0e, 0x01, + 0x26, 0x49, 0x41, 0x5f, 0x37, 0x15, 0xfa, 0x98, 0x20, 0xa4, 0xbc, 0xe9, 0xbe, 0x6e, 0x3e, 0x21, + 0x50, 0x4a, 0xa2, 0x9e, 0x71, 0x92, 0x58, 0x88, 0x44, 0x3d, 0x63, 0x24, 0xcb, 0x34, 0xa3, 0xb6, + 0x5d, 0x5a, 0x3d, 0xc7, 0x02, 0x99, 0xb2, 0xed, 0x06, 0x1f, 0x3e, 0x25, 0xa6, 0x3c, 0x7c, 0x42, + 0x26, 0x14, 0xfc, 0x84, 0xe7, 0x85, 0x89, 0x6d, 0xaa, 0x38, 0xd9, 0xad, 0xca, 0x35, 0x32, 0x1e, + 0xff, 0x83, 0x08, 0x12, 0xfe, 0xdc, 0x09, 0x02, 0x97, 0xff, 0x21, 0x02, 0xf9, 0x10, 0x19, 0x6a, + 0x40, 0x91, 0x0e, 0x3c, 0xd1, 0xa2, 0xbe, 0xe7, 0xbd, 0xd5, 0x25, 0xe8, 0xa9, 0x55, 0x6b, 0xde, + 0x0a, 0xa0, 0x34, 0xf4, 0x01, 0x14, 0x98, 0x28, 0xef, 0x89, 0x50, 0x58, 0xfd, 0x72, 0x54, 0x52, + 0xf8, 0x9d, 0x50, 0xce, 0xf2, 0x61, 0x5a, 0xf0, 0xf5, 0xc3, 0xf2, 0x3f, 0x47, 0x20, 0x1b, 0x48, + 0xa9, 0xe6, 0x50, 0xfc, 0x3f, 0x86, 0xb8, 0xe7, 0x86, 0xbe, 0xd8, 0x56, 0xb0, 0xeb, 0x7b, 0xb8, + 0x1f, 0x47, 0x60, 0x69, 0x5a, 0x82, 0x14, 0x32, 0x4b, 0xa6, 0x8f, 0x73, 0x99, 0xe5, 0xfd, 0x60, + 0xe2, 0xca, 0x74, 0x54, 0x5c, 0xec, 0xfb, 0xa9, 0xeb, 0x6b, 0x9e, 0xa5, 0x30, 0x15, 0x2d, 0x86, + 0x2c, 0x85, 0x94, 0xa2, 0x41, 0x5b, 0xf9, 0xb7, 0x18, 0x14, 0xc2, 0x71, 0x0d, 0x3d, 0x81, 0x64, + 0xcf, 0xb0, 0x4e, 0x54, 0x83, 0xb7, 0x9e, 0xbf, 0x79, 0xad, 0xf0, 0xb8, 0xf1, 0x88, 0xca, 0xd8, + 0x5d, 0x90, 0xb9, 0x34, 0xe4, 0xc0, 0xa2, 0x8d, 0x7b, 0xba, 0x65, 0xaa, 0x86, 0x72, 0x32, 0x62, + 0x8a, 0xc1, 0xcf, 0xa7, 0x7e, 0xbd, 0x21, 0x64, 0x2e, 0x6e, 0x7b, 0x44, 0x09, 0x77, 0x17, 0xe4, + 0xa2, 0x1d, 0x06, 0xa1, 0x3e, 0x14, 0x83, 0x83, 0xda, 0xd6, 0x0b, 0x7e, 0x3b, 0x50, 0xbd, 0xe9, + 0x90, 0xb2, 0xf5, 0x62, 0x97, 0xa6, 0xf5, 0x01, 0xc0, 0xf2, 0xfb, 0x50, 0x1c, 0x9b, 0x14, 0x2a, + 0x43, 0x92, 0xd1, 0xf0, 0xa0, 0x06, 0x9f, 0x9d, 0xaf, 0x26, 0x19, 0x91, 0xcc, 0x31, 0xfc, 0x2c, + 0x6e, 0x43, 0x3e, 0x24, 0x9e, 0x83, 0x0b, 0x90, 0x64, 0x7b, 0x19, 0x34, 0x89, 0x6d, 0x80, 0xb4, + 0xc8, 0x3f, 0xca, 0xeb, 0x90, 0xf1, 0x6a, 0x02, 0x94, 0x83, 0x74, 0xad, 0x71, 0x54, 0xd9, 0xde, + 0xaf, 0xd7, 0xa4, 0x05, 0x94, 0x87, 0x8c, 0x5c, 0xaf, 0xd4, 0x68, 0x93, 0x54, 0x8a, 0x7c, 0x23, + 0xfd, 0xe7, 0x3f, 0x5e, 0x8d, 0xf0, 0x78, 0x92, 0x94, 0x52, 0x7b, 0xf1, 0x34, 0x92, 0x6e, 0x95, + 0x7f, 0x9e, 0x01, 0x54, 0x53, 0x5d, 0x95, 0x6c, 0xc0, 0x15, 0x5a, 0x89, 0xd1, 0x4b, 0xec, 0x2f, + 0xdc, 0xd7, 0x89, 0xdd, 0xa8, 0xaf, 0x33, 0xbd, 0x59, 0x18, 0xff, 0xdd, 0x37, 0x0b, 0xaf, 0xd5, + 0xfa, 0x9c, 0x6c, 0x70, 0x24, 0x6f, 0xd0, 0xe0, 0x78, 0x02, 0x29, 0x96, 0x5f, 0xb3, 0x07, 0x51, + 0xb3, 0xfb, 0x2f, 0x93, 0xe7, 0xcb, 0x3b, 0x55, 0x4e, 0xdd, 0x74, 0xed, 0x91, 0xf7, 0x56, 0x82, + 0xc1, 0xfc, 0xd6, 0x50, 0xfa, 0xfa, 0xad, 0xa1, 0xc9, 0xea, 0x22, 0x33, 0xbb, 0xba, 0xf8, 0x1e, + 0x70, 0x53, 0x12, 0xb9, 0x39, 0x5c, 0xfa, 0x6c, 0x60, 0xca, 0x72, 0x98, 0xed, 0xf0, 0xe4, 0x3c, + 0x67, 0x07, 0xbe, 0x96, 0xdb, 0x00, 0xbc, 0x7c, 0x30, 0xbb, 0xd6, 0x1c, 0xd1, 0x63, 0x05, 0x52, + 0xc4, 0x9f, 0x0e, 0x30, 0x53, 0x72, 0x2f, 0x54, 0x73, 0x20, 0x37, 0xcc, 0x01, 0xe4, 0x82, 0x5b, + 0x88, 0x24, 0x88, 0x3d, 0xc3, 0x23, 0x66, 0x2e, 0x32, 0xf9, 0x13, 0xed, 0x41, 0xc2, 0x4f, 0x28, + 0x66, 0xbf, 0xc2, 0x9d, 0x79, 0x36, 0x64, 0xba, 0x32, 0x13, 0xf1, 0x8d, 0xe8, 0x7b, 0x91, 0xe5, + 0x3f, 0x8b, 0x42, 0x2e, 0xb8, 0x4c, 0xf4, 0x2a, 0xa4, 0xd8, 0x42, 0xd9, 0x43, 0xdf, 0xb0, 0x77, + 0x11, 0x28, 0xd4, 0x84, 0xbc, 0x33, 0xb4, 0x9f, 0xeb, 0xcf, 0x55, 0x43, 0xe9, 0x59, 0xaa, 0x41, + 0xa7, 0x53, 0xd8, 0xba, 0x3f, 0xeb, 0x21, 0x0d, 0xa7, 0x7d, 0x64, 0xa9, 0x86, 0x68, 0xcd, 0x38, + 0x01, 0x18, 0xfa, 0xaa, 0x77, 0x83, 0xa5, 0x70, 0xd7, 0xc6, 0xae, 0x97, 0x0b, 0x5c, 0xfd, 0xc5, + 0x04, 0x44, 0xe7, 0x9a, 0x7d, 0x92, 0x9c, 0x80, 0x1f, 0x31, 0x36, 0x87, 0x7d, 0xff, 0x8e, 0xc1, + 0xcb, 0x09, 0x18, 0x5d, 0xdd, 0x1c, 0xf6, 0xfd, 0x9c, 0xc0, 0xf6, 0x61, 0x22, 0x27, 0xf0, 0x9c, + 0x58, 0xf9, 0x2f, 0x72, 0x50, 0x68, 0x8f, 0x06, 0x41, 0xa7, 0x75, 0xad, 0x0b, 0x9b, 0x69, 0xd7, + 0x32, 0xd1, 0xab, 0x5f, 0xcb, 0x5c, 0xf2, 0x23, 0x0d, 0xa6, 0x73, 0xf1, 0x4b, 0x74, 0xae, 0x06, + 0x71, 0xfa, 0xcc, 0x3e, 0x41, 0xcf, 0x66, 0x96, 0xaf, 0x0c, 0xaf, 0x76, 0x23, 0xf0, 0xd2, 0x9e, + 0x72, 0xa3, 0xef, 0x40, 0x8e, 0x6e, 0x6e, 0x1f, 0xf7, 0x4f, 0xb0, 0x2d, 0x7c, 0xcb, 0xc3, 0xf9, + 0xa4, 0x91, 0x5d, 0x3e, 0xa0, 0x8c, 0xa2, 0x15, 0x83, 0x3d, 0x88, 0x83, 0x1e, 0x42, 0x42, 0x35, + 0x74, 0xea, 0x68, 0x3e, 0xef, 0xe7, 0x1b, 0x8c, 0x10, 0x7d, 0x0b, 0xf2, 0xaa, 0x6d, 0xab, 0x23, + 0xfe, 0x43, 0x04, 0x8d, 0x3a, 0x13, 0xee, 0x25, 0x2f, 0xce, 0x57, 0xb3, 0x15, 0x82, 0xa4, 0xbf, + 0x3d, 0x10, 0x1b, 0x91, 0x55, 0x3d, 0x50, 0xe8, 0x66, 0x29, 0x73, 0xc3, 0x9b, 0x25, 0xf8, 0xdd, + 0x07, 0x8b, 0x49, 0xbf, 0x9f, 0xbd, 0x81, 0xdf, 0xff, 0x01, 0x2c, 0x8b, 0xa7, 0x9c, 0x44, 0xa0, + 0x7f, 0x89, 0x19, 0xf8, 0xa5, 0x49, 0xf9, 0xe2, 0x7c, 0xb5, 0x24, 0xfb, 0x54, 0xfe, 0xae, 0xb1, + 0x02, 0x8e, 0x6c, 0x78, 0xc9, 0x9e, 0x8a, 0xd7, 0x02, 0x11, 0x20, 0x7f, 0xfd, 0x08, 0x10, 0xce, + 0x02, 0x0a, 0x37, 0xca, 0x02, 0x26, 0xa3, 0x49, 0x71, 0x76, 0x34, 0x79, 0x3a, 0x1e, 0x4d, 0xa4, + 0xcb, 0xdb, 0xb7, 0x61, 0x3b, 0xb8, 0x24, 0x92, 0xfc, 0x28, 0x0a, 0xe0, 0x9b, 0x09, 0xfa, 0x3d, + 0xb8, 0x3b, 0x38, 0x1d, 0x39, 0x7a, 0x47, 0x35, 0x14, 0x1b, 0x0f, 0x6c, 0xec, 0x60, 0x93, 0xe5, + 0xf8, 0xd4, 0xf7, 0xe4, 0xe4, 0x3b, 0x02, 0x2d, 0x87, 0xb0, 0xe8, 0x7d, 0xb8, 0x63, 0x58, 0xbd, + 0x69, 0x7c, 0xc1, 0x06, 0xc7, 0x6d, 0x4e, 0x33, 0xc6, 0xac, 0x02, 0x74, 0xd4, 0x81, 0x7a, 0xa2, + 0x1b, 0x7e, 0xcf, 0xe3, 0xfd, 0xab, 0x9a, 0xf8, 0x46, 0xd5, 0x13, 0x21, 0xde, 0xd2, 0xf8, 0x42, + 0xcb, 0xaf, 0x02, 0xf8, 0x78, 0x7a, 0x7f, 0xbe, 0xbf, 0xef, 0xa7, 0x93, 0xfc, 0x26, 0x9e, 0xc7, + 0xc1, 0xc7, 0x63, 0x41, 0x69, 0x32, 0x3c, 0x44, 0xe6, 0x08, 0x0f, 0xfc, 0x4e, 0xfb, 0x1d, 0x88, + 0xd3, 0x9f, 0x09, 0xa5, 0x21, 0x5e, 0x6f, 0x1e, 0x1f, 0x48, 0x0b, 0x28, 0x03, 0x89, 0xca, 0x7e, + 0xa3, 0x72, 0x24, 0x45, 0xd0, 0x12, 0x48, 0x07, 0xc7, 0xfb, 0xed, 0x86, 0x5c, 0x7f, 0xd4, 0x68, + 0x35, 0x15, 0x4a, 0x10, 0x0d, 0xc4, 0x83, 0xff, 0x89, 0x83, 0xc4, 0x9c, 0xf1, 0x4d, 0x23, 0xc2, + 0xec, 0x76, 0xd3, 0xe7, 0x5f, 0xa3, 0x87, 0xb5, 0x3e, 0xfe, 0x5b, 0xc8, 0x7d, 0x13, 0xff, 0xaf, + 0x72, 0xdf, 0xe4, 0x0d, 0x72, 0xdf, 0xd4, 0x0d, 0x7c, 0xe0, 0x17, 0x9d, 0xa3, 0x06, 0x14, 0xed, + 0x27, 0x51, 0x80, 0x80, 0x8a, 0x7d, 0x3b, 0xf8, 0x53, 0xfb, 0xd9, 0xf7, 0xe9, 0x63, 0xb5, 0xe5, + 0xee, 0x82, 0xf8, 0x21, 0xfe, 0x23, 0x48, 0x6b, 0x3c, 0x07, 0xe4, 0xa9, 0xe2, 0x1b, 0x73, 0xa7, + 0x8a, 0xbb, 0x0b, 0xb2, 0xc7, 0x8c, 0xde, 0x0f, 0xfd, 0xc2, 0xf2, 0xc1, 0x5c, 0x3e, 0x61, 0x57, + 0x3c, 0x11, 0xaf, 0x40, 0x92, 0xa5, 0x3f, 0x5c, 0x67, 0x67, 0xfe, 0xd4, 0x6f, 0xcc, 0xc2, 0x48, + 0x8d, 0xcf, 0x18, 0x79, 0x6d, 0x9a, 0x82, 0xc4, 0xd0, 0xd4, 0x2d, 0xf3, 0x4d, 0x39, 0xf8, 0x38, + 0x59, 0xf4, 0x5c, 0x89, 0x13, 0xa1, 0x7f, 0xab, 0x2e, 0xd6, 0xd8, 0xfb, 0x9e, 0x63, 0xf3, 0xb9, + 0x07, 0x88, 0xa0, 0x02, 0x00, 0xc7, 0xeb, 0x66, 0x4f, 0x8a, 0xd2, 0x8a, 0x96, 0x24, 0xde, 0xe4, + 0x2b, 0xf6, 0xe6, 0xb7, 0x40, 0x1a, 0xff, 0xad, 0x61, 0xe0, 0xa5, 0xcf, 0x22, 0xe4, 0x0f, 0x9e, + 0x54, 0xab, 0xed, 0xc6, 0x41, 0xfd, 0xa8, 0x5d, 0x39, 0x38, 0x64, 0xaf, 0x5e, 0xdb, 0xa4, 0x1c, + 0x6e, 0x35, 0x6a, 0x52, 0xf4, 0xcd, 0x6f, 0x41, 0x71, 0x4c, 0x21, 0x10, 0x40, 0xf2, 0xf0, 0x78, + 0x7b, 0xbf, 0x51, 0x9d, 0xfa, 0x52, 0x08, 0x65, 0x21, 0xd5, 0xda, 0xd9, 0xd9, 0x6f, 0x34, 0xeb, + 0x52, 0xec, 0xcd, 0xaf, 0x40, 0x2e, 0x98, 0x1e, 0x23, 0x09, 0x72, 0xdf, 0x6d, 0x35, 0xeb, 0xca, + 0x4e, 0xa5, 0xb1, 0x7f, 0x2c, 0x93, 0x19, 0x20, 0x28, 0x70, 0xf7, 0x24, 0x60, 0x91, 0xed, 0xf5, + 0x5f, 0xfc, 0xf7, 0xca, 0xc2, 0x2f, 0x2e, 0x56, 0x22, 0xbf, 0xbc, 0x58, 0x89, 0xfc, 0xea, 0x62, + 0x25, 0xf2, 0x5f, 0x17, 0x2b, 0x91, 0xbf, 0xfe, 0x74, 0x65, 0xe1, 0x97, 0x9f, 0xae, 0x2c, 0xfc, + 0xea, 0xd3, 0x95, 0x85, 0xef, 0x26, 0xd9, 0x3f, 0x12, 0xf1, 0x7f, 0x01, 0x00, 0x00, 0xff, 0xff, + 0x12, 0xab, 0xa1, 0x05, 0x8f, 0x42, 0x00, 0x00, } diff --git a/pkg/sql/catalog/descpb/structured.proto b/pkg/sql/catalog/descpb/structured.proto index 0d3f6bbd2e99..7d392c22a46c 100644 --- a/pkg/sql/catalog/descpb/structured.proto +++ b/pkg/sql/catalog/descpb/structured.proto @@ -580,7 +580,8 @@ message MaterializedViewRefresh { // NewIndexes are the new set of indexes to backfill the view into. repeated IndexDescriptor new_indexes = 2 [(gogoproto.nullable) = false]; // AsOf is the timestamp to perform the view query at. - optional util.hlc.Timestamp as_of = 3 [(gogoproto.nullable) = false]; + optional util.hlc.Timestamp as_of = 3 [(gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp"]; // ShouldBackfill indicates whether or not the schema changer should backfill // the query into the new indexes. This can be false if the `WITH NO DATA` flag // was specified for the `REFRESH MATERIALIZED VIEW` statement. `WITH NO DATA` @@ -804,7 +805,8 @@ message TableDescriptor { // should live inside of a Descriptor. The Descriptor.Table() method takes an // hlc timestamp to ensure that this field is set properly when extracted from // a Descriptor. - optional util.hlc.Timestamp modification_time = 7 [(gogoproto.nullable) = false]; + optional util.hlc.Timestamp modification_time = 7 [(gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp"]; // A list of draining names. The draining name entries are drained from // the cluster wide name caches by incrementing the version for this // descriptor and ensuring that there are no leases on prior @@ -1006,7 +1008,8 @@ message TableDescriptor { // transaction which created this replacement. In 20.1 and after it is // populated with the read timestamp at which the descriptor being // replaced was read. - optional util.hlc.Timestamp time = 2 [(gogoproto.nullable) = false]; + optional util.hlc.Timestamp time = 2 [(gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp"]; } // ReplacementOf tracks prior IDs by which this table went -- e.g. when @@ -1059,7 +1062,8 @@ message TableDescriptor { // like ModificationTime. See Descriptor.Table(). // CreateAsOfSystemTime is used for CREATE TABLE ... AS ... and was // added in 19.1. - optional util.hlc.Timestamp create_as_of_time = 35 [(gogoproto.nullable) = false]; + optional util.hlc.Timestamp create_as_of_time = 35 [(gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp"]; // outbound_fks contains all foreign key constraints that have this table as // the origin table. @@ -1123,7 +1127,8 @@ message DatabaseDescriptor { optional uint32 id = 2 [(gogoproto.nullable) = false, (gogoproto.customname) = "ID", (gogoproto.casttype) = "ID"]; // Last modification time of the descriptor. - optional util.hlc.Timestamp modification_time = 4 [(gogoproto.nullable) = false]; + optional util.hlc.Timestamp modification_time = 4 [(gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp"]; optional uint32 version = 5 [(gogoproto.nullable) = false, (gogoproto.casttype) = "DescriptorVersion"]; repeated NameInfo draining_names = 6 [(gogoproto.nullable) = false]; @@ -1183,7 +1188,8 @@ message TypeDescriptor { optional uint32 version = 9 [(gogoproto.nullable) = false, (gogoproto.casttype) = "DescriptorVersion"]; // Last modification time of the descriptor. - optional util.hlc.Timestamp modification_time = 10 [(gogoproto.nullable) = false]; + optional util.hlc.Timestamp modification_time = 10 [(gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp"]; repeated NameInfo draining_names = 11 [(gogoproto.nullable) = false]; // privileges contains the privileges for the type. @@ -1289,7 +1295,8 @@ message SchemaDescriptor { optional string offline_reason = 9 [(gogoproto.nullable) = false]; // Last modification time of the descriptor. - optional util.hlc.Timestamp modification_time = 5 [(gogoproto.nullable) = false]; + optional util.hlc.Timestamp modification_time = 5 [(gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp"]; optional uint32 version = 6 [(gogoproto.nullable) = false, (gogoproto.casttype) = "DescriptorVersion"]; repeated NameInfo draining_names = 7 [(gogoproto.nullable) = false]; diff --git a/pkg/sql/catalog/descriptor.go b/pkg/sql/catalog/descriptor.go index 2e18c57ad180..58e632b27023 100644 --- a/pkg/sql/catalog/descriptor.go +++ b/pkg/sql/catalog/descriptor.go @@ -18,8 +18,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util" - "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/errors" "github.com/cockroachdb/redact" ) @@ -53,7 +53,7 @@ type Descriptor interface { // Metadata for descriptor leasing. GetVersion() descpb.DescriptorVersion - GetModificationTime() hlc.Timestamp + GetModificationTime() enginepb.TxnTimestamp GetDrainingNames() []descpb.NameInfo GetPrivileges() *descpb.PrivilegeDescriptor diff --git a/pkg/sql/catalog/descs/txn.go b/pkg/sql/catalog/descs/txn.go index a0102c12c949..ed76c0057904 100644 --- a/pkg/sql/catalog/descs/txn.go +++ b/pkg/sql/catalog/descs/txn.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/catalog/lease" "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/retry" "github.com/cockroachdb/errors" @@ -159,7 +160,7 @@ func CheckTwoVersionInvariant( // Wait until all older version leases have been released or expired. for r := retry.StartWithCtx(ctx, base.DefaultRetryOptions()); r.Next(); { // Use the current clock time. - now := clock.Now() + now := enginepb.TxnTimestamp(clock.Now()) count, err := lease.CountLeases(ctx, ie, descs, now) if err != nil { return false, err diff --git a/pkg/sql/catalog/lease/lease.go b/pkg/sql/catalog/lease/lease.go index 8a440dce70ec..ffda16c82581 100644 --- a/pkg/sql/catalog/lease/lease.go +++ b/pkg/sql/catalog/lease/lease.go @@ -40,6 +40,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/log/logcrash" @@ -81,7 +82,7 @@ type descriptorVersionState struct { // The expiration time is either the expiration time of the lease when a lease // is associated with the version, or the ModificationTime of the next version // when the version isn't associated with a lease. - expiration hlc.Timestamp + expiration enginepb.TxnTimestamp mu struct { syncutil.Mutex @@ -110,7 +111,7 @@ func (s *descriptorVersionState) stringLocked() string { // hasExpired checks if the descriptor is too old to be used (by a txn // operating) at the given timestamp. -func (s *descriptorVersionState) hasExpired(timestamp hlc.Timestamp) bool { +func (s *descriptorVersionState) hasExpired(timestamp enginepb.TxnTimestamp) bool { return s.expiration.LessEq(timestamp) } @@ -137,9 +138,9 @@ func (s *descriptorVersionState) incRefcountLocked() { // The lease expiration stored in the database is of a different type. // We've decided that it's too much work to change the type to -// hlc.Timestamp, so we're using this method to give us the stored +// enginepb.TxnTimestamp, so we're using this method to give us the stored // type: tree.DTimestamp. -func storedLeaseExpiration(expiration hlc.Timestamp) tree.DTimestamp { +func storedLeaseExpiration(expiration enginepb.TxnTimestamp) tree.DTimestamp { return tree.DTimestamp{Time: timeutil.Unix(0, expiration.WallTime).Round(time.Microsecond)} } @@ -189,7 +190,7 @@ func (s storage) jitteredLeaseDuration() time.Duration { // or offline (currently only applicable to tables), the error will be of type // inactiveTableError. The expiration time set for the lease > minExpiration. func (s storage) acquire( - ctx context.Context, minExpiration hlc.Timestamp, id descpb.ID, + ctx context.Context, minExpiration enginepb.TxnTimestamp, id descpb.ID, ) (*descriptorVersionState, error) { var descVersionState *descriptorVersionState err := s.db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { @@ -334,7 +335,7 @@ func (m *Manager) WaitForOneVersion( // Check to see if there are any leases that still exist on the previous // version of the descriptor. - now := m.storage.clock.Now() + now := enginepb.TxnTimestamp(m.storage.clock.Now()) descs := []IDVersion{NewIDVersionPrev(desc.GetName(), desc.GetID(), desc.GetVersion())} version = desc.GetVersion() count, err := CountLeases(ctx, m.storage.internalExecutor, descs, now) @@ -370,7 +371,7 @@ func NewIDVersionPrev(name string, id descpb.ID, currVersion descpb.DescriptorVe // CountLeases returns the number of unexpired leases for a number of descriptors // each at a particular version at a particular time. func CountLeases( - ctx context.Context, executor sqlutil.InternalExecutor, versions []IDVersion, at hlc.Timestamp, + ctx context.Context, executor sqlutil.InternalExecutor, versions []IDVersion, at enginepb.TxnTimestamp, ) (int, error) { var whereClauses []string for _, t := range versions { @@ -404,7 +405,7 @@ func CountLeases( // returns an error when the expiration timestamp is less than the storage // layer GC threshold. func (s storage) getForExpiration( - ctx context.Context, expiration hlc.Timestamp, id descpb.ID, + ctx context.Context, expiration enginepb.TxnTimestamp, id descpb.ID, ) (*descriptorVersionState, error) { var descVersionState *descriptorVersionState err := s.db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { @@ -607,7 +608,7 @@ func ensureVersion( // It returns true if the descriptor returned is the known latest version // of the descriptor. func (t *descriptorState) findForTimestamp( - ctx context.Context, timestamp hlc.Timestamp, + ctx context.Context, timestamp enginepb.TxnTimestamp, ) (*descriptorVersionState, bool, error) { t.mu.Lock() defer t.mu.Unlock() @@ -652,9 +653,9 @@ func (t *descriptorState) findForTimestamp( // 3. Figure out a sane policy on when these descriptors should be purged. // They are currently purged in PurgeOldVersions. func (m *Manager) readOlderVersionForTimestamp( - ctx context.Context, id descpb.ID, timestamp hlc.Timestamp, + ctx context.Context, id descpb.ID, timestamp enginepb.TxnTimestamp, ) ([]*descriptorVersionState, error) { - expiration, done := func() (hlc.Timestamp, bool) { + expiration, done := func() (enginepb.TxnTimestamp, bool) { t := m.findDescriptorState(id, false /* create */) t.mu.Lock() defer t.mu.Unlock() @@ -679,7 +680,7 @@ func (m *Manager) readOlderVersionForTimestamp( } if afterIdx == len(t.mu.active.data) { - return hlc.Timestamp{}, true + return enginepb.TxnTimestamp{}, true } // Read descriptor versions one by one into the past until we @@ -855,7 +856,7 @@ func acquireNodeLease(ctx context.Context, m *Manager, id descpb.ID) (bool, erro return nil, errors.New("cannot acquire lease when draining") } newest := m.findNewest(id) - var minExpiration hlc.Timestamp + var minExpiration enginepb.TxnTimestamp if newest != nil { minExpiration = newest.expiration } @@ -1006,7 +1007,7 @@ func purgeOldVersions( // Acquire a refcount on the descriptor on the latest version to maintain an // active lease, so that it doesn't get released when removeInactives() // is called below. Release this lease after calling removeInactives(). - desc, _, err := t.findForTimestamp(ctx, m.storage.clock.Now()) + desc, _, err := t.findForTimestamp(ctx, enginepb.TxnTimestamp(m.storage.clock.Now())) if isInactive := catalog.HasInactiveDescriptorError(err); err == nil || isInactive { removeInactives(isInactive) if desc != nil { @@ -1171,7 +1172,7 @@ type nameCache struct { // The descriptor's refcount is incremented before returning, so the caller // is responsible for releasing it to the leaseManager. func (c *nameCache) get( - parentID descpb.ID, parentSchemaID descpb.ID, name string, timestamp hlc.Timestamp, + parentID descpb.ID, parentSchemaID descpb.ID, name string, timestamp enginepb.TxnTimestamp, ) *descriptorVersionState { c.mu.Lock() desc, ok := c.descriptors[makeNameCacheKey(parentID, parentSchemaID, name)] @@ -1272,7 +1273,7 @@ type Manager struct { // updatesResolvedTimestamp keeps track of a timestamp before which all // descriptor updates have already been seen. - updatesResolvedTimestamp hlc.Timestamp + updatesResolvedTimestamp enginepb.TxnTimestamp } draining atomic.Value @@ -1332,7 +1333,7 @@ func NewLeaseManager( } lm.stopper.AddCloser(lm.sem.Closer("stopper")) lm.mu.descriptors = make(map[descpb.ID]*descriptorState) - lm.mu.updatesResolvedTimestamp = db.Clock().Now() + lm.mu.updatesResolvedTimestamp = enginepb.TxnTimestamp(db.Clock().Now()) lm.draining.Store(false) return lm @@ -1376,11 +1377,11 @@ func (m *Manager) findNewest(id descpb.ID) *descriptorVersionState { // id and fails because the id has been dropped by the TRUNCATE. func (m *Manager) AcquireByName( ctx context.Context, - timestamp hlc.Timestamp, + timestamp enginepb.TxnTimestamp, parentID descpb.ID, parentSchemaID descpb.ID, name string, -) (catalog.Descriptor, hlc.Timestamp, error) { +) (catalog.Descriptor, enginepb.TxnTimestamp, error) { // Check if we have cached an ID for this name. descVersion := m.names.get(parentID, parentSchemaID, name, timestamp) if descVersion != nil { @@ -1391,19 +1392,19 @@ func (m *Manager) AcquireByName( if t := m.findDescriptorState(descVersion.GetID(), false /* create */); t != nil { if err := t.maybeQueueLeaseRenewal( ctx, m, descVersion.GetID(), name); err != nil { - return nil, hlc.Timestamp{}, err + return nil, enginepb.TxnTimestamp{}, err } } } return descVersion.Descriptor, descVersion.expiration, nil } if err := m.Release(descVersion); err != nil { - return nil, hlc.Timestamp{}, err + return nil, enginepb.TxnTimestamp{}, err } // Return a valid descriptor for the timestamp. desc, expiration, err := m.Acquire(ctx, timestamp, descVersion.GetID()) if err != nil { - return nil, hlc.Timestamp{}, err + return nil, enginepb.TxnTimestamp{}, err } return desc, expiration, nil } @@ -1415,11 +1416,11 @@ func (m *Manager) AcquireByName( var err error id, err := m.resolveName(ctx, timestamp, parentID, parentSchemaID, name) if err != nil { - return nil, hlc.Timestamp{}, err + return nil, enginepb.TxnTimestamp{}, err } desc, expiration, err := m.Acquire(ctx, timestamp, id) if err != nil { - return nil, hlc.Timestamp{}, err + return nil, enginepb.TxnTimestamp{}, err } if !NameMatchesDescriptor(desc, parentID, parentSchemaID, name) { // We resolved name `name`, but the lease has a different name in it. @@ -1459,11 +1460,11 @@ func (m *Manager) AcquireByName( log.Warningf(ctx, "error releasing lease: %s", err) } if err := m.AcquireFreshestFromStore(ctx, id); err != nil { - return nil, hlc.Timestamp{}, err + return nil, enginepb.TxnTimestamp{}, err } desc, expiration, err = m.Acquire(ctx, timestamp, id) if err != nil { - return nil, hlc.Timestamp{}, err + return nil, enginepb.TxnTimestamp{}, err } if !NameMatchesDescriptor(desc, parentID, parentSchemaID, name) { // If the name we had doesn't match the newest descriptor in the DB, then @@ -1471,7 +1472,7 @@ func (m *Manager) AcquireByName( if err := m.Release(desc); err != nil { log.Warningf(ctx, "error releasing lease: %s", err) } - return nil, hlc.Timestamp{}, catalog.ErrDescriptorNotFound + return nil, enginepb.TxnTimestamp{}, catalog.ErrDescriptorNotFound } } return desc, expiration, nil @@ -1482,7 +1483,7 @@ func (m *Manager) AcquireByName( // catalog.ErrDescriptorNotFound is returned. func (m *Manager) resolveName( ctx context.Context, - timestamp hlc.Timestamp, + timestamp enginepb.TxnTimestamp, parentID descpb.ID, parentSchemaID descpb.ID, name string, @@ -1529,8 +1530,8 @@ func (m *Manager) resolveName( // can only return an older version of a descriptor if the latest version // can be leased; as it stands a dropped descriptor cannot be leased. func (m *Manager) Acquire( - ctx context.Context, timestamp hlc.Timestamp, id descpb.ID, -) (catalog.Descriptor, hlc.Timestamp, error) { + ctx context.Context, timestamp enginepb.TxnTimestamp, id descpb.ID, +) (catalog.Descriptor, enginepb.TxnTimestamp, error) { for { t := m.findDescriptorState(id, true /*create*/) desc, latest, err := t.findForTimestamp(ctx, timestamp) @@ -1540,7 +1541,7 @@ func (m *Manager) Acquire( durationUntilExpiry := time.Duration(desc.expiration.WallTime - timestamp.WallTime) if durationUntilExpiry < m.storage.leaseRenewalTimeout { if err := t.maybeQueueLeaseRenewal(ctx, m, id, desc.GetName()); err != nil { - return nil, hlc.Timestamp{}, err + return nil, enginepb.TxnTimestamp{}, err } } } @@ -1555,7 +1556,7 @@ func (m *Manager) Acquire( _, errLease := acquireNodeLease(ctx, m, id) return errLease }(); err != nil { - return nil, hlc.Timestamp{}, err + return nil, enginepb.TxnTimestamp{}, err } if m.testingKnobs.LeaseStoreTestingKnobs.LeaseAcquireResultBlockEvent != nil { @@ -1566,12 +1567,12 @@ func (m *Manager) Acquire( // Read old versions from the store. This can block while reading. versions, errRead := m.readOlderVersionForTimestamp(ctx, id, timestamp) if errRead != nil { - return nil, hlc.Timestamp{}, errRead + return nil, enginepb.TxnTimestamp{}, errRead } m.insertDescriptorVersions(id, versions) default: - return nil, hlc.Timestamp{}, err + return nil, enginepb.TxnTimestamp{}, err } } } @@ -1905,7 +1906,7 @@ func (m *Manager) handleUpdatedSystemCfg( if log.V(2) { log.Info(ctx, "received a new config; will refresh leases") } - var latestTimestamp hlc.Timestamp + var latestTimestamp enginepb.TxnTimestamp cfgFilter.ForModified(cfg, func(kv roachpb.KeyValue) { // Attempt to unmarshal config into a descriptor. var descriptor descpb.Descriptor @@ -1980,7 +1981,7 @@ func (m *Manager) waitForRangefeedsToBeUsable(ctx context.Context, s *stop.Stopp // up to this timestamp. It is set under the gossip path based on the highest // timestamp seen in a system config and under the rangefeed path when a // resolved timestamp is received. -func (m *Manager) setResolvedTimestamp(ts hlc.Timestamp) { +func (m *Manager) setResolvedTimestamp(ts enginepb.TxnTimestamp) { m.mu.Lock() defer m.mu.Unlock() if m.mu.updatesResolvedTimestamp.Less(ts) { @@ -1988,7 +1989,7 @@ func (m *Manager) setResolvedTimestamp(ts hlc.Timestamp) { } } -func (m *Manager) getResolvedTimestamp() hlc.Timestamp { +func (m *Manager) getResolvedTimestamp() enginepb.TxnTimestamp { m.mu.Lock() defer m.mu.Unlock() return m.mu.updatesResolvedTimestamp @@ -2185,15 +2186,15 @@ func (m *Manager) VisitLeases( // This method is useful for testing and is only intended to be used in that // context. func (m *Manager) TestingAcquireAndAssertMinVersion( - ctx context.Context, timestamp hlc.Timestamp, id descpb.ID, minVersion descpb.DescriptorVersion, -) (catalog.Descriptor, hlc.Timestamp, error) { + ctx context.Context, timestamp enginepb.TxnTimestamp, id descpb.ID, minVersion descpb.DescriptorVersion, +) (catalog.Descriptor, enginepb.TxnTimestamp, error) { t := m.findDescriptorState(id, true) if err := ensureVersion(ctx, id, minVersion, m); err != nil { - return nil, hlc.Timestamp{}, err + return nil, enginepb.TxnTimestamp{}, err } desc, _, err := t.findForTimestamp(ctx, timestamp) if err != nil { - return nil, hlc.Timestamp{}, err + return nil, enginepb.TxnTimestamp{}, err } return desc.Descriptor, desc.expiration, nil } diff --git a/pkg/sql/catalog/schemadesc/schema_desc.go b/pkg/sql/catalog/schemadesc/schema_desc.go index 8f2154f1db20..e67b4318fa64 100644 --- a/pkg/sql/catalog/schemadesc/schema_desc.go +++ b/pkg/sql/catalog/schemadesc/schema_desc.go @@ -19,7 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" - "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/errors" "github.com/cockroachdb/redact" @@ -175,7 +175,7 @@ func (desc *Mutable) MaybeIncrementVersion() { return } desc.Version++ - desc.ModificationTime = hlc.Timestamp{} + desc.ModificationTime = enginepb.TxnTimestamp{} } // OriginalName implements the MutableDescriptor interface. diff --git a/pkg/sql/catalog/tabledesc/structured.go b/pkg/sql/catalog/tabledesc/structured.go index f96e9e003330..046dd5608a34 100644 --- a/pkg/sql/catalog/tabledesc/structured.go +++ b/pkg/sql/catalog/tabledesc/structured.go @@ -32,9 +32,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" - "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/interval" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/errors" @@ -1440,7 +1440,7 @@ func (desc *Mutable) MaybeIncrementVersion() { // Starting in 19.2 we use a zero-valued ModificationTime when incrementing // the version, and then, upon reading, use the MVCC timestamp to populate // the ModificationTime. - desc.ModificationTime = hlc.Timestamp{} + desc.ModificationTime = enginepb.TxnTimestamp{} } // OriginalName implements the MutableDescriptor interface. diff --git a/pkg/sql/catalog/tabledesc/table.go b/pkg/sql/catalog/tabledesc/table.go index a2e024a31508..3c97889a4e46 100644 --- a/pkg/sql/catalog/tabledesc/table.go +++ b/pkg/sql/catalog/tabledesc/table.go @@ -24,7 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" - "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" ) @@ -424,7 +424,7 @@ func FindFKOriginIndexInTxn( func InitTableDescriptor( id, parentID, parentSchemaID descpb.ID, name string, - creationTime hlc.Timestamp, + creationTime enginepb.TxnTimestamp, privileges *descpb.PrivilegeDescriptor, persistence tree.Persistence, ) Mutable { diff --git a/pkg/sql/catalog/typedesc/type_desc.go b/pkg/sql/catalog/typedesc/type_desc.go index 9d065e954f2d..02d32f04b1ae 100644 --- a/pkg/sql/catalog/typedesc/type_desc.go +++ b/pkg/sql/catalog/typedesc/type_desc.go @@ -27,7 +27,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/privilege" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" - "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/errors" "github.com/lib/pq/oid" @@ -232,7 +232,7 @@ func (desc *Mutable) MaybeIncrementVersion() { return } desc.Version++ - desc.ModificationTime = hlc.Timestamp{} + desc.ModificationTime = enginepb.TxnTimestamp{} } // OriginalName implements the MutableDescriptor interface. diff --git a/pkg/sql/colfetcher/cfetcher.go b/pkg/sql/colfetcher/cfetcher.go index b8388783ce72..cf181fd78d63 100644 --- a/pkg/sql/colfetcher/cfetcher.go +++ b/pkg/sql/colfetcher/cfetcher.go @@ -36,9 +36,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/scrub" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/encoding" - "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" ) @@ -107,7 +107,7 @@ type cTableInfo struct { // // rowLastModified is the timestamp of the last time any family in the row // was modified in any way. - rowLastModified hlc.Timestamp + rowLastModified enginepb.TxnTimestamp // timestampOutputIdx controls at what row ordinal to write the timestamp. timestampOutputIdx int @@ -781,7 +781,7 @@ func (rf *cFetcher) nextBatch(ctx context.Context) (coldata.Batch, error) { rf.shiftState() case stateDecodeFirstKVOfRow: // Reset MVCC metadata for the table, since this is the first KV of a row. - rf.table.rowLastModified = hlc.Timestamp{} + rf.table.rowLastModified = enginepb.TxnTimestamp{} // foundNull is set when decoding a new index key for a row finds a NULL value // in the index key. This is used when decoding unique secondary indexes in order @@ -999,7 +999,7 @@ func (rf *cFetcher) nextBatch(ctx context.Context) (coldata.Batch, error) { case stateFinalizeRow: // Populate any system columns in the output. if rf.table.timestampOutputIdx != noOutputColumn { - rf.machine.timestampCol[rf.machine.rowIdx] = tree.TimestampToDecimal(rf.table.rowLastModified) + rf.machine.timestampCol[rf.machine.rowIdx] = tree.TimestampToDecimal(rf.table.rowLastModified.ToClockTimestampUnchecked()) } if rf.table.oidOutputIdx != noOutputColumn { rf.machine.tableoidCol.Set(rf.machine.rowIdx, tree.NewDOid(tree.DInt(rf.table.desc.GetID()))) diff --git a/pkg/sql/conn_executor.go b/pkg/sql/conn_executor.go index 0fbfd3b26a47..2d4936ef11d2 100644 --- a/pkg/sql/conn_executor.go +++ b/pkg/sql/conn_executor.go @@ -41,6 +41,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" "github.com/cockroachdb/cockroach/pkg/sql/stmtdiagnostics" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/cancelchecker" "github.com/cockroachdb/cockroach/pkg/util/envutil" @@ -2024,7 +2025,7 @@ func (ex *connExecutor) setTransactionModes( return errors.AssertionFailedf("expected an evaluated AS OF timestamp") } if !asOfTs.IsEmpty() { - ex.state.setHistoricalTimestamp(ex.Ctx(), asOfTs) + ex.state.setHistoricalTimestamp(ex.Ctx(), enginepb.TxnTimestamp(asOfTs)) ex.state.sqlTimestamp = asOfTs.GoTime() if rwMode == tree.UnspecifiedReadWriteMode { rwMode = tree.ReadOnly diff --git a/pkg/sql/conn_executor_exec.go b/pkg/sql/conn_executor_exec.go index 9ec55be2ad85..ef7e297c6196 100644 --- a/pkg/sql/conn_executor_exec.go +++ b/pkg/sql/conn_executor_exec.go @@ -33,6 +33,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/cancelchecker" "github.com/cockroachdb/cockroach/pkg/util/duration" @@ -547,7 +548,7 @@ func (ex *connExecutor) execStmtInOpenState( if asOfTs != nil { p.semaCtx.AsOfTimestamp = asOfTs p.extendedEvalCtx.SetTxnTimestamp(asOfTs.GoTime()) - ex.state.setHistoricalTimestamp(ctx, *asOfTs) + ex.state.setHistoricalTimestamp(ctx, enginepb.TxnTimestamp(*asOfTs)) } } else { // If we're in an explicit txn, we allow AOST but only if it matches with @@ -559,7 +560,7 @@ func (ex *connExecutor) execStmtInOpenState( return makeErrEvent(err) } if ts != nil { - if readTs := ex.state.getReadTimestamp(); *ts != readTs { + if readTs := ex.state.getReadTimestamp(); enginepb.TxnTimestamp(*ts) != readTs { err = pgerror.Newf(pgcode.Syntax, "inconsistent AS OF SYSTEM TIME timestamp; expected: %s", readTs) err = errors.WithHint(err, "try SET TRANSACTION AS OF SYSTEM TIME") @@ -665,7 +666,8 @@ func (ex *connExecutor) execStmtInOpenState( IsCommit: fsm.FromBool(isCommit(ast)), CanAutoRetry: fsm.FromBool(canAutoRetry), } - txn.ManualRestart(ctx, ex.server.cfg.Clock.Now()) + now := ex.server.cfg.Clock.Now() + txn.ManualRestart(ctx, enginepb.TxnTimestamp(now)) payload := eventRetriableErrPayload{ err: roachpb.NewTransactionRetryWithProtoRefreshError( "serializable transaction timestamp pushed (detected by connExecutor)", diff --git a/pkg/sql/conn_executor_prepare.go b/pkg/sql/conn_executor_prepare.go index 75a6f93aef9f..e0d7ad931aa2 100644 --- a/pkg/sql/conn_executor_prepare.go +++ b/pkg/sql/conn_executor_prepare.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/querycache" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/fsm" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/timeutil" @@ -240,7 +241,7 @@ func (ex *connExecutor) populatePrepared( } if protoTS != nil { p.semaCtx.AsOfTimestamp = protoTS - txn.SetFixedTimestamp(ctx, *protoTS) + txn.SetFixedTimestamp(ctx, enginepb.TxnTimestamp(*protoTS)) } // PREPARE has a limited subset of statements it can be run with. Postgres diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index 65b85d008216..5b9a352dd123 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -344,8 +344,8 @@ CREATE TABLE crdb_internal.tables ( tree.NewDString(table.GetName()), dbName, tree.NewDInt(tree.DInt(int64(table.GetVersion()))), - tree.TimestampToInexactDTimestamp(table.GetModificationTime()), - tree.TimestampToDecimalDatum(table.GetModificationTime()), + tree.TimestampToInexactDTimestamp(table.GetModificationTime().ToClockTimestampUnchecked()), + tree.TimestampToDecimalDatum(table.GetModificationTime().ToClockTimestampUnchecked()), tree.NewDString(table.GetFormatVersion().String()), tree.NewDString(table.GetState().String()), leaseNodeDatum, diff --git a/pkg/sql/create_sequence.go b/pkg/sql/create_sequence.go index 96f63d8c65c1..5c3fa07306d3 100644 --- a/pkg/sql/create_sequence.go +++ b/pkg/sql/create_sequence.go @@ -25,7 +25,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/sql/types" - "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/log/eventpb" ) @@ -110,7 +110,7 @@ func doCreateSequence( // // TODO(ajwerner): remove the timestamp from NewSequenceTableDesc, it's // currently relied on in import and restore code and tests. - var creationTime hlc.Timestamp + var creationTime enginepb.TxnTimestamp desc, err := NewSequenceTableDesc( params.ctx, name.Object(), @@ -177,7 +177,7 @@ func NewSequenceTableDesc( parentID descpb.ID, schemaID descpb.ID, id descpb.ID, - creationTime hlc.Timestamp, + creationTime enginepb.TxnTimestamp, privileges *descpb.PrivilegeDescriptor, persistence tree.Persistence, params *runParams, diff --git a/pkg/sql/create_stats.go b/pkg/sql/create_stats.go index 8aa33600cd3b..9c164f247e3f 100644 --- a/pkg/sql/create_stats.go +++ b/pkg/sql/create_stats.go @@ -34,6 +34,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/sql/stats" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/grpcutil" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -499,7 +500,7 @@ func (r *createStatsResumer) Resume( if details.AsOf != nil { p.SemaCtx().AsOfTimestamp = details.AsOf p.ExtendedEvalContext().SetTxnTimestamp(details.AsOf.GoTime()) - txn.SetFixedTimestamp(ctx, *details.AsOf) + txn.SetFixedTimestamp(ctx, enginepb.TxnTimestamp(*details.AsOf)) } planCtx := dsp.NewPlanningCtx(ctx, evalCtx, nil /* planner */, txn, true /* distribute */) diff --git a/pkg/sql/create_table.go b/pkg/sql/create_table.go index aaef174e26fa..d43f0fe5bac3 100644 --- a/pkg/sql/create_table.go +++ b/pkg/sql/create_table.go @@ -45,8 +45,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" - "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log/eventpb" "github.com/cockroachdb/errors" "github.com/lib/pq/oid" @@ -282,7 +282,7 @@ func (n *createTableNode) startExec(params runParams) error { // // TODO(ajwerner): remove the timestamp from newTableDesc and its friends, // it's currently relied on in import and restore code and tests. - var creationTime hlc.Timestamp + var creationTime enginepb.TxnTimestamp if n.n.As() { asCols = planColumns(n.sourcePlan) if !n.run.fromHeuristicPlanner && !n.n.AsHasUserSpecifiedPrimaryKey() { @@ -1225,7 +1225,7 @@ func newTableDescIfAs( params runParams, p *tree.CreateTable, parentID, parentSchemaID, id descpb.ID, - creationTime hlc.Timestamp, + creationTime enginepb.TxnTimestamp, resultColumns []colinfo.ResultColumn, privileges *descpb.PrivilegeDescriptor, evalContext *tree.EvalContext, @@ -1305,7 +1305,7 @@ func NewTableDesc( st *cluster.Settings, n *tree.CreateTable, parentID, parentSchemaID, id descpb.ID, - creationTime hlc.Timestamp, + creationTime enginepb.TxnTimestamp, privileges *descpb.PrivilegeDescriptor, affected map[descpb.ID]*tabledesc.Mutable, semaCtx *tree.SemaContext, @@ -1939,7 +1939,7 @@ func newTableDesc( params runParams, n *tree.CreateTable, parentID, parentSchemaID, id descpb.ID, - creationTime hlc.Timestamp, + creationTime enginepb.TxnTimestamp, privileges *descpb.PrivilegeDescriptor, affected map[descpb.ID]*tabledesc.Mutable, ) (ret *tabledesc.Mutable, err error) { diff --git a/pkg/sql/create_view.go b/pkg/sql/create_view.go index 6663899402ec..b52bc03c64b3 100644 --- a/pkg/sql/create_view.go +++ b/pkg/sql/create_view.go @@ -29,7 +29,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" - "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/log/eventpb" ) @@ -160,7 +160,7 @@ func (n *createViewNode) startExec(params runParams) error { // // TODO(ajwerner): remove the timestamp from MakeViewTableDesc, it's // currently relied on in import and restore code and tests. - var creationTime hlc.Timestamp + var creationTime enginepb.TxnTimestamp desc, err := makeViewTableDesc( params.ctx, viewName, @@ -285,7 +285,7 @@ func makeViewTableDesc( schemaID descpb.ID, id descpb.ID, resultColumns []colinfo.ResultColumn, - creationTime hlc.Timestamp, + creationTime enginepb.TxnTimestamp, privileges *descpb.PrivilegeDescriptor, semaCtx *tree.SemaContext, evalCtx *tree.EvalContext, diff --git a/pkg/sql/distsql_plan_backfill.go b/pkg/sql/distsql_plan_backfill.go index 18619b153d14..95754a89f7e4 100644 --- a/pkg/sql/distsql_plan_backfill.go +++ b/pkg/sql/distsql_plan_backfill.go @@ -18,7 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/physicalplan" "github.com/cockroachdb/cockroach/pkg/sql/types" - "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/errors" ) @@ -27,7 +27,7 @@ func initBackfillerSpec( desc descpb.TableDescriptor, duration time.Duration, chunkSize int64, - readAsOf hlc.Timestamp, + readAsOf enginepb.TxnTimestamp, ) (execinfrapb.BackfillerSpec, error) { ret := execinfrapb.BackfillerSpec{ Table: desc, @@ -56,7 +56,7 @@ func (dsp *DistSQLPlanner) createBackfiller( duration time.Duration, chunkSize int64, spans []roachpb.Span, - readAsOf hlc.Timestamp, + readAsOf enginepb.TxnTimestamp, ) (*PhysicalPlan, error) { spec, err := initBackfillerSpec(backfillType, desc, duration, chunkSize, readAsOf) if err != nil { diff --git a/pkg/sql/doctor/doctor.go b/pkg/sql/doctor/doctor.go index 2dcfc761397b..96a08e2ae580 100644 --- a/pkg/sql/doctor/doctor.go +++ b/pkg/sql/doctor/doctor.go @@ -25,7 +25,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/typedesc" - "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/errors" @@ -36,7 +36,7 @@ import ( type DescriptorTableRow struct { ID int64 DescBytes []byte - ModTime hlc.Timestamp + ModTime enginepb.TxnTimestamp } // DescriptorTable represents data read from `system.descriptor`. diff --git a/pkg/sql/execinfrapb/processors_bulk_io.pb.go b/pkg/sql/execinfrapb/processors_bulk_io.pb.go index be5d0a09a07b..bd02da1f1dba 100644 --- a/pkg/sql/execinfrapb/processors_bulk_io.pb.go +++ b/pkg/sql/execinfrapb/processors_bulk_io.pb.go @@ -17,6 +17,7 @@ import descpb "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" import hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" import time "time" +import github_com_cockroachdb_cockroach_pkg_storage_enginepb "github.com/cockroachdb/cockroach/pkg/storage/enginepb" import github_com_cockroachdb_cockroach_pkg_security "github.com/cockroachdb/cockroach/pkg/security" import github_com_gogo_protobuf_sortkeys "github.com/gogo/protobuf/sortkeys" @@ -71,7 +72,7 @@ func (x *FileCompression) UnmarshalJSON(data []byte) error { return nil } func (FileCompression) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_e62f5714e36bbbd6, []int{0} + return fileDescriptor_processors_bulk_io_496c86f167f7c0c4, []int{0} } type BackfillerSpec_Type int32 @@ -110,7 +111,7 @@ func (x *BackfillerSpec_Type) UnmarshalJSON(data []byte) error { return nil } func (BackfillerSpec_Type) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_e62f5714e36bbbd6, []int{0, 0} + return fileDescriptor_processors_bulk_io_496c86f167f7c0c4, []int{0, 0} } // BackfillerSpec is the specification for a "schema change backfiller". @@ -135,14 +136,14 @@ type BackfillerSpec struct { // of entries backfilled per chunk. ChunkSize int64 `protobuf:"varint,5,opt,name=chunk_size,json=chunkSize" json:"chunk_size"` // The timestamp to perform index backfill historical scans at. - ReadAsOf hlc.Timestamp `protobuf:"bytes,7,opt,name=readAsOf" json:"readAsOf"` + ReadAsOf github_com_cockroachdb_cockroach_pkg_storage_enginepb.TxnTimestamp `protobuf:"bytes,7,opt,name=readAsOf,casttype=github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp" json:"readAsOf"` } func (m *BackfillerSpec) Reset() { *m = BackfillerSpec{} } func (m *BackfillerSpec) String() string { return proto.CompactTextString(m) } func (*BackfillerSpec) ProtoMessage() {} func (*BackfillerSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_e62f5714e36bbbd6, []int{0} + return fileDescriptor_processors_bulk_io_496c86f167f7c0c4, []int{0} } func (m *BackfillerSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -182,7 +183,7 @@ func (m *JobProgress) Reset() { *m = JobProgress{} } func (m *JobProgress) String() string { return proto.CompactTextString(m) } func (*JobProgress) ProtoMessage() {} func (*JobProgress) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_e62f5714e36bbbd6, []int{1} + return fileDescriptor_processors_bulk_io_496c86f167f7c0c4, []int{1} } func (m *JobProgress) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -245,7 +246,7 @@ func (m *ReadImportDataSpec) Reset() { *m = ReadImportDataSpec{} } func (m *ReadImportDataSpec) String() string { return proto.CompactTextString(m) } func (*ReadImportDataSpec) ProtoMessage() {} func (*ReadImportDataSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_e62f5714e36bbbd6, []int{2} + return fileDescriptor_processors_bulk_io_496c86f167f7c0c4, []int{2} } func (m *ReadImportDataSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -283,7 +284,7 @@ func (m *ReadImportDataSpec_ImportTable) Reset() { *m = ReadImportDataSp func (m *ReadImportDataSpec_ImportTable) String() string { return proto.CompactTextString(m) } func (*ReadImportDataSpec_ImportTable) ProtoMessage() {} func (*ReadImportDataSpec_ImportTable) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_e62f5714e36bbbd6, []int{2, 0} + return fileDescriptor_processors_bulk_io_496c86f167f7c0c4, []int{2, 0} } func (m *ReadImportDataSpec_ImportTable) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -329,7 +330,7 @@ func (m *BackupDataSpec) Reset() { *m = BackupDataSpec{} } func (m *BackupDataSpec) String() string { return proto.CompactTextString(m) } func (*BackupDataSpec) ProtoMessage() {} func (*BackupDataSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_e62f5714e36bbbd6, []int{3} + return fileDescriptor_processors_bulk_io_496c86f167f7c0c4, []int{3} } func (m *BackupDataSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -368,7 +369,7 @@ func (m *RestoreSpanEntry) Reset() { *m = RestoreSpanEntry{} } func (m *RestoreSpanEntry) String() string { return proto.CompactTextString(m) } func (*RestoreSpanEntry) ProtoMessage() {} func (*RestoreSpanEntry) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_e62f5714e36bbbd6, []int{4} + return fileDescriptor_processors_bulk_io_496c86f167f7c0c4, []int{4} } func (m *RestoreSpanEntry) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -406,7 +407,7 @@ func (m *RestoreDataSpec) Reset() { *m = RestoreDataSpec{} } func (m *RestoreDataSpec) String() string { return proto.CompactTextString(m) } func (*RestoreDataSpec) ProtoMessage() {} func (*RestoreDataSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_e62f5714e36bbbd6, []int{5} + return fileDescriptor_processors_bulk_io_496c86f167f7c0c4, []int{5} } func (m *RestoreDataSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -440,7 +441,7 @@ func (m *SplitAndScatterSpec) Reset() { *m = SplitAndScatterSpec{} } func (m *SplitAndScatterSpec) String() string { return proto.CompactTextString(m) } func (*SplitAndScatterSpec) ProtoMessage() {} func (*SplitAndScatterSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_e62f5714e36bbbd6, []int{6} + return fileDescriptor_processors_bulk_io_496c86f167f7c0c4, []int{6} } func (m *SplitAndScatterSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -473,7 +474,7 @@ func (m *SplitAndScatterSpec_RestoreEntryChunk) Reset() { *m = SplitAndS func (m *SplitAndScatterSpec_RestoreEntryChunk) String() string { return proto.CompactTextString(m) } func (*SplitAndScatterSpec_RestoreEntryChunk) ProtoMessage() {} func (*SplitAndScatterSpec_RestoreEntryChunk) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_e62f5714e36bbbd6, []int{6, 0} + return fileDescriptor_processors_bulk_io_496c86f167f7c0c4, []int{6, 0} } func (m *SplitAndScatterSpec_RestoreEntryChunk) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -520,7 +521,7 @@ func (m *CSVWriterSpec) Reset() { *m = CSVWriterSpec{} } func (m *CSVWriterSpec) String() string { return proto.CompactTextString(m) } func (*CSVWriterSpec) ProtoMessage() {} func (*CSVWriterSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_e62f5714e36bbbd6, []int{7} + return fileDescriptor_processors_bulk_io_496c86f167f7c0c4, []int{7} } func (m *CSVWriterSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -555,7 +556,7 @@ func (m *BulkRowWriterSpec) Reset() { *m = BulkRowWriterSpec{} } func (m *BulkRowWriterSpec) String() string { return proto.CompactTextString(m) } func (*BulkRowWriterSpec) ProtoMessage() {} func (*BulkRowWriterSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_processors_bulk_io_e62f5714e36bbbd6, []int{8} + return fileDescriptor_processors_bulk_io_496c86f167f7c0c4, []int{8} } func (m *BulkRowWriterSpec) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -3929,114 +3930,116 @@ var ( ) func init() { - proto.RegisterFile("sql/execinfrapb/processors_bulk_io.proto", fileDescriptor_processors_bulk_io_e62f5714e36bbbd6) + proto.RegisterFile("sql/execinfrapb/processors_bulk_io.proto", fileDescriptor_processors_bulk_io_496c86f167f7c0c4) } -var fileDescriptor_processors_bulk_io_e62f5714e36bbbd6 = []byte{ - // 1673 bytes of a gzipped FileDescriptorProto +var fileDescriptor_processors_bulk_io_496c86f167f7c0c4 = []byte{ + // 1699 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x58, 0x4f, 0x6f, 0x1b, 0xc7, - 0x15, 0xd7, 0x92, 0x4b, 0x8a, 0x7c, 0xb4, 0x24, 0x6a, 0xe2, 0xa4, 0x5b, 0x01, 0x95, 0x04, 0x26, - 0x72, 0x59, 0x17, 0x26, 0x11, 0xbb, 0x2d, 0x8c, 0xb6, 0x89, 0x6b, 0x52, 0x96, 0x43, 0x39, 0xb1, - 0xd5, 0xa5, 0xa5, 0x00, 0x41, 0x8b, 0xc5, 0x72, 0x77, 0x44, 0x8d, 0xb9, 0xdc, 0x59, 0xcd, 0xcc, - 0xca, 0xa6, 0x4f, 0x05, 0x7a, 0xea, 0xad, 0x1f, 0xa1, 0xdf, 0xa0, 0xfd, 0x0e, 0xbd, 0xf8, 0x98, - 0x63, 0x4e, 0x46, 0x2b, 0x7f, 0x8b, 0x9e, 0x8a, 0xf9, 0xb3, 0xd4, 0x4a, 0xa6, 0x64, 0x29, 0x86, - 0x2f, 0xf4, 0x7a, 0xde, 0xfc, 0x7e, 0xf3, 0xe6, 0xcd, 0x7b, 0xbf, 0x37, 0x23, 0x68, 0xf2, 0xc3, - 0xa8, 0x8d, 0x5f, 0xe0, 0x80, 0xc4, 0xfb, 0xcc, 0x4f, 0x06, 0xed, 0x84, 0xd1, 0x00, 0x73, 0x4e, - 0x19, 0xf7, 0x06, 0x69, 0x34, 0xf2, 0x08, 0x6d, 0x25, 0x8c, 0x0a, 0x8a, 0x9c, 0x80, 0x06, 0x23, - 0x46, 0xfd, 0xe0, 0xa0, 0xc5, 0x0f, 0xa3, 0x56, 0x48, 0xb8, 0xe0, 0x87, 0x11, 0x4b, 0xe3, 0x95, - 0x4f, 0x9e, 0xd1, 0x01, 0x6f, 0xcb, 0x9f, 0x64, 0xa0, 0xfe, 0xd1, 0x88, 0x15, 0x47, 0xcd, 0x4e, - 0x06, 0x6d, 0x42, 0x6f, 0xed, 0x53, 0x36, 0xf6, 0x45, 0x66, 0xf9, 0x54, 0xae, 0x1a, 0xf8, 0xc2, - 0x8f, 0xe8, 0xb0, 0x1d, 0x62, 0x1e, 0x24, 0x83, 0x36, 0x17, 0x2c, 0x0d, 0x44, 0xca, 0x70, 0x68, - 0x26, 0x6d, 0x5c, 0xe4, 0x9a, 0xcf, 0x71, 0xb6, 0x4a, 0x2a, 0x48, 0xd4, 0x3e, 0x88, 0x82, 0xb6, - 0x20, 0x63, 0xcc, 0x85, 0x3f, 0x4e, 0x8c, 0xe5, 0xfa, 0x90, 0x0e, 0xa9, 0xfa, 0x6c, 0xcb, 0x2f, - 0x33, 0x8a, 0x32, 0xaf, 0x42, 0x5f, 0xf8, 0x66, 0x6c, 0x39, 0x1b, 0xf3, 0x13, 0xa2, 0x87, 0x1a, - 0xff, 0x2c, 0xc2, 0x62, 0xc7, 0x0f, 0x46, 0xfb, 0x24, 0x8a, 0x30, 0xeb, 0x27, 0x38, 0x40, 0x0f, - 0xc1, 0x16, 0x93, 0x04, 0x3b, 0xd6, 0xba, 0xd5, 0x5c, 0xbc, 0x7d, 0xab, 0x75, 0x5e, 0x40, 0x5a, - 0xa7, 0x71, 0xad, 0xa7, 0x93, 0x04, 0x77, 0xec, 0x57, 0xaf, 0xd7, 0xe6, 0x5c, 0x45, 0x80, 0x3a, - 0x50, 0x12, 0xfe, 0x20, 0xc2, 0x4e, 0x61, 0xdd, 0x6a, 0xd6, 0x6e, 0xdf, 0x38, 0xc3, 0xc4, 0x0f, - 0x23, 0xb5, 0xbf, 0xa7, 0x72, 0xce, 0x26, 0xe6, 0x01, 0x23, 0x89, 0xa0, 0xcc, 0x50, 0x68, 0x28, - 0x7a, 0x00, 0x25, 0x9e, 0xf8, 0x31, 0x77, 0x8a, 0xeb, 0xc5, 0x66, 0xed, 0xf6, 0x2f, 0xce, 0xf7, - 0x46, 0xd1, 0xb8, 0xd8, 0x0f, 0xa5, 0x3b, 0x7e, 0x9c, 0xd1, 0x28, 0x34, 0xfa, 0x1c, 0x2a, 0x61, - 0xca, 0x7c, 0x41, 0x68, 0xec, 0xd8, 0xeb, 0x56, 0xb3, 0xd8, 0xf9, 0x58, 0x9a, 0xff, 0xf7, 0x7a, - 0x6d, 0x41, 0x86, 0xb3, 0xb5, 0x69, 0x8c, 0xee, 0x74, 0x1a, 0xfa, 0x14, 0x20, 0x38, 0x48, 0xe3, - 0x91, 0xc7, 0xc9, 0x4b, 0xec, 0x94, 0x14, 0x48, 0x73, 0x56, 0xd5, 0x78, 0x9f, 0xbc, 0xc4, 0xe8, - 0x1e, 0x54, 0x18, 0xf6, 0xc3, 0xfb, 0xfc, 0xc9, 0xbe, 0x33, 0xaf, 0x76, 0xf9, 0xb3, 0x9c, 0x87, - 0xf2, 0xc8, 0x5a, 0x07, 0x51, 0xd0, 0x7a, 0x9a, 0x1d, 0x99, 0x61, 0x98, 0x82, 0x1a, 0x37, 0xc1, - 0x96, 0x71, 0x43, 0x35, 0x98, 0xef, 0xc5, 0x47, 0x7e, 0x44, 0xc2, 0xfa, 0x1c, 0x02, 0x28, 0x77, - 0x69, 0x94, 0x8e, 0xe3, 0xba, 0x85, 0xaa, 0x50, 0xea, 0xc5, 0x21, 0x7e, 0x51, 0x2f, 0x6c, 0xdb, - 0x95, 0x72, 0x7d, 0xbe, 0xf1, 0x1c, 0x6a, 0xdb, 0x74, 0xb0, 0xc3, 0xe8, 0x90, 0x61, 0xce, 0xd1, - 0x67, 0x50, 0x7e, 0x46, 0x07, 0x1e, 0x09, 0xd5, 0x79, 0x15, 0x3b, 0x0b, 0x72, 0x81, 0xe3, 0xd7, - 0x6b, 0xa5, 0x6d, 0x3a, 0xe8, 0x6d, 0xba, 0xa5, 0x67, 0x74, 0xd0, 0x0b, 0x51, 0x13, 0xae, 0x05, - 0x34, 0x16, 0x8c, 0x0c, 0x52, 0x15, 0x03, 0x79, 0x22, 0x05, 0xe3, 0xcc, 0x29, 0x0b, 0x72, 0xc0, - 0xe6, 0x11, 0x15, 0x4e, 0x71, 0xdd, 0x6a, 0x96, 0xb2, 0xe3, 0x94, 0x23, 0x8d, 0x57, 0x15, 0x40, - 0x32, 0xbe, 0xbd, 0x71, 0x42, 0x99, 0xd8, 0xf4, 0x85, 0xaf, 0xd2, 0x65, 0x03, 0x6a, 0xdc, 0x1f, - 0x27, 0x11, 0xd6, 0x81, 0x2a, 0xe4, 0x70, 0xa0, 0x0d, 0x2a, 0x52, 0x0f, 0xa1, 0x92, 0x18, 0x9f, - 0x9d, 0xb2, 0x8a, 0xd4, 0xc6, 0xf9, 0x67, 0x99, 0xdb, 0x60, 0x16, 0xb1, 0x0c, 0x8c, 0x1e, 0x42, - 0x31, 0x65, 0xc4, 0x99, 0x57, 0xf9, 0xf0, 0xeb, 0xf3, 0x39, 0xde, 0x76, 0xb5, 0xb5, 0xcb, 0xc8, - 0x83, 0x58, 0xb0, 0x89, 0x2b, 0x19, 0xd0, 0x17, 0x50, 0xd6, 0xe5, 0xea, 0x54, 0x94, 0x3f, 0x6b, - 0x39, 0x2e, 0x53, 0x28, 0xad, 0xde, 0x93, 0x2d, 0x12, 0xe1, 0x2d, 0x35, 0xcd, 0x78, 0x62, 0x40, - 0x68, 0x0f, 0xca, 0x2a, 0x45, 0xb9, 0x53, 0x55, 0xae, 0xdc, 0xbd, 0x92, 0x2b, 0x2a, 0x5b, 0xb9, - 0xf2, 0x46, 0xf1, 0x5a, 0xae, 0x61, 0x43, 0xf7, 0xe0, 0xa7, 0x7c, 0x44, 0x12, 0x6f, 0x4c, 0x38, - 0x27, 0xf1, 0xd0, 0xdb, 0xa7, 0x0c, 0x93, 0x61, 0xec, 0x8d, 0xf0, 0x84, 0x3b, 0xb0, 0x6e, 0x35, - 0x2b, 0xc6, 0x91, 0x4f, 0xe4, 0xb4, 0x6f, 0xf4, 0xac, 0x2d, 0x3d, 0xe9, 0x11, 0x9e, 0x70, 0x74, - 0x13, 0x16, 0x9e, 0xfb, 0x51, 0x24, 0xf3, 0xfa, 0xb1, 0x1f, 0x53, 0xee, 0xd4, 0x72, 0xb9, 0x7b, - 0xda, 0x84, 0x6e, 0xc3, 0x32, 0x53, 0x25, 0xb3, 0xe3, 0x33, 0x3f, 0x8a, 0x70, 0x44, 0xf8, 0xd8, - 0x59, 0xc8, 0x1d, 0xe1, 0xdb, 0x66, 0xf4, 0x1d, 0x00, 0xc3, 0x3c, 0x1d, 0x63, 0x2f, 0xa1, 0xdc, - 0x59, 0x54, 0x9b, 0xff, 0xdd, 0x95, 0x36, 0xef, 0x2a, 0xf8, 0x0e, 0xd5, 0xfb, 0x77, 0xab, 0x2c, - 0xfb, 0x3f, 0xc2, 0x00, 0x29, 0xc7, 0xcc, 0x53, 0xe2, 0xe4, 0x2c, 0xad, 0x5b, 0xcd, 0x6a, 0x67, - 0xcb, 0x54, 0xea, 0x97, 0x43, 0x22, 0x0e, 0xd2, 0x41, 0x2b, 0xa0, 0xe3, 0xf6, 0x74, 0xb5, 0x70, - 0x70, 0xf2, 0xdd, 0x4e, 0x46, 0xc3, 0x36, 0xc7, 0x41, 0xca, 0x88, 0x98, 0xb4, 0xfa, 0x7f, 0xfc, - 0x7a, 0x97, 0x63, 0x16, 0xfb, 0x63, 0xbc, 0x23, 0xd9, 0xdc, 0xaa, 0x64, 0x56, 0x9f, 0x2b, 0x29, - 0xd4, 0xb4, 0x4b, 0xea, 0x18, 0xd0, 0x1f, 0xc0, 0x96, 0xea, 0xac, 0x2a, 0xe8, 0x6a, 0x3a, 0x65, - 0xb9, 0x0a, 0x89, 0x3e, 0x03, 0x10, 0x3e, 0x1b, 0x62, 0xd1, 0xa5, 0x11, 0x77, 0x0a, 0xeb, 0xc5, - 0x66, 0xd5, 0xd8, 0x73, 0xe3, 0x2b, 0x1c, 0x6a, 0xb9, 0x73, 0x47, 0x75, 0x28, 0x8e, 0xf0, 0x44, - 0xad, 0x5a, 0x75, 0xe5, 0x27, 0x7a, 0x0c, 0xa5, 0x23, 0x3f, 0x4a, 0x33, 0xc5, 0xbc, 0x5a, 0x4a, - 0xe5, 0x76, 0xe4, 0x6a, 0x9a, 0xdf, 0x16, 0xee, 0x5a, 0x2b, 0xbf, 0x81, 0x4a, 0x96, 0xf7, 0xf9, - 0x15, 0x4b, 0x7a, 0xc5, 0xeb, 0xf9, 0x15, 0xab, 0x79, 0xdc, 0xef, 0x61, 0xf1, 0xf4, 0x39, 0xbd, - 0x0b, 0x5d, 0xcc, 0xa1, 0xb7, 0xed, 0x8a, 0xa5, 0x14, 0xab, 0x58, 0xb7, 0xb7, 0xed, 0x8a, 0x5d, - 0x2f, 0x6d, 0xdb, 0x95, 0x52, 0xbd, 0xbc, 0x6d, 0x57, 0xae, 0xd5, 0x17, 0x1a, 0xff, 0x9e, 0xd7, - 0x5d, 0x27, 0x4d, 0xa6, 0x32, 0x72, 0x27, 0x13, 0x7a, 0x4b, 0x25, 0xd4, 0x4f, 0x66, 0x14, 0xe3, - 0xdb, 0xb2, 0xfe, 0x15, 0xd4, 0x49, 0x2c, 0x18, 0x0d, 0xd3, 0x00, 0x87, 0x9e, 0xc6, 0x17, 0x2e, - 0x83, 0x5f, 0x3a, 0x81, 0xf5, 0x15, 0xd3, 0x1d, 0xa8, 0x85, 0x78, 0xdf, 0x4f, 0x23, 0xe1, 0x49, - 0x75, 0x29, 0xaa, 0xcc, 0x43, 0x46, 0x4b, 0x61, 0x53, 0x9b, 0x76, 0xdd, 0x9e, 0x0b, 0x66, 0xda, - 0x2e, 0x23, 0xe8, 0xaf, 0x16, 0x7c, 0x94, 0x32, 0xc2, 0xbd, 0xc1, 0xc4, 0x8b, 0x68, 0xe0, 0x47, - 0x44, 0x4c, 0xbc, 0xd1, 0x91, 0x63, 0x2b, 0x17, 0xbe, 0xbc, 0xb8, 0x73, 0x9e, 0xec, 0x5d, 0xea, - 0x12, 0xef, 0x4c, 0xbe, 0x36, 0x0c, 0x8f, 0x8e, 0xb4, 0x2c, 0x5c, 0x3f, 0x7e, 0xbd, 0x56, 0xdf, - 0x75, 0x7b, 0x79, 0xd3, 0x9e, 0x5b, 0x4f, 0xcf, 0x4c, 0x46, 0x2e, 0xd4, 0xc6, 0x47, 0x41, 0xe0, - 0xed, 0x93, 0x48, 0x60, 0xa6, 0x3a, 0xd5, 0xe2, 0xa9, 0x36, 0x94, 0xed, 0xff, 0x9b, 0xbd, 0x6e, - 0x77, 0x4b, 0x4d, 0x3a, 0xd9, 0xd9, 0xc9, 0x98, 0x0b, 0x92, 0x45, 0x7f, 0xa3, 0xaf, 0x00, 0x70, - 0x1c, 0xb0, 0x49, 0xa2, 0xba, 0x85, 0xd6, 0xeb, 0xe6, 0x0c, 0x4a, 0xa9, 0x8e, 0x0f, 0xa6, 0x13, - 0x9f, 0xa8, 0x5f, 0xee, 0xe6, 0xb0, 0xe8, 0x09, 0x2c, 0x0f, 0xd4, 0x6e, 0x3d, 0x2e, 0x7c, 0x26, - 0x3c, 0xa9, 0x3d, 0x57, 0x69, 0x95, 0x4b, 0x1a, 0xdd, 0x97, 0x60, 0x69, 0x43, 0x8f, 0xc0, 0x0c, - 0x79, 0x38, 0x0e, 0x35, 0x5d, 0xe5, 0xf2, 0x74, 0x0b, 0x1a, 0xfb, 0x20, 0x0e, 0x15, 0xd9, 0x2e, - 0x94, 0x93, 0x91, 0x47, 0xc2, 0x4c, 0xc4, 0xef, 0x5c, 0xfa, 0xcc, 0x76, 0x46, 0xbd, 0xd0, 0xe8, - 0x77, 0x55, 0xb6, 0xdb, 0x9d, 0x47, 0xbd, 0x4d, 0xee, 0x96, 0x12, 0x39, 0x7c, 0x46, 0xc6, 0xe0, - 0x43, 0xc9, 0x58, 0x17, 0x3e, 0x9e, 0x99, 0x3a, 0x33, 0x94, 0xe5, 0xfc, 0x3a, 0xbf, 0x0b, 0x70, - 0xb2, 0x97, 0x3c, 0xd2, 0x9e, 0x81, 0xac, 0xe4, 0x90, 0x8d, 0x7f, 0x59, 0x50, 0x77, 0x31, 0x17, - 0x94, 0x61, 0x59, 0x44, 0x9a, 0xe0, 0x73, 0xb0, 0x65, 0x1d, 0x1a, 0x2d, 0x7d, 0x47, 0x19, 0xaa, - 0xa9, 0xe8, 0x3e, 0x94, 0xf6, 0x89, 0x6c, 0xa4, 0xba, 0x74, 0x37, 0x66, 0xf5, 0x61, 0xa5, 0x6d, - 0x2e, 0x3e, 0x4c, 0x31, 0x17, 0x2a, 0xeb, 0x32, 0x21, 0x50, 0x48, 0x74, 0x03, 0x6a, 0xd9, 0x05, - 0xa1, 0x17, 0xbe, 0x50, 0xe5, 0x9b, 0x75, 0xbc, 0xbc, 0xa1, 0xf1, 0x97, 0x22, 0x2c, 0x19, 0x97, - 0xa7, 0xca, 0xb3, 0x05, 0xd7, 0x98, 0x1e, 0xd2, 0xd9, 0x64, 0x5d, 0x3e, 0x9b, 0x6a, 0x06, 0xa8, - 0x72, 0xe9, 0x74, 0xcd, 0x14, 0xde, 0xa3, 0x66, 0x7a, 0x50, 0x66, 0x58, 0xf5, 0x7b, 0x7d, 0xeb, - 0xfd, 0xe5, 0x3b, 0x23, 0x62, 0x2e, 0xbf, 0x23, 0x3c, 0xc9, 0x6e, 0x29, 0x9a, 0x40, 0xde, 0x52, - 0x4c, 0x82, 0x6b, 0x51, 0xfa, 0xd5, 0x45, 0x2d, 0xe5, 0x54, 0x5c, 0x2e, 0xcc, 0xf0, 0xf7, 0xc8, - 0x9a, 0x7f, 0x14, 0xe0, 0xa3, 0x7e, 0x12, 0x11, 0x71, 0x3f, 0x0e, 0xfb, 0x81, 0x2f, 0x84, 0x79, - 0x76, 0xfc, 0x19, 0xca, 0xea, 0x5e, 0x9d, 0x75, 0x80, 0x7b, 0xe7, 0x7b, 0x3a, 0x03, 0x9e, 0x79, - 0xaf, 0xfc, 0xe9, 0x4a, 0x9e, 0x2c, 0x10, 0x9a, 0x34, 0x17, 0xd3, 0xc2, 0x7b, 0xc6, 0x74, 0xc5, - 0x83, 0xe5, 0xb7, 0x56, 0x43, 0xdb, 0x30, 0x8f, 0xe5, 0x35, 0x1a, 0x67, 0xfe, 0xdf, 0x7c, 0x67, - 0xa4, 0xa7, 0x45, 0x63, 0xf8, 0x33, 0x82, 0xc6, 0xdf, 0x8a, 0xb0, 0xd0, 0xed, 0xef, 0x7d, 0xcb, - 0x48, 0x16, 0x9c, 0x1b, 0xb2, 0x3d, 0x71, 0x41, 0x62, 0xfd, 0x84, 0x51, 0x85, 0x9d, 0xe5, 0x60, - 0xce, 0x80, 0x7e, 0x0e, 0xd7, 0xa4, 0x52, 0x78, 0x89, 0x0a, 0x8c, 0xce, 0xc2, 0xe9, 0x44, 0xa5, - 0x21, 0xda, 0x80, 0xbe, 0x80, 0x79, 0xaa, 0x33, 0x4f, 0x15, 0x4b, 0x6d, 0x66, 0xc3, 0xe8, 0xf6, - 0xf7, 0x4c, 0x7a, 0x66, 0x1e, 0x1a, 0xcc, 0xc9, 0xe3, 0x88, 0xd1, 0xe7, 0xdc, 0xbc, 0xa8, 0xf2, - 0x8f, 0x23, 0x97, 0x3e, 0xe7, 0xe8, 0x4f, 0xb0, 0x1c, 0xd0, 0x71, 0x22, 0x6b, 0x8f, 0xd0, 0xd8, - 0x0b, 0x68, 0x88, 0x03, 0xd3, 0x9e, 0x2e, 0x78, 0xc7, 0xc9, 0xf2, 0xe8, 0x9e, 0xc0, 0x0c, 0x6d, - 0x3d, 0xc7, 0xd4, 0x95, 0x44, 0x67, 0x34, 0xb6, 0xfc, 0x81, 0x34, 0xb6, 0xf1, 0x2d, 0x2c, 0x77, - 0xd2, 0x48, 0x6e, 0x28, 0x77, 0x1c, 0xd3, 0x97, 0xad, 0xf5, 0xa3, 0x5f, 0xb6, 0x37, 0x37, 0x60, - 0xe9, 0xcc, 0x56, 0x51, 0x05, 0xec, 0xc7, 0x34, 0xc6, 0xf5, 0x39, 0xf9, 0xf5, 0xf0, 0x25, 0x49, - 0xea, 0x56, 0xe7, 0xd6, 0xab, 0xff, 0xae, 0xce, 0xbd, 0x3a, 0x5e, 0xb5, 0xbe, 0x3f, 0x5e, 0xb5, - 0x7e, 0x38, 0x5e, 0xb5, 0xfe, 0x73, 0xbc, 0x6a, 0xfd, 0xfd, 0xcd, 0xea, 0xdc, 0xf7, 0x6f, 0x56, - 0xe7, 0x7e, 0x78, 0xb3, 0x3a, 0xf7, 0x5d, 0x2d, 0xf7, 0xc7, 0x83, 0xff, 0x07, 0x00, 0x00, 0xff, - 0xff, 0x3b, 0xa7, 0xba, 0x6f, 0xe9, 0x10, 0x00, 0x00, + 0x15, 0xd7, 0x92, 0x4b, 0x8a, 0x7c, 0xb4, 0x6c, 0x6a, 0xe2, 0xa4, 0x5b, 0x01, 0x95, 0x04, 0x26, + 0x72, 0x59, 0x17, 0x26, 0x11, 0xbb, 0x2d, 0x8c, 0xb6, 0x49, 0x6a, 0x52, 0x96, 0x43, 0x3a, 0xb1, + 0xd5, 0xa5, 0xa5, 0x00, 0x41, 0x8b, 0xc5, 0xfe, 0x19, 0x51, 0x63, 0x2e, 0x77, 0x56, 0x33, 0xb3, + 0xb2, 0xe9, 0x53, 0x81, 0xa2, 0x87, 0xde, 0xfa, 0x11, 0xfa, 0x11, 0xfa, 0x1d, 0x7a, 0xf1, 0x31, + 0xc7, 0x9c, 0x8c, 0x56, 0xfe, 0x0e, 0x3d, 0xf4, 0x54, 0xcc, 0x9f, 0xa5, 0x56, 0xb2, 0x24, 0xcb, + 0x35, 0x72, 0xa1, 0xd7, 0xf3, 0xde, 0xef, 0x37, 0xef, 0xbd, 0x79, 0x7f, 0x66, 0x04, 0x6d, 0x7e, + 0x10, 0x77, 0xf1, 0x73, 0x1c, 0x92, 0x64, 0x8f, 0xf9, 0x69, 0xd0, 0x4d, 0x19, 0x0d, 0x31, 0xe7, + 0x94, 0x71, 0x2f, 0xc8, 0xe2, 0x89, 0x47, 0x68, 0x27, 0x65, 0x54, 0x50, 0xe4, 0x84, 0x34, 0x9c, + 0x30, 0xea, 0x87, 0xfb, 0x1d, 0x7e, 0x10, 0x77, 0x22, 0xc2, 0x05, 0x3f, 0x88, 0x59, 0x96, 0xac, + 0x7c, 0xf4, 0x94, 0x06, 0xbc, 0x2b, 0x7f, 0xd2, 0x40, 0xfd, 0xa3, 0x11, 0x2b, 0x8e, 0xd2, 0x4e, + 0x83, 0x2e, 0xa1, 0xb7, 0xf6, 0x28, 0x9b, 0xfa, 0x22, 0x97, 0x7c, 0x2c, 0x77, 0x0d, 0x7d, 0xe1, + 0xc7, 0x74, 0xdc, 0x8d, 0x30, 0x0f, 0xd3, 0xa0, 0xcb, 0x05, 0xcb, 0x42, 0x91, 0x31, 0x1c, 0x19, + 0xa5, 0x8d, 0x8b, 0x4c, 0xf3, 0x39, 0xce, 0x77, 0xc9, 0x04, 0x89, 0xbb, 0xfb, 0x71, 0xd8, 0x15, + 0x64, 0x8a, 0xb9, 0xf0, 0xa7, 0xa9, 0x91, 0x5c, 0x1f, 0xd3, 0x31, 0x55, 0x9f, 0x5d, 0xf9, 0x65, + 0x56, 0x51, 0x6e, 0x55, 0xe4, 0x0b, 0xdf, 0xac, 0x2d, 0xe7, 0x6b, 0x7e, 0x4a, 0xf4, 0x52, 0xeb, + 0x3f, 0x65, 0xb8, 0xda, 0xf3, 0xc3, 0xc9, 0x1e, 0x89, 0x63, 0xcc, 0x46, 0x29, 0x0e, 0xd1, 0x03, + 0xb0, 0xc5, 0x2c, 0xc5, 0x8e, 0xb5, 0x6e, 0xb5, 0xaf, 0xde, 0xbe, 0xd5, 0x39, 0x2f, 0x20, 0x9d, + 0x93, 0xb8, 0xce, 0x93, 0x59, 0x8a, 0x7b, 0xf6, 0xcb, 0x57, 0x6b, 0x0b, 0xae, 0x22, 0x40, 0x3d, + 0xa8, 0x08, 0x3f, 0x88, 0xb1, 0x53, 0x5a, 0xb7, 0xda, 0x8d, 0xdb, 0x37, 0x4e, 0x31, 0xf1, 0x83, + 0x58, 0xf9, 0xf7, 0x44, 0xea, 0x6c, 0x62, 0x1e, 0x32, 0x92, 0x0a, 0xca, 0x0c, 0x85, 0x86, 0xa2, + 0xfb, 0x50, 0xe1, 0xa9, 0x9f, 0x70, 0xa7, 0xbc, 0x5e, 0x6e, 0x37, 0x6e, 0xff, 0xec, 0x7c, 0x6b, + 0x14, 0x8d, 0x8b, 0xfd, 0x48, 0x9a, 0xe3, 0x27, 0x39, 0x8d, 0x42, 0xa3, 0x4f, 0xa1, 0x16, 0x65, + 0xcc, 0x17, 0x84, 0x26, 0x8e, 0xbd, 0x6e, 0xb5, 0xcb, 0xbd, 0x0f, 0xa5, 0xf8, 0xbf, 0xaf, 0xd6, + 0x96, 0x64, 0x38, 0x3b, 0x9b, 0x46, 0xe8, 0xce, 0xd5, 0xd0, 0xc7, 0x00, 0xe1, 0x7e, 0x96, 0x4c, + 0x3c, 0x4e, 0x5e, 0x60, 0xa7, 0xa2, 0x40, 0x9a, 0xb3, 0xae, 0xd6, 0x47, 0xe4, 0x05, 0x46, 0x7f, + 0xb1, 0xa0, 0xc6, 0xb0, 0x1f, 0xdd, 0xe3, 0x8f, 0xf7, 0x9c, 0x45, 0xe5, 0xe6, 0x4f, 0x0a, 0x26, + 0xca, 0x33, 0xeb, 0xec, 0xc7, 0x61, 0xe7, 0x49, 0x7e, 0x66, 0xbd, 0xa1, 0xd9, 0xb7, 0x37, 0x26, + 0x62, 0x3f, 0x0b, 0x3a, 0x21, 0x9d, 0x76, 0xe7, 0x80, 0x28, 0x38, 0xfe, 0xee, 0xa6, 0x93, 0x71, + 0x97, 0x0b, 0xca, 0xfc, 0x31, 0xee, 0xe2, 0x64, 0x4c, 0x12, 0x9c, 0x06, 0x9d, 0x27, 0xcf, 0x93, + 0x39, 0x97, 0x3b, 0xdf, 0xba, 0x75, 0x13, 0x6c, 0x19, 0x7e, 0xd4, 0x80, 0xc5, 0x41, 0x72, 0xe8, + 0xc7, 0x24, 0x6a, 0x2e, 0x20, 0x80, 0x6a, 0x9f, 0xc6, 0xd9, 0x34, 0x69, 0x5a, 0xa8, 0x0e, 0x95, + 0x41, 0x12, 0xe1, 0xe7, 0xcd, 0xd2, 0xd0, 0xae, 0x55, 0x9b, 0x8b, 0xad, 0x67, 0xd0, 0x18, 0xd2, + 0x60, 0x9b, 0xd1, 0x31, 0xc3, 0x9c, 0xa3, 0x4f, 0xa0, 0xfa, 0x94, 0x06, 0x1e, 0x89, 0xd4, 0xb1, + 0x97, 0x7b, 0x4b, 0xd2, 0xcc, 0xa3, 0x57, 0x6b, 0x95, 0x21, 0x0d, 0x06, 0x9b, 0x6e, 0xe5, 0x29, + 0x0d, 0x06, 0x11, 0x6a, 0xc3, 0x95, 0x90, 0x26, 0x82, 0x91, 0x20, 0x53, 0xa1, 0x94, 0x07, 0x5b, + 0x32, 0x51, 0x39, 0x21, 0x41, 0x0e, 0xd8, 0x3c, 0xa6, 0xc2, 0x29, 0xaf, 0x5b, 0xed, 0x4a, 0x9e, + 0x15, 0x72, 0xa5, 0xf5, 0xb2, 0x06, 0x48, 0x1e, 0xd3, 0x60, 0x9a, 0x52, 0x26, 0x36, 0x7d, 0xe1, + 0xab, 0xac, 0xdb, 0x80, 0x06, 0xf7, 0xa7, 0x69, 0x8c, 0x75, 0xbc, 0x4b, 0x05, 0x1c, 0x68, 0x81, + 0x0a, 0xf8, 0x03, 0xa8, 0xa5, 0xc6, 0x66, 0xa7, 0xaa, 0xe2, 0xbd, 0x71, 0x7e, 0x4a, 0x14, 0x1c, + 0x34, 0x54, 0x73, 0x30, 0x7a, 0x00, 0xe5, 0x8c, 0x11, 0x67, 0x51, 0xa5, 0xd5, 0x2f, 0xcf, 0xe7, + 0x78, 0xd3, 0xd4, 0xce, 0x0e, 0x23, 0xf7, 0x13, 0xc1, 0x66, 0xae, 0x64, 0x40, 0x9f, 0x41, 0x55, + 0x57, 0xbd, 0x53, 0x53, 0xf6, 0xac, 0x15, 0xb8, 0x4c, 0xbd, 0x75, 0x06, 0x8f, 0xb7, 0x48, 0x8c, + 0xb7, 0x94, 0x9a, 0xb1, 0xc4, 0x80, 0xd0, 0x2e, 0x54, 0x55, 0xa6, 0x73, 0xa7, 0xae, 0x4c, 0xb9, + 0xfb, 0x4e, 0xa6, 0xa8, 0xa4, 0xe7, 0xca, 0x1a, 0xc5, 0x6b, 0xb9, 0x86, 0x0d, 0x7d, 0x01, 0x3f, + 0xe6, 0x13, 0x92, 0x7a, 0x53, 0xc2, 0x39, 0x49, 0xc6, 0xde, 0x1e, 0x65, 0x98, 0x8c, 0x13, 0x6f, + 0x82, 0x67, 0xdc, 0x81, 0x75, 0xab, 0x5d, 0x33, 0x86, 0x7c, 0x24, 0xd5, 0xbe, 0xd6, 0x5a, 0x5b, + 0x5a, 0xe9, 0x21, 0x9e, 0x71, 0x74, 0x13, 0x96, 0x9e, 0xf9, 0x71, 0x2c, 0xcb, 0xe3, 0x91, 0x9f, + 0x50, 0xee, 0x34, 0x0a, 0x25, 0x70, 0x52, 0x84, 0x6e, 0xc3, 0x32, 0x53, 0x95, 0xb7, 0xed, 0x33, + 0x3f, 0x8e, 0x71, 0x4c, 0xf8, 0xd4, 0x59, 0x2a, 0x1c, 0xe1, 0x9b, 0x62, 0xf4, 0x2d, 0x00, 0xc3, + 0x3c, 0x9b, 0x62, 0x2f, 0xa5, 0xdc, 0xb9, 0xaa, 0x9c, 0xff, 0xcd, 0x3b, 0x39, 0xef, 0x2a, 0xf8, + 0x36, 0xd5, 0xfe, 0xbb, 0x75, 0x96, 0xff, 0x1f, 0x61, 0x80, 0x8c, 0x63, 0xe6, 0xa9, 0x1e, 0xe7, + 0x5c, 0x5b, 0xb7, 0xda, 0xf5, 0xde, 0x96, 0x29, 0xbc, 0xcf, 0x2f, 0x57, 0x78, 0x38, 0xcc, 0x18, + 0x11, 0xb3, 0xce, 0xe8, 0xf7, 0x5f, 0xed, 0x70, 0xcc, 0x12, 0x7f, 0x8a, 0xb7, 0x25, 0x9b, 0x5b, + 0x97, 0xcc, 0xea, 0x73, 0x25, 0x83, 0x86, 0x36, 0x49, 0x1d, 0x03, 0xfa, 0x1d, 0xd8, 0xb2, 0xc9, + 0xab, 0x0a, 0x7a, 0xb7, 0x76, 0x67, 0xb9, 0x0a, 0x89, 0x3e, 0x01, 0x10, 0x3e, 0x1b, 0x63, 0xd1, + 0xa7, 0x31, 0x77, 0x4a, 0xeb, 0xe5, 0x76, 0xdd, 0xc8, 0x0b, 0xeb, 0x2b, 0x1c, 0x1a, 0x85, 0x73, + 0x47, 0x4d, 0x28, 0x4f, 0xf0, 0x4c, 0xed, 0x5a, 0x77, 0xe5, 0x27, 0x7a, 0x04, 0x95, 0x43, 0x3f, + 0xce, 0xf2, 0xc6, 0xfb, 0x6e, 0x29, 0x55, 0xf0, 0xc8, 0xd5, 0x34, 0xbf, 0x2e, 0xdd, 0xb5, 0x56, + 0x7e, 0x05, 0xb5, 0x3c, 0xef, 0x8b, 0x3b, 0x56, 0xf4, 0x8e, 0xd7, 0x8b, 0x3b, 0xd6, 0x8b, 0xb8, + 0xdf, 0xc2, 0xd5, 0x93, 0xe7, 0xf4, 0x36, 0x74, 0xb9, 0x80, 0x1e, 0xda, 0x35, 0x4b, 0x75, 0xac, + 0x72, 0xd3, 0x1e, 0xda, 0x35, 0xbb, 0x59, 0x19, 0xda, 0xb5, 0x4a, 0xb3, 0x3a, 0xb4, 0x6b, 0x57, + 0x9a, 0x4b, 0xad, 0x7f, 0x2e, 0xea, 0xe1, 0x95, 0xa5, 0xf3, 0x36, 0x72, 0x27, 0x9f, 0x17, 0x96, + 0x4a, 0xa8, 0x1f, 0x9d, 0x51, 0x8c, 0x6f, 0x4e, 0x87, 0x2f, 0xa1, 0x49, 0x12, 0xc1, 0x68, 0x94, + 0x85, 0x38, 0xf2, 0x34, 0xbe, 0x74, 0x19, 0xfc, 0xb5, 0x63, 0xd8, 0x48, 0x31, 0xdd, 0x81, 0x46, + 0x84, 0xf7, 0xfc, 0x2c, 0x16, 0x9e, 0xec, 0x2e, 0x65, 0x95, 0x79, 0xc8, 0xf4, 0x52, 0xd8, 0xd4, + 0xa2, 0x1d, 0x77, 0xe0, 0x82, 0x51, 0xdb, 0x61, 0x04, 0xfd, 0xd9, 0x82, 0x0f, 0x32, 0x46, 0xb8, + 0x17, 0xcc, 0xbc, 0x98, 0x86, 0x7e, 0x4c, 0xc4, 0xcc, 0x9b, 0x1c, 0x3a, 0xb6, 0x32, 0xe1, 0xf3, + 0x8b, 0x07, 0xf0, 0xb1, 0xef, 0xb2, 0x2f, 0xf1, 0xde, 0xec, 0x2b, 0xc3, 0xf0, 0xf0, 0x50, 0xb7, + 0x85, 0xeb, 0x47, 0xaf, 0xd6, 0x9a, 0x3b, 0xee, 0xa0, 0x28, 0xda, 0x75, 0x9b, 0xd9, 0x29, 0x65, + 0xe4, 0x42, 0x63, 0x7a, 0x18, 0x86, 0xde, 0x1e, 0x89, 0x05, 0x66, 0x6a, 0xe0, 0x5d, 0x3d, 0x31, + 0xcc, 0x72, 0xff, 0xbf, 0xde, 0xed, 0xf7, 0xb7, 0x94, 0xd2, 0xb1, 0x67, 0xc7, 0x6b, 0x2e, 0x48, + 0x16, 0xfd, 0x8d, 0xbe, 0x04, 0xc0, 0x49, 0xc8, 0x66, 0xa9, 0x9a, 0x16, 0xba, 0x5f, 0xb7, 0xcf, + 0xa0, 0x94, 0xdd, 0xf1, 0xfe, 0x5c, 0xf1, 0xb1, 0xfa, 0xe5, 0x6e, 0x01, 0x8b, 0x1e, 0xc3, 0x72, + 0xa0, 0xbc, 0xf5, 0xb8, 0xf0, 0x99, 0xf0, 0x64, 0xef, 0xb9, 0xdc, 0xc0, 0x35, 0x27, 0xa5, 0xd1, + 0x23, 0x09, 0x96, 0x32, 0xf4, 0x10, 0xcc, 0x92, 0x87, 0x93, 0x48, 0xd3, 0xd5, 0x2e, 0x4f, 0xb7, + 0xa4, 0xb1, 0xf7, 0x93, 0x48, 0x91, 0xed, 0x40, 0x35, 0x9d, 0x78, 0x24, 0xca, 0x9b, 0xf8, 0x9d, + 0x4b, 0x9f, 0xd9, 0xf6, 0x64, 0x10, 0x99, 0xfe, 0x5d, 0x97, 0xe3, 0x76, 0xfb, 0xe1, 0x60, 0x93, + 0xbb, 0x95, 0x54, 0x2e, 0x9f, 0x6a, 0x63, 0xf0, 0x43, 0xb5, 0xb1, 0x3e, 0x7c, 0x78, 0x66, 0xea, + 0x9c, 0xd1, 0x59, 0xce, 0xaf, 0xf3, 0xbb, 0x00, 0xc7, 0xbe, 0x14, 0x91, 0xf6, 0x19, 0xc8, 0x5a, + 0x01, 0xd9, 0xfa, 0x87, 0x05, 0x4d, 0x17, 0xcb, 0x9b, 0x0e, 0x96, 0x45, 0xa4, 0x09, 0x3e, 0x05, + 0x5b, 0xd6, 0xa1, 0xe9, 0xa5, 0x6f, 0x29, 0x43, 0xa5, 0x8a, 0xee, 0x41, 0x65, 0x8f, 0xc8, 0x41, + 0xaa, 0x4b, 0x77, 0xe3, 0xac, 0x39, 0xac, 0x7a, 0x9b, 0x8b, 0x0f, 0x32, 0xcc, 0x85, 0xca, 0xba, + 0xbc, 0x11, 0x28, 0x24, 0xba, 0x01, 0x8d, 0xfc, 0x82, 0x30, 0x88, 0x9e, 0xab, 0xf2, 0xcd, 0x27, + 0x5e, 0x51, 0xd0, 0xfa, 0x53, 0x19, 0xae, 0x19, 0x93, 0xe7, 0x9d, 0x67, 0x0b, 0xae, 0x30, 0xbd, + 0xa4, 0xb3, 0xc9, 0xba, 0x7c, 0x36, 0x35, 0x0c, 0x50, 0xe5, 0xd2, 0xc9, 0x9a, 0x29, 0xbd, 0x47, + 0xcd, 0x0c, 0xa0, 0xca, 0xb0, 0x9a, 0xf7, 0xfa, 0xf2, 0xfc, 0xf3, 0xb7, 0x46, 0xc4, 0xdc, 0xa1, + 0x27, 0x78, 0x96, 0xdf, 0x52, 0x34, 0x81, 0xbc, 0xa5, 0x98, 0x04, 0xd7, 0x4d, 0xe9, 0x17, 0x17, + 0x8d, 0x94, 0x13, 0x71, 0xb9, 0x30, 0xc3, 0xdf, 0x23, 0x6b, 0xfe, 0x5e, 0x82, 0x0f, 0x46, 0x69, + 0x4c, 0xc4, 0xbd, 0x24, 0x1a, 0x85, 0xbe, 0x10, 0xe6, 0xf5, 0xf2, 0x47, 0xa8, 0xaa, 0xeb, 0x79, + 0x3e, 0x01, 0xbe, 0x38, 0xdf, 0xd2, 0x33, 0xe0, 0xb9, 0xf5, 0xca, 0x9e, 0xbe, 0xe4, 0xc9, 0x03, + 0xa1, 0x49, 0x0b, 0x31, 0x2d, 0xbd, 0x67, 0x4c, 0x57, 0x3c, 0x58, 0x7e, 0x63, 0x37, 0x34, 0x84, + 0x45, 0x2c, 0xaf, 0xd1, 0x38, 0xb7, 0xff, 0xe6, 0x5b, 0x23, 0x3d, 0x2f, 0x1a, 0xc3, 0x9f, 0x13, + 0xb4, 0xfe, 0x5a, 0x86, 0xa5, 0xfe, 0x68, 0xf7, 0x1b, 0x46, 0xf2, 0xe0, 0xdc, 0x90, 0xe3, 0x89, + 0x0b, 0x92, 0xe8, 0x97, 0x90, 0x2a, 0xec, 0x3c, 0x07, 0x0b, 0x02, 0xf4, 0x53, 0xb8, 0x22, 0x3b, + 0x85, 0x97, 0xaa, 0xc0, 0xe8, 0x2c, 0x9c, 0x2b, 0xaa, 0x1e, 0xa2, 0x05, 0xe8, 0x33, 0x58, 0xa4, + 0x3a, 0xf3, 0x54, 0xb1, 0x34, 0xce, 0x1c, 0x18, 0xfd, 0xd1, 0xae, 0x49, 0xcf, 0xdc, 0x42, 0x83, + 0x39, 0x7e, 0x63, 0x31, 0xfa, 0x8c, 0x9b, 0x87, 0x59, 0xf1, 0x8d, 0xe5, 0xd2, 0x67, 0x1c, 0xfd, + 0x01, 0x96, 0x43, 0x3a, 0x4d, 0x65, 0xed, 0x11, 0x9a, 0x78, 0x21, 0x8d, 0x70, 0x68, 0xc6, 0xd3, + 0x05, 0xcf, 0x41, 0x59, 0x1e, 0xfd, 0x63, 0x98, 0xa1, 0x6d, 0x16, 0x98, 0xfa, 0x92, 0xe8, 0x54, + 0x8f, 0xad, 0xfe, 0x40, 0x3d, 0xb6, 0xf5, 0x0d, 0x2c, 0xf7, 0xb2, 0x58, 0x3a, 0x54, 0x38, 0x8e, + 0xf9, 0x03, 0xd9, 0xfa, 0xbf, 0x1f, 0xc8, 0x37, 0x37, 0xe0, 0xda, 0x29, 0x57, 0x51, 0x0d, 0xec, + 0x47, 0x34, 0xc1, 0xcd, 0x05, 0xf9, 0xf5, 0xe0, 0x05, 0x49, 0x9b, 0x56, 0xef, 0xd6, 0xcb, 0x7f, + 0xaf, 0x2e, 0xbc, 0x3c, 0x5a, 0xb5, 0xbe, 0x3b, 0x5a, 0xb5, 0xbe, 0x3f, 0x5a, 0xb5, 0xfe, 0x75, + 0xb4, 0x6a, 0xfd, 0xed, 0xf5, 0xea, 0xc2, 0x77, 0xaf, 0x57, 0x17, 0xbe, 0x7f, 0xbd, 0xba, 0xf0, + 0x6d, 0xa3, 0xf0, 0x37, 0x88, 0xff, 0x05, 0x00, 0x00, 0xff, 0xff, 0xbd, 0x77, 0x20, 0x89, 0x30, + 0x11, 0x00, 0x00, } diff --git a/pkg/sql/execinfrapb/processors_bulk_io.proto b/pkg/sql/execinfrapb/processors_bulk_io.proto index 00e6af3e6565..4d54a04fe8f9 100644 --- a/pkg/sql/execinfrapb/processors_bulk_io.proto +++ b/pkg/sql/execinfrapb/processors_bulk_io.proto @@ -59,7 +59,8 @@ message BackfillerSpec { optional int64 chunk_size = 5 [(gogoproto.nullable) = false]; // The timestamp to perform index backfill historical scans at. - optional util.hlc.Timestamp readAsOf = 7 [(gogoproto.nullable) = false]; + optional util.hlc.Timestamp readAsOf = 7 [(gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/storage/enginepb.TxnTimestamp"]; reserved 6; } diff --git a/pkg/sql/revert.go b/pkg/sql/revert.go index 714c1b10ddbe..8124973d19f5 100644 --- a/pkg/sql/revert.go +++ b/pkg/sql/revert.go @@ -17,7 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" - "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" ) @@ -34,7 +34,7 @@ func RevertTables( db *kv.DB, execCfg *ExecutorConfig, tables []*tabledesc.Immutable, - targetTime hlc.Timestamp, + targetTime enginepb.TxnTimestamp, batchSize int64, ) error { reverting := make(map[descpb.ID]bool, len(tables)) diff --git a/pkg/sql/row/fetcher.go b/pkg/sql/row/fetcher.go index f6eb4fcdd7d8..d67d1e9732aa 100644 --- a/pkg/sql/row/fetcher.go +++ b/pkg/sql/row/fetcher.go @@ -28,9 +28,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/scrub" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/encoding" - "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/mon" "github.com/cockroachdb/cockroach/pkg/util/timeutil" @@ -113,7 +113,7 @@ type tableInfo struct { // // rowLastModified is the timestamp of the last time any family in the row // was modified in any way. - rowLastModified hlc.Timestamp + rowLastModified enginepb.TxnTimestamp // timestampOutputIdx controls at what row ordinal to write the timestamp. timestampOutputIdx int @@ -571,7 +571,7 @@ func (rf *Fetcher) StartScan( func (rf *Fetcher) StartInconsistentScan( ctx context.Context, db *kv.DB, - initialTimestamp hlc.Timestamp, + initialTimestamp enginepb.TxnTimestamp, maxTimestampAge time.Duration, spans roachpb.Spans, limitBatches bool, @@ -965,7 +965,7 @@ func (rf *Fetcher) processKV( // set rowLastModified to a sentinel that's before any real timestamp. // As kvs are iterated for this row, it keeps track of the greatest // timestamp seen. - table.rowLastModified = hlc.Timestamp{} + table.rowLastModified = enginepb.TxnTimestamp{} // All row encodings (both before and after column families) have a // sentinel kv (column family 0) that is always present when a row is // present, even if that row is all NULLs. Thus, a row is deleted if and @@ -1311,7 +1311,7 @@ func (rf *Fetcher) NextRowDecoded( // RowLastModified may only be called after NextRow has returned a non-nil row // and returns the timestamp of the last modification to that row. -func (rf *Fetcher) RowLastModified() hlc.Timestamp { +func (rf *Fetcher) RowLastModified() enginepb.TxnTimestamp { return rf.rowReadyTable.rowLastModified } @@ -1519,7 +1519,8 @@ func (rf *Fetcher) finalizeRow() error { // TODO (rohany): Datums are immutable, so we can't store a DDecimal on the // fetcher and change its contents with each row. If that assumption gets // lifted, then we can avoid an allocation of a new decimal datum here. - dec := rf.alloc.NewDDecimal(tree.DDecimal{Decimal: tree.TimestampToDecimal(rf.RowLastModified())}) + // TODO(nvanbenschoten): what's the deal with this? + dec := rf.alloc.NewDDecimal(tree.DDecimal{Decimal: tree.TimestampToDecimal(rf.RowLastModified().ToClockTimestampUnchecked())}) table.row[table.timestampOutputIdx] = rowenc.EncDatum{Datum: dec} } if table.oidOutputIdx != noOutputColumn { diff --git a/pkg/sql/row/kv_fetcher.go b/pkg/sql/row/kv_fetcher.go index 579b07a599e4..5ca21c3db428 100644 --- a/pkg/sql/row/kv_fetcher.go +++ b/pkg/sql/row/kv_fetcher.go @@ -18,7 +18,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" - "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/mon" ) @@ -122,7 +121,7 @@ func (f *KVFetcher) NextKV( var key []byte var rawBytes []byte var err error - var ts hlc.Timestamp + var ts enginepb.TxnTimestamp switch mvccDecodeStrategy { case MVCCDecodingRequired: key, ts, rawBytes, f.batchResponse, err = enginepb.ScanDecodeKeyValue(f.batchResponse) diff --git a/pkg/sql/rowexec/backfiller.go b/pkg/sql/rowexec/backfiller.go index c3ab1c4c2f92..3492f023d47d 100644 --- a/pkg/sql/rowexec/backfiller.go +++ b/pkg/sql/rowexec/backfiller.go @@ -26,7 +26,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/types" - "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/cockroach/pkg/util/tracing" @@ -48,7 +48,7 @@ type chunkBackfiller interface { mutations []descpb.DescriptorMutation, span roachpb.Span, chunkSize int64, - readAsOf hlc.Timestamp, + readAsOf enginepb.TxnTimestamp, ) (roachpb.Key, error) // CurrentBufferFill returns how fractionally full the configured buffer is. diff --git a/pkg/sql/rowexec/columnbackfiller.go b/pkg/sql/rowexec/columnbackfiller.go index d636939889f0..bb2c155f9d2c 100644 --- a/pkg/sql/rowexec/columnbackfiller.go +++ b/pkg/sql/rowexec/columnbackfiller.go @@ -20,7 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" - "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" ) // columnBackfiller is a processor for backfilling columns. @@ -86,7 +86,7 @@ func (cb *columnBackfiller) runChunk( mutations []descpb.DescriptorMutation, sp roachpb.Span, chunkSize int64, - readAsOf hlc.Timestamp, + readAsOf enginepb.TxnTimestamp, ) (roachpb.Key, error) { var key roachpb.Key err := cb.flowCtx.Cfg.DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { diff --git a/pkg/sql/rowexec/indexbackfiller.go b/pkg/sql/rowexec/indexbackfiller.go index 19dbf2b980cc..9f90883346df 100644 --- a/pkg/sql/rowexec/indexbackfiller.go +++ b/pkg/sql/rowexec/indexbackfiller.go @@ -24,7 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/row" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" - "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/cockroach/pkg/util/tracing" @@ -148,7 +148,7 @@ func (ib *indexBackfiller) runChunk( mutations []descpb.DescriptorMutation, sp roachpb.Span, chunkSize int64, - readAsOf hlc.Timestamp, + readAsOf enginepb.TxnTimestamp, ) (roachpb.Key, error) { knobs := &ib.flowCtx.Cfg.TestingKnobs if knobs.RunBeforeBackfillChunk != nil { diff --git a/pkg/sql/rowexec/rowfetcher.go b/pkg/sql/rowexec/rowfetcher.go index f92444a65cb1..1a98dfb38909 100644 --- a/pkg/sql/rowexec/rowfetcher.go +++ b/pkg/sql/rowexec/rowfetcher.go @@ -23,8 +23,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/row" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util" - "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/mon" ) @@ -37,7 +37,7 @@ type rowFetcher interface { StartInconsistentScan( _ context.Context, _ *kv.DB, - initialTimestamp hlc.Timestamp, + initialTimestamp enginepb.TxnTimestamp, maxTimestampAge time.Duration, spans roachpb.Spans, limitBatches bool, diff --git a/pkg/sql/rowexec/stats.go b/pkg/sql/rowexec/stats.go index fcea1de36bb4..e68f1f91b2d2 100644 --- a/pkg/sql/rowexec/stats.go +++ b/pkg/sql/rowexec/stats.go @@ -22,7 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/row" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" - "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/errors" ) @@ -123,7 +123,7 @@ func (c *rowFetcherStatCollector) StartScan( func (c *rowFetcherStatCollector) StartInconsistentScan( ctx context.Context, db *kv.DB, - initialTimestamp hlc.Timestamp, + initialTimestamp enginepb.TxnTimestamp, maxTimestampAge time.Duration, spans roachpb.Spans, limitBatches bool, diff --git a/pkg/sql/schema_changer.go b/pkg/sql/schema_changer.go index 8b0416077fae..577089db9f4e 100644 --- a/pkg/sql/schema_changer.go +++ b/pkg/sql/schema_changer.go @@ -46,6 +46,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/grpcutil" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -232,7 +233,7 @@ func (sc *SchemaChanger) refreshMaterializedView( } func (sc *SchemaChanger) backfillQueryIntoTable( - ctx context.Context, table *descpb.TableDescriptor, query string, ts hlc.Timestamp, desc string, + ctx context.Context, table *descpb.TableDescriptor, query string, ts enginepb.TxnTimestamp, desc string, ) error { if fn := sc.testingKnobs.RunBeforeQueryBackfill; fn != nil { if err := fn(); err != nil { @@ -1988,7 +1989,7 @@ func (sc *SchemaChanger) txnWithModified( func createSchemaChangeEvalCtx( ctx context.Context, execCfg *ExecutorConfig, - ts hlc.Timestamp, + ts enginepb.TxnTimestamp, ieFactory sqlutil.SessionBoundInternalExecutorFactory, ) extendedEvalContext { diff --git a/pkg/sql/sem/tree/eval.go b/pkg/sql/sem/tree/eval.go index c9caeb9fa90c..7889161a87a0 100644 --- a/pkg/sql/sem/tree/eval.go +++ b/pkg/sql/sem/tree/eval.go @@ -3428,7 +3428,8 @@ func (ctx *EvalContext) GetClusterTimestamp() *DDecimal { if ts.IsEmpty() { panic(errors.AssertionFailedf("zero cluster timestamp in txn")) } - return TimestampToDecimalDatum(ts) + // TODO(nvanbenschoten): what if this is synethtic? + return TimestampToDecimalDatum(ts.ToClockTimestampUnchecked()) } // HasPlaceholders returns true if this EvalContext's placeholders have been diff --git a/pkg/sql/testutils.go b/pkg/sql/testutils.go index c6961f512e5e..a0628bfdd137 100644 --- a/pkg/sql/testutils.go +++ b/pkg/sql/testutils.go @@ -20,7 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" - "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/errors" ) @@ -49,7 +49,7 @@ func CreateTestTableDescriptor( st, n, parentID, keys.PublicSchemaID, id, - hlc.Timestamp{}, /* creationTime */ + enginepb.TxnTimestamp{}, /* creationTime */ privileges, nil, /* affected */ &semaCtx, @@ -68,7 +68,7 @@ func CreateTestTableDescriptor( n.Name.Table(), n.Options, parentID, keys.PublicSchemaID, id, - hlc.Timestamp{}, /* creationTime */ + enginepb.TxnTimestamp{}, /* creationTime */ privileges, tree.PersistencePermanent, nil, /* params */ diff --git a/pkg/sql/txn_state.go b/pkg/sql/txn_state.go index f0a99e7397cf..4d5161a9871b 100644 --- a/pkg/sql/txn_state.go +++ b/pkg/sql/txn_state.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/contextutil" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/metric" @@ -191,7 +192,7 @@ func (ts *txnState) resetForNewSQLTxn( ts.mu.txnStart = timeutil.Now() ts.mu.Unlock() if historicalTimestamp != nil { - ts.setHistoricalTimestamp(ts.Ctx, *historicalTimestamp) + ts.setHistoricalTimestamp(ts.Ctx, enginepb.TxnTimestamp(*historicalTimestamp)) } if err := ts.setReadOnlyMode(readOnly); err != nil { panic(err) @@ -249,7 +250,7 @@ func (ts *txnState) finishExternalTxn() { ts.mu.Unlock() } -func (ts *txnState) setHistoricalTimestamp(ctx context.Context, historicalTimestamp hlc.Timestamp) { +func (ts *txnState) setHistoricalTimestamp(ctx context.Context, historicalTimestamp enginepb.TxnTimestamp) { ts.mu.Lock() ts.mu.txn.SetFixedTimestamp(ctx, historicalTimestamp) ts.mu.Unlock() @@ -257,7 +258,7 @@ func (ts *txnState) setHistoricalTimestamp(ctx context.Context, historicalTimest } // getReadTimestamp returns the transaction's current read timestamp. -func (ts *txnState) getReadTimestamp() hlc.Timestamp { +func (ts *txnState) getReadTimestamp() enginepb.TxnTimestamp { ts.mu.RLock() defer ts.mu.RUnlock() return ts.mu.txn.ReadTimestamp() diff --git a/pkg/sql/virtual_schema.go b/pkg/sql/virtual_schema.go index 7480a1240fed..e4e39a67079d 100644 --- a/pkg/sql/virtual_schema.go +++ b/pkg/sql/virtual_schema.go @@ -33,6 +33,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/errors" @@ -268,9 +269,9 @@ var virtualSchemas = map[descpb.ID]virtualSchema{ catconstants.PgExtensionSchemaID: pgExtension, } -var startTime = hlc.Timestamp{ +var startTime = enginepb.TxnTimestamp(hlc.Timestamp{ WallTime: time.Date(2000, time.January, 1, 0, 0, 0, 0, time.UTC).UnixNano(), -} +}) // // SQL-layer interface to work with virtual schemas. diff --git a/pkg/storage/batch.go b/pkg/storage/batch.go index 0d75f023315d..4a8453a5dd49 100644 --- a/pkg/storage/batch.go +++ b/pkg/storage/batch.go @@ -14,7 +14,6 @@ import ( "encoding/binary" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" - "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/errors" "github.com/cockroachdb/pebble" ) @@ -169,7 +168,7 @@ func encodeKeyToBuf(buf []byte, key MVCCKey, keyLen int) { buf[len(buf)-1] = byte(timestampLength) } -func encodeTimestamp(ts hlc.Timestamp) []byte { +func encodeTimestamp(ts enginepb.TxnTimestamp) []byte { _, encodedTS, _ := enginepb.SplitMVCCKey(EncodeKey(MVCCKey{Timestamp: ts})) return encodedTS } diff --git a/pkg/storage/batch_test.go b/pkg/storage/batch_test.go index 3677903de187..50f57c10b302 100644 --- a/pkg/storage/batch_test.go +++ b/pkg/storage/batch_test.go @@ -22,7 +22,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/testutils" - "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/iterutil" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -207,8 +206,8 @@ func TestReadOnlyBasics(t *testing.T) { func() { ro.NewMVCCIterator(MVCCKeyIterKind, IterOptions{UpperBound: roachpb.KeyMax}).Close() }, func() { ro.NewMVCCIterator(MVCCKeyAndIntentsIterKind, IterOptions{ - MinTimestampHint: hlc.MinTimestamp, - MaxTimestampHint: hlc.MaxTimestamp, + MinTimestampHint: enginepb.MaxTxnTimestamp, + MaxTimestampHint: enginepb.MinTxnTimestamp, UpperBound: roachpb.KeyMax, }).Close() }, @@ -1205,9 +1204,9 @@ func TestDecodeKey(t *testing.T) { tests := []MVCCKey{ {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: enginepb.TxnTimestamp{WallTime: 1}}, + {Key: []byte("foo"), Timestamp: enginepb.TxnTimestamp{WallTime: 1, Logical: 1}}, + {Key: []byte("foo"), Timestamp: enginepb.TxnTimestamp{WallTime: 1, Logical: 1, Flags: 3}}, } for _, test := range tests { t.Run(test.String(), func(t *testing.T) { diff --git a/pkg/storage/bench_pebble_test.go b/pkg/storage/bench_pebble_test.go index d998acb8d73d..ca1e6704738f 100644 --- a/pkg/storage/bench_pebble_test.go +++ b/pkg/storage/bench_pebble_test.go @@ -18,9 +18,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/testutils/skip" "github.com/cockroachdb/cockroach/pkg/util/encoding" - "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/randutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/pebble" @@ -242,10 +242,10 @@ func BenchmarkMVCCPutDelete_Pebble(b *testing.B) { key := encoding.EncodeVarintAscending(nil, blockID) key = encoding.EncodeVarintAscending(key, blockNum) - if err := MVCCPut(ctx, db, nil, key, hlc.Timestamp{}, value, nil /* txn */); err != nil { + if err := MVCCPut(ctx, db, nil, key, enginepb.TxnTimestamp{}, value, nil /* txn */); err != nil { b.Fatal(err) } - if err := MVCCDelete(ctx, db, nil, key, hlc.Timestamp{}, nil /* txn */); err != nil { + if err := MVCCDelete(ctx, db, nil, key, enginepb.TxnTimestamp{}, nil /* txn */); err != nil { b.Fatal(err) } } @@ -365,7 +365,7 @@ func BenchmarkBatchBuilderPut(b *testing.B) { for j := i; j < end; j++ { key := roachpb.Key(encoding.EncodeUvarintAscending(keyBuf[:4], uint64(j))) - ts := hlc.Timestamp{WallTime: int64(j)} + ts := enginepb.TxnTimestamp{WallTime: int64(j)} batch.Put(MVCCKey{key, ts}, value) } batch.Finish() diff --git a/pkg/storage/bench_test.go b/pkg/storage/bench_test.go index 2be4623561d0..3aa8a87e036f 100644 --- a/pkg/storage/bench_test.go +++ b/pkg/storage/bench_test.go @@ -28,7 +28,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/testutils/skip" "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/fileutil" - "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/randutil" "github.com/cockroachdb/cockroach/pkg/util/timeutil" @@ -221,7 +220,7 @@ func loadTestData(dir string, numKeys, numBatches, batchTimeSpan, valueBytes int batch = eng.NewBatch() minWallTime = sstTimestamps[i/scaled] } - timestamp := hlc.Timestamp{WallTime: minWallTime + rand.Int63n(int64(batchTimeSpan))} + timestamp := enginepb.TxnTimestamp{WallTime: minWallTime + rand.Int63n(int64(batchTimeSpan))} value := roachpb.MakeValueFromBytes(randutil.RandBytes(rng, valueBytes)) value.InitChecksum(key) if err := MVCCPut(ctx, batch, nil, key, timestamp, value, nil); err != nil { @@ -310,7 +309,7 @@ func setupMVCCData( value := roachpb.MakeValueFromBytes(randutil.RandBytes(rng, opts.valueBytes)) value.InitChecksum(key) counts[idx]++ - ts := hlc.Timestamp{WallTime: int64(counts[idx] * 5)} + ts := enginepb.TxnTimestamp{WallTime: int64(counts[idx] * 5)} if txn != nil { txn.ReadTimestamp = ts txn.WriteTimestamp = ts @@ -323,7 +322,7 @@ func setupMVCCData( resolveLastIntent := func(batch Batch, idx int) { key := keys[idx] txnMeta := txn.TxnMeta - txnMeta.WriteTimestamp = hlc.Timestamp{WallTime: int64(counts[idx]) * 5} + txnMeta.WriteTimestamp = enginepb.TxnTimestamp{WallTime: int64(counts[idx]) * 5} if _, err := MVCCResolveWriteIntent(ctx, batch, nil /* ms */, roachpb.LockUpdate{ Span: roachpb.Span{Key: key}, Status: roachpb.COMMITTED, @@ -426,7 +425,7 @@ func runMVCCScan(ctx context.Context, b *testing.B, emk engineMaker, opts benchS endKey := roachpb.Key(encoding.EncodeUvarintAscending(endKeyBuf[:4], uint64(keyIdx+int32(opts.numRows)-1))) endKey = endKey.Next() walltime := int64(5 * (rand.Int31n(int32(opts.numVersions)) + 1)) - ts := hlc.Timestamp{WallTime: walltime} + ts := enginepb.TxnTimestamp{WallTime: walltime} res, err := MVCCScan(ctx, eng, startKey, endKey, ts, MVCCScanOptions{ MaxKeys: int64(opts.numRows), Reverse: opts.reverse, @@ -466,7 +465,7 @@ func runMVCCGet(ctx context.Context, b *testing.B, emk engineMaker, opts benchDa keyIdx := rand.Int31n(int32(opts.numKeys)) key := roachpb.Key(encoding.EncodeUvarintAscending(keyBuf[:4], uint64(keyIdx))) walltime := int64(5 * (rand.Int31n(int32(opts.numVersions)) + 1)) - ts := hlc.Timestamp{WallTime: walltime} + ts := enginepb.TxnTimestamp{WallTime: walltime} if v, _, err := MVCCGet(ctx, eng, key, ts, MVCCGetOptions{}); err != nil { b.Fatalf("failed get: %+v", err) } else if v == nil { @@ -494,7 +493,7 @@ func runMVCCPut(ctx context.Context, b *testing.B, emk engineMaker, valueSize in for i := 0; i < b.N; i++ { key := roachpb.Key(encoding.EncodeUvarintAscending(keyBuf[:4], uint64(i))) - ts := hlc.Timestamp{WallTime: timeutil.Now().UnixNano()} + ts := enginepb.TxnTimestamp{WallTime: timeutil.Now().UnixNano()} if err := MVCCPut(ctx, eng, nil, key, ts, value, nil); err != nil { b.Fatalf("failed put: %+v", err) } @@ -516,7 +515,7 @@ func runMVCCBlindPut(ctx context.Context, b *testing.B, emk engineMaker, valueSi for i := 0; i < b.N; i++ { key := roachpb.Key(encoding.EncodeUvarintAscending(keyBuf[:4], uint64(i))) - ts := hlc.Timestamp{WallTime: timeutil.Now().UnixNano()} + ts := enginepb.TxnTimestamp{WallTime: timeutil.Now().UnixNano()} if err := MVCCBlindPut(ctx, eng, nil, key, ts, value, nil); err != nil { b.Fatalf("failed put: %+v", err) } @@ -540,7 +539,7 @@ func runMVCCConditionalPut( if createFirst { for i := 0; i < b.N; i++ { key := roachpb.Key(encoding.EncodeUvarintAscending(keyBuf[:4], uint64(i))) - ts := hlc.Timestamp{WallTime: timeutil.Now().UnixNano()} + ts := enginepb.TxnTimestamp{WallTime: timeutil.Now().UnixNano()} if err := MVCCPut(ctx, eng, nil, key, ts, value, nil); err != nil { b.Fatalf("failed put: %+v", err) } @@ -552,7 +551,7 @@ func runMVCCConditionalPut( for i := 0; i < b.N; i++ { key := roachpb.Key(encoding.EncodeUvarintAscending(keyBuf[:4], uint64(i))) - ts := hlc.Timestamp{WallTime: timeutil.Now().UnixNano()} + ts := enginepb.TxnTimestamp{WallTime: timeutil.Now().UnixNano()} if err := MVCCConditionalPut(ctx, eng, nil, key, ts, value, expected, CPutFailIfMissing, nil); err != nil { b.Fatalf("failed put: %+v", err) } @@ -574,7 +573,7 @@ func runMVCCBlindConditionalPut(ctx context.Context, b *testing.B, emk engineMak for i := 0; i < b.N; i++ { key := roachpb.Key(encoding.EncodeUvarintAscending(keyBuf[:4], uint64(i))) - ts := hlc.Timestamp{WallTime: timeutil.Now().UnixNano()} + ts := enginepb.TxnTimestamp{WallTime: timeutil.Now().UnixNano()} if err := MVCCBlindConditionalPut(ctx, eng, nil, key, ts, value, nil, CPutFailIfMissing, nil); err != nil { b.Fatalf("failed put: %+v", err) } @@ -596,7 +595,7 @@ func runMVCCInitPut(ctx context.Context, b *testing.B, emk engineMaker, valueSiz for i := 0; i < b.N; i++ { key := roachpb.Key(encoding.EncodeUvarintAscending(keyBuf[:4], uint64(i))) - ts := hlc.Timestamp{WallTime: timeutil.Now().UnixNano()} + ts := enginepb.TxnTimestamp{WallTime: timeutil.Now().UnixNano()} if err := MVCCInitPut(ctx, eng, nil, key, ts, value, false, nil); err != nil { b.Fatalf("failed put: %+v", err) } @@ -618,7 +617,7 @@ func runMVCCBlindInitPut(ctx context.Context, b *testing.B, emk engineMaker, val for i := 0; i < b.N; i++ { key := roachpb.Key(encoding.EncodeUvarintAscending(keyBuf[:4], uint64(i))) - ts := hlc.Timestamp{WallTime: timeutil.Now().UnixNano()} + ts := enginepb.TxnTimestamp{WallTime: timeutil.Now().UnixNano()} if err := MVCCBlindInitPut(ctx, eng, nil, key, ts, value, false, nil); err != nil { b.Fatalf("failed put: %+v", err) } @@ -648,7 +647,7 @@ func runMVCCBatchPut(ctx context.Context, b *testing.B, emk engineMaker, valueSi for j := i; j < end; j++ { key := roachpb.Key(encoding.EncodeUvarintAscending(keyBuf[:4], uint64(j))) - ts := hlc.Timestamp{WallTime: timeutil.Now().UnixNano()} + ts := enginepb.TxnTimestamp{WallTime: timeutil.Now().UnixNano()} if err := MVCCPut(ctx, batch, nil, key, ts, value, nil); err != nil { b.Fatalf("failed put: %+v", err) } @@ -694,7 +693,7 @@ func runMVCCBatchTimeSeries(ctx context.Context, b *testing.B, emk engineMaker, b.ResetTimer() - var ts hlc.Timestamp + var ts enginepb.TxnTimestamp for i := 0; i < b.N; i++ { batch := eng.NewBatch() @@ -728,7 +727,7 @@ func runMVCCGetMergedValue( keys[i] = roachpb.Key(fmt.Sprintf("key-%d", i)) } - timestamp := hlc.Timestamp{} + timestamp := enginepb.TxnTimestamp{} for i := 0; i < numKeys; i++ { for j := 0; j < mergesPerKey; j++ { timeseries := &roachpb.InternalTimeSeriesData{ @@ -797,7 +796,7 @@ func runMVCCDeleteRange(ctx context.Context, b *testing.B, emk engineMaker, valu roachpb.KeyMin, roachpb.KeyMax, math.MaxInt64, - hlc.MaxTimestamp, + enginepb.MaxTxnTimestamp, nil, false, ); err != nil { @@ -924,7 +923,7 @@ func runMVCCGarbageCollect( eng := emk(b, "mvcc_gc") defer eng.Close() - ts := hlc.Timestamp{}.Add(time.Date(2000, 1, 1, 0, 0, 0, 0, time.UTC).UnixNano(), 0) + ts := enginepb.TxnTimestamp{}.Add(time.Date(2000, 1, 1, 0, 0, 0, 0, time.UTC).UnixNano(), 0) val := roachpb.MakeValueFromBytes(randutil.RandBytes(rng, opts.valueBytes)) // We write values at ts+(0,i), set now=ts+(1,0) so that we're ahead of all @@ -1006,7 +1005,7 @@ func runBatchApplyBatchRepr( for i := 0; i < batchSize; i++ { key := roachpb.Key(encoding.EncodeUvarintAscending(keyBuf[:4], uint64(order[i]))) - ts := hlc.Timestamp{WallTime: timeutil.Now().UnixNano()} + ts := enginepb.TxnTimestamp{WallTime: timeutil.Now().UnixNano()} if err := MVCCBlindPut(ctx, batch, nil, key, ts, value, nil); err != nil { b.Fatal(err) } @@ -1048,7 +1047,7 @@ func runExportToSst( key = encoding.EncodeUint32Ascending(key, uint32(i)) for j := 0; j < numRevisions; j++ { - err := batch.PutMVCC(MVCCKey{Key: key, Timestamp: hlc.Timestamp{WallTime: int64(j + 1), Logical: 0}}, []byte("foobar")) + err := batch.PutMVCC(MVCCKey{Key: key, Timestamp: enginepb.TxnTimestamp{WallTime: int64(j + 1), Logical: 0}}, []byte("foobar")) if err != nil { b.Fatal(err) } @@ -1064,8 +1063,8 @@ func runExportToSst( b.ResetTimer() for i := 0; i < b.N; i++ { - startTS := hlc.Timestamp{WallTime: int64(numRevisions / 2)} - endTS := hlc.Timestamp{WallTime: int64(numRevisions + 2)} + startTS := enginepb.TxnTimestamp{WallTime: int64(numRevisions / 2)} + endTS := enginepb.TxnTimestamp{WallTime: int64(numRevisions + 2)} _, _, _, err := engine.ExportMVCCToSst(roachpb.KeyMin, roachpb.KeyMax, startTS, endTS, exportAllRevisions, 0 /* targetSize */, 0 /* maxSize */, IterOptions{ LowerBound: roachpb.KeyMin, UpperBound: roachpb.KeyMax, diff --git a/pkg/storage/engine.go b/pkg/storage/engine.go index 20c4e5d6f5e7..1f205ae9e27d 100644 --- a/pkg/storage/engine.go +++ b/pkg/storage/engine.go @@ -22,7 +22,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/storage/fs" "github.com/cockroachdb/cockroach/pkg/util/envutil" - "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/iterutil" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/protoutil" @@ -242,7 +241,7 @@ type IterOptions struct { // not see some interleaved intents. Currently, the only way to correctly // use such an iterator is to use it in concert with an iterator without // timestamp hints, as done by MVCCIncrementalIterator. - MinTimestampHint, MaxTimestampHint hlc.Timestamp + MinTimestampHint, MaxTimestampHint enginepb.TxnTimestamp } // MVCCIterKind is used to inform Reader about the kind of iteration desired @@ -296,7 +295,7 @@ type Reader interface { // This function looks at MVCC versions and intents, and returns an error if an // intent is found. ExportMVCCToSst( - startKey, endKey roachpb.Key, startTS, endTS hlc.Timestamp, + startKey, endKey roachpb.Key, startTS, endTS enginepb.TxnTimestamp, exportAllRevisions bool, targetSize uint64, maxSize uint64, io IterOptions, ) (sst []byte, _ roachpb.BulkOpSummary, resumeKey roachpb.Key, _ error) diff --git a/pkg/storage/engine_key_test.go b/pkg/storage/engine_key_test.go index d6108c664429..992962422969 100644 --- a/pkg/storage/engine_key_test.go +++ b/pkg/storage/engine_key_test.go @@ -21,7 +21,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "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/uuid" "github.com/stretchr/testify/require" @@ -85,9 +84,9 @@ func TestMVCCAndEngineKeyEncodeDecode(t *testing.T) { key MVCCKey }{ {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("glue"), Timestamp: enginepb.TxnTimestamp{WallTime: 89999}}}, + {key: MVCCKey{Key: roachpb.Key("foo"), Timestamp: enginepb.TxnTimestamp{WallTime: 99, Logical: 45}}}, + {key: MVCCKey{Key: roachpb.Key("flags"), Timestamp: enginepb.TxnTimestamp{WallTime: 99, Logical: 45, Flags: 3}}}, } for _, test := range testCases { t.Run("", func(t *testing.T) { diff --git a/pkg/storage/engine_test.go b/pkg/storage/engine_test.go index b8cbeedbdeb2..b39c8cb3f8d4 100644 --- a/pkg/storage/engine_test.go +++ b/pkg/storage/engine_test.go @@ -180,7 +180,7 @@ func TestEngineBatchStaleCachedIterator(t *testing.T) { // Put a value so that the deletion below finds a value to seek // to. - if err := MVCCPut(context.Background(), batch, nil, key, hlc.Timestamp{}, + if err := MVCCPut(context.Background(), batch, nil, key, enginepb.TxnTimestamp{}, roachpb.MakeValueFromString("x"), nil); err != nil { t.Fatal(err) } @@ -188,7 +188,7 @@ func TestEngineBatchStaleCachedIterator(t *testing.T) { // Seek the iterator to `key` and clear the value (but without // telling the iterator about that). if err := MVCCDelete(context.Background(), batch, nil, key, - hlc.Timestamp{}, nil); err != nil { + enginepb.TxnTimestamp{}, nil); err != nil { t.Fatal(err) } @@ -200,7 +200,7 @@ func TestEngineBatchStaleCachedIterator(t *testing.T) { // result back, we'll see the (newly deleted) value (due to the // failure mode above). if v, _, err := MVCCGet(context.Background(), batch, key, - hlc.Timestamp{}, MVCCGetOptions{}); err != nil { + enginepb.TxnTimestamp{}, MVCCGetOptions{}); err != nil { t.Fatal(err) } else if v != nil { t.Fatalf("expected no value, got %+v", v) @@ -606,9 +606,9 @@ func TestEngineTimeBound(t *testing.T) { engine := engineImpl.create() defer engine.Close() - var minTimestamp = hlc.Timestamp{WallTime: 1, Logical: 0} - var maxTimestamp = hlc.Timestamp{WallTime: 3, Logical: 0} - times := []hlc.Timestamp{ + var minTimestamp = enginepb.TxnTimestamp{WallTime: 1, Logical: 0} + var maxTimestamp = enginepb.TxnTimestamp{WallTime: 3, Logical: 0} + times := []enginepb.TxnTimestamp{ {WallTime: 2, Logical: 0}, minTimestamp, maxTimestamp, diff --git a/pkg/storage/enginepb/decode.go b/pkg/storage/enginepb/decode.go index 0e321ecd7960..b935f3c1b0ff 100644 --- a/pkg/storage/enginepb/decode.go +++ b/pkg/storage/enginepb/decode.go @@ -13,7 +13,6 @@ package enginepb import ( "encoding/binary" - "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/errors" ) @@ -43,7 +42,7 @@ func SplitMVCCKey(mvccKey []byte) (key []byte, ts []byte, ok bool) { } // DecodeKey decodes an key/timestamp from its serialized representation. -func DecodeKey(encodedKey []byte) (key []byte, timestamp hlc.Timestamp, _ error) { +func DecodeKey(encodedKey []byte) (key []byte, timestamp TxnTimestamp, _ error) { key, ts, ok := SplitMVCCKey(encodedKey) if !ok { return nil, timestamp, errors.Errorf("invalid encoded mvcc key: %x", encodedKey) @@ -79,7 +78,7 @@ const kvLenSize = 8 // the key/value, the timestamp, and the suffix of data remaining in the batch. func ScanDecodeKeyValue( repr []byte, -) (key []byte, ts hlc.Timestamp, value []byte, orepr []byte, err error) { +) (key []byte, ts TxnTimestamp, value []byte, orepr []byte, err error) { if len(repr) < kvLenSize { return key, ts, nil, repr, errors.Errorf("unexpected batch EOF") } diff --git a/pkg/storage/enginepb/decode_test.go b/pkg/storage/enginepb/decode_test.go index b4437cb35a19..d8e2fa1b847b 100644 --- a/pkg/storage/enginepb/decode_test.go +++ b/pkg/storage/enginepb/decode_test.go @@ -17,12 +17,11 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" - "github.com/cockroachdb/cockroach/pkg/util/hlc" ) func BenchmarkScanDecodeKeyValue(b *testing.B) { key := roachpb.Key("blah blah blah") - ts := hlc.Timestamp{WallTime: int64(1000000)} + ts := enginepb.TxnTimestamp{WallTime: 1000000} value := []byte("foo foo foo") rep := make([]byte, 8) keyBytes := storage.EncodeKey(storage.MVCCKey{Key: key, Timestamp: ts}) diff --git a/pkg/storage/enginepb/mvcc.go b/pkg/storage/enginepb/mvcc.go index 8f8d4f7a1180..0dcef6541028 100644 --- a/pkg/storage/enginepb/mvcc.go +++ b/pkg/storage/enginepb/mvcc.go @@ -16,7 +16,9 @@ import ( "math" "sort" "strings" + "time" + "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/errors" "github.com/cockroachdb/redact" ) @@ -49,6 +51,102 @@ const ( MaxTxnPriority TxnPriority = math.MaxInt32 ) +// TxnTimestamp is an MVCC timestamp that a transaction may read or write at. +// When a transaction commits, it leaves behind MVCC versions for any writes +// that it performed at its timestamp, denoting at which time these writes are +// visible to other transactions. +// +// Transactions begin life with a timestamp pulled from a clock and reflecting +// real time, but a transaction's timestamp may diverge from real time due to +// the closed timestamp on non-blocking ranges or due to conflicts with other +// transactions whose timestamps have already diverged from real time. In these +// cases, a TxnTimestamp may represent a "future timestamp" and will have its +// synthetic flag set. As such, it would be incorrect to introduce back into a +// clock, so TxnTimestamps should never be converted back to hlc.Timestamps. +type TxnTimestamp hlc.Timestamp + +// TxnTimestamp constant values. +var ( + // MaxTxnTimestamp is the max value allowed for TxnTimestamp. + MaxTxnTimestamp = TxnTimestamp(hlc.MaxTimestamp) + // MinTxnTimestamp is the min value allowed for TxnTimestamp. + MinTxnTimestamp = TxnTimestamp(hlc.MinTimestamp) +) + +// NewPopulatedTxnTimestamp ... +func NewPopulatedTxnTimestamp(r randyMvcc3, easy bool) *TxnTimestamp { + return (*TxnTimestamp)(hlc.NewPopulatedTimestamp(r, easy)) +} + +// alias for compact casting. +type hlcTs = hlc.Timestamp + +// ToClockTimestampUnchecked ... +func (t TxnTimestamp) ToClockTimestampUnchecked() hlc.Timestamp { return hlcTs(t) } + +// ToLegacyTimestamp ... +func (t TxnTimestamp) ToLegacyTimestamp() hlc.LegacyTimestamp { return hlcTs(t).ToLegacyTimestamp() } + +// EqOrdering ... +func (t TxnTimestamp) EqOrdering(s TxnTimestamp) bool { return hlcTs(t).EqOrdering(hlcTs(s)) } + +// Less ... +func (t TxnTimestamp) Less(s TxnTimestamp) bool { return hlcTs(t).Less(hlcTs(s)) } + +// LessEq ... +func (t TxnTimestamp) LessEq(s TxnTimestamp) bool { return hlcTs(t).LessEq(hlcTs(s)) } + +// String ... +func (t TxnTimestamp) String() string { return hlcTs(t).String() } + +// SafeValue implements the redact.SafeValue interface. +func (t TxnTimestamp) SafeValue() {} + +// AsOfSystemTime ... +func (t TxnTimestamp) AsOfSystemTime() string { return hlcTs(t).AsOfSystemTime() } + +// IsEmpty ... +func (t TxnTimestamp) IsEmpty() bool { return hlcTs(t).IsEmpty() } + +// GoTime ... +func (t TxnTimestamp) GoTime() time.Time { return hlcTs(t).GoTime() } + +// Add ... +func (t TxnTimestamp) Add(w int64, l int32) TxnTimestamp { return TxnTimestamp(hlcTs(t).Add(w, l)) } + +// Next ... +func (t TxnTimestamp) Next() TxnTimestamp { return TxnTimestamp(hlcTs(t).Next()) } + +// Prev ... +func (t TxnTimestamp) Prev() TxnTimestamp { return TxnTimestamp(hlcTs(t).Prev()) } + +// FloorPrev ... +func (t TxnTimestamp) FloorPrev() TxnTimestamp { return TxnTimestamp(hlcTs(t).FloorPrev()) } + +// Forward ... +func (t *TxnTimestamp) Forward(s TxnTimestamp) bool { return (*hlcTs)(t).Forward(hlcTs(s)) } + +// Backward ... +func (t *TxnTimestamp) Backward(s TxnTimestamp) { (*hlcTs)(t).Backward(hlcTs(s)) } + +// Reset ... +func (t *TxnTimestamp) Reset() { (*hlcTs)(t).Reset() } + +// ProtoMessage ... +func (t *TxnTimestamp) ProtoMessage() {} + +// Size ... +func (t *TxnTimestamp) Size() int { return (*hlcTs)(t).Size() } + +// Equal ... +func (t *TxnTimestamp) Equal(that interface{}) bool { return (*hlcTs)(t).Equal(that) } + +// MarshalTo ... +func (t *TxnTimestamp) MarshalTo(data []byte) (int, error) { return (*hlcTs)(t).MarshalTo(data) } + +// Unmarshal ... +func (t *TxnTimestamp) Unmarshal(data []byte) error { return (*hlcTs)(t).Unmarshal(data) } + // TxnSeqIsIgnored returns true iff the sequence number overlaps with // any range in the ignored array. func TxnSeqIsIgnored(seq TxnSeq, ignored []IgnoredSeqNumRange) bool { @@ -202,6 +300,11 @@ func (meta MVCCMetadata) IsInline() bool { return meta.RawBytes != nil } +// TxnTimestamp ... +func (meta MVCCMetadata) TxnTimestamp() TxnTimestamp { + return TxnTimestamp(meta.Timestamp.ToTimestamp()) +} + // AddToIntentHistory adds the sequence and value to the intent history. func (meta *MVCCMetadata) AddToIntentHistory(seq TxnSeq, val []byte) { meta.IntentHistory = append(meta.IntentHistory, diff --git a/pkg/storage/enginepb/mvcc3.pb.go b/pkg/storage/enginepb/mvcc3.pb.go index ba8e0cd7aa4e..e278804de071 100644 --- a/pkg/storage/enginepb/mvcc3.pb.go +++ b/pkg/storage/enginepb/mvcc3.pb.go @@ -6,7 +6,6 @@ package enginepb import proto "github.com/gogo/protobuf/proto" import fmt "fmt" import math "math" -import hlc "github.com/cockroachdb/cockroach/pkg/util/hlc" import github_com_cockroachdb_cockroach_pkg_util_uuid "github.com/cockroachdb/cockroach/pkg/util/uuid" @@ -92,7 +91,7 @@ type TxnMeta struct { // value will need to be rewritten at the forwarded timestamp if the // transaction commits. // - WriteTimestamp hlc.Timestamp `protobuf:"bytes,5,opt,name=write_timestamp,json=writeTimestamp,proto3" json:"write_timestamp"` + WriteTimestamp TxnTimestamp `protobuf:"bytes,5,opt,name=write_timestamp,json=writeTimestamp,proto3,casttype=TxnTimestamp" json:"write_timestamp"` // The timestamp that the transaction was assigned by its gateway when it // began its first epoch. This is the earliest timestamp that the transaction // could have written any of its intents at. @@ -111,7 +110,7 @@ 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. - MinTimestamp hlc.Timestamp `protobuf:"bytes,9,opt,name=min_timestamp,json=minTimestamp,proto3" json:"min_timestamp"` + MinTimestamp TxnTimestamp `protobuf:"bytes,9,opt,name=min_timestamp,json=minTimestamp,proto3,casttype=TxnTimestamp" 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"` // A zero-indexed sequence number which is increased on each request @@ -125,7 +124,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_be904b17dffa4ef8, []int{0} } func (m *TxnMeta) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -161,7 +160,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_be904b17dffa4ef8, []int{1} } func (m *IgnoredSeqNumRange) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -211,7 +210,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_be904b17dffa4ef8, []int{2} } func (m *MVCCStatsDelta) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -262,7 +261,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_be904b17dffa4ef8, []int{3} } func (m *MVCCPersistentStats) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -307,7 +306,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_be904b17dffa4ef8, []int{4} } func (m *RangeAppliedState) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -335,17 +334,17 @@ var xxx_messageInfo_RangeAppliedState proto.InternalMessageInfo // MVCCWriteValueOp corresponds to a value being written outside of a // transaction. type MVCCWriteValueOp struct { - Key []byte `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"` - Timestamp hlc.Timestamp `protobuf:"bytes,2,opt,name=timestamp,proto3" json:"timestamp"` - Value []byte `protobuf:"bytes,3,opt,name=value,proto3" json:"value,omitempty"` - PrevValue []byte `protobuf:"bytes,4,opt,name=prev_value,json=prevValue,proto3" json:"prev_value,omitempty"` + Key []byte `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"` + Timestamp TxnTimestamp `protobuf:"bytes,2,opt,name=timestamp,proto3,casttype=TxnTimestamp" json:"timestamp"` + Value []byte `protobuf:"bytes,3,opt,name=value,proto3" json:"value,omitempty"` + PrevValue []byte `protobuf:"bytes,4,opt,name=prev_value,json=prevValue,proto3" json:"prev_value,omitempty"` } 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_be904b17dffa4ef8, []int{5} } func (m *MVCCWriteValueOp) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -375,15 +374,15 @@ var xxx_messageInfo_MVCCWriteValueOp proto.InternalMessageInfo type MVCCWriteIntentOp struct { TxnID github_com_cockroachdb_cockroach_pkg_util_uuid.UUID `protobuf:"bytes,1,opt,name=txn_id,json=txnId,proto3,customtype=github.com/cockroachdb/cockroach/pkg/util/uuid.UUID" json:"txn_id"` TxnKey []byte `protobuf:"bytes,2,opt,name=txn_key,json=txnKey,proto3" json:"txn_key,omitempty"` - TxnMinTimestamp hlc.Timestamp `protobuf:"bytes,4,opt,name=txn_min_timestamp,json=txnMinTimestamp,proto3" json:"txn_min_timestamp"` - Timestamp hlc.Timestamp `protobuf:"bytes,3,opt,name=timestamp,proto3" json:"timestamp"` + TxnMinTimestamp TxnTimestamp `protobuf:"bytes,4,opt,name=txn_min_timestamp,json=txnMinTimestamp,proto3,casttype=TxnTimestamp" json:"txn_min_timestamp"` + Timestamp TxnTimestamp `protobuf:"bytes,3,opt,name=timestamp,proto3,casttype=TxnTimestamp" json:"timestamp"` } 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_be904b17dffa4ef8, []int{6} } func (m *MVCCWriteIntentOp) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -412,14 +411,14 @@ var xxx_messageInfo_MVCCWriteIntentOp proto.InternalMessageInfo // timestamp for a given transaction. type MVCCUpdateIntentOp struct { TxnID github_com_cockroachdb_cockroach_pkg_util_uuid.UUID `protobuf:"bytes,1,opt,name=txn_id,json=txnId,proto3,customtype=github.com/cockroachdb/cockroach/pkg/util/uuid.UUID" json:"txn_id"` - Timestamp hlc.Timestamp `protobuf:"bytes,2,opt,name=timestamp,proto3" json:"timestamp"` + Timestamp TxnTimestamp `protobuf:"bytes,2,opt,name=timestamp,proto3,casttype=TxnTimestamp" json:"timestamp"` } 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_be904b17dffa4ef8, []int{7} } func (m *MVCCUpdateIntentOp) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -449,7 +448,7 @@ var xxx_messageInfo_MVCCUpdateIntentOp proto.InternalMessageInfo type MVCCCommitIntentOp struct { TxnID github_com_cockroachdb_cockroach_pkg_util_uuid.UUID `protobuf:"bytes,1,opt,name=txn_id,json=txnId,proto3,customtype=github.com/cockroachdb/cockroach/pkg/util/uuid.UUID" json:"txn_id"` Key []byte `protobuf:"bytes,2,opt,name=key,proto3" json:"key,omitempty"` - Timestamp hlc.Timestamp `protobuf:"bytes,3,opt,name=timestamp,proto3" json:"timestamp"` + Timestamp TxnTimestamp `protobuf:"bytes,3,opt,name=timestamp,proto3,casttype=TxnTimestamp" json:"timestamp"` Value []byte `protobuf:"bytes,4,opt,name=value,proto3" json:"value,omitempty"` PrevValue []byte `protobuf:"bytes,5,opt,name=prev_value,json=prevValue,proto3" json:"prev_value,omitempty"` } @@ -458,7 +457,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_be904b17dffa4ef8, []int{8} } func (m *MVCCCommitIntentOp) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -498,7 +497,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_be904b17dffa4ef8, []int{9} } func (m *MVCCAbortIntentOp) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -534,7 +533,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_be904b17dffa4ef8, []int{10} } func (m *MVCCAbortTxnOp) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -573,7 +572,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_be904b17dffa4ef8, []int{11} } func (m *MVCCLogicalOp) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1450,7 +1449,7 @@ func NewPopulatedTxnMeta(r randyMvcc3, easy bool) *TxnMeta { if r.Intn(2) == 0 { this.Epoch *= -1 } - v3 := hlc.NewPopulatedTimestamp(r, easy) + v3 := NewPopulatedTxnTimestamp(r, easy) this.WriteTimestamp = *v3 this.Priority = TxnPriority(r.Int31()) if r.Intn(2) == 0 { @@ -1460,7 +1459,7 @@ func NewPopulatedTxnMeta(r randyMvcc3, easy bool) *TxnMeta { if r.Intn(2) == 0 { this.Sequence *= -1 } - v4 := hlc.NewPopulatedTimestamp(r, easy) + v4 := NewPopulatedTxnTimestamp(r, easy) this.MinTimestamp = *v4 if !easy && r.Intn(10) != 0 { } @@ -4232,84 +4231,85 @@ var ( ) func init() { - proto.RegisterFile("storage/enginepb/mvcc3.proto", fileDescriptor_mvcc3_7509a297d4cf653a) + proto.RegisterFile("storage/enginepb/mvcc3.proto", fileDescriptor_mvcc3_be904b17dffa4ef8) } -var fileDescriptor_mvcc3_7509a297d4cf653a = []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, - 0xac, 0x34, 0x59, 0xaa, 0xe4, 0xe6, 0x1b, 0xd8, 0x51, 0x4a, 0x1a, 0xc7, 0xe9, 0x9a, 0xb8, 0x87, - 0x4a, 0x5d, 0x0d, 0xcb, 0x74, 0xbd, 0xf2, 0x32, 0xbb, 0xde, 0x1d, 0x08, 0x7c, 0x8b, 0x5e, 0x2a, - 0xf5, 0xd0, 0x56, 0xbe, 0xf5, 0x0b, 0xf4, 0xd0, 0x8f, 0xe0, 0x63, 0xa4, 0x5e, 0xa2, 0x1e, 0x50, - 0x8b, 0x2f, 0xfd, 0x0c, 0xce, 0xa5, 0x9a, 0x99, 0x65, 0x01, 0xa7, 0xc6, 0xa8, 0x6e, 0xad, 0xde, - 0x66, 0x7e, 0xbf, 0xf7, 0x7e, 0xef, 0xcd, 0xf8, 0xc7, 0xbc, 0x35, 0xdc, 0x0d, 0x99, 0x17, 0x60, - 0x9b, 0xd4, 0x08, 0xb5, 0x1d, 0x4a, 0xfc, 0x76, 0xad, 0xdb, 0xb7, 0xac, 0xc7, 0xba, 0x1f, 0x78, - 0xcc, 0x43, 0x77, 0x2c, 0xcf, 0x3a, 0x0a, 0x3c, 0x6c, 0x1d, 0xea, 0x51, 0x9c, 0x3e, 0x89, 0xbb, - 0x53, 0xec, 0x31, 0xc7, 0xad, 0x1d, 0xba, 0x56, 0x8d, 0x39, 0x5d, 0x12, 0x32, 0xdc, 0xf5, 0x65, - 0xd6, 0x9d, 0xdb, 0xb6, 0x67, 0x7b, 0x62, 0x59, 0xe3, 0x2b, 0x89, 0x56, 0xbf, 0x51, 0x61, 0xb5, - 0x35, 0xa0, 0xbb, 0x84, 0x61, 0xf4, 0x19, 0x24, 0x9d, 0x4e, 0x51, 0xa9, 0x28, 0x9b, 0xb9, 0x46, - 0xfd, 0x74, 0x54, 0x4e, 0xfc, 0x3a, 0x2a, 0x3f, 0xb6, 0x1d, 0x76, 0xd8, 0x6b, 0xeb, 0x96, 0xd7, - 0xad, 0xc5, 0x65, 0x3b, 0xed, 0xe9, 0xba, 0xe6, 0x1f, 0xd9, 0x35, 0x51, 0xb4, 0xd7, 0x73, 0x3a, - 0xfa, 0xab, 0x57, 0xcd, 0x9d, 0xf1, 0xa8, 0x9c, 0x6c, 0xee, 0x18, 0x49, 0xa7, 0x83, 0xf2, 0xa0, - 0x1e, 0x91, 0x61, 0x51, 0xe5, 0x9a, 0x06, 0x5f, 0xa2, 0x2a, 0xa4, 0x88, 0xef, 0x59, 0x87, 0x45, - 0xad, 0xa2, 0x6c, 0xa6, 0x1a, 0xb9, 0xf3, 0x51, 0x39, 0xdd, 0x1a, 0xd0, 0x27, 0x1c, 0x33, 0x24, - 0x85, 0x9e, 0xc3, 0xc6, 0xeb, 0xc0, 0x61, 0xc4, 0x8c, 0xcf, 0x50, 0x4c, 0x55, 0x94, 0xcd, 0xec, - 0xa3, 0x7b, 0xfa, 0xf4, 0xe8, 0xbc, 0xa6, 0x7e, 0xe8, 0x5a, 0x7a, 0x6b, 0x12, 0xd4, 0xd0, 0x78, - 0xd3, 0xc6, 0xba, 0xc8, 0x8d, 0x51, 0xf4, 0x11, 0xa4, 0xfd, 0xc0, 0xf1, 0x02, 0x87, 0x0d, 0x8b, - 0x2b, 0xa2, 0xe8, 0xc6, 0xf9, 0xa8, 0x9c, 0x6d, 0x0d, 0xe8, 0xcb, 0x08, 0x36, 0xe2, 0x00, 0xf4, - 0x21, 0xa4, 0x43, 0x72, 0xdc, 0x23, 0xd4, 0x22, 0xc5, 0x55, 0x11, 0x0c, 0xe7, 0xa3, 0xf2, 0x4a, - 0x6b, 0x40, 0xf7, 0xc9, 0xb1, 0x11, 0x73, 0xe8, 0x13, 0x58, 0xeb, 0x3a, 0x74, 0xa6, 0xc1, 0xcc, - 0xf2, 0x0d, 0xe6, 0xba, 0x0e, 0x8d, 0xb1, 0xad, 0xf4, 0xb7, 0x27, 0xe5, 0xc4, 0xcf, 0x27, 0x65, - 0xe5, 0x99, 0x96, 0x4e, 0xe6, 0xd5, 0x67, 0x5a, 0x3a, 0x9d, 0xcf, 0x54, 0xbf, 0x04, 0xd4, 0xb4, - 0xa9, 0x17, 0x90, 0xce, 0x3e, 0x39, 0x7e, 0xd1, 0xeb, 0x1a, 0x98, 0xda, 0x04, 0x55, 0x20, 0x15, - 0x32, 0x1c, 0x30, 0xf1, 0x47, 0x9a, 0x6f, 0x4d, 0x12, 0xe8, 0x2e, 0xa8, 0x84, 0x76, 0x8a, 0xc9, - 0xf7, 0x78, 0x0e, 0x6f, 0xa5, 0x79, 0x9d, 0x3f, 0x4e, 0xca, 0x4a, 0xf5, 0x07, 0x0d, 0xd6, 0x77, - 0x0f, 0xb6, 0xb7, 0xf7, 0x19, 0x66, 0xe1, 0x0e, 0x71, 0x19, 0x46, 0xf7, 0xa1, 0xe0, 0xe2, 0x90, - 0x99, 0x3d, 0xbf, 0x83, 0x19, 0x31, 0x29, 0xa6, 0x5e, 0x28, 0x0a, 0xe5, 0x8d, 0x0d, 0x4e, 0xbc, - 0x12, 0xf8, 0x0b, 0x0e, 0xa3, 0x7b, 0x00, 0x0e, 0x65, 0x84, 0x32, 0x13, 0xdb, 0x44, 0x54, 0xcb, - 0x1b, 0x19, 0x89, 0xd4, 0x6d, 0x82, 0x3e, 0x86, 0x9c, 0x6d, 0x99, 0xed, 0x21, 0x23, 0xa1, 0x08, - 0xe0, 0x7f, 0xff, 0x7c, 0x63, 0x7d, 0x3c, 0x2a, 0xc3, 0xd3, 0xed, 0x06, 0x87, 0xeb, 0x36, 0x31, - 0xc0, 0xb6, 0x26, 0x6b, 0x2e, 0xe8, 0x3a, 0x7d, 0x22, 0x73, 0x84, 0x37, 0x90, 0x91, 0xe1, 0x88, - 0x88, 0x88, 0x69, 0xcb, 0xeb, 0x51, 0x26, 0xcc, 0x10, 0xd1, 0xdb, 0x1c, 0x40, 0xff, 0x87, 0xcc, - 0x11, 0x19, 0x46, 0xc9, 0x2b, 0x82, 0x4d, 0x1f, 0x91, 0xa1, 0xcc, 0x8d, 0x48, 0x99, 0xba, 0x1a, - 0x93, 0x71, 0x66, 0x1f, 0xbb, 0x51, 0x66, 0x5a, 0x92, 0x7d, 0xec, 0xc6, 0x99, 0x9c, 0x94, 0x99, - 0x99, 0x98, 0x94, 0x99, 0x1f, 0x40, 0x2e, 0xba, 0x02, 0x99, 0x0c, 0x82, 0xcf, 0x4a, 0x4c, 0xe6, - 0x4f, 0x43, 0xa4, 0x44, 0x76, 0x36, 0x24, 0xae, 0x1f, 0x0e, 0xc3, 0x48, 0x22, 0x27, 0x4b, 0x84, - 0xc3, 0x30, 0xae, 0xcf, 0x49, 0x99, 0xbc, 0x16, 0x93, 0x32, 0xf3, 0x21, 0x20, 0xcb, 0xa3, 0x0c, - 0x3b, 0x34, 0x34, 0x49, 0xc8, 0x9c, 0x2e, 0xe6, 0x12, 0xeb, 0x15, 0x65, 0x53, 0x35, 0x0a, 0x13, - 0xe6, 0xc9, 0x84, 0x40, 0x9b, 0x90, 0xc7, 0x6d, 0x2f, 0x60, 0x66, 0xe8, 0x63, 0x1a, 0xd5, 0xdb, - 0x10, 0x92, 0xeb, 0x02, 0xdf, 0xf7, 0x31, 0x15, 0x55, 0xb7, 0x34, 0x61, 0x90, 0x1f, 0x35, 0xb8, - 0xc5, 0x0d, 0xf2, 0x92, 0x04, 0xa1, 0x13, 0xf2, 0x86, 0x85, 0x55, 0xfe, 0x6b, 0x2e, 0x51, 0x17, - 0xbb, 0x44, 0x5d, 0xe8, 0x12, 0x75, 0x91, 0x4b, 0xd4, 0x45, 0x2e, 0x51, 0x17, 0xb9, 0x44, 0xbd, - 0xc2, 0x25, 0xea, 0xd5, 0x2e, 0x51, 0xaf, 0x70, 0x89, 0xba, 0xc8, 0x25, 0xea, 0x3f, 0xef, 0x12, - 0xf5, 0x3d, 0x97, 0x4c, 0x9f, 0x92, 0x5f, 0x14, 0x28, 0x88, 0xe7, 0xa9, 0xee, 0xfb, 0xae, 0x43, - 0x3a, 0xdc, 0x27, 0x04, 0x3d, 0x00, 0x14, 0xe0, 0xaf, 0x98, 0x89, 0x25, 0x68, 0x3a, 0xb4, 0x43, - 0x06, 0xc2, 0x28, 0x9a, 0x91, 0xe7, 0x4c, 0x14, 0xdd, 0xe4, 0x38, 0xd2, 0xe1, 0x96, 0x4b, 0x70, - 0x48, 0x2e, 0x84, 0x27, 0x45, 0x78, 0x41, 0x50, 0x73, 0xf1, 0x07, 0x90, 0x0d, 0x78, 0x49, 0x33, - 0xe4, 0xa6, 0x14, 0xce, 0xc9, 0x3e, 0xaa, 0xe9, 0x97, 0x0f, 0x46, 0xfd, 0x2f, 0xbc, 0x1c, 0x3d, - 0xc7, 0x20, 0x94, 0x04, 0x32, 0x73, 0xaa, 0xef, 0x14, 0xc8, 0xf3, 0x9c, 0xcf, 0xf9, 0x30, 0x39, - 0xc0, 0x6e, 0x8f, 0xec, 0xf9, 0x93, 0x71, 0xa6, 0x4c, 0xc7, 0x59, 0x1d, 0x32, 0xd3, 0x19, 0x90, - 0x5c, 0x7e, 0x06, 0x4c, 0xb3, 0xd0, 0x6d, 0x48, 0xf5, 0xb9, 0x7e, 0x34, 0x25, 0xe5, 0x86, 0x7b, - 0xd9, 0x0f, 0x48, 0xdf, 0x94, 0x94, 0x26, 0xa8, 0x0c, 0x47, 0x44, 0x2f, 0xd5, 0xef, 0x93, 0x50, - 0x88, 0xdb, 0x6b, 0x0a, 0xab, 0xec, 0xf9, 0xe8, 0x0b, 0x58, 0x61, 0x03, 0x6a, 0xc6, 0x53, 0x7c, - 0xe7, 0x7a, 0x53, 0x3c, 0xd5, 0x1a, 0xd0, 0xe6, 0x8e, 0x91, 0x62, 0x03, 0xda, 0xec, 0xa0, 0xff, - 0xc1, 0x2a, 0x17, 0xe7, 0x17, 0x90, 0x14, 0xed, 0xf0, 0x5a, 0x9f, 0x5e, 0xbc, 0x03, 0xf5, 0x6f, - 0xdd, 0xc1, 0x1e, 0x14, 0xb8, 0xf6, 0xfc, 0x48, 0xd5, 0x96, 0x97, 0xda, 0x60, 0x03, 0xba, 0x3b, - 0x33, 0x55, 0xab, 0x3f, 0x29, 0x80, 0xf8, 0xfd, 0xc8, 0xe7, 0xe8, 0x66, 0x2e, 0xe8, 0xfa, 0x5e, - 0xa8, 0xbe, 0x8b, 0xda, 0xde, 0xf6, 0xba, 0x5d, 0x87, 0xdd, 0x4c, 0xdb, 0x91, 0xa9, 0x93, 0x97, - 0x98, 0x5a, 0xbd, 0x9e, 0xa9, 0xb5, 0xcb, 0x4d, 0x9d, 0xba, 0x68, 0x6a, 0x5f, 0x7a, 0xba, 0xce, - 0x5f, 0x9a, 0x1b, 0x39, 0x7b, 0xb5, 0x2b, 0xbf, 0x82, 0x44, 0xc5, 0xd6, 0x80, 0xfe, 0xdb, 0xe5, - 0xde, 0xa9, 0xb0, 0xc6, 0xeb, 0x3d, 0xf7, 0x6c, 0xc7, 0xc2, 0xee, 0x9e, 0x8f, 0x76, 0x21, 0x2b, - 0x3f, 0x75, 0xe5, 0x95, 0x28, 0xe2, 0xb2, 0x1f, 0x5c, 0xf5, 0x90, 0xcd, 0x3e, 0x4a, 0x06, 0xbc, - 0x8e, 0x77, 0xe8, 0x25, 0xe4, 0xa4, 0x9c, 0x9c, 0x1e, 0x91, 0x0b, 0x1f, 0x2e, 0xa5, 0x37, 0xb9, - 0x71, 0x43, 0x76, 0x24, 0xb7, 0x68, 0x1f, 0xd6, 0xa2, 0x51, 0x1f, 0x49, 0x4a, 0x3f, 0xe8, 0x57, - 0x49, 0xce, 0xff, 0xf0, 0x8c, 0x5c, 0x6f, 0x66, 0xcf, 0x45, 0x2d, 0xe1, 0xf0, 0x89, 0xa8, 0xb6, - 0x9c, 0xe8, 0xfc, 0xcf, 0xc2, 0xc8, 0x59, 0x33, 0x7b, 0x7e, 0x76, 0x39, 0xbb, 0x22, 0xcd, 0xd4, - 0x72, 0x67, 0x9f, 0x73, 0x9b, 0x91, 0xc5, 0xd3, 0x2d, 0x7a, 0x0a, 0x19, 0xa9, 0xc8, 0x06, 0x54, - 0x7c, 0x30, 0x64, 0x1f, 0xdd, 0x5f, 0x4a, 0x4e, 0x58, 0xc9, 0x48, 0xe3, 0x68, 0xbd, 0xa5, 0x9d, - 0x9e, 0x94, 0x95, 0x46, 0xe5, 0xf4, 0xf7, 0x52, 0xe2, 0x74, 0x5c, 0x52, 0xde, 0x8c, 0x4b, 0xca, - 0xdb, 0x71, 0x49, 0xf9, 0x6d, 0x5c, 0x52, 0xbe, 0x3e, 0x2b, 0x25, 0xde, 0x9c, 0x95, 0x12, 0x6f, - 0xcf, 0x4a, 0x89, 0xf6, 0x8a, 0xf8, 0xa7, 0xec, 0xf1, 0x9f, 0x01, 0x00, 0x00, 0xff, 0xff, 0x8e, - 0xd4, 0xe0, 0x2a, 0x00, 0x0e, 0x00, 0x00, +var fileDescriptor_mvcc3_be904b17dffa4ef8 = []byte{ + // 1206 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xcc, 0x57, 0xc1, 0x4f, 0xe3, 0xc6, + 0x17, 0x8e, 0x63, 0x07, 0x92, 0x49, 0x80, 0x64, 0x16, 0xe9, 0x17, 0xf1, 0xdb, 0x8d, 0xd3, 0x1c, + 0x2a, 0xb4, 0xdd, 0x75, 0x2a, 0xb8, 0x71, 0x23, 0xb0, 0x5a, 0x65, 0x5b, 0x16, 0xd6, 0x04, 0x5a, + 0xa9, 0x55, 0xad, 0x89, 0x33, 0x35, 0x16, 0xf6, 0xd8, 0xd8, 0x93, 0xac, 0xf3, 0x37, 0xf4, 0xd2, + 0x63, 0x4f, 0x15, 0xb7, 0x5e, 0x7a, 0xad, 0xd4, 0x4b, 0x6f, 0x3d, 0x70, 0x5c, 0xa9, 0x97, 0x55, + 0x0f, 0x51, 0x1b, 0x2e, 0xfd, 0x1b, 0xe8, 0xa5, 0x9a, 0x19, 0xc7, 0x49, 0xd8, 0x12, 0x10, 0xa5, + 0xa8, 0xb7, 0x99, 0xf7, 0xbd, 0xf7, 0xbd, 0x37, 0x2f, 0xdf, 0xcc, 0x73, 0xc0, 0xc3, 0x90, 0x7a, + 0x01, 0xb2, 0x70, 0x1d, 0x13, 0xcb, 0x26, 0xd8, 0x6f, 0xd7, 0xdd, 0x9e, 0x69, 0xae, 0x6b, 0x7e, + 0xe0, 0x51, 0x0f, 0xae, 0x98, 0x9e, 0x79, 0x1c, 0x78, 0xc8, 0x3c, 0xd2, 0x62, 0x3f, 0x6d, 0xe4, + 0xb7, 0x52, 0xee, 0x52, 0xdb, 0xa9, 0x1f, 0x39, 0x66, 0x9d, 0xda, 0x2e, 0x0e, 0x29, 0x72, 0x7d, + 0x11, 0xb5, 0xb2, 0x6c, 0x79, 0x96, 0xc7, 0x97, 0x75, 0xb6, 0x12, 0xd6, 0xda, 0x0f, 0x32, 0x98, + 0x6f, 0x45, 0x64, 0x07, 0x53, 0x04, 0x5f, 0x81, 0xb4, 0xdd, 0x29, 0x4b, 0x55, 0x69, 0xb5, 0xd0, + 0xd8, 0x3c, 0x1b, 0xa8, 0xa9, 0x5f, 0x07, 0xea, 0xba, 0x65, 0xd3, 0xa3, 0x6e, 0x5b, 0x33, 0x3d, + 0xb7, 0x9e, 0xa4, 0xed, 0xb4, 0xc7, 0xeb, 0xba, 0x7f, 0x6c, 0xd5, 0x79, 0xd2, 0x6e, 0xd7, 0xee, + 0x68, 0x07, 0x07, 0xcd, 0xed, 0xe1, 0x40, 0x4d, 0x37, 0xb7, 0xf5, 0xb4, 0xdd, 0x81, 0x45, 0x20, + 0x1f, 0xe3, 0x7e, 0x59, 0x66, 0x9c, 0x3a, 0x5b, 0xc2, 0x1a, 0xc8, 0x60, 0xdf, 0x33, 0x8f, 0xca, + 0x4a, 0x55, 0x5a, 0xcd, 0x34, 0x0a, 0x17, 0x03, 0x35, 0xdb, 0x8a, 0xc8, 0x33, 0x66, 0xd3, 0x05, + 0x04, 0x3f, 0x07, 0x4b, 0xaf, 0x03, 0x9b, 0x62, 0x23, 0x39, 0x43, 0x39, 0x53, 0x95, 0x56, 0xf3, + 0x6b, 0x8f, 0xb4, 0xf1, 0xd1, 0x59, 0x4e, 0xed, 0xc8, 0x31, 0xb5, 0xd6, 0xc8, 0xa9, 0xb1, 0xcc, + 0x8a, 0xbe, 0x18, 0xa8, 0x85, 0x56, 0x44, 0x12, 0xab, 0xbe, 0xc8, 0xb9, 0x92, 0x3d, 0xfc, 0x00, + 0x64, 0xfd, 0xc0, 0xf6, 0x02, 0x9b, 0xf6, 0xcb, 0x73, 0xbc, 0x88, 0xa5, 0x8b, 0x81, 0x9a, 0x6f, + 0x45, 0x64, 0x2f, 0x36, 0xeb, 0x89, 0x03, 0x7c, 0x1f, 0x64, 0x43, 0x7c, 0xd2, 0xc5, 0xc4, 0xc4, + 0xe5, 0x79, 0xee, 0x0c, 0x2e, 0x06, 0xea, 0x5c, 0x2b, 0x22, 0xfb, 0xf8, 0x44, 0x4f, 0x30, 0xf8, + 0x29, 0x58, 0x70, 0x6d, 0x32, 0x51, 0x70, 0xee, 0xf6, 0x05, 0x17, 0x5c, 0x7b, 0xbc, 0xdb, 0xc8, + 0x7e, 0x73, 0xaa, 0xa6, 0x7e, 0x3c, 0x55, 0xa5, 0x17, 0x4a, 0x36, 0x5d, 0x94, 0x5f, 0x28, 0xd9, + 0x6c, 0x31, 0x57, 0xfb, 0x02, 0xc0, 0xa6, 0x45, 0xbc, 0x00, 0x77, 0xf6, 0xf1, 0xc9, 0xcb, 0xae, + 0xab, 0x23, 0x62, 0x61, 0x58, 0x05, 0x99, 0x90, 0xa2, 0x80, 0xf2, 0x1f, 0x71, 0xba, 0x54, 0x01, + 0xc0, 0x87, 0x40, 0xc6, 0xa4, 0x53, 0x4e, 0xbf, 0x83, 0x33, 0xf3, 0x46, 0x96, 0xe5, 0xf9, 0xe3, + 0x54, 0x95, 0x6a, 0xdf, 0x2a, 0x60, 0x71, 0xe7, 0x70, 0x6b, 0x6b, 0x9f, 0x22, 0x1a, 0x6e, 0x63, + 0x87, 0x22, 0xf8, 0x18, 0x94, 0x1c, 0x14, 0x52, 0xa3, 0xeb, 0x77, 0x10, 0xc5, 0x06, 0x41, 0xc4, + 0x0b, 0x79, 0xa2, 0xa2, 0xbe, 0xc4, 0x80, 0x03, 0x6e, 0x7f, 0xc9, 0xcc, 0xf0, 0x11, 0x00, 0x36, + 0xa1, 0x98, 0x50, 0x03, 0x59, 0x98, 0x67, 0x2b, 0xea, 0x39, 0x61, 0xd9, 0xb4, 0x30, 0xfc, 0x10, + 0x14, 0x2c, 0xd3, 0x68, 0xf7, 0x29, 0x0e, 0xb9, 0x03, 0xd3, 0x47, 0xb1, 0xb1, 0x38, 0x1c, 0xa8, + 0xe0, 0xf9, 0x56, 0x83, 0x99, 0x37, 0x2d, 0xac, 0x03, 0xcb, 0x1c, 0xad, 0x19, 0xa1, 0x63, 0xf7, + 0xb0, 0x88, 0xe1, 0xda, 0x81, 0x7a, 0x8e, 0x59, 0xb8, 0x47, 0x02, 0x9b, 0x5e, 0x97, 0x50, 0x2e, + 0x96, 0x18, 0xde, 0x62, 0x06, 0xf8, 0x7f, 0x90, 0x3b, 0xc6, 0xfd, 0x38, 0x78, 0x8e, 0xa3, 0xd9, + 0x63, 0xdc, 0x17, 0xb1, 0x31, 0x28, 0x42, 0xe7, 0x13, 0x30, 0x89, 0xec, 0x21, 0x27, 0x8e, 0xcc, + 0x0a, 0xb0, 0x87, 0x9c, 0x24, 0x92, 0x81, 0x22, 0x32, 0x97, 0x80, 0x22, 0xf2, 0x3d, 0x50, 0x88, + 0x5b, 0x20, 0x82, 0x01, 0xc7, 0xf3, 0xc2, 0x26, 0xe2, 0xc7, 0x2e, 0x82, 0x22, 0x3f, 0xe9, 0x92, + 0xe4, 0x0f, 0xfb, 0x61, 0x4c, 0x51, 0x10, 0x29, 0xc2, 0x7e, 0x98, 0xe4, 0x67, 0xa0, 0x08, 0x5e, + 0x48, 0x40, 0x11, 0xf9, 0x14, 0x40, 0xd3, 0x23, 0x14, 0xd9, 0x24, 0x34, 0x70, 0x48, 0x6d, 0x17, + 0x31, 0x8a, 0xc5, 0xaa, 0xb4, 0x2a, 0xeb, 0xa5, 0x11, 0xf2, 0x6c, 0x04, 0xc0, 0x55, 0x50, 0x44, + 0x6d, 0x2f, 0xa0, 0x46, 0xe8, 0x23, 0x12, 0xe7, 0x5b, 0xe2, 0x94, 0x8b, 0xdc, 0xbe, 0xef, 0x23, + 0xc2, 0xb3, 0x6e, 0x28, 0x5c, 0x20, 0xdf, 0x29, 0xe0, 0x01, 0x13, 0xc8, 0x1e, 0x0e, 0x42, 0x3b, + 0x64, 0x05, 0x73, 0xa9, 0xfc, 0xd7, 0x54, 0x22, 0xcf, 0x56, 0x89, 0x3c, 0x53, 0x25, 0xf2, 0x2c, + 0x95, 0xc8, 0xb3, 0x54, 0x22, 0xcf, 0x52, 0x89, 0x7c, 0x8d, 0x4a, 0xe4, 0xeb, 0x55, 0x22, 0x5f, + 0xa3, 0x12, 0x79, 0x96, 0x4a, 0xe4, 0xbb, 0x57, 0x89, 0xfc, 0x8e, 0x4a, 0xc6, 0x4f, 0xc9, 0x2f, + 0x12, 0x28, 0xf1, 0xe7, 0x69, 0xd3, 0xf7, 0x1d, 0x1b, 0x77, 0x98, 0x4e, 0x30, 0x7c, 0x02, 0x60, + 0x80, 0xbe, 0xa4, 0x06, 0x12, 0x46, 0xc3, 0x26, 0x1d, 0x1c, 0x71, 0xa1, 0x28, 0x7a, 0x91, 0x21, + 0xb1, 0x77, 0x93, 0xd9, 0xa1, 0x06, 0x1e, 0x38, 0x18, 0x85, 0xf8, 0x92, 0x7b, 0x9a, 0xbb, 0x97, + 0x38, 0x34, 0xe5, 0x7f, 0x08, 0xf2, 0x01, 0x4b, 0x69, 0x84, 0x4c, 0x94, 0x5c, 0x39, 0xf9, 0xb5, + 0xba, 0x76, 0xf5, 0xe0, 0xd4, 0xfe, 0x46, 0xcb, 0x0d, 0x85, 0x3d, 0xcf, 0x3a, 0xe0, 0x4c, 0xdc, + 0x32, 0x71, 0xaa, 0xef, 0x25, 0x50, 0x64, 0x31, 0x9f, 0xb0, 0xe1, 0x72, 0x88, 0x9c, 0x2e, 0xde, + 0xf5, 0x47, 0xe3, 0x4e, 0x1a, 0x8f, 0xbb, 0x57, 0x20, 0x37, 0x9e, 0x09, 0xe9, 0xdb, 0xcf, 0x84, + 0x31, 0x0b, 0x5c, 0x06, 0x99, 0x1e, 0xcb, 0x17, 0x4f, 0x55, 0xb1, 0x61, 0xda, 0xf6, 0x03, 0xdc, + 0x33, 0x04, 0xa4, 0x70, 0x28, 0xc7, 0x2c, 0xbc, 0xb6, 0xda, 0x4f, 0x69, 0x50, 0x4a, 0xca, 0x6d, + 0x72, 0xe9, 0xec, 0xfa, 0xf0, 0x33, 0x30, 0x47, 0x23, 0x62, 0x24, 0x53, 0x7f, 0xfb, 0x9f, 0x4d, + 0xfd, 0x4c, 0x2b, 0x22, 0xcd, 0x6d, 0x3d, 0x43, 0x23, 0xd2, 0xec, 0xc0, 0xff, 0x81, 0x79, 0x46, + 0xce, 0x1a, 0x92, 0xe6, 0xe5, 0xb0, 0x5c, 0x1f, 0x5d, 0xee, 0x89, 0x7c, 0x27, 0x3d, 0x31, 0x40, + 0x89, 0xe5, 0x9a, 0x1e, 0xc1, 0xca, 0xed, 0xa9, 0x97, 0x68, 0x44, 0x76, 0x26, 0xa6, 0x70, 0xed, + 0x67, 0x09, 0x40, 0xd6, 0x3f, 0xf1, 0x7c, 0xdd, 0x4f, 0x03, 0xef, 0x5e, 0x3b, 0xb5, 0xaf, 0xd2, + 0xe2, 0x18, 0x5b, 0x9e, 0xeb, 0xda, 0xf4, 0x7e, 0x8e, 0x11, 0x5f, 0x8a, 0xf4, 0x15, 0x97, 0x42, + 0xbe, 0xdb, 0x4b, 0xa1, 0x5c, 0x7d, 0x29, 0x32, 0x97, 0x2f, 0x85, 0x2f, 0xee, 0xc4, 0x26, 0x7b, + 0xb9, 0xee, 0xa5, 0x17, 0x35, 0x57, 0x7c, 0x55, 0xf1, 0x8c, 0xad, 0x88, 0xfc, 0xdb, 0xe9, 0xfe, + 0x94, 0xc1, 0x02, 0xcb, 0xf7, 0xb1, 0x67, 0xd9, 0x26, 0x72, 0x76, 0x7d, 0xb8, 0x03, 0xf2, 0xe2, + 0xd3, 0x5a, 0xb4, 0x44, 0xe2, 0xcd, 0x7f, 0x72, 0xdd, 0xc3, 0x38, 0xf9, 0xc8, 0xe9, 0xe0, 0x75, + 0xb2, 0x83, 0x7b, 0xa0, 0x20, 0xe8, 0xc4, 0x34, 0x8a, 0x55, 0xfa, 0xf4, 0x46, 0x7c, 0xa3, 0x8e, + 0xeb, 0xa2, 0x22, 0xb1, 0x85, 0xfb, 0x60, 0x21, 0xfe, 0x74, 0x88, 0x29, 0x85, 0x3e, 0xb4, 0xeb, + 0x28, 0xa7, 0x2f, 0xa6, 0x5e, 0xe8, 0x4e, 0xec, 0x19, 0xa9, 0xc9, 0x15, 0x3f, 0x22, 0x55, 0x6e, + 0x46, 0x3a, 0x7d, 0x4d, 0xf4, 0x82, 0x39, 0xb1, 0x67, 0x67, 0x17, 0xb3, 0x30, 0xe6, 0xcc, 0xdc, + 0xec, 0xec, 0x53, 0x6a, 0xd3, 0xf3, 0x68, 0xbc, 0x85, 0xcf, 0x41, 0x4e, 0x30, 0xd2, 0x88, 0xf0, + 0x0f, 0x90, 0xfc, 0xda, 0xe3, 0x1b, 0xd1, 0x71, 0x29, 0xe9, 0x59, 0x14, 0xaf, 0x37, 0x94, 0xb3, + 0x53, 0x55, 0x6a, 0x54, 0xcf, 0x7e, 0xaf, 0xa4, 0xce, 0x86, 0x15, 0xe9, 0xcd, 0xb0, 0x22, 0xbd, + 0x1d, 0x56, 0xa4, 0xdf, 0x86, 0x15, 0xe9, 0xeb, 0xf3, 0x4a, 0xea, 0xcd, 0x79, 0x25, 0xf5, 0xf6, + 0xbc, 0x92, 0x6a, 0xcf, 0xf1, 0x3f, 0x81, 0xeb, 0x7f, 0x05, 0x00, 0x00, 0xff, 0xff, 0x12, 0xac, + 0x2d, 0x43, 0x70, 0x0e, 0x00, 0x00, } diff --git a/pkg/storage/enginepb/mvcc3.proto b/pkg/storage/enginepb/mvcc3.proto index bd8f3e8bf20e..d702e2e0a3b8 100644 --- a/pkg/storage/enginepb/mvcc3.proto +++ b/pkg/storage/enginepb/mvcc3.proto @@ -87,7 +87,8 @@ message TxnMeta { // value will need to be rewritten at the forwarded timestamp if the // transaction commits. // - util.hlc.Timestamp write_timestamp = 5 [(gogoproto.nullable) = false]; + util.hlc.Timestamp write_timestamp = 5 [(gogoproto.nullable) = false, + (gogoproto.casttype) = "TxnTimestamp"]; // The timestamp that the transaction was assigned by its gateway when it // began its first epoch. This is the earliest timestamp that the transaction // could have written any of its intents at. @@ -106,7 +107,8 @@ 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. - util.hlc.Timestamp min_timestamp = 9 [(gogoproto.nullable) = false]; + util.hlc.Timestamp min_timestamp = 9 [(gogoproto.nullable) = false, + (gogoproto.casttype) = "TxnTimestamp"]; // The transaction's priority, ratcheted on transaction pushes. int32 priority = 6 [(gogoproto.casttype) = "TxnPriority"]; // A zero-indexed sequence number which is increased on each request @@ -202,7 +204,7 @@ message RangeAppliedState { // transaction. message MVCCWriteValueOp { bytes key = 1; - util.hlc.Timestamp timestamp = 2 [(gogoproto.nullable) = false]; + util.hlc.Timestamp timestamp = 2 [(gogoproto.nullable) = false, (gogoproto.casttype) = "TxnTimestamp"]; bytes value = 3; bytes prev_value = 4; } @@ -215,8 +217,8 @@ message MVCCWriteIntentOp { (gogoproto.customname) = "TxnID", (gogoproto.nullable) = false]; bytes txn_key = 2; - util.hlc.Timestamp txn_min_timestamp = 4 [(gogoproto.nullable) = false]; - util.hlc.Timestamp timestamp = 3 [(gogoproto.nullable) = false]; + util.hlc.Timestamp txn_min_timestamp = 4 [(gogoproto.nullable) = false, (gogoproto.casttype) = "TxnTimestamp"]; + util.hlc.Timestamp timestamp = 3 [(gogoproto.nullable) = false, (gogoproto.casttype) = "TxnTimestamp"]; } // MVCCUpdateIntentOp corresponds to an intent being updates at a larger @@ -226,7 +228,7 @@ message MVCCUpdateIntentOp { (gogoproto.customtype) = "github.com/cockroachdb/cockroach/pkg/util/uuid.UUID", (gogoproto.customname) = "TxnID", (gogoproto.nullable) = false]; - util.hlc.Timestamp timestamp = 2 [(gogoproto.nullable) = false]; + util.hlc.Timestamp timestamp = 2 [(gogoproto.nullable) = false, (gogoproto.casttype) = "TxnTimestamp"]; } // MVCCCommitIntentOp corresponds to an intent being committed for a given @@ -237,7 +239,7 @@ message MVCCCommitIntentOp { (gogoproto.customname) = "TxnID", (gogoproto.nullable) = false]; bytes key = 2; - util.hlc.Timestamp timestamp = 3 [(gogoproto.nullable) = false]; + util.hlc.Timestamp timestamp = 3 [(gogoproto.nullable) = false, (gogoproto.casttype) = "TxnTimestamp"]; bytes value = 4; bytes prev_value = 5; } diff --git a/pkg/storage/enginepb/mvcc_test.go b/pkg/storage/enginepb/mvcc_test.go index 946517c24c52..35413716cad6 100644 --- a/pkg/storage/enginepb/mvcc_test.go +++ b/pkg/storage/enginepb/mvcc_test.go @@ -16,7 +16,6 @@ import ( "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/uuid" "github.com/stretchr/testify/assert" ) @@ -26,7 +25,7 @@ func TestFormatMVCCMetadata(t *testing.T) { if err != nil { t.Fatal(err) } - ts := hlc.Timestamp{Logical: 1} + ts := enginepb.TxnTimestamp{Logical: 1} tmeta := &enginepb.TxnMeta{ Key: roachpb.Key("a"), ID: txnID, diff --git a/pkg/storage/intent_interleaving_iter_test.go b/pkg/storage/intent_interleaving_iter_test.go index 4c6b10b74679..ff305e5e698c 100644 --- a/pkg/storage/intent_interleaving_iter_test.go +++ b/pkg/storage/intent_interleaving_iter_test.go @@ -256,7 +256,7 @@ func TestIntentInterleavingIter(t *testing.T) { d.ScanArgs(t, "ts", &ts) var value string d.ScanArgs(t, "v", &value) - mvccKey := MVCCKey{Key: key, Timestamp: hlc.Timestamp{WallTime: int64(ts)}} + mvccKey := MVCCKey{Key: key, Timestamp: enginepb.TxnTimestamp{WallTime: int64(ts)}} if err := batch.PutMVCC(mvccKey, []byte(value)); err != nil { return err.Error() } @@ -380,7 +380,7 @@ func generateRandomData(t *testing.T, rng *rand.Rand) (lkv []lockKeyValue, mvcck } for _, ts := range timestamps { mvcckv = append(mvcckv, MVCCKeyValue{ - Key: MVCCKey{Key: key, Timestamp: hlc.Timestamp{WallTime: int64(ts) + 1}}, + Key: MVCCKey{Key: key, Timestamp: enginepb.TxnTimestamp{WallTime: int64(ts) + 1}}, Value: []byte("value"), }) } @@ -433,7 +433,7 @@ func generateIterOps(rng *rand.Rand, mvcckv []MVCCKeyValue) []string { useTimestamp := rng.Intn(2) == 0 seekKey := mvcckv[seekIndex].Key if !useTimestamp { - seekKey.Timestamp = hlc.Timestamp{} + seekKey.Timestamp = enginepb.TxnTimestamp{} } op := "seek-ge" fwdDirection := true @@ -582,7 +582,7 @@ func writeBenchData( } for j := versionsPerKey; j >= 1; j-- { require.NoError(b, batch.PutMVCC( - MVCCKey{Key: key, Timestamp: hlc.Timestamp{WallTime: int64(j)}}, []byte("value"))) + MVCCKey{Key: key, Timestamp: enginepb.TxnTimestamp{WallTime: int64(j)}}, []byte("value"))) } } require.NoError(b, batch.Commit(true)) diff --git a/pkg/storage/metamorphic/generator.go b/pkg/storage/metamorphic/generator.go index 9647ec876006..a73a8ff37e38 100644 --- a/pkg/storage/metamorphic/generator.go +++ b/pkg/storage/metamorphic/generator.go @@ -23,7 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/storage" - "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/pebble" ) @@ -511,7 +511,7 @@ func (m *metaTestRunner) printComment(comment string) { // Monotonically increasing timestamp generator. type tsGenerator struct { - lastTS hlc.Timestamp + lastTS enginepb.TxnTimestamp zipf *rand.Zipf } @@ -519,13 +519,13 @@ func (t *tsGenerator) init(rng *rand.Rand) { t.zipf = rand.NewZipf(rng, 2, 5, zipfMax) } -func (t *tsGenerator) generate() hlc.Timestamp { +func (t *tsGenerator) generate() enginepb.TxnTimestamp { t.lastTS.WallTime++ return t.lastTS } -func (t *tsGenerator) randomPastTimestamp(rng *rand.Rand) hlc.Timestamp { - var result hlc.Timestamp +func (t *tsGenerator) randomPastTimestamp(rng *rand.Rand) enginepb.TxnTimestamp { + var result enginepb.TxnTimestamp // Return a result that's skewed toward the latest wall time. result.WallTime = int64(float64(t.lastTS.WallTime) * float64((zipfMax - t.zipf.Uint64())) / float64(zipfMax)) diff --git a/pkg/storage/metamorphic/operands.go b/pkg/storage/metamorphic/operands.go index 5b574741cbed..6a97b190bdab 100644 --- a/pkg/storage/metamorphic/operands.go +++ b/pkg/storage/metamorphic/operands.go @@ -17,7 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" - "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" ) type operandType int @@ -274,12 +274,12 @@ func (t *pastTSGenerator) closeAll() { // No-op. } -func (t *pastTSGenerator) toString(ts hlc.Timestamp) string { +func (t *pastTSGenerator) toString(ts enginepb.TxnTimestamp) string { return fmt.Sprintf("%d", ts.WallTime) } -func (t *pastTSGenerator) parse(input string) hlc.Timestamp { - var ts hlc.Timestamp +func (t *pastTSGenerator) parse(input string) enginepb.TxnTimestamp { + var ts enginepb.TxnTimestamp wallTime, err := strconv.ParseInt(input, 10, 0) if err != nil { panic(err) diff --git a/pkg/storage/metamorphic/operations.go b/pkg/storage/metamorphic/operations.go index 036dfd8522a7..db61a612513b 100644 --- a/pkg/storage/metamorphic/operations.go +++ b/pkg/storage/metamorphic/operations.go @@ -22,7 +22,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" - "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/uint128" "github.com/cockroachdb/cockroach/pkg/util/uuid" ) @@ -115,7 +114,7 @@ func generateMVCCScan( if endKey.Less(key) { key, endKey = endKey, key } - var ts hlc.Timestamp + var ts enginepb.TxnTimestamp var txn txnID if inconsistent { ts = m.pastTSGenerator.parse(args[2]) @@ -152,7 +151,7 @@ type mvccGetOp struct { m *metaTestRunner reader readWriterID key roachpb.Key - ts hlc.Timestamp + ts enginepb.TxnTimestamp txn txnID inconsistent bool } @@ -286,8 +285,8 @@ type mvccClearTimeRangeOp struct { writer readWriterID key roachpb.Key endKey roachpb.Key - startTime hlc.Timestamp - endTime hlc.Timestamp + startTime enginepb.TxnTimestamp + endTime enginepb.TxnTimestamp } func (m mvccClearTimeRangeOp) run(ctx context.Context) string { @@ -343,7 +342,7 @@ type mvccScanOp struct { m *metaTestRunner key roachpb.Key endKey roachpb.Key - ts hlc.Timestamp + ts enginepb.TxnTimestamp txn txnID inconsistent bool reverse bool @@ -380,7 +379,7 @@ func (m mvccScanOp) run(ctx context.Context) string { type txnOpenOp struct { m *metaTestRunner id txnID - ts hlc.Timestamp + ts enginepb.TxnTimestamp } func (t txnOpenOp) run(ctx context.Context) string { diff --git a/pkg/storage/multi_iterator_test.go b/pkg/storage/multi_iterator_test.go index a9634a7ed09d..95516a9496fd 100644 --- a/pkg/storage/multi_iterator_test.go +++ b/pkg/storage/multi_iterator_test.go @@ -18,7 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" ) @@ -88,10 +88,10 @@ func TestMultiIterator(t *testing.T) { break } k := []byte{input[i]} - ts := hlc.Timestamp{WallTime: int64(input[i+1])} + ts := enginepb.TxnTimestamp{WallTime: int64(input[i+1])} var v []byte if i+1 < len(input) && input[i+1] == 'M' { - ts = hlc.Timestamp{} + ts = enginepb.TxnTimestamp{} v = nil } else if i+2 < len(input) && input[i+2] == 'X' { v = nil diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go index 9955a7aeafd5..0a0019d17df9 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -94,7 +94,7 @@ func emptyKeyError() error { // of a timestamp. type MVCCKey struct { Key roachpb.Key - Timestamp hlc.Timestamp + Timestamp enginepb.TxnTimestamp } // MakeMVCCMetadataKey creates an MVCCKey from a roachpb.Key. @@ -705,7 +705,7 @@ func MVCCGetProto( ctx context.Context, reader Reader, key roachpb.Key, - timestamp hlc.Timestamp, + timestamp enginepb.TxnTimestamp, msg protoutil.Message, opts MVCCGetOptions, ) (bool, error) { @@ -731,7 +731,7 @@ func MVCCPutProto( rw ReadWriter, ms *enginepb.MVCCStats, key roachpb.Key, - timestamp hlc.Timestamp, + timestamp enginepb.TxnTimestamp, txn *roachpb.Transaction, msg protoutil.Message, ) error { @@ -751,7 +751,7 @@ func MVCCBlindPutProto( writer Writer, ms *enginepb.MVCCStats, key roachpb.Key, - timestamp hlc.Timestamp, + timestamp enginepb.TxnTimestamp, msg protoutil.Message, txn *roachpb.Transaction, ) error { @@ -806,7 +806,7 @@ func newMVCCIterator(reader Reader, inlineMeta bool, opts IterOptions) MVCCItera // Note that transactional gets must be consistent. Put another way, only // non-transactional gets may be inconsistent. // -// If the timestamp is specified as hlc.Timestamp{}, the value is expected to be +// If the timestamp is specified as enginepb.TxnTimestamp{}, the value is expected to be // "inlined". See MVCCPut(). // // When reading in "fail on more recent" mode, a WriteTooOldError will be @@ -815,7 +815,7 @@ func newMVCCIterator(reader Reader, inlineMeta bool, opts IterOptions) MVCCItera // observes another transaction's intent, even if it has a timestamp above // the read timestamp. func MVCCGet( - ctx context.Context, reader Reader, key roachpb.Key, timestamp hlc.Timestamp, opts MVCCGetOptions, + ctx context.Context, reader Reader, key roachpb.Key, timestamp enginepb.TxnTimestamp, opts MVCCGetOptions, ) (*roachpb.Value, *roachpb.Intent, error) { iter := newMVCCIterator(reader, timestamp.IsEmpty(), IterOptions{Prefix: true}) defer iter.Close() @@ -827,7 +827,7 @@ func mvccGet( ctx context.Context, iter MVCCIterator, key roachpb.Key, - timestamp hlc.Timestamp, + timestamp enginepb.TxnTimestamp, opts MVCCGetOptions, ) (value optionalValue, intent *roachpb.Intent, err error) { if len(key) == 0 { @@ -903,7 +903,7 @@ func MVCCGetAsTxn( ctx context.Context, reader Reader, key roachpb.Key, - timestamp hlc.Timestamp, + timestamp enginepb.TxnTimestamp, txnMeta enginepb.TxnMeta, ) (*roachpb.Value, *roachpb.Intent, error) { return MVCCGet(ctx, reader, key, timestamp, MVCCGetOptions{ @@ -1031,7 +1031,7 @@ func (b *putBuffer) putIntentMeta( txnDidNotUpdateMeta bool, meta *enginepb.MVCCMetadata, ) (keyBytes, valBytes int64, err error) { - if meta.Txn != nil && meta.Timestamp.ToTimestamp() != meta.Txn.WriteTimestamp { + if meta.Txn != nil && meta.TxnTimestamp() != meta.Txn.WriteTimestamp { // The timestamps are supposed to be in sync. If they weren't, it wouldn't // be clear for readers which one to use for what. return 0, 0, errors.AssertionFailedf( @@ -1071,7 +1071,7 @@ func (b *putBuffer) putIntentMeta( // dictate the timestamp of the operation, and the timestamp parameter is // confusing and redundant. See the comment on mvccPutInternal for details. // -// If the timestamp is specified as hlc.Timestamp{}, the value is +// If the timestamp is specified as enginepb.TxnTimestamp{}, the value is // inlined instead of being written as a timestamp-versioned value. A // zero timestamp write to a key precludes a subsequent write using a // non-zero timestamp and vice versa. Inlined values require only a @@ -1083,7 +1083,7 @@ func MVCCPut( rw ReadWriter, ms *enginepb.MVCCStats, key roachpb.Key, - timestamp hlc.Timestamp, + timestamp enginepb.TxnTimestamp, value roachpb.Value, txn *roachpb.Transaction, ) error { @@ -1113,7 +1113,7 @@ func MVCCBlindPut( writer Writer, ms *enginepb.MVCCStats, key roachpb.Key, - timestamp hlc.Timestamp, + timestamp enginepb.TxnTimestamp, value roachpb.Value, txn *roachpb.Transaction, ) error { @@ -1131,7 +1131,7 @@ func MVCCDelete( rw ReadWriter, ms *enginepb.MVCCStats, key roachpb.Key, - timestamp hlc.Timestamp, + timestamp enginepb.TxnTimestamp, txn *roachpb.Transaction, ) error { iter := newMVCCIterator(rw, timestamp.IsEmpty(), IterOptions{Prefix: true}) @@ -1152,7 +1152,7 @@ func mvccPutUsingIter( iter MVCCIterator, ms *enginepb.MVCCStats, key roachpb.Key, - timestamp hlc.Timestamp, + timestamp enginepb.TxnTimestamp, value roachpb.Value, txn *roachpb.Transaction, valueFn func(optionalValue) ([]byte, error), @@ -1183,7 +1183,7 @@ func maybeGetValue( key roachpb.Key, value []byte, exists bool, - readTimestamp hlc.Timestamp, + readTimestamp enginepb.TxnTimestamp, txn *roachpb.Transaction, valueFn func(optionalValue) ([]byte, error), ) ([]byte, error) { @@ -1251,7 +1251,7 @@ func replayTransactionalWrite( iter MVCCIterator, meta *enginepb.MVCCMetadata, key roachpb.Key, - timestamp hlc.Timestamp, + timestamp enginepb.TxnTimestamp, value []byte, txn *roachpb.Transaction, valueFn func(optionalValue) ([]byte, error), @@ -1341,15 +1341,15 @@ func replayTransactionalWrite( // In an attempt to reduce confusion about which timestamp applies, when writing // transactionally, the timestamp parameter must be equal to the transaction's // read timestamp. (One could imagine instead requiring that the timestamp -// parameter be set to hlc.Timestamp{} when writing transactionally, but -// hlc.Timestamp{} is already used as a sentinel for inline puts.) +// parameter be set to enginepb.TxnTimestamp{} when writing transactionally, but +// enginepb.TxnTimestamp{} is already used as a sentinel for inline puts.) func mvccPutInternal( ctx context.Context, writer Writer, iter MVCCIterator, ms *enginepb.MVCCStats, key roachpb.Key, - timestamp hlc.Timestamp, + timestamp enginepb.TxnTimestamp, value []byte, txn *roachpb.Transaction, buf *putBuffer, @@ -1444,7 +1444,7 @@ func mvccPutInternal( writeTimestamp = txn.WriteTimestamp } - timestamp = hlc.Timestamp{} // prevent accidental use below + timestamp = enginepb.TxnTimestamp{} // prevent accidental use below // Determine what the logical operation is. Are we writing an intent // or a value directly? @@ -1463,7 +1463,7 @@ func mvccPutInternal( if ok { // There is existing metadata for this key; ensure our write is permitted. meta = &buf.meta - metaTimestamp := meta.Timestamp.ToTimestamp() + metaTimestamp := meta.TxnTimestamp() if meta.Txn != nil { // There is an uncommitted write intent. @@ -1760,7 +1760,7 @@ func MVCCIncrement( rw ReadWriter, ms *enginepb.MVCCStats, key roachpb.Key, - timestamp hlc.Timestamp, + timestamp enginepb.TxnTimestamp, txn *roachpb.Transaction, inc int64, ) (int64, error) { @@ -1830,7 +1830,7 @@ func MVCCConditionalPut( rw ReadWriter, ms *enginepb.MVCCStats, key roachpb.Key, - timestamp hlc.Timestamp, + timestamp enginepb.TxnTimestamp, value roachpb.Value, expVal []byte, allowIfDoesNotExist CPutMissingBehavior, @@ -1856,7 +1856,7 @@ func MVCCBlindConditionalPut( writer Writer, ms *enginepb.MVCCStats, key roachpb.Key, - timestamp hlc.Timestamp, + timestamp enginepb.TxnTimestamp, value roachpb.Value, expVal []byte, allowIfDoesNotExist CPutMissingBehavior, @@ -1871,7 +1871,7 @@ func mvccConditionalPutUsingIter( iter MVCCIterator, ms *enginepb.MVCCStats, key roachpb.Key, - timestamp hlc.Timestamp, + timestamp enginepb.TxnTimestamp, value roachpb.Value, expBytes []byte, allowNoExisting CPutMissingBehavior, @@ -1909,7 +1909,7 @@ func MVCCInitPut( rw ReadWriter, ms *enginepb.MVCCStats, key roachpb.Key, - timestamp hlc.Timestamp, + timestamp enginepb.TxnTimestamp, value roachpb.Value, failOnTombstones bool, txn *roachpb.Transaction, @@ -1932,7 +1932,7 @@ func MVCCBlindInitPut( rw ReadWriter, ms *enginepb.MVCCStats, key roachpb.Key, - timestamp hlc.Timestamp, + timestamp enginepb.TxnTimestamp, value roachpb.Value, failOnTombstones bool, txn *roachpb.Transaction, @@ -1946,7 +1946,7 @@ func mvccInitPutUsingIter( iter MVCCIterator, ms *enginepb.MVCCStats, key roachpb.Key, - timestamp hlc.Timestamp, + timestamp enginepb.TxnTimestamp, value roachpb.Value, failOnTombstones bool, txn *roachpb.Transaction, @@ -1996,7 +1996,7 @@ func MVCCMerge( rw ReadWriter, ms *enginepb.MVCCStats, key roachpb.Key, - timestamp hlc.Timestamp, + timestamp enginepb.TxnTimestamp, value roachpb.Value, ) error { if len(key) == 0 { @@ -2049,7 +2049,7 @@ func MVCCClearTimeRange( rw ReadWriter, ms *enginepb.MVCCStats, key, endKey roachpb.Key, - startTime, endTime hlc.Timestamp, + startTime, endTime enginepb.TxnTimestamp, maxBatchSize int64, ) (*roachpb.Span, error) { var batchSize int64 @@ -2179,7 +2179,7 @@ func MVCCClearTimeRange( if err := it.ValueProto(&meta); err != nil { return nil, err } - ts := meta.Timestamp.ToTimestamp() + ts := meta.TxnTimestamp() if meta.Txn != nil && startTime.Less(ts) && ts.LessEq(endTime) { err := &roachpb.WriteIntentError{ Intents: []roachpb.Intent{ @@ -2231,7 +2231,7 @@ func MVCCDeleteRange( ms *enginepb.MVCCStats, key, endKey roachpb.Key, max int64, - timestamp hlc.Timestamp, + timestamp enginepb.TxnTimestamp, txn *roachpb.Transaction, returnKeys bool, ) ([]roachpb.Key, *roachpb.Span, int64, error) { @@ -2275,7 +2275,7 @@ func mvccScanToBytes( ctx context.Context, iter MVCCIterator, key, endKey roachpb.Key, - timestamp hlc.Timestamp, + timestamp enginepb.TxnTimestamp, opts MVCCScanOptions, ) (MVCCScanResult, error) { if len(endKey) == 0 { @@ -2337,7 +2337,7 @@ func mvccScanToKvs( ctx context.Context, iter MVCCIterator, key, endKey roachpb.Key, - timestamp hlc.Timestamp, + timestamp enginepb.TxnTimestamp, opts MVCCScanOptions, ) (MVCCScanResult, error) { res, err := mvccScanToBytes(ctx, iter, key, endKey, timestamp, opts) @@ -2479,7 +2479,7 @@ func MVCCScan( ctx context.Context, reader Reader, key, endKey roachpb.Key, - timestamp hlc.Timestamp, + timestamp enginepb.TxnTimestamp, opts MVCCScanOptions, ) (MVCCScanResult, error) { iter := newMVCCIterator(reader, timestamp.IsEmpty(), IterOptions{LowerBound: key, UpperBound: endKey}) @@ -2492,7 +2492,7 @@ func MVCCScanToBytes( ctx context.Context, reader Reader, key, endKey roachpb.Key, - timestamp hlc.Timestamp, + timestamp enginepb.TxnTimestamp, opts MVCCScanOptions, ) (MVCCScanResult, error) { iter := newMVCCIterator(reader, timestamp.IsEmpty(), IterOptions{LowerBound: key, UpperBound: endKey}) @@ -2510,7 +2510,7 @@ func MVCCScanAsTxn( ctx context.Context, reader Reader, key, endKey roachpb.Key, - timestamp hlc.Timestamp, + timestamp enginepb.TxnTimestamp, txnMeta enginepb.TxnMeta, ) (MVCCScanResult, error) { return MVCCScan(ctx, reader, key, endKey, timestamp, MVCCScanOptions{ @@ -2533,7 +2533,7 @@ func MVCCIterate( ctx context.Context, reader Reader, key, endKey roachpb.Key, - timestamp hlc.Timestamp, + timestamp enginepb.TxnTimestamp, opts MVCCScanOptions, f func(roachpb.KeyValue) error, ) ([]roachpb.Intent, error) { @@ -2673,7 +2673,7 @@ func mvccResolveWriteIntent( if !ok || meta.Txn == nil || intent.Txn.ID != meta.Txn.ID { return false, nil } - metaTimestamp := meta.Timestamp.ToTimestamp() + metaTimestamp := meta.TxnTimestamp() precedingIntentState := ExistingIntentInterleaved if isIntentSeparated { precedingIntentState = ExistingIntentSeparated @@ -3111,7 +3111,7 @@ func MVCCGarbageCollect( rw ReadWriter, ms *enginepb.MVCCStats, keys []roachpb.GCRequest_GCKey, - timestamp hlc.Timestamp, + timestamp enginepb.TxnTimestamp, ) error { var count int64 @@ -3163,7 +3163,7 @@ func MVCCGarbageCollect( // being removed. We had this faulty functionality at some point; it // should no longer be necessary since the higher levels already make // sure each individual GCRequest does bounded work. - if meta.Timestamp.ToTimestamp().LessEq(gcKey.Timestamp) { + if meta.TxnTimestamp().LessEq(gcKey.Timestamp) { // For version keys, don't allow GC'ing the meta key if it's // not marked deleted. However, for inline values we allow it; // they are internal and GCing them directly saves the extra diff --git a/pkg/storage/mvcc_history_test.go b/pkg/storage/mvcc_history_test.go index 94a24ff4ab69..5dc1929dbd78 100644 --- a/pkg/storage/mvcc_history_test.go +++ b/pkg/storage/mvcc_history_test.go @@ -22,7 +22,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/testutils/skip" - "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/protoutil" @@ -866,12 +865,12 @@ func (e *evalCtx) getResolve() (bool, roachpb.TransactionStatus) { return true, e.getTxnStatus() } -func (e *evalCtx) getTs(txn *roachpb.Transaction) hlc.Timestamp { +func (e *evalCtx) getTs(txn *roachpb.Transaction) enginepb.TxnTimestamp { return e.getTsWithName(txn, "ts") } -func (e *evalCtx) getTsWithName(txn *roachpb.Transaction, name string) hlc.Timestamp { - var ts hlc.Timestamp +func (e *evalCtx) getTsWithName(txn *roachpb.Transaction, name string) enginepb.TxnTimestamp { + var ts enginepb.TxnTimestamp if txn != nil { ts = txn.ReadTimestamp } @@ -994,7 +993,7 @@ func (e *evalCtx) getKeyRange() (sk, ek roachpb.Key) { } func (e *evalCtx) newTxn( - txnName string, ts, maxTs hlc.Timestamp, key roachpb.Key, + txnName string, ts, maxTs enginepb.TxnTimestamp, key roachpb.Key, ) (*roachpb.Transaction, error) { if _, ok := e.txns[txnName]; ok { e.Fatalf("txn %s already open", txnName) diff --git a/pkg/storage/mvcc_incremental_iterator.go b/pkg/storage/mvcc_incremental_iterator.go index c0881f53703a..5354e6272123 100644 --- a/pkg/storage/mvcc_incremental_iterator.go +++ b/pkg/storage/mvcc_incremental_iterator.go @@ -13,7 +13,6 @@ package storage import ( "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/protoutil" "github.com/cockroachdb/errors" ) @@ -78,8 +77,8 @@ type MVCCIncrementalIterator struct { // iterator. timeBoundIter MVCCIterator - startTime hlc.Timestamp - endTime hlc.Timestamp + startTime enginepb.TxnTimestamp + endTime enginepb.TxnTimestamp err error valid bool @@ -97,8 +96,8 @@ type MVCCIncrementalIterOptions struct { // EndTime]. Note that if {Min,Max}TimestampHints are specified in // IterOptions, the timestamp hints interval should include the start and end // time. - StartTime hlc.Timestamp - EndTime hlc.Timestamp + StartTime enginepb.TxnTimestamp + EndTime enginepb.TxnTimestamp } // NewMVCCIncrementalIterator creates an MVCCIncrementalIterator with the @@ -303,7 +302,7 @@ func (i *MVCCIncrementalIterator) advance() { return } - metaTimestamp := i.meta.Timestamp.ToTimestamp() + metaTimestamp := i.meta.TxnTimestamp() if i.meta.Txn != nil { if i.startTime.Less(metaTimestamp) && metaTimestamp.LessEq(i.endTime) { i.err = &roachpb.WriteIntentError{ diff --git a/pkg/storage/mvcc_incremental_iterator_test.go b/pkg/storage/mvcc_incremental_iterator_test.go index 28fb9f0dd017..faa0cb4723c1 100644 --- a/pkg/storage/mvcc_incremental_iterator_test.go +++ b/pkg/storage/mvcc_incremental_iterator_test.go @@ -22,7 +22,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/testutils" - "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/uuid" @@ -37,7 +36,7 @@ const all, latest = true, false func iterateExpectErr( e Engine, startKey, endKey roachpb.Key, - startTime, endTime hlc.Timestamp, + startTime, endTime enginepb.TxnTimestamp, revisions bool, errString string, ) func(*testing.T) { @@ -73,7 +72,7 @@ func assertExportedKVs( t *testing.T, e Engine, startKey, endKey roachpb.Key, - startTime, endTime hlc.Timestamp, + startTime, endTime enginepb.TxnTimestamp, revisions bool, io IterOptions, expected []MVCCKeyValue, @@ -112,7 +111,7 @@ func assertIteratedKVs( t *testing.T, e Engine, startKey, endKey roachpb.Key, - startTime, endTime hlc.Timestamp, + startTime, endTime enginepb.TxnTimestamp, revisions bool, io IterOptions, expected []MVCCKeyValue, @@ -155,7 +154,7 @@ func assertIteratedKVs( func assertEqualKVs( e Engine, startKey, endKey roachpb.Key, - startTime, endTime hlc.Timestamp, + startTime, endTime enginepb.TxnTimestamp, revisions bool, expected []MVCCKeyValue, ) func(*testing.T) { @@ -202,15 +201,15 @@ func TestMVCCIncrementalIterator(t *testing.T) { // Use a non-zero min, since we use IsEmpty to decide if a ts should be used // as upper/lower-bound during iterator initialization. - tsMin = hlc.Timestamp{WallTime: 0, Logical: 1} - ts1 = hlc.Timestamp{WallTime: 1, Logical: 0} - ts2 = hlc.Timestamp{WallTime: 2, Logical: 0} - ts3 = hlc.Timestamp{WallTime: 3, Logical: 0} - ts4 = hlc.Timestamp{WallTime: 4, Logical: 0} - tsMax = hlc.Timestamp{WallTime: math.MaxInt64, Logical: 0} + tsMin = enginepb.TxnTimestamp{WallTime: 0, Logical: 1} + ts1 = enginepb.TxnTimestamp{WallTime: 1, Logical: 0} + ts2 = enginepb.TxnTimestamp{WallTime: 2, Logical: 0} + ts3 = enginepb.TxnTimestamp{WallTime: 3, Logical: 0} + ts4 = enginepb.TxnTimestamp{WallTime: 4, Logical: 0} + tsMax = enginepb.TxnTimestamp{WallTime: math.MaxInt64, Logical: 0} ) - makeKVT := func(key roachpb.Key, value []byte, ts hlc.Timestamp) MVCCKeyValue { + makeKVT := func(key roachpb.Key, value []byte, ts enginepb.TxnTimestamp) MVCCKeyValue { return MVCCKeyValue{Key: MVCCKey{Key: key, Timestamp: ts}, Value: value} } @@ -399,7 +398,7 @@ func TestMVCCIncrementalIterator(t *testing.T) { } func slurpKVsInTimeRange( - reader Reader, prefix roachpb.Key, startTime, endTime hlc.Timestamp, + reader Reader, prefix roachpb.Key, startTime, endTime enginepb.TxnTimestamp, ) ([]MVCCKeyValue, error) { endKey := prefix.PrefixEnd() iter := NewMVCCIncrementalIterator(reader, MVCCIncrementalIterOptions{ @@ -441,10 +440,10 @@ func TestMVCCIncrementalIteratorIntentRewrittenConcurrently(t *testing.T) { kA := roachpb.Key("kA") vA1 := roachpb.MakeValueFromString("vA1") vA2 := roachpb.MakeValueFromString("vA2") - ts0 := hlc.Timestamp{WallTime: 0} - ts1 := hlc.Timestamp{WallTime: 1} - ts2 := hlc.Timestamp{WallTime: 2} - ts3 := hlc.Timestamp{WallTime: 3} + ts0 := enginepb.TxnTimestamp{WallTime: 0} + ts1 := enginepb.TxnTimestamp{WallTime: 1} + ts2 := enginepb.TxnTimestamp{WallTime: 2} + ts3 := enginepb.TxnTimestamp{WallTime: 3} txn := &roachpb.Transaction{ TxnMeta: enginepb.TxnMeta{ Key: roachpb.Key("b"), @@ -505,7 +504,7 @@ func TestMVCCIncrementalIteratorIntentDeletion(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - txn := func(key roachpb.Key, ts hlc.Timestamp) *roachpb.Transaction { + txn := func(key roachpb.Key, ts enginepb.TxnTimestamp) *roachpb.Transaction { return &roachpb.Transaction{ TxnMeta: enginepb.TxnMeta{ Key: key, @@ -531,10 +530,10 @@ func TestMVCCIncrementalIteratorIntentDeletion(t *testing.T) { vB1 := roachpb.MakeValueFromString("vB1") kC := roachpb.Key("kC") vC1 := roachpb.MakeValueFromString("vC1") - ts0 := hlc.Timestamp{WallTime: 0} - ts1 := hlc.Timestamp{WallTime: 1} - ts2 := hlc.Timestamp{WallTime: 2} - ts3 := hlc.Timestamp{WallTime: 3} + ts0 := enginepb.TxnTimestamp{WallTime: 0} + ts1 := enginepb.TxnTimestamp{WallTime: 1} + ts2 := enginepb.TxnTimestamp{WallTime: 2} + ts3 := enginepb.TxnTimestamp{WallTime: 3} txnA1 := txn(kA, ts1) txnA3 := txn(kA, ts3) txnB1 := txn(kB, ts1) @@ -619,7 +618,7 @@ func TestMVCCIncrementalIteratorIntentStraddlesSStables(t *testing.T) { put := func(key, value string, ts int64, txn *roachpb.Transaction) { v := roachpb.MakeValueFromString(value) if err := MVCCPut( - ctx, db1, nil, roachpb.Key(key), hlc.Timestamp{WallTime: ts}, v, txn, + ctx, db1, nil, roachpb.Key(key), enginepb.TxnTimestamp{WallTime: ts}, v, txn, ); err != nil { t.Fatal(err) } @@ -631,9 +630,9 @@ func TestMVCCIncrementalIteratorIntentStraddlesSStables(t *testing.T) { Key: roachpb.Key("b"), ID: uuid.MakeV4(), Epoch: 1, - WriteTimestamp: hlc.Timestamp{WallTime: 2}, + WriteTimestamp: enginepb.TxnTimestamp{WallTime: 2}, }, - ReadTimestamp: hlc.Timestamp{WallTime: 2}, + ReadTimestamp: enginepb.TxnTimestamp{WallTime: 2}, }) // Create a second DB in which we'll create a specific SSTable structure: the @@ -699,8 +698,8 @@ func TestMVCCIncrementalIteratorIntentStraddlesSStables(t *testing.T) { // intent error. it := NewMVCCIncrementalIterator(db2, MVCCIncrementalIterOptions{ IterOptions: IterOptions{UpperBound: keys.MaxKey}, - StartTime: hlc.Timestamp{WallTime: 1}, - EndTime: hlc.Timestamp{WallTime: 2}, + StartTime: enginepb.TxnTimestamp{WallTime: 1}, + EndTime: enginepb.TxnTimestamp{WallTime: 2}, }) defer it.Close() for it.SeekGE(MVCCKey{Key: keys.MinKey}); ; it.Next() { @@ -740,27 +739,27 @@ func TestMVCCIterateTimeBound(t *testing.T) { defer eng.Close() for _, testCase := range []struct { - start hlc.Timestamp - end hlc.Timestamp + start enginepb.TxnTimestamp + end enginepb.TxnTimestamp }{ // entire time range - {hlc.Timestamp{WallTime: 0, Logical: 0}, hlc.Timestamp{WallTime: 110, Logical: 0}}, + {enginepb.TxnTimestamp{WallTime: 0, Logical: 0}, enginepb.TxnTimestamp{WallTime: 110, Logical: 0}}, // one SST - {hlc.Timestamp{WallTime: 10, Logical: 0}, hlc.Timestamp{WallTime: 19, Logical: 0}}, + {enginepb.TxnTimestamp{WallTime: 10, Logical: 0}, enginepb.TxnTimestamp{WallTime: 19, Logical: 0}}, // one SST, plus the min of the following SST - {hlc.Timestamp{WallTime: 10, Logical: 0}, hlc.Timestamp{WallTime: 20, Logical: 0}}, + {enginepb.TxnTimestamp{WallTime: 10, Logical: 0}, enginepb.TxnTimestamp{WallTime: 20, Logical: 0}}, // one SST, plus the max of the preceding SST - {hlc.Timestamp{WallTime: 9, Logical: 0}, hlc.Timestamp{WallTime: 19, Logical: 0}}, + {enginepb.TxnTimestamp{WallTime: 9, Logical: 0}, enginepb.TxnTimestamp{WallTime: 19, Logical: 0}}, // one SST, plus the min of the following and the max of the preceding SST - {hlc.Timestamp{WallTime: 9, Logical: 0}, hlc.Timestamp{WallTime: 21, Logical: 0}}, + {enginepb.TxnTimestamp{WallTime: 9, Logical: 0}, enginepb.TxnTimestamp{WallTime: 21, Logical: 0}}, // one SST, not min or max - {hlc.Timestamp{WallTime: 17, Logical: 0}, hlc.Timestamp{WallTime: 18, Logical: 0}}, + {enginepb.TxnTimestamp{WallTime: 17, Logical: 0}, enginepb.TxnTimestamp{WallTime: 18, Logical: 0}}, // one SST's max - {hlc.Timestamp{WallTime: 18, Logical: 0}, hlc.Timestamp{WallTime: 19, Logical: 0}}, + {enginepb.TxnTimestamp{WallTime: 18, Logical: 0}, enginepb.TxnTimestamp{WallTime: 19, Logical: 0}}, // one SST's min - {hlc.Timestamp{WallTime: 19, Logical: 0}, hlc.Timestamp{WallTime: 20, Logical: 0}}, + {enginepb.TxnTimestamp{WallTime: 19, Logical: 0}, enginepb.TxnTimestamp{WallTime: 20, Logical: 0}}, // random endpoints - {hlc.Timestamp{WallTime: 32, Logical: 0}, hlc.Timestamp{WallTime: 78, Logical: 0}}, + {enginepb.TxnTimestamp{WallTime: 32, Logical: 0}, enginepb.TxnTimestamp{WallTime: 78, Logical: 0}}, } { t.Run(fmt.Sprintf("%s-%s", testCase.start, testCase.end), func(t *testing.T) { defer leaktest.AfterTest(t)() diff --git a/pkg/storage/mvcc_logical_ops.go b/pkg/storage/mvcc_logical_ops.go index b74afcce384e..2040bed00da8 100644 --- a/pkg/storage/mvcc_logical_ops.go +++ b/pkg/storage/mvcc_logical_ops.go @@ -17,7 +17,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/bufalloc" - "github.com/cockroachdb/cockroach/pkg/util/hlc" ) // MVCCLogicalOpType is an enum with values corresponding to each of the @@ -50,7 +49,7 @@ const ( type MVCCLogicalOpDetails struct { Txn enginepb.TxnMeta Key roachpb.Key - Timestamp hlc.Timestamp + Timestamp enginepb.TxnTimestamp // Safe indicates that the values in this struct will never be invalidated // at a later point. If the details object cannot promise that its values diff --git a/pkg/storage/mvcc_logical_ops_test.go b/pkg/storage/mvcc_logical_ops_test.go index 9508763df608..32e914c93ec9 100644 --- a/pkg/storage/mvcc_logical_ops_test.go +++ b/pkg/storage/mvcc_logical_ops_test.go @@ -19,7 +19,6 @@ 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/log" "github.com/kr/pretty" @@ -39,17 +38,17 @@ func TestMVCCOpLogWriter(t *testing.T) { defer ol.Close() // Write a value and an intent. - if err := MVCCPut(ctx, ol, nil, testKey1, hlc.Timestamp{Logical: 1}, value1, nil); err != nil { + if err := MVCCPut(ctx, ol, nil, testKey1, enginepb.TxnTimestamp{Logical: 1}, value1, nil); err != nil { t.Fatal(err) } - txn1ts := makeTxn(*txn1, hlc.Timestamp{Logical: 2}) + txn1ts := makeTxn(*txn1, enginepb.TxnTimestamp{Logical: 2}) if err := MVCCPut(ctx, ol, nil, testKey1, txn1ts.ReadTimestamp, value2, txn1ts); err != nil { t.Fatal(err) } // Write a value and an intent on local keys. localKey := keys.MakeRangeIDPrefix(1) - if err := MVCCPut(ctx, ol, nil, localKey, hlc.Timestamp{Logical: 1}, value1, nil); err != nil { + if err := MVCCPut(ctx, ol, nil, localKey, enginepb.TxnTimestamp{Logical: 1}, value1, nil); err != nil { t.Fatal(err) } if err := MVCCPut(ctx, ol, nil, localKey, txn1ts.ReadTimestamp, value2, txn1ts); err != nil { @@ -59,7 +58,7 @@ func TestMVCCOpLogWriter(t *testing.T) { // Update the intents and write another. Use a distinct batch. olDist := ol.Distinct() txn1ts.Sequence++ - txn1ts.WriteTimestamp = hlc.Timestamp{Logical: 3} + txn1ts.WriteTimestamp = enginepb.TxnTimestamp{Logical: 3} if err := MVCCPut(ctx, olDist, nil, testKey1, txn1ts.ReadTimestamp, value2, txn1ts); err != nil { t.Fatal(err) } @@ -67,8 +66,8 @@ func TestMVCCOpLogWriter(t *testing.T) { t.Fatal(err) } // Set the txn timestamp to a larger value than the intent. - txn1LargerTS := makeTxn(*txn1, hlc.Timestamp{Logical: 4}) - txn1LargerTS.WriteTimestamp = hlc.Timestamp{Logical: 4} + txn1LargerTS := makeTxn(*txn1, enginepb.TxnTimestamp{Logical: 4}) + txn1LargerTS.WriteTimestamp = enginepb.TxnTimestamp{Logical: 4} if err := MVCCPut(ctx, olDist, nil, testKey2, txn1LargerTS.ReadTimestamp, value3, txn1LargerTS); err != nil { t.Fatal(err) } @@ -76,7 +75,7 @@ func TestMVCCOpLogWriter(t *testing.T) { // Resolve all three intent. txn1CommitTS := *txn1Commit - txn1CommitTS.WriteTimestamp = hlc.Timestamp{Logical: 4} + txn1CommitTS.WriteTimestamp = enginepb.TxnTimestamp{Logical: 4} if _, _, err := MVCCResolveWriteIntentRange(ctx, ol, nil, roachpb.MakeLockUpdate( &txn1CommitTS, @@ -93,12 +92,12 @@ func TestMVCCOpLogWriter(t *testing.T) { } // Write another intent, push it, then abort it. - txn2ts := makeTxn(*txn2, hlc.Timestamp{Logical: 5}) + txn2ts := makeTxn(*txn2, enginepb.TxnTimestamp{Logical: 5}) if err := MVCCPut(ctx, ol, nil, testKey3, txn2ts.ReadTimestamp, value4, txn2ts); err != nil { t.Fatal(err) } txn2Pushed := *txn2 - txn2Pushed.WriteTimestamp = hlc.Timestamp{Logical: 6} + txn2Pushed.WriteTimestamp = enginepb.TxnTimestamp{Logical: 6} if _, err := MVCCResolveWriteIntent(ctx, ol, nil, roachpb.MakeLockUpdate(&txn2Pushed, roachpb.Span{Key: testKey3}), ); err != nil { @@ -121,43 +120,43 @@ func TestMVCCOpLogWriter(t *testing.T) { exp := []enginepb.MVCCLogicalOp{ makeOp(&enginepb.MVCCWriteValueOp{ Key: testKey1, - Timestamp: hlc.Timestamp{Logical: 1}, + Timestamp: enginepb.TxnTimestamp{Logical: 1}, }), makeOp(&enginepb.MVCCWriteIntentOp{ TxnID: txn1.ID, TxnKey: txn1.Key, TxnMinTimestamp: txn1.MinTimestamp, - Timestamp: hlc.Timestamp{Logical: 2}, + Timestamp: enginepb.TxnTimestamp{Logical: 2}, }), makeOp(&enginepb.MVCCUpdateIntentOp{ TxnID: txn1.ID, - Timestamp: hlc.Timestamp{Logical: 3}, + Timestamp: enginepb.TxnTimestamp{Logical: 3}, }), makeOp(&enginepb.MVCCWriteIntentOp{ TxnID: txn1.ID, TxnKey: txn1.Key, TxnMinTimestamp: txn1.MinTimestamp, - Timestamp: hlc.Timestamp{Logical: 4}, + Timestamp: enginepb.TxnTimestamp{Logical: 4}, }), makeOp(&enginepb.MVCCCommitIntentOp{ TxnID: txn1.ID, Key: testKey1, - Timestamp: hlc.Timestamp{Logical: 4}, + Timestamp: enginepb.TxnTimestamp{Logical: 4}, }), makeOp(&enginepb.MVCCCommitIntentOp{ TxnID: txn1.ID, Key: testKey2, - Timestamp: hlc.Timestamp{Logical: 4}, + Timestamp: enginepb.TxnTimestamp{Logical: 4}, }), makeOp(&enginepb.MVCCWriteIntentOp{ TxnID: txn2.ID, TxnKey: txn2.Key, TxnMinTimestamp: txn2.MinTimestamp, - Timestamp: hlc.Timestamp{Logical: 5}, + Timestamp: enginepb.TxnTimestamp{Logical: 5}, }), makeOp(&enginepb.MVCCUpdateIntentOp{ TxnID: txn2.ID, - Timestamp: hlc.Timestamp{Logical: 6}, + Timestamp: enginepb.TxnTimestamp{Logical: 6}, }), makeOp(&enginepb.MVCCAbortIntentOp{ TxnID: txn2.ID, diff --git a/pkg/storage/mvcc_stats_test.go b/pkg/storage/mvcc_stats_test.go index 898a8c41e04a..6a4ce04f07e2 100644 --- a/pkg/storage/mvcc_stats_test.go +++ b/pkg/storage/mvcc_stats_test.go @@ -76,7 +76,7 @@ func TestMVCCStatsDeleteCommitMovesTimestamp(t *testing.T) { assertEq(t, engine, "initially", aggMS, &enginepb.MVCCStats{}) key := roachpb.Key("a") - ts1 := hlc.Timestamp{WallTime: 1e9} + ts1 := enginepb.TxnTimestamp{WallTime: 1e9} // Put a value. value := roachpb.MakeValueFromString("value") if err := MVCCPut(ctx, engine, aggMS, key, ts1, value, nil); err != nil { @@ -99,7 +99,7 @@ func TestMVCCStatsDeleteCommitMovesTimestamp(t *testing.T) { assertEq(t, engine, "after put", aggMS, &expMS) // Delete the value at ts=3. We'll commit this at ts=4 later. - ts3 := hlc.Timestamp{WallTime: 3 * 1e9} + ts3 := enginepb.TxnTimestamp{WallTime: 3 * 1e9} txn := &roachpb.Transaction{ TxnMeta: enginepb.TxnMeta{ID: uuid.MakeV4(), WriteTimestamp: ts3}, ReadTimestamp: ts3, @@ -110,7 +110,7 @@ func TestMVCCStatsDeleteCommitMovesTimestamp(t *testing.T) { // Now commit the value, but with a timestamp gap (i.e. this is a // push-commit as it would happen for a SNAPSHOT txn) - ts4 := hlc.Timestamp{WallTime: 4 * 1e9} + ts4 := enginepb.TxnTimestamp{WallTime: 4 * 1e9} txn.Status = roachpb.COMMITTED txn.WriteTimestamp.Forward(ts4) if _, err := MVCCResolveWriteIntent(ctx, engine, aggMS, @@ -155,7 +155,7 @@ func TestMVCCStatsPutCommitMovesTimestamp(t *testing.T) { assertEq(t, engine, "initially", aggMS, &enginepb.MVCCStats{}) key := roachpb.Key("a") - ts1 := hlc.Timestamp{WallTime: 1e9} + ts1 := enginepb.TxnTimestamp{WallTime: 1e9} txn := &roachpb.Transaction{ TxnMeta: enginepb.TxnMeta{ID: uuid.MakeV4(), WriteTimestamp: ts1}, ReadTimestamp: ts1, @@ -195,7 +195,7 @@ func TestMVCCStatsPutCommitMovesTimestamp(t *testing.T) { // Now commit the intent, but with a timestamp gap (i.e. this is a // push-commit as it would happen for a SNAPSHOT txn) - ts4 := hlc.Timestamp{WallTime: 4 * 1e9} + ts4 := enginepb.TxnTimestamp{WallTime: 4 * 1e9} txn.Status = roachpb.COMMITTED txn.WriteTimestamp.Forward(ts4) if _, err := MVCCResolveWriteIntent(ctx, engine, aggMS, @@ -239,7 +239,7 @@ func TestMVCCStatsPutPushMovesTimestamp(t *testing.T) { assertEq(t, engine, "initially", aggMS, &enginepb.MVCCStats{}) key := roachpb.Key("a") - ts1 := hlc.Timestamp{WallTime: 1e9} + ts1 := enginepb.TxnTimestamp{WallTime: 1e9} txn := &roachpb.Transaction{ TxnMeta: enginepb.TxnMeta{ID: uuid.MakeV4(), WriteTimestamp: ts1}, ReadTimestamp: ts1, @@ -279,7 +279,7 @@ func TestMVCCStatsPutPushMovesTimestamp(t *testing.T) { // Now push the value, but with a timestamp gap (i.e. this is a // push as it would happen for a SNAPSHOT txn) - ts4 := hlc.Timestamp{WallTime: 4 * 1e9} + ts4 := enginepb.TxnTimestamp{WallTime: 4 * 1e9} txn.WriteTimestamp.Forward(ts4) if _, err := MVCCResolveWriteIntent(ctx, engine, aggMS, roachpb.MakeLockUpdate(txn, roachpb.Span{Key: key}), @@ -328,8 +328,8 @@ func TestMVCCStatsDeleteMovesTimestamp(t *testing.T) { assertEq(t, engine, "initially", aggMS, &enginepb.MVCCStats{}) - ts1 := hlc.Timestamp{WallTime: 1e9} - ts2 := hlc.Timestamp{WallTime: 2 * 1e9} + ts1 := enginepb.TxnTimestamp{WallTime: 1e9} + ts2 := enginepb.TxnTimestamp{WallTime: 2 * 1e9} key := roachpb.Key("a") txn := &roachpb.Transaction{ @@ -451,8 +451,8 @@ func TestMVCCStatsPutMovesDeletionTimestamp(t *testing.T) { assertEq(t, engine, "initially", aggMS, &enginepb.MVCCStats{}) - ts1 := hlc.Timestamp{WallTime: 1e9} - ts2 := hlc.Timestamp{WallTime: 2 * 1e9} + ts1 := enginepb.TxnTimestamp{WallTime: 1e9} + ts2 := enginepb.TxnTimestamp{WallTime: 2 * 1e9} key := roachpb.Key("a") txn := &roachpb.Transaction{ @@ -567,9 +567,9 @@ func TestMVCCStatsDelDelCommitMovesTimestamp(t *testing.T) { key := roachpb.Key("a") - ts1 := hlc.Timestamp{WallTime: 1e9} - ts2 := hlc.Timestamp{WallTime: 2e9} - ts3 := hlc.Timestamp{WallTime: 3e9} + ts1 := enginepb.TxnTimestamp{WallTime: 1e9} + ts2 := enginepb.TxnTimestamp{WallTime: 2e9} + ts3 := enginepb.TxnTimestamp{WallTime: 3e9} // Write a non-transactional tombstone at t=1s. if err := MVCCDelete(ctx, engine, aggMS, key, ts1, nil /* txn */); err != nil { @@ -711,9 +711,9 @@ func TestMVCCStatsPutDelPutMovesTimestamp(t *testing.T) { key := roachpb.Key("a") - ts1 := hlc.Timestamp{WallTime: 1e9} - ts2 := hlc.Timestamp{WallTime: 2e9} - ts3 := hlc.Timestamp{WallTime: 3e9} + ts1 := enginepb.TxnTimestamp{WallTime: 1e9} + ts2 := enginepb.TxnTimestamp{WallTime: 2e9} + ts3 := enginepb.TxnTimestamp{WallTime: 3e9} // Write a non-transactional value at t=1s. value := roachpb.MakeValueFromString("value") @@ -884,8 +884,8 @@ func TestMVCCStatsDelDelGC(t *testing.T) { assertEq(t, engine, "initially", aggMS, &enginepb.MVCCStats{}) key := roachpb.Key("a") - ts1 := hlc.Timestamp{WallTime: 1e9} - ts2 := hlc.Timestamp{WallTime: 2e9} + ts1 := enginepb.TxnTimestamp{WallTime: 1e9} + ts2 := enginepb.TxnTimestamp{WallTime: 2e9} // Write tombstones at ts1 and ts2. if err := MVCCDelete(ctx, engine, aggMS, key, ts1, nil); err != nil { @@ -961,8 +961,8 @@ func TestMVCCStatsPutIntentTimestampNotPutTimestamp(t *testing.T) { assertEq(t, engine, "initially", aggMS, &enginepb.MVCCStats{}) key := roachpb.Key("a") - ts201 := hlc.Timestamp{WallTime: 2e9 + 1} - ts099 := hlc.Timestamp{WallTime: 1e9 - 1} + ts201 := enginepb.TxnTimestamp{WallTime: 2e9 + 1} + ts099 := enginepb.TxnTimestamp{WallTime: 1e9 - 1} txn := &roachpb.Transaction{ TxnMeta: enginepb.TxnMeta{ID: uuid.MakeV4(), WriteTimestamp: ts201}, ReadTimestamp: ts099, @@ -1058,8 +1058,8 @@ func TestMVCCStatsPutWaitDeleteGC(t *testing.T) { key := roachpb.Key("a") - ts1 := hlc.Timestamp{WallTime: 1e9} - ts2 := hlc.Timestamp{WallTime: 2e9} + ts1 := enginepb.TxnTimestamp{WallTime: 1e9} + ts2 := enginepb.TxnTimestamp{WallTime: 2e9} // Write a value at ts1. val1 := roachpb.MakeValueFromString("value") @@ -1146,8 +1146,8 @@ func TestMVCCStatsTxnSysPutPut(t *testing.T) { key := keys.RangeDescriptorKey(roachpb.RKey("a")) - ts1 := hlc.Timestamp{WallTime: 1e9} - ts2 := hlc.Timestamp{WallTime: 2e9} + ts1 := enginepb.TxnTimestamp{WallTime: 1e9} + ts2 := enginepb.TxnTimestamp{WallTime: 2e9} txn := &roachpb.Transaction{ TxnMeta: enginepb.TxnMeta{ID: uuid.MakeV4(), WriteTimestamp: ts1}, @@ -1241,7 +1241,7 @@ func TestMVCCStatsTxnSysPutAbort(t *testing.T) { key := keys.RangeDescriptorKey(roachpb.RKey("a")) - ts1 := hlc.Timestamp{WallTime: 1e9} + ts1 := enginepb.TxnTimestamp{WallTime: 1e9} txn := &roachpb.Transaction{ TxnMeta: enginepb.TxnMeta{ID: uuid.MakeV4(), WriteTimestamp: ts1}, ReadTimestamp: ts1, @@ -1317,8 +1317,8 @@ func TestMVCCStatsSysPutPut(t *testing.T) { key := keys.RangeDescriptorKey(roachpb.RKey("a")) - ts1 := hlc.Timestamp{WallTime: 1e9} - ts2 := hlc.Timestamp{WallTime: 2e9} + ts1 := enginepb.TxnTimestamp{WallTime: 1e9} + ts2 := enginepb.TxnTimestamp{WallTime: 2e9} // Write a value at ts1. val1 := roachpb.MakeValueFromString("value") @@ -1383,7 +1383,7 @@ var mvccStatsTests = []struct { type state struct { MS *enginepb.MVCCStats - TS hlc.Timestamp + TS enginepb.TxnTimestamp Txn *roachpb.Transaction eng Engine @@ -1420,7 +1420,7 @@ func (s *randomTest) step(t *testing.T) { if !s.inline { // Jump up to a few seconds into the future. In ~1% of cases, jump // backwards instead (this exercises intactness on WriteTooOld, etc). - s.TS = hlc.Timestamp{ + s.TS = enginepb.TxnTimestamp{ WallTime: s.TS.WallTime + int64((s.state.rng.Float32()-0.01)*4e9), Logical: int32(s.rng.Intn(10)), } @@ -1431,7 +1431,7 @@ func (s *randomTest) step(t *testing.T) { s.TS.WallTime = 0 } } else { - s.TS = hlc.Timestamp{} + s.TS = enginepb.TxnTimestamp{} } restart := s.Txn != nil && s.rng.Intn(2) == 0 @@ -1519,7 +1519,7 @@ func TestMVCCStatsRandomized(t *testing.T) { } actions["EnsureTxn"] = func(s *state) string { if s.Txn == nil { - txn := roachpb.MakeTransaction("test", nil, 0, s.TS, 0) + txn := roachpb.MakeTransaction("test", nil, 0, hlc.Timestamp(s.TS), 0) s.Txn = &txn } return "" @@ -1558,7 +1558,7 @@ func TestMVCCStatsRandomized(t *testing.T) { } actions["GC"] = func(s *state) string { // Sometimes GC everything, sometimes only older versions. - gcTS := hlc.Timestamp{ + gcTS := enginepb.TxnTimestamp{ WallTime: s.rng.Int63n(s.TS.WallTime + 1 /* avoid zero */), } if err := MVCCGarbageCollect( diff --git a/pkg/storage/mvcc_test.go b/pkg/storage/mvcc_test.go index 439954b9387d..67752c9aefa9 100644 --- a/pkg/storage/mvcc_test.go +++ b/pkg/storage/mvcc_test.go @@ -55,8 +55,8 @@ var ( testKey6 = roachpb.Key("/db6") txn1ID = uuid.MakeV4() txn2ID = uuid.MakeV4() - txn1TS = hlc.Timestamp{Logical: 1} - txn2TS = hlc.Timestamp{Logical: 2} + txn1TS = enginepb.TxnTimestamp{Logical: 1} + txn2TS = enginepb.TxnTimestamp{Logical: 2} txn1 = &roachpb.Transaction{TxnMeta: enginepb.TxnMeta{Key: roachpb.Key("a"), ID: txn1ID, Epoch: 1, WriteTimestamp: txn1TS, MinTimestamp: txn1TS}, ReadTimestamp: txn1TS} txn1Commit = &roachpb.Transaction{TxnMeta: enginepb.TxnMeta{Key: roachpb.Key("a"), ID: txn1ID, Epoch: 1, WriteTimestamp: txn1TS, MinTimestamp: txn1TS}, ReadTimestamp: txn1TS, Status: roachpb.COMMITTED} txn1Abort = &roachpb.Transaction{TxnMeta: enginepb.TxnMeta{Key: roachpb.Key("a"), ID: txn1ID, Epoch: 1, WriteTimestamp: txn1TS, MinTimestamp: txn1TS}, Status: roachpb.ABORTED} @@ -92,14 +92,14 @@ var mvccEngineImpls = []struct { // makeTxn creates a new transaction using the specified base // txn and timestamp. -func makeTxn(baseTxn roachpb.Transaction, ts hlc.Timestamp) *roachpb.Transaction { +func makeTxn(baseTxn roachpb.Transaction, ts enginepb.TxnTimestamp) *roachpb.Transaction { txn := baseTxn.Clone() txn.ReadTimestamp = ts txn.WriteTimestamp = ts return txn } -func mvccVersionKey(key roachpb.Key, ts hlc.Timestamp) MVCCKey { +func mvccVersionKey(key roachpb.Key, ts enginepb.TxnTimestamp) MVCCKey { return MVCCKey{Key: key, Timestamp: ts} } @@ -263,13 +263,13 @@ func TestMVCCKeys(t *testing.T) { a0Key := roachpb.Key("a\x00") keys := mvccKeys{ mvccKey(aKey), - mvccVersionKey(aKey, hlc.Timestamp{WallTime: math.MaxInt64}), - mvccVersionKey(aKey, hlc.Timestamp{WallTime: 1}), - mvccVersionKey(aKey, hlc.Timestamp{Logical: 1}), + mvccVersionKey(aKey, enginepb.TxnTimestamp{WallTime: math.MaxInt64}), + mvccVersionKey(aKey, enginepb.TxnTimestamp{WallTime: 1}), + mvccVersionKey(aKey, enginepb.TxnTimestamp{Logical: 1}), mvccKey(a0Key), - mvccVersionKey(a0Key, hlc.Timestamp{WallTime: math.MaxInt64}), - mvccVersionKey(a0Key, hlc.Timestamp{WallTime: 1}), - mvccVersionKey(a0Key, hlc.Timestamp{Logical: 1}), + mvccVersionKey(a0Key, enginepb.TxnTimestamp{WallTime: math.MaxInt64}), + mvccVersionKey(a0Key, enginepb.TxnTimestamp{WallTime: 1}), + mvccVersionKey(a0Key, enginepb.TxnTimestamp{Logical: 1}), } sortKeys := make(mvccKeys, len(keys)) copy(sortKeys, keys) @@ -289,7 +289,7 @@ func TestMVCCGetNotExist(t *testing.T) { engine := engineImpl.create() defer engine.Close() - value, _, err := MVCCGet(context.Background(), engine, testKey1, hlc.Timestamp{Logical: 1}, + value, _, err := MVCCGet(context.Background(), engine, testKey1, enginepb.TxnTimestamp{Logical: 1}, MVCCGetOptions{}) if err != nil { t.Fatal(err) @@ -324,14 +324,14 @@ func TestMVCCGetNoMoreOldVersion(t *testing.T) { engine := engineImpl.create() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, enginepb.TxnTimestamp{WallTime: 3}, value1, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, enginepb.TxnTimestamp{WallTime: 1}, value2, nil); err != nil { t.Fatal(err) } - value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 2}, MVCCGetOptions{}) + value, _, err := MVCCGet(ctx, engine, testKey1, enginepb.TxnTimestamp{WallTime: 2}, MVCCGetOptions{}) if err != nil { t.Fatal(err) } @@ -353,10 +353,10 @@ func TestMVCCGetAndDelete(t *testing.T) { engine := engineImpl.create() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, enginepb.TxnTimestamp{WallTime: 1}, value1, nil); err != nil { t.Fatal(err) } - value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 2}, MVCCGetOptions{}) + value, _, err := MVCCGet(ctx, engine, testKey1, enginepb.TxnTimestamp{WallTime: 2}, MVCCGetOptions{}) if err != nil { t.Fatal(err) } @@ -364,13 +364,13 @@ func TestMVCCGetAndDelete(t *testing.T) { t.Fatal("the value should not be empty") } - err = MVCCDelete(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, nil) + err = MVCCDelete(ctx, engine, nil, testKey1, enginepb.TxnTimestamp{WallTime: 3}, nil) if err != nil { t.Fatal(err) } // Read the latest version which should be deleted. - value, _, err = MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 4}, MVCCGetOptions{}) + value, _, err = MVCCGet(ctx, engine, testKey1, enginepb.TxnTimestamp{WallTime: 4}, MVCCGetOptions{}) if err != nil { t.Fatal(err) } @@ -378,7 +378,7 @@ func TestMVCCGetAndDelete(t *testing.T) { t.Fatal("the value should be empty") } // Read the latest version with tombstone. - value, _, err = MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 4}, + value, _, err = MVCCGet(ctx, engine, testKey1, enginepb.TxnTimestamp{WallTime: 4}, MVCCGetOptions{Tombstones: true}) if err != nil { t.Fatal(err) @@ -388,7 +388,7 @@ func TestMVCCGetAndDelete(t *testing.T) { // Read the old version which should still exist. for _, logical := range []int32{0, math.MaxInt32} { - value, _, err = MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 2, Logical: logical}, + value, _, err = MVCCGet(ctx, engine, testKey1, enginepb.TxnTimestamp{WallTime: 2, Logical: logical}, MVCCGetOptions{}) if err != nil { t.Fatal(err) @@ -414,20 +414,20 @@ func TestMVCCWriteWithOlderTimestampAfterDeletionOfNonexistentKey(t *testing.T) defer engine.Close() if err := MVCCDelete( - context.Background(), engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, nil, + context.Background(), engine, nil, testKey1, enginepb.TxnTimestamp{WallTime: 3}, nil, ); err != nil { t.Fatal(err) } if err := MVCCPut( - context.Background(), engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, value1, nil, + context.Background(), engine, nil, testKey1, enginepb.TxnTimestamp{WallTime: 1}, value1, nil, ); !testutils.IsError( err, "write at timestamp 0.000000001,0 too old; wrote at 0.000000003,1", ) { t.Fatal(err) } - value, _, err := MVCCGet(context.Background(), engine, testKey1, hlc.Timestamp{WallTime: 2}, + value, _, err := MVCCGet(context.Background(), engine, testKey1, enginepb.TxnTimestamp{WallTime: 2}, MVCCGetOptions{}) if err != nil { t.Fatal(err) @@ -439,7 +439,7 @@ func TestMVCCWriteWithOlderTimestampAfterDeletionOfNonexistentKey(t *testing.T) } // Read the latest version which will be the value written with the timestamp pushed. - value, _, err = MVCCGet(context.Background(), engine, testKey1, hlc.Timestamp{WallTime: 4}, + value, _, err = MVCCGet(context.Background(), engine, testKey1, enginepb.TxnTimestamp{WallTime: 4}, MVCCGetOptions{}) if err != nil { t.Fatal(err) @@ -450,7 +450,7 @@ func TestMVCCWriteWithOlderTimestampAfterDeletionOfNonexistentKey(t *testing.T) if !bytes.Equal(value.RawBytes, value1.RawBytes) { t.Errorf("expected %q; got %q", value1.RawBytes, value.RawBytes) } - if expTS := (hlc.Timestamp{WallTime: 3, Logical: 1}); value.Timestamp != expTS { + if expTS := (enginepb.TxnTimestamp{WallTime: 3, Logical: 1}); value.Timestamp != expTS { t.Fatalf("timestamp was not pushed: %s, expected %s", value.Timestamp, expTS) } }) @@ -468,12 +468,12 @@ func TestMVCCInlineWithTxn(t *testing.T) { defer engine.Close() // Put an inline value. - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, enginepb.TxnTimestamp{}, value1, nil); err != nil { t.Fatal(err) } // Now verify inline get. - value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{}, MVCCGetOptions{}) + value, _, err := MVCCGet(ctx, engine, testKey1, enginepb.TxnTimestamp{}, MVCCGetOptions{}) if err != nil { t.Fatal(err) } @@ -483,14 +483,14 @@ func TestMVCCInlineWithTxn(t *testing.T) { // Verify inline get with txn does still work (this will happen on a // scan if the distributed sender is forced to wrap it in a txn). - if _, _, err = MVCCGet(ctx, engine, testKey1, hlc.Timestamp{}, MVCCGetOptions{ + if _, _, err = MVCCGet(ctx, engine, testKey1, enginepb.TxnTimestamp{}, MVCCGetOptions{ Txn: txn1, }); err != nil { t.Error(err) } // Verify inline put with txn is an error. - err = MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{}, value2, txn2) + err = MVCCPut(ctx, engine, nil, testKey2, enginepb.TxnTimestamp{}, value2, txn2) if !testutils.IsError(err, "writes not allowed within transactions") { t.Errorf("unexpected error: %+v", err) } @@ -508,7 +508,7 @@ func TestMVCCDeleteMissingKey(t *testing.T) { engine := engineImpl.create() defer engine.Close() - if err := MVCCDelete(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, nil); err != nil { + if err := MVCCDelete(ctx, engine, nil, testKey1, enginepb.TxnTimestamp{WallTime: 1}, nil); err != nil { t.Fatal(err) } // Verify nothing is written to the engine. @@ -530,13 +530,13 @@ func TestMVCCGetAndDeleteInTxn(t *testing.T) { engine := engineImpl.create() defer engine.Close() - txn := makeTxn(*txn1, hlc.Timestamp{WallTime: 1}) + txn := makeTxn(*txn1, enginepb.TxnTimestamp{WallTime: 1}) txn.Sequence++ if err := MVCCPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, value1, txn); err != nil { t.Fatal(err) } - if value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 2}, MVCCGetOptions{ + if value, _, err := MVCCGet(ctx, engine, testKey1, enginepb.TxnTimestamp{WallTime: 2}, MVCCGetOptions{ Txn: txn, }); err != nil { t.Fatal(err) @@ -545,13 +545,13 @@ func TestMVCCGetAndDeleteInTxn(t *testing.T) { } txn.Sequence++ - txn.WriteTimestamp = hlc.Timestamp{WallTime: 3} + txn.WriteTimestamp = enginepb.TxnTimestamp{WallTime: 3} if err := MVCCDelete(ctx, engine, nil, testKey1, txn.ReadTimestamp, txn); err != nil { t.Fatal(err) } // Read the latest version which should be deleted. - if value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 4}, MVCCGetOptions{ + if value, _, err := MVCCGet(ctx, engine, testKey1, enginepb.TxnTimestamp{WallTime: 4}, MVCCGetOptions{ Txn: txn, }); err != nil { t.Fatal(err) @@ -559,7 +559,7 @@ func TestMVCCGetAndDeleteInTxn(t *testing.T) { t.Fatal("the value should be empty") } // Read the latest version with tombstone. - if value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 4}, MVCCGetOptions{ + if value, _, err := MVCCGet(ctx, engine, testKey1, enginepb.TxnTimestamp{WallTime: 4}, MVCCGetOptions{ Tombstones: true, Txn: txn, }); err != nil { @@ -570,7 +570,7 @@ func TestMVCCGetAndDeleteInTxn(t *testing.T) { // Read the old version which shouldn't exist, as within a // transaction, we delete previous values. - if value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 2}, MVCCGetOptions{}); err != nil { + if value, _, err := MVCCGet(ctx, engine, testKey1, enginepb.TxnTimestamp{WallTime: 2}, MVCCGetOptions{}); err != nil { t.Fatal(err) } else if value != nil { t.Fatalf("expected value nil, got: %s", value) @@ -594,11 +594,11 @@ func TestMVCCGetWriteIntentError(t *testing.T) { t.Fatal(err) } - if _, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, MVCCGetOptions{}); err == nil { + if _, _, err := MVCCGet(ctx, engine, testKey1, enginepb.TxnTimestamp{WallTime: 1}, MVCCGetOptions{}); err == nil { t.Fatal("cannot read the value of a write intent without TxnID") } - if _, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, MVCCGetOptions{ + if _, _, err := MVCCGet(ctx, engine, testKey1, enginepb.TxnTimestamp{WallTime: 1}, MVCCGetOptions{ Txn: txn2, }); err == nil { t.Fatal("cannot read the value of a write intent from a different TxnID") @@ -607,7 +607,7 @@ func TestMVCCGetWriteIntentError(t *testing.T) { } } -func mkVal(s string, ts hlc.Timestamp) roachpb.Value { +func mkVal(s string, ts enginepb.TxnTimestamp) roachpb.Value { v := roachpb.MakeValueFromString(s) v.Timestamp = ts return v @@ -623,7 +623,7 @@ func TestMVCCScanWriteIntentError(t *testing.T) { engine := engineImpl.create() defer engine.Close() - ts := []hlc.Timestamp{{Logical: 1}, {Logical: 2}, {Logical: 3}, {Logical: 4}, {Logical: 5}, {Logical: 6}} + ts := []enginepb.TxnTimestamp{{Logical: 1}, {Logical: 2}, {Logical: 3}, {Logical: 4}, {Logical: 5}, {Logical: 6}} txn1ts := makeTxn(*txn1, ts[2]) txn2ts := makeTxn(*txn2, ts[5]) @@ -644,7 +644,7 @@ func TestMVCCScanWriteIntentError(t *testing.T) { txn = txn2ts } v := *protoutil.Clone(&kv.Value).(*roachpb.Value) - v.Timestamp = hlc.Timestamp{} + v.Timestamp = enginepb.TxnTimestamp{} if err := MVCCPut(ctx, engine, nil, kv.Key, kv.Value.Timestamp, v, txn); err != nil { t.Fatal(err) } @@ -699,7 +699,7 @@ func TestMVCCScanWriteIntentError(t *testing.T) { cStr = "consistent" } res, err := MVCCScan(ctx, engine, testKey1, testKey4.Next(), - hlc.Timestamp{WallTime: 1}, MVCCScanOptions{Inconsistent: !scan.consistent, Txn: scan.txn}) + enginepb.TxnTimestamp{WallTime: 1}, MVCCScanOptions{Inconsistent: !scan.consistent, Txn: scan.txn}) var wiErr *roachpb.WriteIntentError _ = errors.As(err, &wiErr) if (err == nil) != (wiErr == nil) { @@ -748,16 +748,16 @@ func TestMVCCGetInconsistent(t *testing.T) { defer engine.Close() // Put two values to key 1, the latest with a txn. - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, enginepb.TxnTimestamp{WallTime: 1}, value1, nil); err != nil { t.Fatal(err) } - txn1ts := makeTxn(*txn1, hlc.Timestamp{WallTime: 2}) + txn1ts := makeTxn(*txn1, enginepb.TxnTimestamp{WallTime: 2}) if err := MVCCPut(ctx, engine, nil, testKey1, txn1ts.ReadTimestamp, value2, txn1ts); err != nil { t.Fatal(err) } // A get with consistent=false should fail in a txn. - if _, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, MVCCGetOptions{ + if _, _, err := MVCCGet(ctx, engine, testKey1, enginepb.TxnTimestamp{WallTime: 1}, MVCCGetOptions{ Inconsistent: true, Txn: txn1, }); err == nil { @@ -765,9 +765,9 @@ func TestMVCCGetInconsistent(t *testing.T) { } // Inconsistent get will fetch value1 for any timestamp. - for _, ts := range []hlc.Timestamp{{WallTime: 1}, {WallTime: 2}} { + for _, ts := range []enginepb.TxnTimestamp{{WallTime: 1}, {WallTime: 2}} { val, intent, err := MVCCGet(ctx, engine, testKey1, ts, MVCCGetOptions{Inconsistent: true}) - if ts.Less(hlc.Timestamp{WallTime: 2}) { + if ts.Less(enginepb.TxnTimestamp{WallTime: 2}) { if err != nil { t.Fatal(err) } @@ -785,7 +785,7 @@ func TestMVCCGetInconsistent(t *testing.T) { if err := MVCCPut(ctx, engine, nil, testKey2, txn2.ReadTimestamp, value1, txn2); err != nil { t.Fatal(err) } - val, intent, err := MVCCGet(ctx, engine, testKey2, hlc.Timestamp{WallTime: 2}, + val, intent, err := MVCCGet(ctx, engine, testKey2, enginepb.TxnTimestamp{WallTime: 2}, MVCCGetOptions{Inconsistent: true}) if intent == nil || !intent.Key.Equal(testKey2) { t.Fatal(err) @@ -822,16 +822,16 @@ func TestMVCCGetProtoInconsistent(t *testing.T) { v2 := roachpb.MakeValueFromBytes(bytes2) // Put two values to key 1, the latest with a txn. - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, v1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, enginepb.TxnTimestamp{WallTime: 1}, v1, nil); err != nil { t.Fatal(err) } - txn1ts := makeTxn(*txn1, hlc.Timestamp{WallTime: 2}) + txn1ts := makeTxn(*txn1, enginepb.TxnTimestamp{WallTime: 2}) if err := MVCCPut(ctx, engine, nil, testKey1, txn1ts.ReadTimestamp, v2, txn1ts); err != nil { t.Fatal(err) } // An inconsistent get should fail in a txn. - if _, err := MVCCGetProto(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, nil, MVCCGetOptions{ + if _, err := MVCCGetProto(ctx, engine, testKey1, enginepb.TxnTimestamp{WallTime: 1}, nil, MVCCGetOptions{ Inconsistent: true, Txn: txn1, }); err == nil { @@ -842,12 +842,12 @@ func TestMVCCGetProtoInconsistent(t *testing.T) { // Inconsistent get will fetch value1 for any timestamp. - for _, ts := range []hlc.Timestamp{{WallTime: 1}, {WallTime: 2}} { + for _, ts := range []enginepb.TxnTimestamp{{WallTime: 1}, {WallTime: 2}} { val := roachpb.Value{} found, err := MVCCGetProto(ctx, engine, testKey1, ts, &val, MVCCGetOptions{ Inconsistent: true, }) - if ts.Less(hlc.Timestamp{WallTime: 2}) { + if ts.Less(enginepb.TxnTimestamp{WallTime: 2}) { if err != nil { t.Fatal(err) } @@ -872,7 +872,7 @@ func TestMVCCGetProtoInconsistent(t *testing.T) { t.Fatal(err) } val := roachpb.Value{} - found, err := MVCCGetProto(ctx, engine, testKey2, hlc.Timestamp{WallTime: 2}, &val, MVCCGetOptions{ + found, err := MVCCGetProto(ctx, engine, testKey2, enginepb.TxnTimestamp{WallTime: 2}, &val, MVCCGetOptions{ Inconsistent: true, }) if err != nil { @@ -887,14 +887,14 @@ func TestMVCCGetProtoInconsistent(t *testing.T) { // Write a malformed value (not an encoded MVCCKeyValue) and a // write intent to key 3; the parse error is returned instead of the // write intent. - if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 1}, value3, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey3, enginepb.TxnTimestamp{WallTime: 1}, value3, nil); err != nil { t.Fatal(err) } if err := MVCCPut(ctx, engine, nil, testKey3, txn1ts.ReadTimestamp, v2, txn1ts); err != nil { t.Fatal(err) } val := roachpb.Value{} - found, err := MVCCGetProto(ctx, engine, testKey3, hlc.Timestamp{WallTime: 1}, &val, MVCCGetOptions{ + found, err := MVCCGetProto(ctx, engine, testKey3, enginepb.TxnTimestamp{WallTime: 1}, &val, MVCCGetOptions{ Inconsistent: true, }) if err == nil { @@ -924,8 +924,8 @@ func TestMVCCInvalidateIterator(t *testing.T) { defer engine.Close() ctx := context.Background() - ts1 := hlc.Timestamp{WallTime: 1} - ts2 := hlc.Timestamp{WallTime: 2} + ts1 := enginepb.TxnTimestamp{WallTime: 1} + ts2 := enginepb.TxnTimestamp{WallTime: 2} key := roachpb.Key("a") if err := MVCCPut(ctx, engine, nil, key, ts1, value1, nil); err != nil { @@ -989,23 +989,23 @@ func TestMVCCPutAfterBatchIterCreate(t *testing.T) { engine := engineImpl.create() defer engine.Close() - err := engine.PutMVCC(MVCCKey{testKey1, hlc.Timestamp{WallTime: 5}}, []byte("foobar")) + err := engine.PutMVCC(MVCCKey{testKey1, enginepb.TxnTimestamp{WallTime: 5}}, []byte("foobar")) if err != nil { t.Fatal(err) } - err = engine.PutMVCC(MVCCKey{testKey2, hlc.Timestamp{WallTime: 5}}, []byte("foobar")) + err = engine.PutMVCC(MVCCKey{testKey2, enginepb.TxnTimestamp{WallTime: 5}}, []byte("foobar")) if err != nil { t.Fatal(err) } - err = engine.PutMVCC(MVCCKey{testKey2, hlc.Timestamp{WallTime: 3}}, []byte("foobar")) + err = engine.PutMVCC(MVCCKey{testKey2, enginepb.TxnTimestamp{WallTime: 3}}, []byte("foobar")) if err != nil { t.Fatal(err) } - err = engine.PutMVCC(MVCCKey{testKey3, hlc.Timestamp{WallTime: 5}}, []byte("foobar")) + err = engine.PutMVCC(MVCCKey{testKey3, enginepb.TxnTimestamp{WallTime: 5}}, []byte("foobar")) if err != nil { t.Fatal(err) } - err = engine.PutMVCC(MVCCKey{testKey4, hlc.Timestamp{WallTime: 5}}, []byte("foobar")) + err = engine.PutMVCC(MVCCKey{testKey4, enginepb.TxnTimestamp{WallTime: 5}}, []byte("foobar")) if err != nil { t.Fatal(err) } @@ -1014,19 +1014,19 @@ func TestMVCCPutAfterBatchIterCreate(t *testing.T) { defer batch.Close() txn := &roachpb.Transaction{ TxnMeta: enginepb.TxnMeta{ - WriteTimestamp: hlc.Timestamp{WallTime: 10}, + WriteTimestamp: enginepb.TxnTimestamp{WallTime: 10}, }, Name: "test", Status: roachpb.PENDING, - ReadTimestamp: hlc.Timestamp{WallTime: 10}, - MaxTimestamp: hlc.Timestamp{WallTime: 10}, + ReadTimestamp: enginepb.TxnTimestamp{WallTime: 10}, + MaxTimestamp: enginepb.TxnTimestamp{WallTime: 10}, } iter := batch.NewMVCCIterator(MVCCKeyAndIntentsIterKind, IterOptions{ LowerBound: testKey1, UpperBound: testKey5, }) defer iter.Close() - iter.SeekGE(MVCCKey{testKey1, hlc.Timestamp{WallTime: 5}}) + iter.SeekGE(MVCCKey{testKey1, enginepb.TxnTimestamp{WallTime: 5}}) iter.Next() // key2/5 // Lay down an intent on key3, which will go at key3/0 and sort before key3/5. @@ -1046,33 +1046,33 @@ func TestMVCCPutAfterBatchIterCreate(t *testing.T) { } func mvccScanTest(ctx context.Context, t *testing.T, engine Engine) { - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, enginepb.TxnTimestamp{WallTime: 1}, value1, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 2}, value4, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, enginepb.TxnTimestamp{WallTime: 2}, value4, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, enginepb.TxnTimestamp{WallTime: 1}, value2, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 3}, value3, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, enginepb.TxnTimestamp{WallTime: 3}, value3, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 1}, value3, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey3, enginepb.TxnTimestamp{WallTime: 1}, value3, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 4}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey3, enginepb.TxnTimestamp{WallTime: 4}, value2, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, value4, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey4, enginepb.TxnTimestamp{WallTime: 1}, value4, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 5}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey4, enginepb.TxnTimestamp{WallTime: 5}, value1, nil); err != nil { t.Fatal(err) } res, err := MVCCScan(ctx, engine, testKey2, testKey4, - hlc.Timestamp{WallTime: 1}, MVCCScanOptions{}) + enginepb.TxnTimestamp{WallTime: 1}, MVCCScanOptions{}) if err != nil { t.Fatal(err) } @@ -1090,7 +1090,7 @@ func mvccScanTest(ctx context.Context, t *testing.T, engine Engine) { } res, err = MVCCScan(ctx, engine, testKey2, testKey4, - hlc.Timestamp{WallTime: 4}, MVCCScanOptions{}) + enginepb.TxnTimestamp{WallTime: 4}, MVCCScanOptions{}) if err != nil { t.Fatal(err) } @@ -1108,7 +1108,7 @@ func mvccScanTest(ctx context.Context, t *testing.T, engine Engine) { } res, err = MVCCScan( - ctx, engine, testKey4, keyMax, hlc.Timestamp{WallTime: 1}, MVCCScanOptions{}, + ctx, engine, testKey4, keyMax, enginepb.TxnTimestamp{WallTime: 1}, MVCCScanOptions{}, ) if err != nil { t.Fatal(err) @@ -1124,13 +1124,13 @@ func mvccScanTest(ctx context.Context, t *testing.T, engine Engine) { t.Fatalf("resumeSpan = %+v", resumeSpan) } - if _, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, MVCCGetOptions{ + if _, _, err := MVCCGet(ctx, engine, testKey1, enginepb.TxnTimestamp{WallTime: 1}, MVCCGetOptions{ Txn: txn2, }); err != nil { t.Fatal(err) } res, err = MVCCScan(ctx, engine, keyMin, testKey2, - hlc.Timestamp{WallTime: 1}, MVCCScanOptions{}) + enginepb.TxnTimestamp{WallTime: 1}, MVCCScanOptions{}) if err != nil { t.Fatal(err) } @@ -1167,24 +1167,24 @@ func TestMVCCScanMaxNum(t *testing.T) { engine := engineImpl.create() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, enginepb.TxnTimestamp{WallTime: 1}, value1, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, enginepb.TxnTimestamp{WallTime: 1}, value2, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 1}, value3, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey3, enginepb.TxnTimestamp{WallTime: 1}, value3, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, value4, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey4, enginepb.TxnTimestamp{WallTime: 1}, value4, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey6, hlc.Timestamp{WallTime: 1}, value4, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey6, enginepb.TxnTimestamp{WallTime: 1}, value4, nil); err != nil { t.Fatal(err) } res, err := MVCCScan(ctx, engine, testKey2, testKey4, - hlc.Timestamp{WallTime: 1}, MVCCScanOptions{MaxKeys: 1}) + enginepb.TxnTimestamp{WallTime: 1}, MVCCScanOptions{MaxKeys: 1}) if err != nil { t.Fatal(err) } @@ -1198,7 +1198,7 @@ func TestMVCCScanMaxNum(t *testing.T) { } res, err = MVCCScan(ctx, engine, testKey2, testKey4, - hlc.Timestamp{WallTime: 1}, MVCCScanOptions{MaxKeys: -1}) + enginepb.TxnTimestamp{WallTime: 1}, MVCCScanOptions{MaxKeys: -1}) if err != nil { t.Fatal(err) } @@ -1212,7 +1212,7 @@ func TestMVCCScanMaxNum(t *testing.T) { // Note: testKey6, though not scanned directly, is important in testing that // the computed resume span does not extend beyond the upper bound of a scan. res, err = MVCCScan(ctx, engine, testKey4, testKey5, - hlc.Timestamp{WallTime: 1}, MVCCScanOptions{MaxKeys: 1}) + enginepb.TxnTimestamp{WallTime: 1}, MVCCScanOptions{MaxKeys: 1}) if err != nil { t.Fatal(err) } @@ -1224,7 +1224,7 @@ func TestMVCCScanMaxNum(t *testing.T) { } res, err = MVCCScan(ctx, engine, testKey5, testKey6.Next(), - hlc.Timestamp{WallTime: 1}, MVCCScanOptions{Reverse: true, MaxKeys: 1}) + enginepb.TxnTimestamp{WallTime: 1}, MVCCScanOptions{Reverse: true, MaxKeys: 1}) if err != nil { t.Fatal(err) } @@ -1259,24 +1259,24 @@ func TestMVCCScanWithKeyPrefix(t *testing.T) { // b // In this case, if we scan from "a"-"b", we wish to skip // a and a and find "aa'. - if err := MVCCPut(ctx, engine, nil, roachpb.Key("/a"), hlc.Timestamp{WallTime: 1}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, roachpb.Key("/a"), enginepb.TxnTimestamp{WallTime: 1}, value1, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, roachpb.Key("/a"), hlc.Timestamp{WallTime: 2}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, roachpb.Key("/a"), enginepb.TxnTimestamp{WallTime: 2}, value2, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, roachpb.Key("/aa"), hlc.Timestamp{WallTime: 2}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, roachpb.Key("/aa"), enginepb.TxnTimestamp{WallTime: 2}, value2, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, roachpb.Key("/aa"), hlc.Timestamp{WallTime: 3}, value3, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, roachpb.Key("/aa"), enginepb.TxnTimestamp{WallTime: 3}, value3, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, roachpb.Key("/b"), hlc.Timestamp{WallTime: 1}, value3, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, roachpb.Key("/b"), enginepb.TxnTimestamp{WallTime: 1}, value3, nil); err != nil { t.Fatal(err) } res, err := MVCCScan(ctx, engine, roachpb.Key("/a"), roachpb.Key("/b"), - hlc.Timestamp{WallTime: 2}, MVCCScanOptions{}) + enginepb.TxnTimestamp{WallTime: 2}, MVCCScanOptions{}) if err != nil { t.Fatal(err) } @@ -1301,22 +1301,22 @@ func TestMVCCScanInTxn(t *testing.T) { engine := engineImpl.create() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, enginepb.TxnTimestamp{WallTime: 1}, value1, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, enginepb.TxnTimestamp{WallTime: 1}, value2, nil); err != nil { t.Fatal(err) } - txn := makeTxn(*txn1, hlc.Timestamp{WallTime: 1}) + txn := makeTxn(*txn1, enginepb.TxnTimestamp{WallTime: 1}) if err := MVCCPut(ctx, engine, nil, testKey3, txn.ReadTimestamp, value3, txn); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, value4, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey4, enginepb.TxnTimestamp{WallTime: 1}, value4, nil); err != nil { t.Fatal(err) } res, err := MVCCScan(ctx, engine, testKey2, testKey4, - hlc.Timestamp{WallTime: 1}, MVCCScanOptions{Txn: txn1}) + enginepb.TxnTimestamp{WallTime: 1}, MVCCScanOptions{Txn: txn1}) if err != nil { t.Fatal(err) } @@ -1329,7 +1329,7 @@ func TestMVCCScanInTxn(t *testing.T) { } if _, err := MVCCScan( - ctx, engine, testKey2, testKey4, hlc.Timestamp{WallTime: 1}, MVCCScanOptions{}, + ctx, engine, testKey2, testKey4, enginepb.TxnTimestamp{WallTime: 1}, MVCCScanOptions{}, ); err == nil { t.Fatal("expected error on uncommitted write intent") } @@ -1351,18 +1351,18 @@ func TestMVCCScanInconsistent(t *testing.T) { // A scan with consistent=false should fail in a txn. if _, err := MVCCScan( - ctx, engine, keyMin, keyMax, hlc.Timestamp{WallTime: 1}, + ctx, engine, keyMin, keyMax, enginepb.TxnTimestamp{WallTime: 1}, MVCCScanOptions{Inconsistent: true, Txn: txn1}, ); err == nil { t.Error("expected an error scanning with consistent=false in txn") } - ts1 := hlc.Timestamp{WallTime: 1} - ts2 := hlc.Timestamp{WallTime: 2} - ts3 := hlc.Timestamp{WallTime: 3} - ts4 := hlc.Timestamp{WallTime: 4} - ts5 := hlc.Timestamp{WallTime: 5} - ts6 := hlc.Timestamp{WallTime: 6} + ts1 := enginepb.TxnTimestamp{WallTime: 1} + ts2 := enginepb.TxnTimestamp{WallTime: 2} + ts3 := enginepb.TxnTimestamp{WallTime: 3} + ts4 := enginepb.TxnTimestamp{WallTime: 4} + ts5 := enginepb.TxnTimestamp{WallTime: 5} + ts6 := enginepb.TxnTimestamp{WallTime: 6} if err := MVCCPut(ctx, engine, nil, testKey1, ts1, value1, nil); err != nil { t.Fatal(err) } @@ -1389,7 +1389,7 @@ func TestMVCCScanInconsistent(t *testing.T) { roachpb.MakeIntent(&txn2ts5.TxnMeta, testKey3), } res, err := MVCCScan( - ctx, engine, testKey1, testKey4.Next(), hlc.Timestamp{WallTime: 7}, + ctx, engine, testKey1, testKey4.Next(), enginepb.TxnTimestamp{WallTime: 7}, MVCCScanOptions{Inconsistent: true}, ) if err != nil { @@ -1399,7 +1399,7 @@ func TestMVCCScanInconsistent(t *testing.T) { t.Fatalf("expected %v, but found %v", expIntents, res.Intents) } - makeTimestampedValue := func(v roachpb.Value, ts hlc.Timestamp) roachpb.Value { + makeTimestampedValue := func(v roachpb.Value, ts enginepb.TxnTimestamp) roachpb.Value { v.Timestamp = ts return v } @@ -1416,7 +1416,7 @@ func TestMVCCScanInconsistent(t *testing.T) { // Now try a scan at a historical timestamp. expIntents = expIntents[:1] res, err = MVCCScan(ctx, engine, testKey1, testKey4.Next(), - hlc.Timestamp{WallTime: 3}, MVCCScanOptions{Inconsistent: true}) + enginepb.TxnTimestamp{WallTime: 3}, MVCCScanOptions{Inconsistent: true}) if !reflect.DeepEqual(res.Intents, expIntents) { t.Fatal(err) } @@ -1441,28 +1441,28 @@ func TestMVCCDeleteRange(t *testing.T) { engine := engineImpl.create() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, enginepb.TxnTimestamp{WallTime: 1}, value1, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, enginepb.TxnTimestamp{WallTime: 1}, value2, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 1}, value3, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey3, enginepb.TxnTimestamp{WallTime: 1}, value3, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, value4, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey4, enginepb.TxnTimestamp{WallTime: 1}, value4, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey5, hlc.Timestamp{WallTime: 1}, value5, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey5, enginepb.TxnTimestamp{WallTime: 1}, value5, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey6, hlc.Timestamp{WallTime: 1}, value6, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey6, enginepb.TxnTimestamp{WallTime: 1}, value6, nil); err != nil { t.Fatal(err) } // Attempt to delete two keys. deleted, resumeSpan, num, err := MVCCDeleteRange( - ctx, engine, nil, testKey2, testKey6, 2, hlc.Timestamp{WallTime: 2}, nil, false, + ctx, engine, nil, testKey2, testKey6, 2, enginepb.TxnTimestamp{WallTime: 2}, nil, false, ) if err != nil { t.Fatal(err) @@ -1477,7 +1477,7 @@ func TestMVCCDeleteRange(t *testing.T) { t.Fatalf("expected = %+v, resumeSpan = %+v", expected, resumeSpan) } res, _ := MVCCScan(ctx, engine, keyMin, keyMax, - hlc.Timestamp{WallTime: 2}, MVCCScanOptions{}) + enginepb.TxnTimestamp{WallTime: 2}, MVCCScanOptions{}) if len(res.KVs) != 4 || !bytes.Equal(res.KVs[0].Key, testKey1) || !bytes.Equal(res.KVs[1].Key, testKey4) || @@ -1493,7 +1493,7 @@ func TestMVCCDeleteRange(t *testing.T) { // Try again, but with tombstones set to true to fetch the deleted keys as well. kvs := []roachpb.KeyValue{} if _, err = MVCCIterate( - ctx, engine, keyMin, keyMax, hlc.Timestamp{WallTime: 2}, MVCCScanOptions{Tombstones: true}, + ctx, engine, keyMin, keyMax, enginepb.TxnTimestamp{WallTime: 2}, MVCCScanOptions{Tombstones: true}, func(kv roachpb.KeyValue) error { kvs = append(kvs, kv) return nil @@ -1519,7 +1519,7 @@ func TestMVCCDeleteRange(t *testing.T) { // Attempt to delete no keys. deleted, resumeSpan, num, err = MVCCDeleteRange( - ctx, engine, nil, testKey2, testKey6, -1, hlc.Timestamp{WallTime: 2}, nil, false) + ctx, engine, nil, testKey2, testKey6, -1, enginepb.TxnTimestamp{WallTime: 2}, nil, false) if err != nil { t.Fatal(err) } @@ -1532,7 +1532,7 @@ func TestMVCCDeleteRange(t *testing.T) { if expected := (roachpb.Span{Key: testKey2, EndKey: testKey6}); !resumeSpan.EqualValue(expected) { t.Fatalf("expected = %+v, resumeSpan = %+v", expected, resumeSpan) } - res, _ = MVCCScan(ctx, engine, keyMin, keyMax, hlc.Timestamp{WallTime: 2}, + res, _ = MVCCScan(ctx, engine, keyMin, keyMax, enginepb.TxnTimestamp{WallTime: 2}, MVCCScanOptions{}) if len(res.KVs) != 4 || !bytes.Equal(res.KVs[0].Key, testKey1) || @@ -1547,7 +1547,7 @@ func TestMVCCDeleteRange(t *testing.T) { } deleted, resumeSpan, num, err = MVCCDeleteRange( - ctx, engine, nil, testKey4, keyMax, 0, hlc.Timestamp{WallTime: 2}, nil, false) + ctx, engine, nil, testKey4, keyMax, 0, enginepb.TxnTimestamp{WallTime: 2}, nil, false) if err != nil { t.Fatal(err) } @@ -1560,7 +1560,7 @@ func TestMVCCDeleteRange(t *testing.T) { if resumeSpan != nil { t.Fatalf("wrong resume key: expected nil, found %v", resumeSpan) } - res, err = MVCCScan(ctx, engine, keyMin, keyMax, hlc.Timestamp{WallTime: 2}, + res, err = MVCCScan(ctx, engine, keyMin, keyMax, enginepb.TxnTimestamp{WallTime: 2}, MVCCScanOptions{}) if err != nil { t.Fatal(err) @@ -1572,7 +1572,7 @@ func TestMVCCDeleteRange(t *testing.T) { } deleted, resumeSpan, num, err = MVCCDeleteRange( - ctx, engine, nil, keyMin, testKey2, 0, hlc.Timestamp{WallTime: 2}, nil, false) + ctx, engine, nil, keyMin, testKey2, 0, enginepb.TxnTimestamp{WallTime: 2}, nil, false) if err != nil { t.Fatal(err) } @@ -1585,7 +1585,7 @@ func TestMVCCDeleteRange(t *testing.T) { if resumeSpan != nil { t.Fatalf("wrong resume key: expected nil, found %v", resumeSpan) } - res, _ = MVCCScan(ctx, engine, keyMin, keyMax, hlc.Timestamp{WallTime: 2}, + res, _ = MVCCScan(ctx, engine, keyMin, keyMax, enginepb.TxnTimestamp{WallTime: 2}, MVCCScanOptions{}) if err != nil { t.Fatal(err) @@ -1607,28 +1607,28 @@ func TestMVCCDeleteRangeReturnKeys(t *testing.T) { engine := engineImpl.create() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, enginepb.TxnTimestamp{WallTime: 1}, value1, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, enginepb.TxnTimestamp{WallTime: 1}, value2, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 1}, value3, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey3, enginepb.TxnTimestamp{WallTime: 1}, value3, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, value4, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey4, enginepb.TxnTimestamp{WallTime: 1}, value4, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey5, hlc.Timestamp{WallTime: 1}, value5, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey5, enginepb.TxnTimestamp{WallTime: 1}, value5, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey6, hlc.Timestamp{WallTime: 1}, value6, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey6, enginepb.TxnTimestamp{WallTime: 1}, value6, nil); err != nil { t.Fatal(err) } // Attempt to delete two keys. deleted, resumeSpan, num, err := MVCCDeleteRange( - ctx, engine, nil, testKey2, testKey6, 2, hlc.Timestamp{WallTime: 2}, nil, true) + ctx, engine, nil, testKey2, testKey6, 2, enginepb.TxnTimestamp{WallTime: 2}, nil, true) if err != nil { t.Fatal(err) } @@ -1647,7 +1647,7 @@ func TestMVCCDeleteRangeReturnKeys(t *testing.T) { if expected := (roachpb.Span{Key: testKey4, EndKey: testKey6}); !resumeSpan.EqualValue(expected) { t.Fatalf("expected = %+v, resumeSpan = %+v", expected, resumeSpan) } - res, _ := MVCCScan(ctx, engine, keyMin, keyMax, hlc.Timestamp{WallTime: 2}, + res, _ := MVCCScan(ctx, engine, keyMin, keyMax, enginepb.TxnTimestamp{WallTime: 2}, MVCCScanOptions{}) if len(res.KVs) != 4 || !bytes.Equal(res.KVs[0].Key, testKey1) || @@ -1663,7 +1663,7 @@ func TestMVCCDeleteRangeReturnKeys(t *testing.T) { // Attempt to delete no keys. deleted, resumeSpan, num, err = MVCCDeleteRange( - ctx, engine, nil, testKey2, testKey6, -1, hlc.Timestamp{WallTime: 2}, nil, true) + ctx, engine, nil, testKey2, testKey6, -1, enginepb.TxnTimestamp{WallTime: 2}, nil, true) if err != nil { t.Fatal(err) } @@ -1676,7 +1676,7 @@ func TestMVCCDeleteRangeReturnKeys(t *testing.T) { if expected := (roachpb.Span{Key: testKey2, EndKey: testKey6}); !resumeSpan.EqualValue(expected) { t.Fatalf("expected = %+v, resumeSpan = %+v", expected, resumeSpan) } - res, _ = MVCCScan(ctx, engine, keyMin, keyMax, hlc.Timestamp{WallTime: 2}, + res, _ = MVCCScan(ctx, engine, keyMin, keyMax, enginepb.TxnTimestamp{WallTime: 2}, MVCCScanOptions{}) if len(res.KVs) != 4 || !bytes.Equal(res.KVs[0].Key, testKey1) || @@ -1691,7 +1691,7 @@ func TestMVCCDeleteRangeReturnKeys(t *testing.T) { } deleted, resumeSpan, num, err = MVCCDeleteRange( - ctx, engine, nil, testKey4, keyMax, math.MaxInt64, hlc.Timestamp{WallTime: 2}, nil, true) + ctx, engine, nil, testKey4, keyMax, math.MaxInt64, enginepb.TxnTimestamp{WallTime: 2}, nil, true) if err != nil { t.Fatal(err) } @@ -1713,7 +1713,7 @@ func TestMVCCDeleteRangeReturnKeys(t *testing.T) { if resumeSpan != nil { t.Fatalf("wrong resume key: expected nil, found %v", resumeSpan) } - res, _ = MVCCScan(ctx, engine, keyMin, keyMax, hlc.Timestamp{WallTime: 2}, + res, _ = MVCCScan(ctx, engine, keyMin, keyMax, enginepb.TxnTimestamp{WallTime: 2}, MVCCScanOptions{}) if len(res.KVs) != 1 || !bytes.Equal(res.KVs[0].Key, testKey1) || @@ -1722,7 +1722,7 @@ func TestMVCCDeleteRangeReturnKeys(t *testing.T) { } deleted, resumeSpan, num, err = MVCCDeleteRange( - ctx, engine, nil, keyMin, testKey2, math.MaxInt64, hlc.Timestamp{WallTime: 2}, nil, true) + ctx, engine, nil, keyMin, testKey2, math.MaxInt64, enginepb.TxnTimestamp{WallTime: 2}, nil, true) if err != nil { t.Fatal(err) } @@ -1738,7 +1738,7 @@ func TestMVCCDeleteRangeReturnKeys(t *testing.T) { if resumeSpan != nil { t.Fatalf("wrong resume key: %v", resumeSpan) } - res, _ = MVCCScan(ctx, engine, keyMin, keyMax, hlc.Timestamp{WallTime: 2}, + res, _ = MVCCScan(ctx, engine, keyMin, keyMax, enginepb.TxnTimestamp{WallTime: 2}, MVCCScanOptions{}) if len(res.KVs) != 0 { t.Fatal("the value should be empty") @@ -1757,8 +1757,8 @@ func TestMVCCDeleteRangeFailed(t *testing.T) { engine := engineImpl.create() defer engine.Close() - txn := makeTxn(*txn1, hlc.Timestamp{WallTime: 1}) - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, value1, nil); err != nil { + txn := makeTxn(*txn1, enginepb.TxnTimestamp{WallTime: 1}) + if err := MVCCPut(ctx, engine, nil, testKey1, enginepb.TxnTimestamp{WallTime: 1}, value1, nil); err != nil { t.Fatal(err) } txn.Sequence++ @@ -1769,12 +1769,12 @@ func TestMVCCDeleteRangeFailed(t *testing.T) { if err := MVCCPut(ctx, engine, nil, testKey3, txn.ReadTimestamp, value3, txn); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, value4, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey4, enginepb.TxnTimestamp{WallTime: 1}, value4, nil); err != nil { t.Fatal(err) } if _, _, _, err := MVCCDeleteRange( - ctx, engine, nil, testKey2, testKey4, math.MaxInt64, hlc.Timestamp{WallTime: 1}, nil, false, + ctx, engine, nil, testKey2, testKey4, math.MaxInt64, enginepb.TxnTimestamp{WallTime: 1}, nil, false, ); err == nil { t.Fatal("expected error on uncommitted write intent") } @@ -1799,10 +1799,10 @@ func TestMVCCDeleteRangeConcurrentTxn(t *testing.T) { engine := engineImpl.create() defer engine.Close() - txn1ts := makeTxn(*txn1, hlc.Timestamp{WallTime: 1}) - txn2ts := makeTxn(*txn2, hlc.Timestamp{WallTime: 2}) + txn1ts := makeTxn(*txn1, enginepb.TxnTimestamp{WallTime: 1}) + txn2ts := makeTxn(*txn2, enginepb.TxnTimestamp{WallTime: 2}) - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, enginepb.TxnTimestamp{WallTime: 1}, value1, nil); err != nil { t.Fatal(err) } if err := MVCCPut(ctx, engine, nil, testKey2, txn1ts.ReadTimestamp, value2, txn1ts); err != nil { @@ -1811,7 +1811,7 @@ func TestMVCCDeleteRangeConcurrentTxn(t *testing.T) { if err := MVCCPut(ctx, engine, nil, testKey3, txn2ts.ReadTimestamp, value3, txn2ts); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, value4, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey4, enginepb.TxnTimestamp{WallTime: 1}, value4, nil); err != nil { t.Fatal(err) } @@ -1837,28 +1837,28 @@ func TestMVCCUncommittedDeleteRangeVisible(t *testing.T) { defer engine.Close() if err := MVCCPut( - ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, value1, nil, + ctx, engine, nil, testKey1, enginepb.TxnTimestamp{WallTime: 1}, value1, nil, ); err != nil { t.Fatal(err) } if err := MVCCPut( - ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, value2, nil, + ctx, engine, nil, testKey2, enginepb.TxnTimestamp{WallTime: 1}, value2, nil, ); err != nil { t.Fatal(err) } if err := MVCCPut( - ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 1}, value3, nil, + ctx, engine, nil, testKey3, enginepb.TxnTimestamp{WallTime: 1}, value3, nil, ); err != nil { t.Fatal(err) } if err := MVCCDelete( - ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 2, Logical: 1}, nil, + ctx, engine, nil, testKey2, enginepb.TxnTimestamp{WallTime: 2, Logical: 1}, nil, ); err != nil { t.Fatal(err) } - txn := makeTxn(*txn1, hlc.Timestamp{WallTime: 2, Logical: 2}) + txn := makeTxn(*txn1, enginepb.TxnTimestamp{WallTime: 2, Logical: 2}) if _, _, _, err := MVCCDeleteRange( ctx, engine, nil, testKey1, testKey4, math.MaxInt64, txn.ReadTimestamp, txn, false, ); err != nil { @@ -1867,7 +1867,7 @@ func TestMVCCUncommittedDeleteRangeVisible(t *testing.T) { txn.Epoch++ res, _ := MVCCScan(ctx, engine, testKey1, testKey4, - hlc.Timestamp{WallTime: 3}, MVCCScanOptions{Txn: txn}) + enginepb.TxnTimestamp{WallTime: 3}, MVCCScanOptions{Txn: txn}) if e := 2; len(res.KVs) != e { t.Fatalf("e = %d, got %d", e, len(res.KVs)) } @@ -1886,15 +1886,15 @@ func TestMVCCDeleteRangeOldTimestamp(t *testing.T) { t.Run(engineImpl.name, func(t *testing.T) { engine := engineImpl.create() defer engine.Close() - err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, value1, nil) + err := MVCCPut(ctx, engine, nil, testKey1, enginepb.TxnTimestamp{WallTime: 1}, value1, nil) if err != nil { t.Fatal(err) } - err = MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 3}, value2, nil) + err = MVCCPut(ctx, engine, nil, testKey2, enginepb.TxnTimestamp{WallTime: 3}, value2, nil) if err != nil { t.Fatal(err) } - err = MVCCDelete(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 5}, nil) + err = MVCCDelete(ctx, engine, nil, testKey2, enginepb.TxnTimestamp{WallTime: 5}, nil) if err != nil { t.Fatal(err) } @@ -1903,7 +1903,7 @@ func TestMVCCDeleteRangeOldTimestamp(t *testing.T) { b := engine.NewBatch() defer b.Close() keys, resume, keyCount, err := MVCCDeleteRange( - ctx, b, nil, testKey1, testKey4, math.MaxInt64, hlc.Timestamp{WallTime: 4}, nil, true, + ctx, b, nil, testKey1, testKey4, math.MaxInt64, enginepb.TxnTimestamp{WallTime: 4}, nil, true, ) require.Nil(t, keys) require.Nil(t, resume) @@ -1915,7 +1915,7 @@ func TestMVCCDeleteRangeOldTimestamp(t *testing.T) { b = engine.NewBatch() defer b.Close() keys, resume, keyCount, err = MVCCDeleteRange( - ctx, b, nil, testKey1, testKey4, math.MaxInt64, hlc.Timestamp{WallTime: 5}, nil, true, + ctx, b, nil, testKey1, testKey4, math.MaxInt64, enginepb.TxnTimestamp{WallTime: 5}, nil, true, ) require.Nil(t, keys) require.Nil(t, resume) @@ -1928,7 +1928,7 @@ func TestMVCCDeleteRangeOldTimestamp(t *testing.T) { b = engine.NewBatch() defer b.Close() keys, resume, keyCount, err = MVCCDeleteRange( - ctx, b, nil, testKey1, testKey4, math.MaxInt64, hlc.Timestamp{WallTime: 6}, nil, true, + ctx, b, nil, testKey1, testKey4, math.MaxInt64, enginepb.TxnTimestamp{WallTime: 6}, nil, true, ) require.Equal(t, []roachpb.Key{testKey1}, keys) require.Nil(t, resume) @@ -1959,19 +1959,19 @@ func TestMVCCDeleteRangeInline(t *testing.T) { {testKey4, value4}, {testKey5, value5}, } { - if err := MVCCPut(ctx, engine, nil, kv.key, hlc.Timestamp{Logical: 0}, kv.value, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, kv.key, enginepb.TxnTimestamp{Logical: 0}, kv.value, nil); err != nil { t.Fatalf("%d: %+v", i, err) } } // Create one non-inline value (non-zero timestamp). - if err := MVCCPut(ctx, engine, nil, testKey6, hlc.Timestamp{WallTime: 1}, value6, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey6, enginepb.TxnTimestamp{WallTime: 1}, value6, nil); err != nil { t.Fatal(err) } // Attempt to delete two inline keys, should succeed. deleted, resumeSpan, num, err := MVCCDeleteRange( - ctx, engine, nil, testKey2, testKey6, 2, hlc.Timestamp{Logical: 0}, nil, true, + ctx, engine, nil, testKey2, testKey6, 2, enginepb.TxnTimestamp{Logical: 0}, nil, true, ) if err != nil { t.Fatal(err) @@ -1989,7 +1989,7 @@ func TestMVCCDeleteRangeInline(t *testing.T) { // Attempt to delete inline keys at a timestamp; should fail. const inlineMismatchErrString = "put is inline" if _, _, _, err := MVCCDeleteRange( - ctx, engine, nil, testKey1, testKey6, 1, hlc.Timestamp{WallTime: 2}, nil, true, + ctx, engine, nil, testKey1, testKey6, 1, enginepb.TxnTimestamp{WallTime: 2}, nil, true, ); !testutils.IsError(err, inlineMismatchErrString) { t.Fatalf("got error %v, expected error with text '%s'", err, inlineMismatchErrString) } @@ -1997,14 +1997,14 @@ func TestMVCCDeleteRangeInline(t *testing.T) { // Attempt to delete non-inline key at zero timestamp; should fail. const writeTooOldErrString = "WriteTooOldError" if _, _, _, err := MVCCDeleteRange( - ctx, engine, nil, testKey6, keyMax, 1, hlc.Timestamp{Logical: 0}, nil, true, + ctx, engine, nil, testKey6, keyMax, 1, enginepb.TxnTimestamp{Logical: 0}, nil, true, ); !testutils.IsError(err, writeTooOldErrString) { t.Fatalf("got error %v, expected error with text '%s'", err, writeTooOldErrString) } // Attempt to delete inline keys in a transaction; should fail. if _, _, _, err := MVCCDeleteRange( - ctx, engine, nil, testKey2, testKey6, 2, hlc.Timestamp{Logical: 0}, txn1, true, + ctx, engine, nil, testKey2, testKey6, 2, enginepb.TxnTimestamp{Logical: 0}, txn1, true, ); !testutils.IsError(err, "writes not allowed within transactions") { t.Errorf("unexpected error: %+v", err) } @@ -2028,7 +2028,7 @@ func TestMVCCDeleteRangeInline(t *testing.T) { Value: value6, }, } - res, err := MVCCScan(ctx, engine, keyMin, keyMax, hlc.Timestamp{WallTime: 2}, + res, err := MVCCScan(ctx, engine, keyMin, keyMax, enginepb.TxnTimestamp{WallTime: 2}, MVCCScanOptions{}) if err != nil { t.Fatal(err) @@ -2036,7 +2036,7 @@ func TestMVCCDeleteRangeInline(t *testing.T) { if a, e := len(res.KVs), len(expectedKvs); a != e { t.Fatalf("engine scan found %d keys; expected %d", a, e) } - res.KVs[3].Value.Timestamp = hlc.Timestamp{} + res.KVs[3].Value.Timestamp = enginepb.TxnTimestamp{} if !reflect.DeepEqual(expectedKvs, res.KVs) { t.Fatalf( "engine scan found key/values: %v; expected %v. Diff: %s", @@ -2057,29 +2057,29 @@ func TestMVCCClearTimeRange(t *testing.T) { for _, engineImpl := range mvccEngineImpls { t.Run(engineImpl.name, func(t *testing.T) { - ts0 := hlc.Timestamp{WallTime: 0} + ts0 := enginepb.TxnTimestamp{WallTime: 0} ts0Content := []roachpb.KeyValue{} - ts1 := hlc.Timestamp{WallTime: 10} + ts1 := enginepb.TxnTimestamp{WallTime: 10} v1 := value1 v1.Timestamp = ts1 ts1Content := []roachpb.KeyValue{{Key: testKey2, Value: v1}} - ts2 := hlc.Timestamp{WallTime: 20} + ts2 := enginepb.TxnTimestamp{WallTime: 20} v2 := value2 v2.Timestamp = ts2 ts2Content := []roachpb.KeyValue{{Key: testKey2, Value: v2}, {Key: testKey5, Value: v2}} - ts3 := hlc.Timestamp{WallTime: 30} + ts3 := enginepb.TxnTimestamp{WallTime: 30} v3 := value3 v3.Timestamp = ts3 ts3Content := []roachpb.KeyValue{ {Key: testKey1, Value: v3}, {Key: testKey2, Value: v2}, {Key: testKey5, Value: v2}, } - ts4 := hlc.Timestamp{WallTime: 40} + ts4 := enginepb.TxnTimestamp{WallTime: 40} v4 := value4 v4.Timestamp = ts4 ts4Content := []roachpb.KeyValue{ {Key: testKey1, Value: v3}, {Key: testKey2, Value: v4}, {Key: testKey5, Value: v4}, } - ts5 := hlc.Timestamp{WallTime: 50} + ts5 := enginepb.TxnTimestamp{WallTime: 50} // setupKVs will generate an engine with the key-time space as follows: // 50 - @@ -2109,7 +2109,7 @@ func TestMVCCClearTimeRange(t *testing.T) { return engine } - assertKVs := func(t *testing.T, reader Reader, at hlc.Timestamp, expected []roachpb.KeyValue) { + assertKVs := func(t *testing.T, reader Reader, at enginepb.TxnTimestamp, expected []roachpb.KeyValue) { t.Helper() res, err := MVCCScan(ctx, reader, keyMin, keyMax, at, MVCCScanOptions{}) require.NoError(t, err) @@ -2209,7 +2209,7 @@ func TestMVCCClearTimeRange(t *testing.T) { }) // Add an intent at k3@ts3. - txn := roachpb.MakeTransaction("test", nil, roachpb.NormalUserPriority, ts3, 1) + txn := roachpb.MakeTransaction("test", nil, roachpb.NormalUserPriority, hlc.Timestamp(ts3), 1) setupKVsWithIntent := func(t *testing.T) Engine { e := setupKVs(t) require.NoError(t, MVCCPut(ctx, e, nil, testKey3, ts3, value3, &txn)) @@ -2295,7 +2295,7 @@ func TestMVCCClearTimeRangeOnRandomData(t *testing.T) { e := engineImpl.create() defer e.Close() - now := hlc.Timestamp{WallTime: 100000000} + now := enginepb.TxnTimestamp{WallTime: 100000000} var ms enginepb.MVCCStats @@ -2324,24 +2324,24 @@ func TestMVCCClearTimeRangeOnRandomData(t *testing.T) { key := roachpb.Key(fmt.Sprintf("%05d", k)) if rand.Float64() > 0.8 { - require.NoError(t, MVCCDelete(ctx, e, &ms, key, hlc.Timestamp{WallTime: ts}, nil)) + require.NoError(t, MVCCDelete(ctx, e, &ms, key, enginepb.TxnTimestamp{WallTime: ts}, nil)) } else { v := roachpb.MakeValueFromString(fmt.Sprintf("v-%d", i)) - require.NoError(t, MVCCPut(ctx, e, &ms, key, hlc.Timestamp{WallTime: ts}, v, nil)) + require.NoError(t, MVCCPut(ctx, e, &ms, key, enginepb.TxnTimestamp{WallTime: ts}, v, nil)) } } swathTime := rand.Intn(randTimeRange-100) + 100 for i := swathStart; i < swathEnd; i++ { key := roachpb.Key(fmt.Sprintf("%05d", i)) v := roachpb.MakeValueFromString(fmt.Sprintf("v-%d", i)) - require.NoError(t, MVCCPut(ctx, e, &ms, key, hlc.Timestamp{WallTime: int64(swathTime)}, v, nil)) + require.NoError(t, MVCCPut(ctx, e, &ms, key, enginepb.TxnTimestamp{WallTime: int64(swathTime)}, v, nil)) } // Add another swath of keys above to exercise an after-iteration range flush. for i := keyRange; i < keyRange+200; i++ { key := roachpb.Key(fmt.Sprintf("%05d", i)) v := roachpb.MakeValueFromString(fmt.Sprintf("v-%d", i)) - require.NoError(t, MVCCPut(ctx, e, &ms, key, hlc.Timestamp{WallTime: int64(randTimeRange + 1)}, v, nil)) + require.NoError(t, MVCCPut(ctx, e, &ms, key, enginepb.TxnTimestamp{WallTime: int64(randTimeRange + 1)}, v, nil)) } ms.AgeTo(2000) @@ -2361,7 +2361,7 @@ func TestMVCCClearTimeRangeOnRandomData(t *testing.T) { for i := len(reverts) - 1; i >= 0; i-- { t.Run(fmt.Sprintf("revert-%d", i), func(t *testing.T) { - revertTo := hlc.Timestamp{WallTime: int64(reverts[i])} + revertTo := enginepb.TxnTimestamp{WallTime: int64(reverts[i])} // MVCC-Scan at the revert time. resBefore, err := MVCCScan(ctx, e, keyMin, keyMax, revertTo, MVCCScanOptions{MaxKeys: numKVs}) require.NoError(t, err) @@ -2399,25 +2399,25 @@ func TestMVCCInitPut(t *testing.T) { engine := engineImpl.create() defer engine.Close() - err := MVCCInitPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 1}, value1, false, nil) + err := MVCCInitPut(ctx, engine, nil, testKey1, enginepb.TxnTimestamp{Logical: 1}, value1, false, nil) if err != nil { t.Fatal(err) } // A repeat of the command will still succeed - err = MVCCInitPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 2}, value1, false, nil) + err = MVCCInitPut(ctx, engine, nil, testKey1, enginepb.TxnTimestamp{Logical: 2}, value1, false, nil) if err != nil { t.Fatal(err) } // Delete. - err = MVCCDelete(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 3}, nil) + err = MVCCDelete(ctx, engine, nil, testKey1, enginepb.TxnTimestamp{Logical: 3}, nil) if err != nil { t.Fatal(err) } // Reinserting the value fails if we fail on tombstones. - err = MVCCInitPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 4}, value1, true, nil) + err = MVCCInitPut(ctx, engine, nil, testKey1, enginepb.TxnTimestamp{Logical: 4}, value1, true, nil) if e := (*roachpb.ConditionFailedError)(nil); errors.As(err, &e) { if !bytes.Equal(e.ActualValue.RawBytes, nil) { t.Fatalf("the value %s in get result is not a tombstone", e.ActualValue.RawBytes) @@ -2429,13 +2429,13 @@ func TestMVCCInitPut(t *testing.T) { } // But doesn't if we *don't* fail on tombstones. - err = MVCCInitPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 5}, value1, false, nil) + err = MVCCInitPut(ctx, engine, nil, testKey1, enginepb.TxnTimestamp{Logical: 5}, value1, false, nil) if err != nil { t.Fatal(err) } // A repeat of the command with a different value will fail. - err = MVCCInitPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 6}, value2, false, nil) + err = MVCCInitPut(ctx, engine, nil, testKey1, enginepb.TxnTimestamp{Logical: 6}, value2, false, nil) if e := (*roachpb.ConditionFailedError)(nil); errors.As(err, &e) { if !bytes.Equal(e.ActualValue.RawBytes, value1.RawBytes) { t.Fatalf("the value %s in get result does not match the value %s in request", @@ -2449,13 +2449,13 @@ func TestMVCCInitPut(t *testing.T) { // Ensure that the timestamps were correctly updated. for _, check := range []struct { - ts, expTS hlc.Timestamp + ts, expTS enginepb.TxnTimestamp }{ - {ts: hlc.Timestamp{Logical: 1}, expTS: hlc.Timestamp{Logical: 1}}, - {ts: hlc.Timestamp{Logical: 2}, expTS: hlc.Timestamp{Logical: 2}}, + {ts: enginepb.TxnTimestamp{Logical: 1}, expTS: enginepb.TxnTimestamp{Logical: 1}}, + {ts: enginepb.TxnTimestamp{Logical: 2}, expTS: enginepb.TxnTimestamp{Logical: 2}}, // If we're checking the future wall time case, the rewrite after delete // will be present. - {ts: hlc.Timestamp{WallTime: 1}, expTS: hlc.Timestamp{Logical: 5}}, + {ts: enginepb.TxnTimestamp{WallTime: 1}, expTS: enginepb.TxnTimestamp{Logical: 5}}, } { value, _, err := MVCCGet(ctx, engine, testKey1, check.ts, MVCCGetOptions{}) if err != nil { @@ -2470,7 +2470,7 @@ func TestMVCCInitPut(t *testing.T) { } } - value, _, pErr := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{Logical: 0}, MVCCGetOptions{}) + value, _, pErr := MVCCGet(ctx, engine, testKey1, enginepb.TxnTimestamp{Logical: 0}, MVCCGetOptions{}) if pErr != nil { t.Fatal(pErr) } @@ -2519,14 +2519,14 @@ func TestMVCCInitPutWithTxn(t *testing.T) { // Commit value3. txnCommit := txn txnCommit.Status = roachpb.COMMITTED - txnCommit.WriteTimestamp = clock.Now().Add(1, 0) + txnCommit.WriteTimestamp = enginepb.TxnTimestamp(clock.Now().Add(1, 0)) if _, err := MVCCResolveWriteIntent(ctx, engine, nil, roachpb.MakeLockUpdate(&txnCommit, roachpb.Span{Key: testKey1})); err != nil { t.Fatal(err) } // Write value4 with an old timestamp without txn...should get an error. - err = MVCCInitPut(ctx, engine, nil, testKey1, clock.Now(), value4, false, nil) + err = MVCCInitPut(ctx, engine, nil, testKey1, enginepb.TxnTimestamp(clock.Now()), value4, false, nil) if e := (*roachpb.ConditionFailedError)(nil); errors.As(err, &e) { if !bytes.Equal(e.ActualValue.RawBytes, value2.RawBytes) { t.Fatalf("the value %s in get result does not match the value %s in request", @@ -2551,33 +2551,33 @@ func TestMVCCReverseScan(t *testing.T) { engine := engineImpl.create() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, enginepb.TxnTimestamp{WallTime: 1}, value1, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 2}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, enginepb.TxnTimestamp{WallTime: 2}, value2, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, value3, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, enginepb.TxnTimestamp{WallTime: 1}, value3, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 3}, value4, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, enginepb.TxnTimestamp{WallTime: 3}, value4, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 1}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey3, enginepb.TxnTimestamp{WallTime: 1}, value1, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey4, hlc.Timestamp{WallTime: 1}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey4, enginepb.TxnTimestamp{WallTime: 1}, value2, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey5, hlc.Timestamp{WallTime: 3}, value5, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey5, enginepb.TxnTimestamp{WallTime: 3}, value5, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey6, hlc.Timestamp{WallTime: 3}, value6, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey6, enginepb.TxnTimestamp{WallTime: 3}, value6, nil); err != nil { t.Fatal(err) } res, err := MVCCScan(ctx, engine, testKey2, testKey4, - hlc.Timestamp{WallTime: 1}, MVCCScanOptions{Reverse: true}) + enginepb.TxnTimestamp{WallTime: 1}, MVCCScanOptions{Reverse: true}) if err != nil { t.Fatal(err) @@ -2593,7 +2593,7 @@ func TestMVCCReverseScan(t *testing.T) { t.Fatalf("resumeSpan = %+v", res.ResumeSpan) } - res, err = MVCCScan(ctx, engine, testKey2, testKey4, hlc.Timestamp{WallTime: 1}, + res, err = MVCCScan(ctx, engine, testKey2, testKey4, enginepb.TxnTimestamp{WallTime: 1}, MVCCScanOptions{Reverse: true, MaxKeys: 1}) if err != nil { @@ -2608,7 +2608,7 @@ func TestMVCCReverseScan(t *testing.T) { t.Fatalf("expected = %+v, resumeSpan = %+v", expected, res.ResumeSpan) } - res, err = MVCCScan(ctx, engine, testKey2, testKey4, hlc.Timestamp{WallTime: 1}, + res, err = MVCCScan(ctx, engine, testKey2, testKey4, enginepb.TxnTimestamp{WallTime: 1}, MVCCScanOptions{Reverse: true, MaxKeys: -1}) if err != nil { @@ -2623,7 +2623,7 @@ func TestMVCCReverseScan(t *testing.T) { // The first key we encounter has multiple versions and we need to read the // latest. - res, err = MVCCScan(ctx, engine, testKey2, testKey3, hlc.Timestamp{WallTime: 4}, + res, err = MVCCScan(ctx, engine, testKey2, testKey3, enginepb.TxnTimestamp{WallTime: 4}, MVCCScanOptions{Reverse: true, MaxKeys: 1}) if err != nil { @@ -2637,7 +2637,7 @@ func TestMVCCReverseScan(t *testing.T) { // The first key we encounter is newer than our read timestamp and we need to // back up to the previous key. - res, err = MVCCScan(ctx, engine, testKey4, testKey6, hlc.Timestamp{WallTime: 1}, + res, err = MVCCScan(ctx, engine, testKey4, testKey6, enginepb.TxnTimestamp{WallTime: 1}, MVCCScanOptions{Reverse: true, MaxKeys: 1}) if err != nil { @@ -2650,7 +2650,7 @@ func TestMVCCReverseScan(t *testing.T) { } // Scan only the first key in the key space. - res, err = MVCCScan(ctx, engine, testKey1, testKey1.Next(), hlc.Timestamp{WallTime: 1}, + res, err = MVCCScan(ctx, engine, testKey1, testKey1.Next(), enginepb.TxnTimestamp{WallTime: 1}, MVCCScanOptions{Reverse: true, MaxKeys: 1}) if err != nil { @@ -2687,15 +2687,15 @@ func TestMVCCReverseScanFirstKeyInFuture(t *testing.T) { // Before fixing #17825, the MVCC version scan on key3 would fall out of the // scan bounds and if it never found another valid key before reaching // KeyMax, would stop the ReverseScan from continuing. - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: 1}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, enginepb.TxnTimestamp{WallTime: 1}, value2, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey3, hlc.Timestamp{WallTime: 3}, value3, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey3, enginepb.TxnTimestamp{WallTime: 3}, value3, nil); err != nil { t.Fatal(err) } res, err := MVCCScan(ctx, engine, testKey1, testKey4, - hlc.Timestamp{WallTime: 2}, MVCCScanOptions{Reverse: true}) + enginepb.TxnTimestamp{WallTime: 2}, MVCCScanOptions{Reverse: true}) if err != nil { t.Fatal(err) } @@ -2727,17 +2727,17 @@ func TestMVCCReverseScanSeeksOverRepeatedKeys(t *testing.T) { // written. Repeat the key enough times to make sure the `SeekForPrev()` // optimization will be used. for i := 1; i <= 10; i++ { - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{WallTime: int64(i)}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, enginepb.TxnTimestamp{WallTime: int64(i)}, value2, nil); err != nil { t.Fatal(err) } } - txn1ts := makeTxn(*txn1, hlc.Timestamp{WallTime: 11}) + txn1ts := makeTxn(*txn1, enginepb.TxnTimestamp{WallTime: 11}) if err := MVCCPut(ctx, engine, nil, testKey2, txn1ts.ReadTimestamp, value2, txn1ts); err != nil { t.Fatal(err) } res, err := MVCCScan(ctx, engine, testKey1, testKey3, - hlc.Timestamp{WallTime: 1}, MVCCScanOptions{Reverse: true}) + enginepb.TxnTimestamp{WallTime: 1}, MVCCScanOptions{Reverse: true}) if err != nil { t.Fatal(err) } @@ -2779,13 +2779,13 @@ func TestMVCCReverseScanStopAtSmallestKey(t *testing.T) { defer engine.Close() for i := 1; i <= numPuts; i++ { - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: int64(i)}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, enginepb.TxnTimestamp{WallTime: int64(i)}, value1, nil); err != nil { t.Fatal(err) } } res, err := MVCCScan(ctx, engine, testKey1, testKey3, - hlc.Timestamp{WallTime: ts}, MVCCScanOptions{Reverse: true}) + enginepb.TxnTimestamp{WallTime: ts}, MVCCScanOptions{Reverse: true}) if err != nil { t.Fatal(err) } @@ -2825,7 +2825,7 @@ func TestMVCCResolveTxn(t *testing.T) { } { - value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{Logical: 1}, MVCCGetOptions{ + value, _, err := MVCCGet(ctx, engine, testKey1, enginepb.TxnTimestamp{Logical: 1}, MVCCGetOptions{ Txn: txn1, }) if err != nil { @@ -2844,7 +2844,7 @@ func TestMVCCResolveTxn(t *testing.T) { } { - value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{Logical: 1}, MVCCGetOptions{}) + value, _, err := MVCCGet(ctx, engine, testKey1, enginepb.TxnTimestamp{Logical: 1}, MVCCGetOptions{}) if err != nil { t.Fatal(err) } @@ -2886,7 +2886,7 @@ func TestMVCCResolveNewerIntent(t *testing.T) { t.Fatal(err) } - value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{Logical: 2}, MVCCGetOptions{}) + value, _, err := MVCCGet(ctx, engine, testKey1, enginepb.TxnTimestamp{Logical: 2}, MVCCGetOptions{}) if err != nil { t.Fatal(err) } @@ -2930,16 +2930,16 @@ func TestMVCCResolveIntentTxnTimestampMismatch(t *testing.T) { } for i, test := range []struct { - hlc.Timestamp + enginepb.TxnTimestamp found bool }{ // Check that the intent has indeed moved to where we pushed it. {tsEarly, false}, {intent.Txn.WriteTimestamp.Prev(), false}, {intent.Txn.WriteTimestamp, true}, - {hlc.MaxTimestamp, true}, + {enginepb.MaxTxnTimestamp, true}, } { - _, _, err := MVCCGet(ctx, engine, testKey1, test.Timestamp, MVCCGetOptions{}) + _, _, err := MVCCGet(ctx, engine, testKey1, test.TxnTimestamp, MVCCGetOptions{}) if errors.HasType(err, (*roachpb.WriteIntentError)(nil)) != test.found { t.Fatalf("%d: expected write intent error: %t, got %v", i, test.found, err) } @@ -2963,17 +2963,17 @@ func TestMVCCConditionalPutOldTimestamp(t *testing.T) { t.Run(engineImpl.name, func(t *testing.T) { engine := engineImpl.create() defer engine.Close() - err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, value1, nil) + err := MVCCPut(ctx, engine, nil, testKey1, enginepb.TxnTimestamp{WallTime: 1}, value1, nil) if err != nil { t.Fatal(err) } - err = MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, value2, nil) + err = MVCCPut(ctx, engine, nil, testKey1, enginepb.TxnTimestamp{WallTime: 3}, value2, nil) if err != nil { t.Fatal(err) } // Check nothing is written if the value doesn't match. - err = MVCCConditionalPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 2}, value3, value1.TagAndDataBytes(), CPutFailIfMissing, nil) + err = MVCCConditionalPut(ctx, engine, nil, testKey1, enginepb.TxnTimestamp{WallTime: 2}, value3, value1.TagAndDataBytes(), CPutFailIfMissing, nil) if err == nil { t.Errorf("unexpected success on conditional put") } @@ -2983,7 +2983,7 @@ func TestMVCCConditionalPutOldTimestamp(t *testing.T) { // But if value does match the most recently written version, we'll get // a write too old error but still write updated value. - err = MVCCConditionalPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 2}, value3, value2.TagAndDataBytes(), CPutFailIfMissing, nil) + err = MVCCConditionalPut(ctx, engine, nil, testKey1, enginepb.TxnTimestamp{WallTime: 2}, value3, value2.TagAndDataBytes(), CPutFailIfMissing, nil) if err == nil { t.Errorf("unexpected success on conditional put") } @@ -2991,7 +2991,7 @@ func TestMVCCConditionalPutOldTimestamp(t *testing.T) { t.Errorf("unexpected error on conditional put: %+v", err) } // Verify new value was actually written at (3, 1). - ts := hlc.Timestamp{WallTime: 3, Logical: 1} + ts := enginepb.TxnTimestamp{WallTime: 3, Logical: 1} value, _, err := MVCCGet(ctx, engine, testKey1, ts, MVCCGetOptions{}) if err != nil || value.Timestamp != ts || !bytes.Equal(value3.RawBytes, value.RawBytes) { t.Fatalf("expected err=nil (got %s), timestamp=%s (got %s), value=%q (got %q)", @@ -3016,25 +3016,25 @@ func TestMVCCMultiplePutOldTimestamp(t *testing.T) { engine := engineImpl.create() defer engine.Close() - err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, value1, nil) + err := MVCCPut(ctx, engine, nil, testKey1, enginepb.TxnTimestamp{WallTime: 3}, value1, nil) if err != nil { t.Fatal(err) } // Verify the first txn Put returns a write too old error, but the // intent is written at the advanced timestamp. - txn := makeTxn(*txn1, hlc.Timestamp{WallTime: 1}) + txn := makeTxn(*txn1, enginepb.TxnTimestamp{WallTime: 1}) txn.Sequence++ err = MVCCPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, value2, txn) if !errors.HasType(err, (*roachpb.WriteTooOldError)(nil)) { t.Errorf("expected WriteTooOldError on Put; got %v", err) } // Verify new value was actually written at (3, 1). - value, _, err := MVCCGet(ctx, engine, testKey1, hlc.MaxTimestamp, MVCCGetOptions{Txn: txn}) + value, _, err := MVCCGet(ctx, engine, testKey1, enginepb.MaxTxnTimestamp, MVCCGetOptions{Txn: txn}) if err != nil { t.Fatal(err) } - expTS := hlc.Timestamp{WallTime: 3, Logical: 1} + expTS := enginepb.TxnTimestamp{WallTime: 3, Logical: 1} if value.Timestamp != expTS || !bytes.Equal(value2.RawBytes, value.RawBytes) { t.Fatalf("expected timestamp=%s (got %s), value=%q (got %q)", value.Timestamp, expTS, value2.RawBytes, value.RawBytes) @@ -3048,7 +3048,7 @@ func TestMVCCMultiplePutOldTimestamp(t *testing.T) { t.Error(err) } // Verify new value was actually written at (3, 1). - value, _, err = MVCCGet(ctx, engine, testKey1, hlc.MaxTimestamp, MVCCGetOptions{Txn: txn}) + value, _, err = MVCCGet(ctx, engine, testKey1, enginepb.MaxTxnTimestamp, MVCCGetOptions{Txn: txn}) if err != nil { t.Fatal(err) } @@ -3070,7 +3070,7 @@ func TestMVCCPutNegativeTimestampError(t *testing.T) { engine := engineImpl.create() defer engine.Close() - timestamp := hlc.Timestamp{WallTime: -1} + timestamp := enginepb.TxnTimestamp{WallTime: -1} expectedErrorString := fmt.Sprintf("cannot write to %q at timestamp %s", testKey1, timestamp) err := MVCCPut(ctx, engine, nil, testKey1, timestamp, value1, nil) @@ -3096,7 +3096,7 @@ func TestMVCCPutOldOrigTimestampNewCommitTimestamp(t *testing.T) { engine := engineImpl.create() defer engine.Close() - err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, value1, nil) + err := MVCCPut(ctx, engine, nil, testKey1, enginepb.TxnTimestamp{WallTime: 3}, value1, nil) if err != nil { t.Fatal(err) } @@ -3104,8 +3104,8 @@ func TestMVCCPutOldOrigTimestampNewCommitTimestamp(t *testing.T) { // Perform a transactional Put with a transaction whose original timestamp is // below the existing key's timestamp and whose provisional commit timestamp // is above the existing key's timestamp. - txn := makeTxn(*txn1, hlc.Timestamp{WallTime: 1}) - txn.WriteTimestamp = hlc.Timestamp{WallTime: 5} + txn := makeTxn(*txn1, enginepb.TxnTimestamp{WallTime: 1}) + txn.WriteTimestamp = enginepb.TxnTimestamp{WallTime: 5} txn.Sequence++ err = MVCCPut(ctx, engine, nil, testKey1, txn.ReadTimestamp, value2, txn) @@ -3118,7 +3118,7 @@ func TestMVCCPutOldOrigTimestampNewCommitTimestamp(t *testing.T) { // Verify new value was actually written at the transaction's provisional // commit timestamp. - value, _, err := MVCCGet(ctx, engine, testKey1, hlc.MaxTimestamp, MVCCGetOptions{Txn: txn}) + value, _, err := MVCCGet(ctx, engine, testKey1, enginepb.MaxTxnTimestamp, MVCCGetOptions{Txn: txn}) if err != nil { t.Fatal(err) } @@ -3145,7 +3145,7 @@ func TestMVCCAbortTxn(t *testing.T) { } txn1AbortWithTS := txn1Abort.Clone() - txn1AbortWithTS.WriteTimestamp = hlc.Timestamp{Logical: 1} + txn1AbortWithTS.WriteTimestamp = enginepb.TxnTimestamp{Logical: 1} if _, err := MVCCResolveWriteIntent(ctx, engine, nil, roachpb.MakeLockUpdate(txn1AbortWithTS, roachpb.Span{Key: testKey1}), @@ -3154,7 +3154,7 @@ func TestMVCCAbortTxn(t *testing.T) { } if value, _, err := MVCCGet( - ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, MVCCGetOptions{}, + ctx, engine, testKey1, enginepb.TxnTimestamp{WallTime: 1}, MVCCGetOptions{}, ); err != nil { t.Fatal(err) } else if value != nil { @@ -3179,19 +3179,19 @@ func TestMVCCAbortTxnWithPreviousVersion(t *testing.T) { engine := engineImpl.create() defer engine.Close() - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 1}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, enginepb.TxnTimestamp{Logical: 1}, value1, nil); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, enginepb.TxnTimestamp{WallTime: 1}, value2, nil); err != nil { t.Fatal(err) } - txn1ts := makeTxn(*txn1, hlc.Timestamp{WallTime: 2}) + txn1ts := makeTxn(*txn1, enginepb.TxnTimestamp{WallTime: 2}) if err := MVCCPut(ctx, engine, nil, testKey1, txn1ts.ReadTimestamp, value3, txn1ts); err != nil { t.Fatal(err) } txn1AbortWithTS := txn1Abort.Clone() - txn1AbortWithTS.WriteTimestamp = hlc.Timestamp{WallTime: 2} + txn1AbortWithTS.WriteTimestamp = enginepb.TxnTimestamp{WallTime: 2} if _, err := MVCCResolveWriteIntent(ctx, engine, nil, roachpb.MakeLockUpdate(txn1AbortWithTS, roachpb.Span{Key: testKey1}), @@ -3206,10 +3206,10 @@ func TestMVCCAbortTxnWithPreviousVersion(t *testing.T) { } if value, _, err := MVCCGet( - ctx, engine, testKey1, hlc.Timestamp{WallTime: 3}, MVCCGetOptions{}, + ctx, engine, testKey1, enginepb.TxnTimestamp{WallTime: 3}, MVCCGetOptions{}, ); err != nil { t.Fatal(err) - } else if expTS := (hlc.Timestamp{WallTime: 1}); value.Timestamp != expTS { + } else if expTS := (enginepb.TxnTimestamp{WallTime: 1}); value.Timestamp != expTS { t.Fatalf("expected timestamp %+v == %+v", value.Timestamp, expTS) } else if !bytes.Equal(value2.RawBytes, value.RawBytes) { t.Fatalf("the value %q in get result does not match the value %q in request", @@ -3239,13 +3239,13 @@ func TestMVCCWriteWithDiffTimestampsAndEpochs(t *testing.T) { txne2 := txn txne2.Sequence++ txne2.Epoch = 2 - txne2.WriteTimestamp = hlc.Timestamp{WallTime: 1} + txne2.WriteTimestamp = enginepb.TxnTimestamp{WallTime: 1} if err := MVCCPut(ctx, engine, nil, testKey1, txne2.ReadTimestamp, value2, &txne2); err != nil { t.Fatal(err) } // Try a write with an earlier timestamp; this is just ignored. txne2.Sequence++ - txne2.WriteTimestamp = hlc.Timestamp{WallTime: 1} + txne2.WriteTimestamp = enginepb.TxnTimestamp{WallTime: 1} if err := MVCCPut(ctx, engine, nil, testKey1, txne2.ReadTimestamp, value1, &txne2); err != nil { t.Fatal(err) } @@ -3261,7 +3261,7 @@ func TestMVCCWriteWithDiffTimestampsAndEpochs(t *testing.T) { // Resolve the intent. txne2Commit := txne2 txne2Commit.Status = roachpb.COMMITTED - txne2Commit.WriteTimestamp = hlc.Timestamp{WallTime: 1} + txne2Commit.WriteTimestamp = enginepb.TxnTimestamp{WallTime: 1} if _, err := MVCCResolveWriteIntent(ctx, engine, nil, roachpb.MakeLockUpdate(&txne2Commit, roachpb.Span{Key: testKey1})); err != nil { t.Fatal(err) @@ -3270,7 +3270,7 @@ func TestMVCCWriteWithDiffTimestampsAndEpochs(t *testing.T) { expTS := txne2Commit.WriteTimestamp.Add(0, 1) // Now try writing an earlier value without a txn--should get WriteTooOldError. - err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 1}, value4, nil) + err := MVCCPut(ctx, engine, nil, testKey1, enginepb.TxnTimestamp{Logical: 1}, value4, nil) if wtoErr := (*roachpb.WriteTooOldError)(nil); !errors.As(err, &wtoErr) { t.Fatal("unexpected success") } else if wtoErr.ActualTimestamp != expTS { @@ -3297,16 +3297,16 @@ func TestMVCCWriteWithDiffTimestampsAndEpochs(t *testing.T) { err, value.Timestamp, intentTS, value5.RawBytes, value.RawBytes) } // Attempt to read older timestamp; should fail. - value, _, err = MVCCGet(ctx, engine, testKey1, hlc.Timestamp{Logical: 0}, MVCCGetOptions{}) + value, _, err = MVCCGet(ctx, engine, testKey1, enginepb.TxnTimestamp{Logical: 0}, MVCCGetOptions{}) if value != nil || err != nil { t.Fatalf("expected value nil, err nil; got %+v, %v", value, err) } // Read at correct timestamp. - value, _, err = MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, MVCCGetOptions{}) + value, _, err = MVCCGet(ctx, engine, testKey1, enginepb.TxnTimestamp{WallTime: 1}, MVCCGetOptions{}) if err != nil { t.Fatal(err) } - if expTS := (hlc.Timestamp{WallTime: 1}); value.Timestamp != expTS { + if expTS := (enginepb.TxnTimestamp{WallTime: 1}); value.Timestamp != expTS { t.Fatalf("expected timestamp %+v == %+v", value.Timestamp, expTS) } if !bytes.Equal(value3.RawBytes, value.RawBytes) { @@ -3331,11 +3331,11 @@ func TestMVCCGetWithDiffEpochs(t *testing.T) { defer engine.Close() // Write initial value without a txn. - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 1}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, enginepb.TxnTimestamp{Logical: 1}, value1, nil); err != nil { t.Fatal(err) } // Now write using txn1, epoch 1. - txn1ts := makeTxn(*txn1, hlc.Timestamp{WallTime: 1}) + txn1ts := makeTxn(*txn1, enginepb.TxnTimestamp{WallTime: 1}) if err := MVCCPut(ctx, engine, nil, testKey1, txn1ts.ReadTimestamp, value2, txn1ts); err != nil { t.Fatal(err) } @@ -3356,7 +3356,7 @@ func TestMVCCGetWithDiffEpochs(t *testing.T) { } for i, test := range testCases { t.Run(strconv.Itoa(i), func(t *testing.T) { - value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 2}, MVCCGetOptions{ + value, _, err := MVCCGet(ctx, engine, testKey1, enginepb.TxnTimestamp{WallTime: 2}, MVCCGetOptions{ Txn: test.txn, }) if test.expErr { @@ -3392,17 +3392,17 @@ func TestMVCCGetWithDiffEpochsAndTimestamps(t *testing.T) { defer engine.Close() // Write initial value without a txn at timestamp 1. - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 1}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, enginepb.TxnTimestamp{WallTime: 1}, value1, nil); err != nil { t.Fatal(err) } // Write another value without a txn at timestamp 3. - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{WallTime: 3}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, enginepb.TxnTimestamp{WallTime: 3}, value2, nil); err != nil { t.Fatal(err) } // Now write using txn1, epoch 1. - txn1ts := makeTxn(*txn1, hlc.Timestamp{WallTime: 1}) + txn1ts := makeTxn(*txn1, enginepb.TxnTimestamp{WallTime: 1}) // Bump epoch 1's write timestamp to timestamp 4. - txn1ts.WriteTimestamp = hlc.Timestamp{WallTime: 4} + txn1ts.WriteTimestamp = enginepb.TxnTimestamp{WallTime: 4} // Expected to hit WriteTooOld error but to still lay down intent. err := MVCCPut(ctx, engine, nil, testKey1, txn1ts.ReadTimestamp, value3, txn1ts) if wtoErr := (*roachpb.WriteTooOldError)(nil); !errors.As(err, &wtoErr) { @@ -3413,29 +3413,29 @@ func TestMVCCGetWithDiffEpochsAndTimestamps(t *testing.T) { // Try reading using different epochs & timestamps. testCases := []struct { txn *roachpb.Transaction - readTS hlc.Timestamp + readTS enginepb.TxnTimestamp expValue *roachpb.Value }{ // Epoch 1, read 1; should see new value3. - {txn1, hlc.Timestamp{WallTime: 1}, &value3}, + {txn1, enginepb.TxnTimestamp{WallTime: 1}, &value3}, // Epoch 1, read 2; should see new value3. - {txn1, hlc.Timestamp{WallTime: 2}, &value3}, + {txn1, enginepb.TxnTimestamp{WallTime: 2}, &value3}, // Epoch 1, read 3; should see new value3. - {txn1, hlc.Timestamp{WallTime: 3}, &value3}, + {txn1, enginepb.TxnTimestamp{WallTime: 3}, &value3}, // Epoch 1, read 4; should see new value3. - {txn1, hlc.Timestamp{WallTime: 4}, &value3}, + {txn1, enginepb.TxnTimestamp{WallTime: 4}, &value3}, // Epoch 1, read 5; should see new value3. - {txn1, hlc.Timestamp{WallTime: 5}, &value3}, + {txn1, enginepb.TxnTimestamp{WallTime: 5}, &value3}, // Epoch 2, read 1; should see committed value1. - {txn1e2, hlc.Timestamp{WallTime: 1}, &value1}, + {txn1e2, enginepb.TxnTimestamp{WallTime: 1}, &value1}, // Epoch 2, read 2; should see committed value1. - {txn1e2, hlc.Timestamp{WallTime: 2}, &value1}, + {txn1e2, enginepb.TxnTimestamp{WallTime: 2}, &value1}, // Epoch 2, read 3; should see committed value2. - {txn1e2, hlc.Timestamp{WallTime: 3}, &value2}, + {txn1e2, enginepb.TxnTimestamp{WallTime: 3}, &value2}, // Epoch 2, read 4; should see committed value2. - {txn1e2, hlc.Timestamp{WallTime: 4}, &value2}, + {txn1e2, enginepb.TxnTimestamp{WallTime: 4}, &value2}, // Epoch 2, read 5; should see committed value2. - {txn1e2, hlc.Timestamp{WallTime: 5}, &value2}, + {txn1e2, enginepb.TxnTimestamp{WallTime: 5}, &value2}, } for i, test := range testCases { t.Run(strconv.Itoa(i), func(t *testing.T) { @@ -3464,7 +3464,7 @@ func TestMVCCGetWithOldEpoch(t *testing.T) { if err := MVCCPut(ctx, engine, nil, testKey1, txn1e2.ReadTimestamp, value2, txn1e2); err != nil { t.Fatal(err) } - _, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 2}, MVCCGetOptions{ + _, _, err := MVCCGet(ctx, engine, testKey1, enginepb.TxnTimestamp{WallTime: 2}, MVCCGetOptions{ Txn: txn1, }) if err == nil { @@ -3571,13 +3571,13 @@ func TestMVCCGetWithPushedTimestamp(t *testing.T) { t.Fatal(err) } // Resolve the intent, pushing its timestamp forward. - txn := makeTxn(*txn1, hlc.Timestamp{WallTime: 1}) + txn := makeTxn(*txn1, enginepb.TxnTimestamp{WallTime: 1}) if _, err := MVCCResolveWriteIntent(ctx, engine, nil, roachpb.MakeLockUpdate(txn, roachpb.Span{Key: testKey1})); err != nil { t.Fatal(err) } // Attempt to read using naive txn's previous timestamp. - value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{Logical: 1}, MVCCGetOptions{ + value, _, err := MVCCGet(ctx, engine, testKey1, enginepb.TxnTimestamp{Logical: 1}, MVCCGetOptions{ Txn: txn1, }) if err != nil || value == nil || !bytes.Equal(value.RawBytes, value1.RawBytes) { @@ -3614,13 +3614,13 @@ func TestMVCCResolveWithDiffEpochs(t *testing.T) { // Verify key1 is empty, as resolution with epoch 2 would have // aborted the epoch 1 intent. - value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{Logical: 1}, MVCCGetOptions{}) + value, _, err := MVCCGet(ctx, engine, testKey1, enginepb.TxnTimestamp{Logical: 1}, MVCCGetOptions{}) if value != nil || err != nil { t.Errorf("expected value nil, err nil; got %+v, %v", value, err) } // Key2 should be committed. - value, _, err = MVCCGet(ctx, engine, testKey2, hlc.Timestamp{Logical: 1}, MVCCGetOptions{}) + value, _, err = MVCCGet(ctx, engine, testKey2, enginepb.TxnTimestamp{Logical: 1}, MVCCGetOptions{}) if err != nil { t.Fatal(err) } @@ -3646,7 +3646,7 @@ func TestMVCCResolveWithUpdatedTimestamp(t *testing.T) { t.Fatal(err) } - value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, MVCCGetOptions{ + value, _, err := MVCCGet(ctx, engine, testKey1, enginepb.TxnTimestamp{WallTime: 1}, MVCCGetOptions{ Txn: txn1, }) if err != nil { @@ -3659,22 +3659,22 @@ func TestMVCCResolveWithUpdatedTimestamp(t *testing.T) { // Resolve with a higher commit timestamp -- this should rewrite the // intent when making it permanent. - txn := makeTxn(*txn1Commit, hlc.Timestamp{WallTime: 1}) + txn := makeTxn(*txn1Commit, enginepb.TxnTimestamp{WallTime: 1}) if _, err = MVCCResolveWriteIntent(ctx, engine, nil, roachpb.MakeLockUpdate(txn, roachpb.Span{Key: testKey1})); err != nil { t.Fatal(err) } - value, _, err = MVCCGet(ctx, engine, testKey1, hlc.Timestamp{Logical: 1}, MVCCGetOptions{}) + value, _, err = MVCCGet(ctx, engine, testKey1, enginepb.TxnTimestamp{Logical: 1}, MVCCGetOptions{}) if value != nil || err != nil { t.Fatalf("expected both value and err to be nil: %+v, %v", value, err) } - value, _, err = MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, MVCCGetOptions{}) + value, _, err = MVCCGet(ctx, engine, testKey1, enginepb.TxnTimestamp{WallTime: 1}, MVCCGetOptions{}) if err != nil { t.Error(err) } - if expTS := (hlc.Timestamp{WallTime: 1}); value.Timestamp != expTS { + if expTS := (enginepb.TxnTimestamp{WallTime: 1}); value.Timestamp != expTS { t.Fatalf("expected timestamp %+v == %+v", value.Timestamp, expTS) } if !bytes.Equal(value1.RawBytes, value.RawBytes) { @@ -3698,7 +3698,7 @@ func TestMVCCResolveWithPushedTimestamp(t *testing.T) { if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, value1, txn1); err != nil { t.Fatal(err) } - value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, MVCCGetOptions{ + value, _, err := MVCCGet(ctx, engine, testKey1, enginepb.TxnTimestamp{WallTime: 1}, MVCCGetOptions{ Txn: txn1, }) if err != nil { @@ -3711,25 +3711,25 @@ func TestMVCCResolveWithPushedTimestamp(t *testing.T) { // Resolve with a higher commit timestamp, but with still-pending transaction. // This represents a straightforward push (i.e. from a read/write conflict). - txn := makeTxn(*txn1, hlc.Timestamp{WallTime: 1}) + txn := makeTxn(*txn1, enginepb.TxnTimestamp{WallTime: 1}) if _, err = MVCCResolveWriteIntent(ctx, engine, nil, roachpb.MakeLockUpdate(txn, roachpb.Span{Key: testKey1})); err != nil { t.Fatal(err) } - value, _, err = MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, MVCCGetOptions{}) + value, _, err = MVCCGet(ctx, engine, testKey1, enginepb.TxnTimestamp{WallTime: 1}, MVCCGetOptions{}) if value != nil || err == nil { t.Fatalf("expected both value nil and err to be a writeIntentError: %+v", value) } // Can still fetch the value using txn1. - value, _, err = MVCCGet(ctx, engine, testKey1, hlc.Timestamp{WallTime: 1}, MVCCGetOptions{ + value, _, err = MVCCGet(ctx, engine, testKey1, enginepb.TxnTimestamp{WallTime: 1}, MVCCGetOptions{ Txn: txn1, }) if err != nil { t.Error(err) } - if expTS := (hlc.Timestamp{WallTime: 1}); value.Timestamp != expTS { + if expTS := (enginepb.TxnTimestamp{WallTime: 1}); value.Timestamp != expTS { t.Fatalf("expected timestamp %+v == %+v", value.Timestamp, expTS) } if !bytes.Equal(value1.RawBytes, value.RawBytes) { @@ -3757,7 +3757,7 @@ func TestMVCCResolveTxnNoOps(t *testing.T) { } // Add key and resolve despite there being no intent. - if err := MVCCPut(ctx, engine, nil, testKey1, hlc.Timestamp{Logical: 1}, value1, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey1, enginepb.TxnTimestamp{Logical: 1}, value1, nil); err != nil { t.Fatal(err) } if _, err := MVCCResolveWriteIntent(ctx, engine, nil, @@ -3771,7 +3771,7 @@ func TestMVCCResolveTxnNoOps(t *testing.T) { } txn1CommitWithTS := txn2Commit.Clone() - txn1CommitWithTS.WriteTimestamp = hlc.Timestamp{WallTime: 1} + txn1CommitWithTS.WriteTimestamp = enginepb.TxnTimestamp{WallTime: 1} if _, err := MVCCResolveWriteIntent(ctx, engine, nil, roachpb.MakeLockUpdate(txn1CommitWithTS, roachpb.Span{Key: testKey2})); err != nil { t.Fatal(err) @@ -3793,7 +3793,7 @@ func TestMVCCResolveTxnRange(t *testing.T) { if err := MVCCPut(ctx, engine, nil, testKey1, txn1.ReadTimestamp, value1, txn1); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, testKey2, hlc.Timestamp{Logical: 1}, value2, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, testKey2, enginepb.TxnTimestamp{Logical: 1}, value2, nil); err != nil { t.Fatal(err) } if err := MVCCPut(ctx, engine, nil, testKey3, txn2.ReadTimestamp, value3, txn2); err != nil { @@ -3815,7 +3815,7 @@ func TestMVCCResolveTxnRange(t *testing.T) { } { - value, _, err := MVCCGet(ctx, engine, testKey1, hlc.Timestamp{Logical: 1}, MVCCGetOptions{}) + value, _, err := MVCCGet(ctx, engine, testKey1, enginepb.TxnTimestamp{Logical: 1}, MVCCGetOptions{}) if err != nil { t.Fatal(err) } @@ -3825,7 +3825,7 @@ func TestMVCCResolveTxnRange(t *testing.T) { } } { - value, _, err := MVCCGet(ctx, engine, testKey2, hlc.Timestamp{Logical: 1}, MVCCGetOptions{}) + value, _, err := MVCCGet(ctx, engine, testKey2, enginepb.TxnTimestamp{Logical: 1}, MVCCGetOptions{}) if err != nil { t.Fatal(err) } @@ -3835,7 +3835,7 @@ func TestMVCCResolveTxnRange(t *testing.T) { } } { - value, _, err := MVCCGet(ctx, engine, testKey3, hlc.Timestamp{Logical: 1}, MVCCGetOptions{ + value, _, err := MVCCGet(ctx, engine, testKey3, enginepb.TxnTimestamp{Logical: 1}, MVCCGetOptions{ Txn: txn2, }) if err != nil { @@ -3847,7 +3847,7 @@ func TestMVCCResolveTxnRange(t *testing.T) { } } { - value, _, err := MVCCGet(ctx, engine, testKey4, hlc.Timestamp{Logical: 1}, MVCCGetOptions{}) + value, _, err := MVCCGet(ctx, engine, testKey4, enginepb.TxnTimestamp{Logical: 1}, MVCCGetOptions{}) if err != nil { t.Fatal(err) } @@ -3878,11 +3878,11 @@ func TestMVCCResolveTxnRangeResume(t *testing.T) { if err := MVCCPut(ctx, engine, nil, key0, txn1.ReadTimestamp, value1, txn1); err != nil { t.Fatal(err) } - txn2ts := makeTxn(*txn2, hlc.Timestamp{Logical: 2}) + txn2ts := makeTxn(*txn2, enginepb.TxnTimestamp{Logical: 2}) if err := MVCCPut(ctx, engine, nil, key1, txn2ts.ReadTimestamp, value2, txn2ts); err != nil { t.Fatal(err) } - if err := MVCCPut(ctx, engine, nil, key2, hlc.Timestamp{Logical: 3}, value3, nil); err != nil { + if err := MVCCPut(ctx, engine, nil, key2, enginepb.TxnTimestamp{Logical: 3}, value3, nil); err != nil { t.Fatal(err) } } @@ -3960,7 +3960,7 @@ func TestFindSplitKey(t *testing.T) { v := strings.Repeat("X", 10-len(k)) val := roachpb.MakeValueFromString(v) // Write the key and value through MVCC - if err := MVCCPut(ctx, engine, ms, []byte(k), hlc.Timestamp{Logical: 1}, val, nil); err != nil { + if err := MVCCPut(ctx, engine, ms, []byte(k), enginepb.TxnTimestamp{Logical: 1}, val, nil); err != nil { t.Fatal(err) } } @@ -4348,7 +4348,7 @@ func TestFindValidSplitKeys(t *testing.T) { // Add three MVCC versions of every key. Splits are not allowed // between MVCC versions, so this shouldn't have any effect. for j := 1; j <= 3; j++ { - ts := hlc.Timestamp{Logical: int32(j)} + ts := enginepb.TxnTimestamp{Logical: int32(j)} if err := MVCCPut(ctx, engine, ms, []byte(k), ts, val, nil); err != nil { t.Fatal(err) } @@ -4452,7 +4452,7 @@ func TestFindBalancedSplitKeys(t *testing.T) { expKey = key } val := roachpb.MakeValueFromString(strings.Repeat("X", test.valSizes[j])) - if err := MVCCPut(ctx, engine, ms, key, hlc.Timestamp{Logical: 1}, val, nil); err != nil { + if err := MVCCPut(ctx, engine, ms, key, enginepb.TxnTimestamp{Logical: 1}, val, nil); err != nil { t.Fatal(err) } } @@ -4486,13 +4486,13 @@ func TestMVCCGarbageCollect(t *testing.T) { ms := &enginepb.MVCCStats{} bytes := []byte("value") - ts1 := hlc.Timestamp{WallTime: 1e9} - ts2 := hlc.Timestamp{WallTime: 2e9} - ts3 := hlc.Timestamp{WallTime: 3e9} + ts1 := enginepb.TxnTimestamp{WallTime: 1e9} + ts2 := enginepb.TxnTimestamp{WallTime: 2e9} + ts3 := enginepb.TxnTimestamp{WallTime: 3e9} val1 := roachpb.MakeValueFromBytesAndTimestamp(bytes, ts1) val2 := roachpb.MakeValueFromBytesAndTimestamp(bytes, ts2) val3 := roachpb.MakeValueFromBytesAndTimestamp(bytes, ts3) - valInline := roachpb.MakeValueFromBytesAndTimestamp(bytes, hlc.Timestamp{}) + valInline := roachpb.MakeValueFromBytesAndTimestamp(bytes, enginepb.TxnTimestamp{}) testData := []struct { key roachpb.Key @@ -4519,7 +4519,7 @@ func TestMVCCGarbageCollect(t *testing.T) { continue } valCpy := *protoutil.Clone(&val).(*roachpb.Value) - valCpy.Timestamp = hlc.Timestamp{} + valCpy.Timestamp = enginepb.TxnTimestamp{} if err := MVCCPut(ctx, engine, ms, test.key, val.Timestamp, valCpy, nil); err != nil { t.Fatal(err) } @@ -4541,10 +4541,10 @@ func TestMVCCGarbageCollect(t *testing.T) { {Key: roachpb.Key("a-del"), Timestamp: ts2}, {Key: roachpb.Key("b"), Timestamp: ts1}, {Key: roachpb.Key("b-del"), Timestamp: ts2}, - {Key: roachpb.Key("inline"), Timestamp: hlc.Timestamp{}}, + {Key: roachpb.Key("inline"), Timestamp: enginepb.TxnTimestamp{}}, // Keys that don't exist, which should result in a no-op. {Key: roachpb.Key("a-bad"), Timestamp: ts2}, - {Key: roachpb.Key("inline-bad"), Timestamp: hlc.Timestamp{}}, + {Key: roachpb.Key("inline-bad"), Timestamp: enginepb.TxnTimestamp{}}, } if err := MVCCGarbageCollect( context.Background(), engine, ms, keys, ts3, @@ -4600,11 +4600,11 @@ func TestMVCCGarbageCollectNonDeleted(t *testing.T) { defer engine.Close() s := "string" - ts1 := hlc.Timestamp{WallTime: 1e9} - ts2 := hlc.Timestamp{WallTime: 2e9} + ts1 := enginepb.TxnTimestamp{WallTime: 1e9} + ts2 := enginepb.TxnTimestamp{WallTime: 2e9} val1 := mkVal(s, ts1) val2 := mkVal(s, ts2) - valInline := mkVal(s, hlc.Timestamp{}) + valInline := mkVal(s, enginepb.TxnTimestamp{}) testData := []struct { key roachpb.Key @@ -4618,7 +4618,7 @@ func TestMVCCGarbageCollectNonDeleted(t *testing.T) { for _, test := range testData { for _, val := range test.vals { valCpy := *protoutil.Clone(&val).(*roachpb.Value) - valCpy.Timestamp = hlc.Timestamp{} + valCpy.Timestamp = enginepb.TxnTimestamp{} if err := MVCCPut(ctx, engine, nil, test.key, val.Timestamp, valCpy, nil); err != nil { t.Fatal(err) } @@ -4648,8 +4648,8 @@ func TestMVCCGarbageCollectIntent(t *testing.T) { defer engine.Close() bytes := []byte("value") - ts1 := hlc.Timestamp{WallTime: 1e9} - ts2 := hlc.Timestamp{WallTime: 2e9} + ts1 := enginepb.TxnTimestamp{WallTime: 1e9} + ts2 := enginepb.TxnTimestamp{WallTime: 2e9} key := roachpb.Key("a") { val1 := roachpb.MakeValueFromBytes(bytes) @@ -4717,8 +4717,8 @@ func TestMVCCGarbageCollectUsesSeekLTAppropriately(t *testing.T) { keys []testCaseKey } bytes := []byte("value") - toHLC := func(seconds int) hlc.Timestamp { - return hlc.Timestamp{WallTime: (time.Duration(seconds) * time.Second).Nanoseconds()} + toHLC := func(seconds int) enginepb.TxnTimestamp { + return enginepb.TxnTimestamp{WallTime: (time.Duration(seconds) * time.Second).Nanoseconds()} } engineBatchIteratorSupportsPrev := func(engine Engine) bool { batch := engine.NewBatch() @@ -4921,12 +4921,12 @@ func TestTimeSeriesMVCCStats(t *testing.T) { // Perform a sequence of merges on the same key // and record the MVCC stats for it. - if err := MVCCMerge(ctx, engine, &ms, testKey1, hlc.Timestamp{Logical: 1}, tsvalue1); err != nil { + if err := MVCCMerge(ctx, engine, &ms, testKey1, enginepb.TxnTimestamp{Logical: 1}, tsvalue1); err != nil { t.Fatal(err) } firstMS := ms - if err := MVCCMerge(ctx, engine, &ms, testKey1, hlc.Timestamp{Logical: 1}, tsvalue1); err != nil { + if err := MVCCMerge(ctx, engine, &ms, testKey1, enginepb.TxnTimestamp{Logical: 1}, tsvalue1); err != nil { t.Fatal(err) } secondMS := ms @@ -4964,10 +4964,10 @@ func TestMVCCTimeSeriesPartialMerge(t *testing.T) { vals := make([]*roachpb.Value, 2) for i, k := range []roachpb.Key{testKey1, testKey2} { - if err := MVCCMerge(ctx, engine, nil, k, hlc.Timestamp{Logical: 1}, tsvalue1); err != nil { + if err := MVCCMerge(ctx, engine, nil, k, enginepb.TxnTimestamp{Logical: 1}, tsvalue1); err != nil { t.Fatal(err) } - if err := MVCCMerge(ctx, engine, nil, k, hlc.Timestamp{Logical: 2}, tsvalue2); err != nil { + if err := MVCCMerge(ctx, engine, nil, k, enginepb.TxnTimestamp{Logical: 2}, tsvalue2); err != nil { t.Fatal(err) } @@ -4977,10 +4977,10 @@ func TestMVCCTimeSeriesPartialMerge(t *testing.T) { } } - if err := MVCCMerge(ctx, engine, nil, k, hlc.Timestamp{Logical: 2}, tsvalue2); err != nil { + if err := MVCCMerge(ctx, engine, nil, k, enginepb.TxnTimestamp{Logical: 2}, tsvalue2); err != nil { t.Fatal(err) } - if err := MVCCMerge(ctx, engine, nil, k, hlc.Timestamp{Logical: 1}, tsvalue1); err != nil { + if err := MVCCMerge(ctx, engine, nil, k, enginepb.TxnTimestamp{Logical: 1}, tsvalue1); err != nil { t.Fatal(err) } @@ -4990,7 +4990,7 @@ func TestMVCCTimeSeriesPartialMerge(t *testing.T) { } } - if v, _, err := MVCCGet(ctx, engine, k, hlc.Timestamp{}, MVCCGetOptions{}); err != nil { + if v, _, err := MVCCGet(ctx, engine, k, enginepb.TxnTimestamp{}, MVCCGetOptions{}); err != nil { t.Fatal(err) } else { vals[i] = v diff --git a/pkg/storage/pebble.go b/pkg/storage/pebble.go index 1ca4e3e99e72..107763b9618d 100644 --- a/pkg/storage/pebble.go +++ b/pkg/storage/pebble.go @@ -30,7 +30,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/storage/fs" "github.com/cockroachdb/cockroach/pkg/util/envutil" - "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/uuid" @@ -263,7 +262,7 @@ func (t *pebbleTimeBoundPropCollector) Finish(userProps map[string]string) error return nil //nolint:returnerrcheck } if meta.Txn != nil { - ts := encodeTimestamp(meta.Timestamp.ToTimestamp()) + ts := encodeTimestamp(meta.TxnTimestamp()) t.updateBounds(ts) } } @@ -649,7 +648,7 @@ func (p *Pebble) Closed() bool { // ExportMVCCToSst is part of the engine.Reader interface. func (p *Pebble) ExportMVCCToSst( startKey, endKey roachpb.Key, - startTS, endTS hlc.Timestamp, + startTS, endTS enginepb.TxnTimestamp, exportAllRevisions bool, targetSize, maxSize uint64, io IterOptions, @@ -1091,8 +1090,8 @@ func (p *Pebble) Compact() error { // CompactRange implements the Engine interface. func (p *Pebble) CompactRange(start, end roachpb.Key, forceBottommost bool) error { - bufStart := EncodeKey(MVCCKey{start, hlc.Timestamp{}}) - bufEnd := EncodeKey(MVCCKey{end, hlc.Timestamp{}}) + bufStart := EncodeKey(MVCCKey{start, enginepb.TxnTimestamp{}}) + bufEnd := EncodeKey(MVCCKey{end, enginepb.TxnTimestamp{}}) return p.db.Compact(bufStart, bufEnd) } @@ -1234,7 +1233,7 @@ func (p *pebbleReadOnly) Closed() bool { // ExportMVCCToSst is part of the engine.Reader interface. func (p *pebbleReadOnly) ExportMVCCToSst( startKey, endKey roachpb.Key, - startTS, endTS hlc.Timestamp, + startTS, endTS enginepb.TxnTimestamp, exportAllRevisions bool, targetSize, maxSize uint64, io IterOptions, @@ -1439,7 +1438,7 @@ func (p *pebbleSnapshot) Closed() bool { // ExportMVCCToSst is part of the engine.Reader interface. func (p *pebbleSnapshot) ExportMVCCToSst( startKey, endKey roachpb.Key, - startTS, endTS hlc.Timestamp, + startTS, endTS enginepb.TxnTimestamp, exportAllRevisions bool, targetSize, maxSize uint64, io IterOptions, @@ -1525,7 +1524,7 @@ func pebbleGetProto( func pebbleExportToSst( reader Reader, startKey, endKey roachpb.Key, - startTS, endTS hlc.Timestamp, + startTS, endTS enginepb.TxnTimestamp, exportAllRevisions bool, targetSize, maxSize uint64, io IterOptions, diff --git a/pkg/storage/pebble_batch.go b/pkg/storage/pebble_batch.go index 7cc5af60de4f..91cb76ee38a6 100644 --- a/pkg/storage/pebble_batch.go +++ b/pkg/storage/pebble_batch.go @@ -14,7 +14,7 @@ import ( "sync" "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" @@ -120,7 +120,7 @@ func (p *pebbleBatch) Closed() bool { // ExportMVCCToSst is part of the engine.Reader interface. func (p *pebbleBatch) ExportMVCCToSst( startKey, endKey roachpb.Key, - startTS, endTS hlc.Timestamp, + startTS, endTS enginepb.TxnTimestamp, exportAllRevisions bool, targetSize, maxSize uint64, io IterOptions, diff --git a/pkg/storage/pebble_mvcc_scanner.go b/pkg/storage/pebble_mvcc_scanner.go index 683f27d24e77..d232a12fd273 100644 --- a/pkg/storage/pebble_mvcc_scanner.go +++ b/pkg/storage/pebble_mvcc_scanner.go @@ -18,7 +18,6 @@ import ( "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/protoutil" "github.com/cockroachdb/errors" "github.com/cockroachdb/pebble" @@ -96,7 +95,7 @@ type pebbleMVCCScanner struct { // Iteration bounds. Does not contain MVCC timestamp. start, end roachpb.Key // Timestamp with which MVCCScan/MVCCGet was called. - ts hlc.Timestamp + ts enginepb.TxnTimestamp // Max number of keys to return. Note that targetBytes below is implemented // by mutating maxKeys. (In particular, one must not assume that if maxKeys // is zero initially it will always be zero). @@ -131,7 +130,7 @@ type pebbleMVCCScanner struct { // above the scan timestamp. Only applicable if failOnMoreRecent is true. If // set and no other error is hit, a WriteToOld error will be returned from // the scan. - mostRecentTS hlc.Timestamp + mostRecentTS enginepb.TxnTimestamp // Stores any error returned. If non-nil, iteration short circuits. err error // Number of iterations to try before we do a Seek/SeekReverse. Stays within @@ -275,7 +274,7 @@ func (p *pebbleMVCCScanner) maybeFailOnMoreRecent() { } // Returns an uncertainty error with the specified timestamp and p.txn. -func (p *pebbleMVCCScanner) uncertaintyError(ts hlc.Timestamp) bool { +func (p *pebbleMVCCScanner) uncertaintyError(ts enginepb.TxnTimestamp) bool { p.err = roachpb.NewReadWithinUncertaintyIntervalError(p.ts, ts, p.txn) p.results.clear() p.intents.Reset() @@ -360,7 +359,7 @@ func (p *pebbleMVCCScanner) getAndAdvance() bool { p.err = errors.Errorf("intent without transaction") return false } - metaTS := p.meta.Timestamp.ToTimestamp() + metaTS := p.meta.TxnTimestamp() // metaTS is the timestamp of an intent value, which we may or may // not end up ignoring, depending on factors codified below. If we do ignore @@ -624,7 +623,7 @@ func (p *pebbleMVCCScanner) addAndAdvance(rawKey []byte, val []byte) bool { // Seeks to the latest revision of the current key that's still less than or // equal to the specified timestamp, adds it to the result set, then moves onto // the next user key. -func (p *pebbleMVCCScanner) seekVersion(ts hlc.Timestamp, uncertaintyCheck bool) bool { +func (p *pebbleMVCCScanner) seekVersion(ts enginepb.TxnTimestamp, uncertaintyCheck bool) bool { key := MVCCKey{Key: p.curKey.Key, Timestamp: ts} p.keyBuf = EncodeKeyToBuf(p.keyBuf[:0], key) origKey := p.keyBuf[:len(p.curKey.Key)] diff --git a/pkg/storage/pebble_test.go b/pkg/storage/pebble_test.go index e153d9e59601..6b38e9c7149c 100644 --- a/pkg/storage/pebble_test.go +++ b/pkg/storage/pebble_test.go @@ -25,7 +25,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "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/log" "github.com/cockroachdb/cockroach/pkg/util/protoutil" @@ -62,7 +61,7 @@ func TestPebbleTimeBoundPropCollector(t *testing.T) { ikey := pebble.InternalKey{ UserKey: EncodeKey(MVCCKey{ Key: key, - Timestamp: hlc.Timestamp{WallTime: int64(timestamp)}, + Timestamp: enginepb.TxnTimestamp{WallTime: int64(timestamp)}, }), } @@ -123,7 +122,7 @@ func TestPebbleIterReuse(t *testing.T) { batch := eng.NewBatch() for i := 0; i < 100; i++ { - key := MVCCKey{[]byte{byte(i)}, hlc.Timestamp{WallTime: 100}} + key := MVCCKey{[]byte{byte(i)}, enginepb.TxnTimestamp{WallTime: 100}} if err := batch.PutMVCC(key, []byte("foo")); err != nil { t.Fatal(err) } @@ -213,14 +212,14 @@ func TestPebbleSeparatorSuccessor(t *testing.T) { {makeMVCCKey("1\xfd\xff"), makeMVCCKey("1\xff"), makeMVCCKey("1\xfe")}, {MVCCKey{ Key: []byte("1\xff\xff"), - Timestamp: hlc.Timestamp{WallTime: 20, Logical: 3}, + Timestamp: enginepb.TxnTimestamp{WallTime: 20, Logical: 3}, }, makeMVCCKey("9"), makeMVCCKey("2")}, {MVCCKey{ Key: []byte("1\xff\xff"), - Timestamp: hlc.Timestamp{WallTime: 20, Logical: 3}, + Timestamp: enginepb.TxnTimestamp{WallTime: 20, Logical: 3}, }, makeMVCCKey("19"), MVCCKey{ Key: []byte("1\xff\xff"), - Timestamp: hlc.Timestamp{WallTime: 20, Logical: 3}, + Timestamp: enginepb.TxnTimestamp{WallTime: 20, Logical: 3}, }, }, // Empty b values. @@ -254,7 +253,7 @@ func TestPebbleSeparatorSuccessor(t *testing.T) { {makeMVCCKey("13\xff"), makeMVCCKey("2")}, {MVCCKey{ Key: []byte("1\xff\xff"), - Timestamp: hlc.Timestamp{WallTime: 20, Logical: 3}, + Timestamp: enginepb.TxnTimestamp{WallTime: 20, Logical: 3}, }, makeMVCCKey("2")}, {makeMVCCKey("\xff"), makeMVCCKey("\xff")}, {makeMVCCKey("\xff\xff"), makeMVCCKey("\xff\xff")}, @@ -262,10 +261,10 @@ func TestPebbleSeparatorSuccessor(t *testing.T) { {makeMVCCKey("\xfe\xff\xff"), makeMVCCKey("\xff")}, {MVCCKey{ Key: []byte("\xff\xff"), - Timestamp: hlc.Timestamp{WallTime: 20, Logical: 3}, + Timestamp: enginepb.TxnTimestamp{WallTime: 20, Logical: 3}, }, MVCCKey{ Key: []byte("\xff\xff"), - Timestamp: hlc.Timestamp{WallTime: 20, Logical: 3}, + Timestamp: enginepb.TxnTimestamp{WallTime: 20, Logical: 3}, }}, } for _, tc := range succCases { @@ -304,7 +303,7 @@ func BenchmarkMVCCKeyCompare(b *testing.B) { for i := range keys { k := MVCCKey{ Key: randutil.RandBytes(rng, 8), - Timestamp: hlc.Timestamp{ + Timestamp: enginepb.TxnTimestamp{ WallTime: int64(rng.Intn(5)), }, } diff --git a/pkg/storage/sst_iterator_test.go b/pkg/storage/sst_iterator_test.go index f202c7e97436..a8669bf6dad2 100644 --- a/pkg/storage/sst_iterator_test.go +++ b/pkg/storage/sst_iterator_test.go @@ -16,8 +16,8 @@ import ( "reflect" "testing" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/testutils" - "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" ) @@ -82,7 +82,7 @@ func TestSSTIterator(t *testing.T) { kv := MVCCKeyValue{ Key: MVCCKey{ Key: []byte{'A' + byte(i)}, - Timestamp: hlc.Timestamp{WallTime: int64(i)}, + Timestamp: enginepb.TxnTimestamp{WallTime: int64(i)}, }, Value: []byte{'a' + byte(i)}, } @@ -131,15 +131,15 @@ func TestCockroachComparer(t *testing.T) { } keyA2 := MVCCKey{ Key: []byte("a"), - Timestamp: hlc.Timestamp{WallTime: 2}, + Timestamp: enginepb.TxnTimestamp{WallTime: 2}, } keyA1 := MVCCKey{ Key: []byte("a"), - Timestamp: hlc.Timestamp{WallTime: 1}, + Timestamp: enginepb.TxnTimestamp{WallTime: 1}, } keyB2 := MVCCKey{ Key: []byte("b"), - Timestamp: hlc.Timestamp{WallTime: 2}, + Timestamp: enginepb.TxnTimestamp{WallTime: 2}, } if x := EngineComparer.Compare(EncodeKey(keyAMetadata), EncodeKey(keyA1)); x != -1 {