Skip to content

Commit

Permalink
*: use Timestamp.IsEmpty where possible
Browse files Browse the repository at this point in the history
Be more explicit and avoid any issues with structural equality.
  • Loading branch information
nvanbenschoten committed Nov 12, 2020
1 parent fdc60c8 commit 30ce153
Show file tree
Hide file tree
Showing 54 changed files with 83 additions and 92 deletions.
2 changes: 1 addition & 1 deletion pkg/ccl/backupccl/backup_planning.go
Original file line number Diff line number Diff line change
Expand Up @@ -824,7 +824,7 @@ func backupPlanHook(
keys.MinKey,
p.User(),
func(span covering.Range, start, end hlc.Timestamp) error {
if (start == hlc.Timestamp{}) {
if start.IsEmpty() {
newSpans = append(newSpans, roachpb.Span{Key: span.Start, EndKey: span.End})
return nil
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/changefeedccl/cdctest/validator.go
Original file line number Diff line number Diff line change
Expand Up @@ -467,7 +467,7 @@ func (v *fingerprintValidator) NoteResolved(partition string, resolved hlc.Times
// we fingerprint at `updated.Prev()` since we want to catch cases where one or
// more row updates are missed. For example: If k1 was written at t1, t2, t3 and
// the update for t2 was missed.
if v.previousRowUpdateTs != (hlc.Timestamp{}) && v.previousRowUpdateTs.Less(row.updated) {
if !v.previousRowUpdateTs.IsEmpty() && v.previousRowUpdateTs.Less(row.updated) {
if err := v.fingerprint(row.updated.Prev()); err != nil {
return err
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/changefeedccl/changefeed.go
Original file line number Diff line number Diff line change
Expand Up @@ -199,7 +199,7 @@ func kvsToRows(
}
schemaTimestamp := kv.Value.Timestamp
prevSchemaTimestamp := schemaTimestamp
if backfillTs := input.BackfillTimestamp(); backfillTs != (hlc.Timestamp{}) {
if backfillTs := input.BackfillTimestamp(); !backfillTs.IsEmpty() {
schemaTimestamp = backfillTs
prevSchemaTimestamp = schemaTimestamp.Prev()
}
Expand Down
4 changes: 2 additions & 2 deletions pkg/ccl/changefeedccl/changefeed_dist.go
Original file line number Diff line number Diff line change
Expand Up @@ -80,7 +80,7 @@ func distChangefeedFlow(
// based on whether we should perform an initial scan.
{
h := progress.GetHighWater()
noHighWater := (h == nil || *h == (hlc.Timestamp{}))
noHighWater := (h == nil || h.IsEmpty())
// We want to set the highWater and thus avoid an initial scan if either
// this is a cursor and there was no request for one, or we don't have a
// cursor but we have a request to not have an initial scan.
Expand All @@ -92,7 +92,7 @@ func distChangefeedFlow(

spansTS := details.StatementTime
var initialHighWater hlc.Timestamp
if h := progress.GetHighWater(); h != nil && *h != (hlc.Timestamp{}) {
if h := progress.GetHighWater(); h != nil && !h.IsEmpty() {
initialHighWater = *h
// If we have a high-water set, use it to compute the spans, since the
// ones at the statement time may have been garbage collected by now.
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/changefeedccl/kvfeed/buffer.go
Original file line number Diff line number Diff line change
Expand Up @@ -126,7 +126,7 @@ func (b *Event) Timestamp() hlc.Timestamp {
case ResolvedEvent:
return b.resolved.Timestamp
case KVEvent:
if b.backfillTimestamp != (hlc.Timestamp{}) {
if !b.backfillTimestamp.IsEmpty() {
return b.backfillTimestamp
}
return b.kv.Value.Timestamp
Expand Down
4 changes: 2 additions & 2 deletions pkg/ccl/changefeedccl/schemafeed/schema_feed.go
Original file line number Diff line number Diff line change
Expand Up @@ -369,7 +369,7 @@ func (tf *SchemaFeed) waitForTS(ctx context.Context, ts hlc.Timestamp) error {
tf.mu.Lock()
highWater := tf.mu.highWater
var err error
if tf.mu.errTS != (hlc.Timestamp{}) && tf.mu.errTS.LessEq(ts) {
if !tf.mu.errTS.IsEmpty() && tf.mu.errTS.LessEq(ts) {
err = tf.mu.err
}
fastPath := err != nil || ts.LessEq(highWater)
Expand Down Expand Up @@ -437,7 +437,7 @@ func (tf *SchemaFeed) adjustTimestamps(startTS, endTS hlc.Timestamp, validateErr

if validateErr != nil {
// don't care about startTS in the invalid case
if tf.mu.errTS == (hlc.Timestamp{}) || endTS.Less(tf.mu.errTS) {
if tf.mu.errTS.IsEmpty() || endTS.Less(tf.mu.errTS) {
tf.mu.errTS = endTS
tf.mu.err = validateErr
newWaiters := make([]tableHistoryWaiter, 0, len(tf.mu.waiters))
Expand Down
3 changes: 1 addition & 2 deletions pkg/ccl/storageccl/import.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/settings"
"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/util/humanizeutil"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/retry"
Expand Down Expand Up @@ -220,7 +219,7 @@ func evalImport(ctx context.Context, cArgs batcheval.CommandArgs) (*roachpb.Impo
break
}

if args.EndTime != (hlc.Timestamp{}) {
if !args.EndTime.IsEmpty() {
// TODO(dan): If we have to skip past a lot of versions to find the
// latest one before args.EndTime, then this could be slow.
if args.EndTime.Less(iter.UnsafeKey().Timestamp) {
Expand Down
2 changes: 1 addition & 1 deletion pkg/cli/debug.go
Original file line number Diff line number Diff line change
Expand Up @@ -349,7 +349,7 @@ func loadRangeDescriptor(
) (roachpb.RangeDescriptor, error) {
var desc roachpb.RangeDescriptor
handleKV := func(kv storage.MVCCKeyValue) error {
if kv.Key.Timestamp == (hlc.Timestamp{}) {
if kv.Key.Timestamp.IsEmpty() {
// We only want values, not MVCCMetadata.
return nil
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/jobs/jobs_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1499,7 +1499,7 @@ func TestShowJobs(t *testing.T) {
progress := &jobspb.Progress{
ModifiedMicros: in.modified.UnixNano() / time.Microsecond.Nanoseconds(),
}
if in.highWater != (hlc.Timestamp{}) {
if !in.highWater.IsEmpty() {
progress.Progress = &jobspb.Progress_HighWater{
HighWater: &in.highWater,
}
Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/client_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -338,7 +338,7 @@ func TestClientGetAndPut(t *testing.T) {
if !bytes.Equal(value, gr.ValueBytes()) {
t.Errorf("expected values equal; %s != %s", value, gr.ValueBytes())
}
if gr.Value.Timestamp == (hlc.Timestamp{}) {
if gr.Value.Timestamp.IsEmpty() {
t.Fatalf("expected non-zero timestamp; got empty")
}
}
Expand All @@ -361,7 +361,7 @@ func TestClientPutInline(t *testing.T) {
if !bytes.Equal(value, gr.ValueBytes()) {
t.Errorf("expected values equal; %s != %s", value, gr.ValueBytes())
}
if ts := gr.Value.Timestamp; ts != (hlc.Timestamp{}) {
if ts := gr.Value.Timestamp; !ts.IsEmpty() {
t.Fatalf("expected zero timestamp; got %s", ts)
}
}
Expand Down
6 changes: 3 additions & 3 deletions pkg/kv/kvclient/kvcoord/dist_sender.go
Original file line number Diff line number Diff line change
Expand Up @@ -574,7 +574,7 @@ 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 == (hlc.Timestamp{}) {
if ba.ReadConsistency != roachpb.CONSISTENT && ba.Timestamp.IsEmpty() {
ba.Timestamp = ds.clock.Now()
}

Expand Down Expand Up @@ -1954,10 +1954,10 @@ func (ds *DistSender) sendToReplicas(
// If the reply contains a timestamp, update the local HLC with it.
if br.Error != nil {
log.VErrEventf(ctx, 2, "%v", br.Error)
if br.Error.Now != (hlc.Timestamp{}) {
if !br.Error.Now.IsEmpty() {
ds.clock.Update(br.Error.Now)
}
} else if br.Now != (hlc.Timestamp{}) {
} else if !br.Now.IsEmpty() {
ds.clock.Update(br.Now)
}

Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvclient/kvcoord/split_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -290,7 +290,7 @@ func TestRangeSplitsStickyBit(t *testing.T) {
if err != nil {
t.Fatal(err)
}
if (desc.GetStickyBit() == hlc.Timestamp{}) {
if desc.GetStickyBit().IsEmpty() {
t.Fatal("Sticky bit not set after splitting")
}

Expand All @@ -309,7 +309,7 @@ func TestRangeSplitsStickyBit(t *testing.T) {
if err != nil {
t.Fatal(err)
}
if (desc.GetStickyBit() == hlc.Timestamp{}) {
if desc.GetStickyBit().IsEmpty() {
t.Fatal("Sticky bit not set after splitting")
}
}
4 changes: 2 additions & 2 deletions pkg/kv/kvserver/batcheval/cmd_end_transaction.go
Original file line number Diff line number Diff line change
Expand Up @@ -634,7 +634,7 @@ func RunCommitTrigger(
}
if sbt := ct.GetStickyBitTrigger(); sbt != nil {
newDesc := *rec.Desc()
if sbt.StickyBit != (hlc.Timestamp{}) {
if !sbt.StickyBit.IsEmpty() {
newDesc.StickyBit = &sbt.StickyBit
} else {
newDesc.StickyBit = nil
Expand Down Expand Up @@ -945,7 +945,7 @@ func splitTriggerHelper(
if err != nil {
return enginepb.MVCCStats{}, result.Result{}, errors.Wrap(err, "unable to load GCThreshold")
}
if (*gcThreshold == hlc.Timestamp{}) {
if gcThreshold.IsEmpty() {
log.VEventf(ctx, 1, "LHS's GCThreshold of split is not set")
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/client_metrics_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -386,7 +386,7 @@ func TestStoreMaxBehindNanosOnlyTracksEpochBasedLeases(t *testing.T) {
testutils.SucceedsSoon(t, func() error {
_, metaRepl := getFirstStoreReplica(t, tc.Server(1), keys.Meta2Prefix)
l, _ := metaRepl.GetLease()
if l.Start == (hlc.Timestamp{}) {
if l.Start.IsEmpty() {
return errors.Errorf("don't have a lease for meta1 yet: %v %v", l, meta2Repl1)
}
sinceExpBasedLeaseStart := timeutil.Since(timeutil.Unix(0, l.Start.WallTime))
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/client_split_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1771,7 +1771,7 @@ func TestStoreSplitTimestampCacheDifferentLeaseHolder(t *testing.T) {
}

// Verify that the txn's safe timestamp was set.
if txnOld.TestingCloneTxn().ReadTimestamp == (hlc.Timestamp{}) {
if txnOld.TestingCloneTxn().ReadTimestamp.IsEmpty() {
t.Fatal("expected non-zero refreshed timestamp")
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/debug_print.go
Original file line number Diff line number Diff line change
Expand Up @@ -235,7 +235,7 @@ func tryTxn(kv storage.MVCCKeyValue) (string, error) {
}

func tryRangeIDKey(kv storage.MVCCKeyValue) (string, error) {
if kv.Key.Timestamp != (hlc.Timestamp{}) {
if !kv.Key.Timestamp.IsEmpty() {
return "", fmt.Errorf("range ID keys shouldn't have timestamps: %s", kv.Key)
}
_, _, suffix, _, err := keys.DecodeRangeIDKey(kv.Key.Key)
Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvserver/gc/data_distribution_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -57,10 +57,10 @@ func (ds dataDistribution) setupTest(
} else {
// TODO(ajwerner): Decide if using MVCCPut is worth it.
ts := kv.Key.Timestamp
if txn.ReadTimestamp == (hlc.Timestamp{}) {
if txn.ReadTimestamp.IsEmpty() {
txn.ReadTimestamp = ts
}
if txn.WriteTimestamp == (hlc.Timestamp{}) {
if txn.WriteTimestamp.IsEmpty() {
txn.WriteTimestamp = ts
}
err := storage.MVCCPut(ctx, eng, &ms, kv.Key.Key, ts,
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/gc/gc_old_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -125,7 +125,7 @@ func runGCOld(
startIdx = 2
}
// See if any values may be GC'd.
if idx, gcTS := gc.Filter(keys[startIdx:], vals[startIdx:]); gcTS != (hlc.Timestamp{}) {
if idx, gcTS := gc.Filter(keys[startIdx:], vals[startIdx:]); !gcTS.IsEmpty() {
// Batch keys after the total size of version keys exceeds
// the threshold limit. This avoids sending potentially large
// GC requests through Raft. Iterate through the keys in reverse
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/gc_queue.go
Original file line number Diff line number Diff line change
Expand Up @@ -281,7 +281,7 @@ func makeGCQueueScoreImpl(
) gcQueueScore {
ms.Forward(now.WallTime)
var r gcQueueScore
if (gcThreshold != hlc.Timestamp{}) {
if !gcThreshold.IsEmpty() {
r.LikelyLastGC = time.Duration(now.WallTime - gcThreshold.Add(r.TTL.Nanoseconds(), 0).WallTime)
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/log.go
Original file line number Diff line number Diff line change
Expand Up @@ -214,7 +214,7 @@ func (s *Store) logChange(
// *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 {
if input == (hlc.Timestamp{}) {
if input.IsEmpty() {
return s.Clock().PhysicalTime()
}
return input.GoTime()
Expand Down
1 change: 0 additions & 1 deletion pkg/kv/kvserver/protectedts/ptstorage/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@ go_library(
"//pkg/sql/sem/tree",
"//pkg/sql/sessiondata",
"//pkg/sql/sqlutil",
"//pkg/util/hlc",
"//pkg/util/log",
"//pkg/util/protoutil",
"//pkg/util/uuid",
Expand Down
3 changes: 1 addition & 2 deletions pkg/kv/kvserver/protectedts/ptstorage/storage.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@ 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/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
Expand Down Expand Up @@ -266,7 +265,7 @@ var (
)

func validateRecordForProtect(r *ptpb.Record) error {
if r.Timestamp == (hlc.Timestamp{}) {
if r.Timestamp.IsEmpty() {
return errZeroTimestamp
}
if r.ID == uuid.Nil {
Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvserver/queue.go
Original file line number Diff line number Diff line change
Expand Up @@ -212,15 +212,15 @@ func isExpectedQueueError(err error) bool {
// Returns a bool for whether to queue as well as a priority based
// on how long it's been since last processed.
func shouldQueueAgain(now, last hlc.Timestamp, minInterval time.Duration) (bool, float64) {
if minInterval == 0 || last == (hlc.Timestamp{}) {
if minInterval == 0 || last.IsEmpty() {
return true, 0
}
if diff := now.GoTime().Sub(last.GoTime()); diff >= minInterval {
priority := float64(1)
// If there's a non-zero last processed timestamp, adjust the
// priority by a multiple of how long it's been since the last
// time this replica was processed.
if last != (hlc.Timestamp{}) {
if !last.IsEmpty() {
priority = float64(diff.Nanoseconds()) / float64(minInterval.Nanoseconds())
}
return true, priority
Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvserver/rangefeed/processor.go
Original file line number Diff line number Diff line change
Expand Up @@ -471,7 +471,7 @@ func (p *Processor) ForwardClosedTS(closedTS hlc.Timestamp) bool {
if p == nil {
return true
}
if closedTS == (hlc.Timestamp{}) {
if closedTS.IsEmpty() {
return true
}
return p.sendEvent(event{ct: closedTS}, p.EventChanTimeout)
Expand Down Expand Up @@ -538,7 +538,7 @@ func (p *Processor) consumeEvent(ctx context.Context, e *event) {
switch {
case len(e.ops) > 0:
p.consumeLogicalOps(ctx, e.ops)
case e.ct != hlc.Timestamp{}:
case !e.ct.IsEmpty():
p.forwardClosedTS(ctx, e.ct)
case e.initRTS:
p.initResolvedTS(ctx)
Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvserver/replica_command.go
Original file line number Diff line number Diff line change
Expand Up @@ -172,7 +172,7 @@ func setStickyBit(desc *roachpb.RangeDescriptor, expiration hlc.Timestamp) {
// byte representation of setting the stickyBit to nil is different than
// setting it to hlc.Timestamp{}. This check ensures that CPuts would not
// fail on older versions.
if (expiration != hlc.Timestamp{}) {
if !expiration.IsEmpty() {
desc.StickyBit = &expiration
}
}
Expand Down Expand Up @@ -462,7 +462,7 @@ func (r *Replica) adminUnsplitWithDescriptor(
// mixed version clusters that don't support StickyBit, all range descriptor
// sticky bits are guaranteed to be nil, so we can skip checking the cluster
// version.
if (desc.GetStickyBit() == hlc.Timestamp{}) {
if desc.GetStickyBit().IsEmpty() {
return reply, nil
}

Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvserver/replica_consistency_diff.go
Original file line number Diff line number Diff line change
Expand Up @@ -113,9 +113,9 @@ func diffRange(l, r *roachpb.RaftSnapshotData) ReplicaSnapshotDiffSlice {
// Timestamp sorting is weird. Timestamp{} sorts first, the
// remainder sort in descending order. See storage/engine/doc.go.
if e.Timestamp != v.Timestamp {
if e.Timestamp == (hlc.Timestamp{}) {
if e.Timestamp.IsEmpty() {
addLeaseHolder()
} else if v.Timestamp == (hlc.Timestamp{}) {
} else if v.Timestamp.IsEmpty() {
addReplica()
} else if v.Timestamp.Less(e.Timestamp) {
addLeaseHolder()
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/replica_destroy.go
Original file line number Diff line number Diff line change
Expand Up @@ -243,7 +243,7 @@ func writeTombstoneKey(
tombstone := &roachpb.RangeTombstone{
NextReplicaID: nextReplicaID,
}
// "Blind" because ms == nil and timestamp == hlc.Timestamp{}.
// "Blind" because ms == nil and timestamp.IsEmpty().
return storage.MVCCBlindPutProto(ctx, writer, nil, tombstoneKey,
hlc.Timestamp{}, tombstone, nil)
}
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/replica_proposal.go
Original file line number Diff line number Diff line change
Expand Up @@ -726,7 +726,7 @@ func (r *Replica) evaluateProposal(
ba *roachpb.BatchRequest,
latchSpans *spanset.SpanSet,
) (*result.Result, bool, *roachpb.Error) {
if ba.Timestamp == (hlc.Timestamp{}) {
if ba.Timestamp.IsEmpty() {
return nil, false, roachpb.NewErrorf("can't propose Raft command with zero timestamp")
}

Expand Down
3 changes: 1 addition & 2 deletions pkg/kv/kvserver/replica_send.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/txnwait"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"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"
Expand Down Expand Up @@ -644,7 +643,7 @@ func (r *Replica) executeAdminBatch(
// TODO(tschottdorf): should check that request is contained in range and that
// EndTxn only occurs at the very end.
func (r *Replica) checkBatchRequest(ba *roachpb.BatchRequest, isReadOnly bool) error {
if ba.Timestamp == (hlc.Timestamp{}) {
if ba.Timestamp.IsEmpty() {
// For transactional requests, Store.Send sets the timestamp. For non-
// transactional requests, the client sets the timestamp. Either way, we
// need to have a timestamp at this point.
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/replica_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -324,7 +324,7 @@ func (tc *testContext) Sender() kv.Sender {
if ba.RangeID == 0 {
ba.RangeID = 1
}
if ba.Timestamp == (hlc.Timestamp{}) {
if ba.Timestamp.IsEmpty() {
if err := ba.SetActiveTimestamp(tc.Clock().Now); err != nil {
tc.Fatal(err)
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/stateloader/initial.go
Original file line number Diff line number Diff line change
Expand Up @@ -70,7 +70,7 @@ func WriteInitialReplicaState(

if existingGCThreshold, err := rsl.LoadGCThreshold(ctx, readWriter); err != nil {
return enginepb.MVCCStats{}, errors.Wrap(err, "error reading GCThreshold")
} else if (*existingGCThreshold != hlc.Timestamp{}) {
} else if !existingGCThreshold.IsEmpty() {
log.Fatalf(ctx, "expected trivial GChreshold, but found %+v", existingGCThreshold)
}

Expand Down
Loading

0 comments on commit 30ce153

Please sign in to comment.