diff --git a/pkg/ccl/backupccl/file_sst_sink.go b/pkg/ccl/backupccl/file_sst_sink.go index 03dab88aba95..5c3d669db333 100644 --- a/pkg/ccl/backupccl/file_sst_sink.go +++ b/pkg/ccl/backupccl/file_sst_sink.go @@ -283,8 +283,8 @@ func (s *fileSSTSink) write(ctx context.Context, resp exportedSpan) (roachpb.Key } else if len(s.flushedFiles) > 0 { last := s.flushedFiles[len(s.flushedFiles)-1] extend = last.Span.EndKey.Equal(span.Key) && - last.EndTime.EqOrdering(resp.metadata.EndTime) && - last.StartTime.EqOrdering(resp.metadata.StartTime) && + last.EndTime == resp.metadata.EndTime && + last.StartTime == resp.metadata.StartTime && last.EntryCounts.DataSize < fileSpanByteLimit } diff --git a/pkg/ccl/changefeedccl/changefeed_processors.go b/pkg/ccl/changefeedccl/changefeed_processors.go index 9f58f2842b1c..fb448ad1d2be 100644 --- a/pkg/ccl/changefeedccl/changefeed_processors.go +++ b/pkg/ccl/changefeedccl/changefeed_processors.go @@ -461,7 +461,7 @@ func (ca *changeAggregator) makeKVFeedCfg( filters := config.Opts.GetFilters() cfg := ca.FlowCtx.Cfg - initialScanOnly := config.EndTime.EqOrdering(initialHighWater) + initialScanOnly := config.EndTime == initialHighWater var sf schemafeed.SchemaFeed if schemaChange.Policy == changefeedbase.OptSchemaChangePolicyIgnore || initialScanOnly { diff --git a/pkg/ccl/changefeedccl/changefeed_test.go b/pkg/ccl/changefeedccl/changefeed_test.go index 053c3b5db10f..a40bcab649be 100644 --- a/pkg/ccl/changefeedccl/changefeed_test.go +++ b/pkg/ccl/changefeedccl/changefeed_test.go @@ -2128,7 +2128,7 @@ func TestChangefeedLaggingSpanCheckpointing(t *testing.T) { // (this is mostly due to racy updates sent from aggregators to the frontier. // However, the checkpoint timestamp should be at least at the cursor. progress := loadProgress() - require.True(t, progress.GetHighWater().IsEmpty() || progress.GetHighWater().EqOrdering(cursor), + require.True(t, progress.GetHighWater().IsEmpty() || *progress.GetHighWater() == cursor, "expected empty highwater or %s, found %s", cursor, progress.GetHighWater()) require.NotNil(t, progress.GetChangefeed().Checkpoint) require.Less(t, 0, len(progress.GetChangefeed().Checkpoint.Spans)) @@ -7513,7 +7513,7 @@ func TestChangefeedEndTimeWithCursor(t *testing.T) { // event with end_time timestamp. That is: verify frontier.Frontier() is at end_time. expectedFrontier := endTime.Prev() testutils.SucceedsWithin(t, func() error { - if expectedFrontier.EqOrdering(frontier.Frontier()) { + if expectedFrontier == frontier.Frontier() { return nil } return errors.Newf("still waiting for frontier to reach %s, current %s", diff --git a/pkg/ccl/changefeedccl/kvfeed/kv_feed.go b/pkg/ccl/changefeedccl/kvfeed/kv_feed.go index 4080d61f21fc..d5a6a0c5fb26 100644 --- a/pkg/ccl/changefeedccl/kvfeed/kv_feed.go +++ b/pkg/ccl/changefeedccl/kvfeed/kv_feed.go @@ -330,7 +330,7 @@ func (f *kvFeed) run(ctx context.Context) (err error) { for i := 0; ; i++ { initialScan := i == 0 - initialScanOnly := f.endTime.EqOrdering(f.initialHighWater) + initialScanOnly := f.endTime == f.initialHighWater scannedSpans, scannedTS, err := f.scanIfShould(ctx, initialScan, initialScanOnly, rangeFeedResumeFrontier.Frontier()) if err != nil { return err @@ -754,7 +754,7 @@ func copyFromSourceToDestUntilTableEvent( return false, false, err } - return skipEvent, frontier.Frontier().EqOrdering(boundaryResolvedTimestamp), nil + return skipEvent, frontier.Frontier() == boundaryResolvedTimestamp, nil case kvevent.TypeFlush: // TypeFlush events have a timestamp of zero and should have already // been processed by the timestamp check above. We include this here diff --git a/pkg/kv/kvserver/batcheval/cmd_refresh_range.go b/pkg/kv/kvserver/batcheval/cmd_refresh_range.go index 43306c8e1bd5..bb602171e4b7 100644 --- a/pkg/kv/kvserver/batcheval/cmd_refresh_range.go +++ b/pkg/kv/kvserver/batcheval/cmd_refresh_range.go @@ -135,7 +135,7 @@ func refreshRange( } else if !ok { return errors.Errorf("expected provisional value for intent") } - if !meta.Timestamp.ToTimestamp().EqOrdering(iter.UnsafeKey().Timestamp) { + if meta.Timestamp.ToTimestamp() != iter.UnsafeKey().Timestamp { return errors.Errorf("expected provisional value for intent with ts %s, found %s", meta.Timestamp, iter.UnsafeKey().Timestamp) } diff --git a/pkg/kv/kvserver/liveness/livenesspb/liveness.go b/pkg/kv/kvserver/liveness/livenesspb/liveness.go index eeb94c73bce4..5ba3eb36409d 100644 --- a/pkg/kv/kvserver/liveness/livenesspb/liveness.go +++ b/pkg/kv/kvserver/liveness/livenesspb/liveness.go @@ -58,7 +58,7 @@ func (l Liveness) Compare(o Liveness) int { } return +1 } - if !l.Expiration.EqOrdering(o.Expiration) { + if l.Expiration != o.Expiration { if l.Expiration.Less(o.Expiration) { return -1 } diff --git a/pkg/kv/kvserver/rangefeed/catchup_scan.go b/pkg/kv/kvserver/rangefeed/catchup_scan.go index aaee93a67d73..4b2940fcd1db 100644 --- a/pkg/kv/kvserver/rangefeed/catchup_scan.go +++ b/pkg/kv/kvserver/rangefeed/catchup_scan.go @@ -266,7 +266,7 @@ func (i *CatchUpIterator) CatchUpScan( } else if !ok { return errors.Errorf("expected provisional value for intent") } - if !meta.Timestamp.ToTimestamp().EqOrdering(i.UnsafeKey().Timestamp) { + if meta.Timestamp.ToTimestamp() != i.UnsafeKey().Timestamp { return errors.Errorf("expected provisional value for intent with ts %s, found %s", meta.Timestamp, i.UnsafeKey().Timestamp) } diff --git a/pkg/kv/kvserver/rangefeed/resolved_timestamp.go b/pkg/kv/kvserver/rangefeed/resolved_timestamp.go index ad9689a8721e..20faf2c6169d 100644 --- a/pkg/kv/kvserver/rangefeed/resolved_timestamp.go +++ b/pkg/kv/kvserver/rangefeed/resolved_timestamp.go @@ -280,7 +280,7 @@ func (rts *resolvedTimestamp) recompute(ctx context.Context) bool { func (rts *resolvedTimestamp) assertNoChange(ctx context.Context) { before := rts.resolvedTS changed := rts.recompute(ctx) - if changed || !before.EqOrdering(rts.resolvedTS) { + if changed || before != rts.resolvedTS { log.Fatalf(ctx, "unexpected resolved timestamp change on recomputation, "+ "was %s, recomputed as %s", before, rts.resolvedTS) } @@ -374,7 +374,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.EqOrdering(h[j].timestamp) { + if h[i].timestamp == h[j].timestamp { return bytes.Compare(h[i].txnID.GetBytes(), h[j].txnID.GetBytes()) < 0 } return h[i].timestamp.Less(h[j].timestamp) diff --git a/pkg/kv/kvserver/readsummary/rspb/summary.go b/pkg/kv/kvserver/readsummary/rspb/summary.go index 37d512df1215..074706929e64 100644 --- a/pkg/kv/kvserver/readsummary/rspb/summary.go +++ b/pkg/kv/kvserver/readsummary/rspb/summary.go @@ -95,7 +95,7 @@ func (c *Segment) Clone() Segment { // commutative and idempotent. func (c *Segment) Merge(o Segment) { // Forward the low water mark. - if !c.LowWater.EqOrdering(o.LowWater) { + if c.LowWater != o.LowWater { if c.LowWater.Less(o.LowWater) { // c.LowWater < o.LowWater, filter c.ReadSpans. c.ReadSpans = filterSpans(c.ReadSpans, o.LowWater) diff --git a/pkg/kv/kvserver/store_test.go b/pkg/kv/kvserver/store_test.go index 492a4462c373..a48de7dfdf70 100644 --- a/pkg/kv/kvserver/store_test.go +++ b/pkg/kv/kvserver/store_test.go @@ -2174,11 +2174,11 @@ func TestStoreSkipLockedTSCache(t *testing.T) { // Verify the timestamp cache has been set for "a" and "c", but not for "b". t2TS := makeTS(t2.UnixNano(), 0) rTS, _ := store.tsCache.GetMax(ctx, roachpb.Key("a"), nil) - require.True(t, rTS.EqOrdering(t2TS)) + require.Equal(t, t2TS, rTS) rTS, _ = store.tsCache.GetMax(ctx, roachpb.Key("b"), nil) require.True(t, rTS.Less(t2TS)) rTS, _ = store.tsCache.GetMax(ctx, roachpb.Key("c"), nil) - require.True(t, rTS.EqOrdering(t2TS)) + require.Equal(t, t2TS, rTS) }) } } diff --git a/pkg/sql/conn_executor_test.go b/pkg/sql/conn_executor_test.go index c73e41d3c823..bebdd71d1730 100644 --- a/pkg/sql/conn_executor_test.go +++ b/pkg/sql/conn_executor_test.go @@ -1337,7 +1337,7 @@ CREATE TABLE t1.test (k INT PRIMARY KEY, v TEXT); } err = txn.Commit() require.NoError(t, err) - locked(func() { require.True(t, fs.Expiration().EqOrdering(mu.txnDeadline)) }) + locked(func() { require.Equal(t, fs.Expiration(), mu.txnDeadline) }) }) t.Run("lease_deadline_overrides_session_expiry", func(t *testing.T) { diff --git a/pkg/sql/sem/asof/as_of.go b/pkg/sql/sem/asof/as_of.go index 7f82dd78e724..ee37b31e3eec 100644 --- a/pkg/sql/sem/asof/as_of.go +++ b/pkg/sql/sem/asof/as_of.go @@ -293,7 +293,7 @@ func DatumToHLC( return ts, convErr } zero := hlc.Timestamp{} - if ts.EqOrdering(zero) { + if ts == 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") diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go index d9253c18858b..dc129f7b3966 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -1439,7 +1439,7 @@ func MVCCGetForKnownTimestampWithNoIntent( if val == nil { return nil, enginepb.MVCCValueHeader{}, errors.Errorf("value missing for key %v", key) } - if !val.Timestamp.EqOrdering(timestamp) { + if val.Timestamp != timestamp { return nil, enginepb.MVCCValueHeader{}, errors.Errorf( "expected timestamp %v and found %v for key %v", timestamp, val.Timestamp, key) } diff --git a/pkg/storage/mvcc_incremental_iterator.go b/pkg/storage/mvcc_incremental_iterator.go index 4f6320d6e80c..3263ad59975a 100644 --- a/pkg/storage/mvcc_incremental_iterator.go +++ b/pkg/storage/mvcc_incremental_iterator.go @@ -915,7 +915,7 @@ func (i *MVCCIncrementalIterator) assertInvariants() error { // i.meta should match the underlying iterator's key. if hasPoint, _ := i.iter.HasPointAndRange(); hasPoint { metaTS := i.meta.Timestamp.ToTimestamp() - if iterKey.Timestamp.IsSet() && !metaTS.EqOrdering(iterKey.Timestamp) { + if iterKey.Timestamp.IsSet() && metaTS != iterKey.Timestamp { return errors.AssertionFailedf("i.meta.Timestamp %s differs from i.iter.UnsafeKey %s", metaTS, iterKey) } diff --git a/pkg/storage/mvcc_key.go b/pkg/storage/mvcc_key.go index 77d72e2e015e..429767fb08fb 100644 --- a/pkg/storage/mvcc_key.go +++ b/pkg/storage/mvcc_key.go @@ -116,7 +116,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.EqOrdering(l.Timestamp) + return k.Key.Compare(l.Key) == 0 && k.Timestamp == l.Timestamp } // IsValue returns true iff the timestamp is non-zero. diff --git a/pkg/storage/pebble_mvcc_scanner.go b/pkg/storage/pebble_mvcc_scanner.go index e57021b0d91e..4d3ea65695ff 100644 --- a/pkg/storage/pebble_mvcc_scanner.go +++ b/pkg/storage/pebble_mvcc_scanner.go @@ -829,7 +829,7 @@ func (p *pebbleMVCCScanner) getOne(ctx context.Context) (ok, added bool) { } // ts == read_ts - if p.curUnsafeKey.Timestamp.EqOrdering(p.ts) { + if p.curUnsafeKey.Timestamp == p.ts { if p.failOnMoreRecent { // 2. Our txn's read timestamp is equal to the most recent // version's timestamp and the scanner has been configured to diff --git a/pkg/util/hlc/timestamp.go b/pkg/util/hlc/timestamp.go index 677acdd4755b..c601253a8d69 100644 --- a/pkg/util/hlc/timestamp.go +++ b/pkg/util/hlc/timestamp.go @@ -318,11 +318,6 @@ func (t Timestamp) ToLegacyTimestamp() LegacyTimestamp { return LegacyTimestamp( // ToTimestamp converts a LegacyTimestamp to a Timestamp. func (t LegacyTimestamp) ToTimestamp() Timestamp { return Timestamp(t) } -// EqOrdering returns whether the receiver sorts equally to the parameter. -func (t LegacyTimestamp) EqOrdering(s LegacyTimestamp) bool { - return t.ToTimestamp().EqOrdering(s.ToTimestamp()) -} - // Less returns whether the receiver is less than the parameter. func (t LegacyTimestamp) Less(s LegacyTimestamp) bool { return t.ToTimestamp().Less(s.ToTimestamp()) diff --git a/pkg/util/hlc/timestamp_test.go b/pkg/util/hlc/timestamp_test.go index 60a7c7345204..201a45acf161 100644 --- a/pkg/util/hlc/timestamp_test.go +++ b/pkg/util/hlc/timestamp_test.go @@ -56,22 +56,6 @@ func TestCompare(t *testing.T) { } } -func TestEqOrdering(t *testing.T) { - a := Timestamp{} - b := Timestamp{} - if !a.EqOrdering(b) { - t.Errorf("expected %+v == %+v", a, b) - } - b = makeTS(1, 0) - if a.EqOrdering(b) { - t.Errorf("expected %+v != %+v", a, b) - } - a = makeTS(1, 1) - if a.EqOrdering(b) { - t.Errorf("expected %+v != %+v", b, a) - } -} - func TestLess(t *testing.T) { a := Timestamp{} b := Timestamp{} diff --git a/pkg/util/span/frontier.go b/pkg/util/span/frontier.go index d4edd4dc2991..b7d6e0cfc7f9 100644 --- a/pkg/util/span/frontier.go +++ b/pkg/util/span/frontier.go @@ -699,7 +699,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.EqOrdering(h[j].ts) { + if h[i].ts == h[j].ts { return h[i].Start.Compare(h[j].Start) < 0 } return h[i].ts.Less(h[j].ts) diff --git a/pkg/util/span/llrb_frontier.go b/pkg/util/span/llrb_frontier.go index 566d553f9142..4f2d7dd199a6 100644 --- a/pkg/util/span/llrb_frontier.go +++ b/pkg/util/span/llrb_frontier.go @@ -60,7 +60,7 @@ func (h llrbFrontierHeap) Len() int { return len(h) } // Less implements heap.Interface. func (h llrbFrontierHeap) Less(i, j int) bool { - if h[i].ts.EqOrdering(h[j].ts) { + if h[i].ts == h[j].ts { return h[i].span.Key.Compare(h[j].span.Key) < 0 } return h[i].ts.Less(h[j].ts)