Skip to content

Commit

Permalink
[DNM] enginepb: strongly type enginepb.TxnTimestamp, split from hlc.T…
Browse files Browse the repository at this point in the history
…imestamp

This commit splits off a new enginepb.TxnTimestamp type (see `pkg/storage/enginepb/mvcc.go`)
from the existing hlc.Timestamp type through a type alias. While the two types
share the same memory and proto representation, they have different purposes and
properties. hlc.Timestamp continues to serve in its original role — representing
a real timestamp pulled from a clock on one of the nodes in the system.
enginepb.TxnTimestamp is introduced to serve in hlc.Timestamp's current
secondary role – representing MVCC time and the time that transactions read and
write at. In the words of cockroachdb#56373 (review),
this splits "clock-domain timestamps" from "transaction-domain timestamps", allowing
us to use to type system to help keep them separate and to make their interactions
more explicit.

This results in boiling most of the ocean, but comes with some benefits in the
end.

- It makes conversion from the clock-domain to the transaction-domain explicit.
This must pass through a call to `enginepb.ToTxnTimestamp`.

- It makes conversions from the transaction-domain to the clock-domain impossible
(not quite there yet).

- Similarly, it makes it impossible for transaction-domain timestamps to be
passed into `hlc.Clock.Update`.

- It allows us to more clearly state that clock-domain timestamps must always trail
present time but transaction-domain timestamps can move into the future. As such,
only transaction-domain timestamps will ever have the synthetic bit set.

I'm interested to get people's take on this. These benefits are nice, but the
size of this diff (it still doesn't come close to compiling) and the newly
introduced burden of needing to choose between multiple timestamp types is not.
  • Loading branch information
nvanbenschoten committed Dec 22, 2020
1 parent cea9423 commit a144eb8
Show file tree
Hide file tree
Showing 177 changed files with 3,429 additions and 3,223 deletions.
3 changes: 2 additions & 1 deletion pkg/ccl/importccl/import_stmt.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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")
}
Expand Down
3 changes: 2 additions & 1 deletion pkg/ccl/importccl/import_table_creation.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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,
Expand Down
9 changes: 5 additions & 4 deletions pkg/cli/debug.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -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}),
}
}

Expand Down Expand Up @@ -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)
Expand Down Expand Up @@ -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{
Expand Down
4 changes: 2 additions & 2 deletions pkg/gossip/infostore.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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,
Expand Down
4 changes: 2 additions & 2 deletions pkg/jobs/jobsprotectedts/jobs_protected_ts.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
)
Expand Down Expand Up @@ -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,
Expand Down
6 changes: 3 additions & 3 deletions pkg/kv/bulk/buffering_adder.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand All @@ -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
Expand Down Expand Up @@ -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) {
Expand Down
6 changes: 3 additions & 3 deletions pkg/kv/db.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
}
Expand Down
7 changes: 5 additions & 2 deletions pkg/kv/kvclient/kvcoord/dist_sender.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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 {
Expand Down Expand Up @@ -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,
Expand Down
12 changes: 6 additions & 6 deletions pkg/kv/kvclient/kvcoord/dist_sender_rangefeed.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand All @@ -31,7 +31,7 @@ import (

type singleRangeInfo struct {
rs roachpb.RSpan
ts hlc.Timestamp
ts enginepb.TxnTimestamp
token rangecache.EvictionToken
}

Expand All @@ -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 {
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -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{
Expand Down
13 changes: 6 additions & 7 deletions pkg/kv/kvclient/kvcoord/txn_coord_sender.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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(
Expand Down Expand Up @@ -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
Expand All @@ -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
Expand All @@ -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()
Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvclient/kvcoord/txn_interceptor_span_refresher.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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
Expand Down
13 changes: 6 additions & 7 deletions pkg/kv/kvserver/abortspan/abortspan.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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
}

Expand All @@ -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 {
Expand All @@ -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.
Expand All @@ -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
Expand All @@ -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
Expand Down Expand Up @@ -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"))
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/batcheval/cmd_clear_range.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
}
}
Expand Down
Loading

0 comments on commit a144eb8

Please sign in to comment.