Skip to content

Commit

Permalink
storage: ensure proper equality tests on Timestamp
Browse files Browse the repository at this point in the history
Now that structural equality is different than semantic equality because
of the new flags field in Timestamp, we need to be careful about
comparing by value.

I detected these by adding a `_ func()` field to the struct and observed
what fell out when compiling.
  • Loading branch information
nvanbenschoten committed Nov 12, 2020
1 parent 30ce153 commit db9ef7e
Show file tree
Hide file tree
Showing 8 changed files with 8 additions and 8 deletions.
2 changes: 1 addition & 1 deletion pkg/ccl/changefeedccl/kvfeed/kv_feed.go
Original file line number Diff line number Diff line change
Expand Up @@ -362,7 +362,7 @@ func copyFromSourceToSinkUntilTableEvent(
return false, false
}
frontier.Forward(resolved.Span, boundaryResolvedTimestamp)
return true, frontier.Frontier() == boundaryResolvedTimestamp
return true, frontier.Frontier().EqOrdering(boundaryResolvedTimestamp)
default:
log.Fatal(ctx, "unknown event type")
return false, false
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/liveness/livenesspb/liveness.go
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,7 @@ func (l *Liveness) Compare(o Liveness) int {
}
return +1
}
if l.Expiration != o.Expiration {
if !l.Expiration.EqOrdering(o.Expiration) {
if l.Expiration.Less(o.Expiration) {
return -1
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/rangefeed/resolved_timestamp.go
Original file line number Diff line number Diff line change
Expand Up @@ -312,7 +312,7 @@ func (h unresolvedTxnHeap) Less(i, j int) bool {
// container/heap constructs a min-heap by default, so prioritize the txn
// with the smaller timestamp. Break ties by comparing IDs to establish a
// total order.
if h[i].timestamp == h[j].timestamp {
if h[i].timestamp.EqOrdering(h[j].timestamp) {
return bytes.Compare(h[i].txnID.GetBytes(), h[j].txnID.GetBytes()) < 0
}
return h[i].timestamp.Less(h[j].timestamp)
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/replica_consistency_diff.go
Original file line number Diff line number Diff line change
Expand Up @@ -112,7 +112,7 @@ func diffRange(l, r *roachpb.RaftSnapshotData) ReplicaSnapshotDiffSlice {
case 0:
// 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.EqOrdering(v.Timestamp) {
if e.Timestamp.IsEmpty() {
addLeaseHolder()
} else if v.Timestamp.IsEmpty() {
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/sem/tree/as_of.go
Original file line number Diff line number Diff line change
Expand Up @@ -132,7 +132,7 @@ func DatumToHLC(evalCtx *EvalContext, stmtTimestamp time.Time, d Datum) (hlc.Tim
return ts, convErr
}
zero := hlc.Timestamp{}
if ts == zero {
if ts.EqOrdering(zero) {
return ts, errors.Errorf("zero timestamp is invalid")
} else if ts.Less(zero) {
return ts, errors.Errorf("timestamp before 1970-01-01T00:00:00Z is invalid")
Expand Down
2 changes: 1 addition & 1 deletion pkg/storage/multi_iterator.go
Original file line number Diff line number Diff line change
Expand Up @@ -155,7 +155,7 @@ func (f *multiIterator) advance() {
// The iterator at iterIdx has the same key as the current best, add
// it to itersWithCurrentKey and check how the timestamps compare.
f.itersWithCurrentKey = append(f.itersWithCurrentKey, iterIdx)
if proposedMVCCKey.Timestamp == iterMVCCKey.Timestamp {
if proposedMVCCKey.Timestamp.EqOrdering(iterMVCCKey.Timestamp) {
// We have two exactly equal mvcc keys (both key and timestamps
// match). The one in the later iterator takes precedence and
// the one in the earlier iterator should be omitted from
Expand Down
2 changes: 1 addition & 1 deletion pkg/storage/mvcc.go
Original file line number Diff line number Diff line change
Expand Up @@ -137,7 +137,7 @@ func (k MVCCKey) Less(l MVCCKey) bool {

// Equal returns whether two keys are identical.
func (k MVCCKey) Equal(l MVCCKey) bool {
return k.Key.Compare(l.Key) == 0 && k.Timestamp == l.Timestamp
return k.Key.Compare(l.Key) == 0 && k.Timestamp.EqOrdering(l.Timestamp)
}

// IsValue returns true iff the timestamp is non-zero.
Expand Down
2 changes: 1 addition & 1 deletion pkg/util/span/frontier.go
Original file line number Diff line number Diff line change
Expand Up @@ -60,7 +60,7 @@ func (h frontierHeap) Len() int { return len(h) }

// Less implements heap.Interface.
func (h frontierHeap) Less(i, j int) bool {
if h[i].ts == h[j].ts {
if h[i].ts.EqOrdering(h[j].ts) {
return h[i].span.Key.Compare(h[j].span.Key) < 0
}
return h[i].ts.Less(h[j].ts)
Expand Down

0 comments on commit db9ef7e

Please sign in to comment.