From 62ebe56640091abfe3da6d0aa043cb7b9231c3e9 Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Tue, 19 Dec 2023 17:51:33 -0500 Subject: [PATCH] hlc: remove the Synthetic field from Timestamp and LegacyTimestamp Closes #101938. This commit completes the work to remove the Synthetic field from Timestamp and LegacyTimestamp. It removes the fields from the proto definitions and removes all access to the fields in methods. The commit also cleans up the remaining references in the code which were just waiting for the field to be removed. Release note: None --- pkg/kv/kvserver/below_raft_protos_test.go | 10 +- pkg/kv/kvserver/kvserverpb/proposer_kv.go | 5 +- pkg/kv/kvserver/replica_consistency.go | 2 - .../MVCCMetadata | 2 +- .../MVCCMetadataSubsetForMergeSerialization | 2 +- .../RangeAppliedState | 2 +- pkg/roachpb/data_test.go | 23 +-- pkg/roachpb/span_config_test.go | 10 +- pkg/roachpb/string_test.go | 4 +- pkg/storage/enginepb/mvcc3_test.go | 2 +- pkg/storage/mvcc_value.go | 11 -- pkg/util/hlc/legacy_timestamp.proto | 12 +- pkg/util/hlc/timestamp.go | 148 +++++------------- pkg/util/hlc/timestamp.proto | 34 +--- pkg/util/hlc/timestamp_test.go | 83 +--------- 15 files changed, 62 insertions(+), 288 deletions(-) diff --git a/pkg/kv/kvserver/below_raft_protos_test.go b/pkg/kv/kvserver/below_raft_protos_test.go index f3f22a0c0ad5..bdc8dd8c4514 100644 --- a/pkg/kv/kvserver/below_raft_protos_test.go +++ b/pkg/kv/kvserver/below_raft_protos_test.go @@ -43,10 +43,6 @@ func TestBelowRaftProtosDontChange(t *testing.T) { func(r *rand.Rand) protoutil.Message { m := enginepb.NewPopulatedMVCCMetadata(r, false) m.Txn = nil // never populated below Raft - m.Timestamp.Synthetic = nil // never populated below Raft - if m.MergeTimestamp != nil { - m.MergeTimestamp.Synthetic = nil // never populated below Raft - } m.TxnDidNotUpdateMeta = nil // never populated below Raft return m }, @@ -85,11 +81,7 @@ func TestBelowRaftProtosDontChange(t *testing.T) { return roachpb.NewPopulatedInternalTimeSeriesData(r, false) }, func(r *rand.Rand) protoutil.Message { - m := enginepb.NewPopulatedMVCCMetadataSubsetForMergeSerialization(r, false) - if m.MergeTimestamp != nil { - m.MergeTimestamp.Synthetic = nil // never populated below Raft - } - return m + return enginepb.NewPopulatedMVCCMetadataSubsetForMergeSerialization(r, false) }, func(r *rand.Rand) protoutil.Message { return kvserverpb.NewPopulatedRaftReplicaID(r, false) diff --git a/pkg/kv/kvserver/kvserverpb/proposer_kv.go b/pkg/kv/kvserver/kvserverpb/proposer_kv.go index 6b02942bfeb2..f982cce6eba0 100644 --- a/pkg/kv/kvserver/kvserverpb/proposer_kv.go +++ b/pkg/kv/kvserver/kvserverpb/proposer_kv.go @@ -20,9 +20,8 @@ import ( var maxRaftCommandFooterSize = (&RaftCommandFooter{ MaxLeaseIndex: math.MaxUint64, ClosedTimestamp: hlc.Timestamp{ - WallTime: math.MaxInt64, - Logical: math.MaxInt32, - Synthetic: true, + WallTime: math.MaxInt64, + Logical: math.MaxInt32, }, }).Size() diff --git a/pkg/kv/kvserver/replica_consistency.go b/pkg/kv/kvserver/replica_consistency.go index 6291789ac02c..4cc3c53e6437 100644 --- a/pkg/kv/kvserver/replica_consistency.go +++ b/pkg/kv/kvserver/replica_consistency.go @@ -680,8 +680,6 @@ func CalcReplicaDigest( result.PersistedMS = rangeAppliedState.RangeStats.ToStats() if statsOnly { - // Unset the synthetic flag, to ease the migration of deleting this field. - rangeAppliedState.RaftClosedTimestamp.Synthetic = false b, err := protoutil.Marshal(rangeAppliedState) if err != nil { return nil, err diff --git a/pkg/kv/kvserver/testdata/TestBelowRaftProtosDontChange/MVCCMetadata b/pkg/kv/kvserver/testdata/TestBelowRaftProtosDontChange/MVCCMetadata index d58d9b06ce06..94b600ca3b5a 100644 --- a/pkg/kv/kvserver/testdata/TestBelowRaftProtosDontChange/MVCCMetadata +++ b/pkg/kv/kvserver/testdata/TestBelowRaftProtosDontChange/MVCCMetadata @@ -1,3 +1,3 @@ echo ---- -16747849667884397839 +10315043615536467344 diff --git a/pkg/kv/kvserver/testdata/TestBelowRaftProtosDontChange/MVCCMetadataSubsetForMergeSerialization b/pkg/kv/kvserver/testdata/TestBelowRaftProtosDontChange/MVCCMetadataSubsetForMergeSerialization index cd042b3d7e72..13e49d12525f 100644 --- a/pkg/kv/kvserver/testdata/TestBelowRaftProtosDontChange/MVCCMetadataSubsetForMergeSerialization +++ b/pkg/kv/kvserver/testdata/TestBelowRaftProtosDontChange/MVCCMetadataSubsetForMergeSerialization @@ -1,3 +1,3 @@ echo ---- -12542053708208219209 +12249944093974449552 diff --git a/pkg/kv/kvserver/testdata/TestBelowRaftProtosDontChange/RangeAppliedState b/pkg/kv/kvserver/testdata/TestBelowRaftProtosDontChange/RangeAppliedState index 064632aeebce..eecc7cd9043d 100644 --- a/pkg/kv/kvserver/testdata/TestBelowRaftProtosDontChange/RangeAppliedState +++ b/pkg/kv/kvserver/testdata/TestBelowRaftProtosDontChange/RangeAppliedState @@ -1,3 +1,3 @@ echo ---- -2796048770313977431 +4296154144683375258 diff --git a/pkg/roachpb/data_test.go b/pkg/roachpb/data_test.go index 32f261f7624c..fa6ca7ff95f8 100644 --- a/pkg/roachpb/data_test.go +++ b/pkg/roachpb/data_test.go @@ -57,14 +57,6 @@ func makeTS(walltime int64, logical int32) hlc.Timestamp { } } -func makeSynTS(walltime int64, logical int32) hlc.Timestamp { - return hlc.Timestamp{ - WallTime: walltime, - Logical: logical, - Synthetic: true, - } -} - func TestKeyClone(t *testing.T) { k := Key{0x01, 0x02, 0x03} c := k.Clone() @@ -551,23 +543,22 @@ var nonZeroTxn = Transaction{ Key: Key("foo"), IsoLevel: isolation.Snapshot, Epoch: 2, - WriteTimestamp: makeSynTS(20, 21), - MinTimestamp: makeSynTS(10, 11), + WriteTimestamp: makeTS(20, 21), + MinTimestamp: makeTS(10, 11), Priority: 957356782, Sequence: 123, CoordinatorNodeID: 3, }, Name: "name", Status: COMMITTED, - LastHeartbeat: makeSynTS(1, 2), - ReadTimestamp: makeSynTS(20, 22), - GlobalUncertaintyLimit: makeSynTS(40, 41), + LastHeartbeat: makeTS(1, 2), + ReadTimestamp: makeTS(20, 22), + GlobalUncertaintyLimit: makeTS(40, 41), ObservedTimestamps: []ObservedTimestamp{{ NodeID: 1, Timestamp: hlc.ClockTimestamp{ - WallTime: 1, - Logical: 2, - Synthetic: true, // normally not set, but needed for zerofields.NoZeroField + WallTime: 1, + Logical: 2, }, }}, WriteTooOld: true, diff --git a/pkg/roachpb/span_config_test.go b/pkg/roachpb/span_config_test.go index 9aea50fbcc2b..e88d9f4bc4e4 100644 --- a/pkg/roachpb/span_config_test.go +++ b/pkg/roachpb/span_config_test.go @@ -25,9 +25,8 @@ func TestSpanConfigHasConfigurationChange(t *testing.T) { TTLSeconds: 10, ProtectionPolicies: []ProtectionPolicy{{ ProtectedTimestamp: hlc.Timestamp{ - WallTime: 1, - Logical: 1, - Synthetic: false, + WallTime: 1, + Logical: 1, }, IgnoreIfExcludedFromBackup: false, }}, @@ -57,9 +56,8 @@ func TestSpanConfigHasConfigurationChange(t *testing.T) { TTLSeconds: 10, ProtectionPolicies: []ProtectionPolicy{{ ProtectedTimestamp: hlc.Timestamp{ - WallTime: 2, - Logical: 2, - Synthetic: false, + WallTime: 2, + Logical: 2, }, IgnoreIfExcludedFromBackup: false, }}, diff --git a/pkg/roachpb/string_test.go b/pkg/roachpb/string_test.go index e10787e572ff..fe910ee32569 100644 --- a/pkg/roachpb/string_test.go +++ b/pkg/roachpb/string_test.go @@ -43,10 +43,10 @@ func TestTransactionString(t *testing.T) { Status: roachpb.COMMITTED, LastHeartbeat: hlc.Timestamp{WallTime: 10, Logical: 11}, ReadTimestamp: hlc.Timestamp{WallTime: 30, Logical: 31}, - GlobalUncertaintyLimit: hlc.Timestamp{WallTime: 40, Logical: 41, Synthetic: true}, + GlobalUncertaintyLimit: hlc.Timestamp{WallTime: 40, Logical: 41}, } expStr := `"name" meta={id=d7aa0f5e key="foo" iso=Serializable 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 gul=0.000000040,41?` + ` lock=true stat=COMMITTED rts=0.000000030,31 wto=false gul=0.000000040,41` if str := txn.String(); str != expStr { t.Errorf( diff --git a/pkg/storage/enginepb/mvcc3_test.go b/pkg/storage/enginepb/mvcc3_test.go index 90b3ab5c5471..565fc3f37b3d 100644 --- a/pkg/storage/enginepb/mvcc3_test.go +++ b/pkg/storage/enginepb/mvcc3_test.go @@ -29,7 +29,7 @@ func TestTxnMetaSizeOf(t *testing.T) { func populatedMVCCValueHeader() MVCCValueHeader { allFieldsSet := MVCCValueHeader{ - LocalTimestamp: hlc.ClockTimestamp{WallTime: 1, Logical: 1, Synthetic: true}, + LocalTimestamp: hlc.ClockTimestamp{WallTime: 1, Logical: 1}, OmitInRangefeeds: true, } allFieldsSet.KVNemesisSeq.Set(123) diff --git a/pkg/storage/mvcc_value.go b/pkg/storage/mvcc_value.go index 0718418b8a12..85c288999f35 100644 --- a/pkg/storage/mvcc_value.go +++ b/pkg/storage/mvcc_value.go @@ -105,17 +105,6 @@ func (v MVCCValue) LocalTimestampNeeded(keyTS hlc.Timestamp) bool { // provided key version timestamp and returned. func (v MVCCValue) GetLocalTimestamp(keyTS hlc.Timestamp) hlc.ClockTimestamp { if v.LocalTimestamp.IsEmpty() { - if keyTS.Synthetic { - // A synthetic version timestamp means that the version timestamp is - // disconnected from real time and did not come from an HLC clock on the - // leaseholder that wrote the value or from somewhere else in the system. - // As a result, the version timestamp cannot be cast to a clock timestamp, - // so we return min_clock_timestamp instead. The effect of this is that - // observed timestamps can not be used to avoid uncertainty retries for - // values without a local timestamp and with a synthetic version - // timestamp. - return hlc.MinClockTimestamp - } return hlc.ClockTimestamp(keyTS) } return v.LocalTimestamp diff --git a/pkg/util/hlc/legacy_timestamp.proto b/pkg/util/hlc/legacy_timestamp.proto index 4bde3653f770..08bd3c575301 100644 --- a/pkg/util/hlc/legacy_timestamp.proto +++ b/pkg/util/hlc/legacy_timestamp.proto @@ -30,15 +30,5 @@ message LegacyTimestamp { // skew)/(minimal ns between events) and nearly impossible to // overflow. optional int32 logical = 2 [(gogoproto.nullable) = false]; - // Indicates that the Timestamp did not come from an HLC clock somewhere - // in the system and, therefore, does not have the ability to update a - // peer's HLC clock. If set to true, the "synthetic timestamp" may be - // arbitrarily disconnected from real time. - // - // See the commentary on Timestamp.synthetic for more information. - // - // The field is nullable so that it is not serialized when set to false. - // This ensures that the timestamp encoding does not change across nodes - // that are and are not aware of this field. - optional bool synthetic = 3; + reserved 3; } diff --git a/pkg/util/hlc/timestamp.go b/pkg/util/hlc/timestamp.go index 83ec08640ad2..4301340d928f 100644 --- a/pkg/util/hlc/timestamp.go +++ b/pkg/util/hlc/timestamp.go @@ -40,6 +40,9 @@ var ( // operator) because it does not consider differences in flags and only // considers whether the walltime and logical time differ between the // timestamps. +// +// TODO(nvanbenschoten): remove this method and its uses. It's now equivalent to +// a direct struct comparison. func (t Timestamp) EqOrdering(s Timestamp) bool { return t.WallTime == s.WallTime && t.Logical == s.Logical } @@ -77,8 +80,7 @@ func (t Timestamp) Compare(s Timestamp) int { } // String implements the fmt.Stringer interface. -// Outputs: seconds.nanos,logical[?] -// ? is added if synthetic is set. +// Outputs: seconds.nanos,logical func (t Timestamp) String() string { // The following code was originally written as // fmt.Sprintf("%d.%09d,%d", t.WallTime/1e9, t.WallTime%1e9, t.Logical). @@ -126,11 +128,6 @@ func (t Timestamp) String() string { zeroBuf[0] = byte('0' + ns%10) } buf = strconv.AppendInt(buf, int64(t.Logical), 10) - - if t.Synthetic { - buf = append(buf, '?') - } - return *(*string)(unsafe.Pointer(&buf)) } @@ -139,12 +136,11 @@ func (Timestamp) SafeValue() {} var ( timestampRegexp = regexp.MustCompile( - `^(?P-)?(?P\d{1,19})(?:\.(?P\d{1,20}))?(?:,(?P-?\d{1,10}))?(?P\?)?$`) - signSubexp = 1 - secsSubexp = 2 - nanosSubexp = 3 - logicalSubexp = 4 - syntheticSubexp = 5 + `^(?P-)?(?P\d{1,19})(?:\.(?P\d{1,20}))?(?:,(?P-?\d{1,10}))?$`) + signSubexp = 1 + secsSubexp = 2 + nanosSubexp = 3 + logicalSubexp = 4 ) // ParseTimestamp attempts to parse the string generated from @@ -181,22 +177,16 @@ func ParseTimestamp(str string) (_ Timestamp, err error) { return Timestamp{}, err } } - synthetic := matches[syntheticSubexp] != "" t := Timestamp{ - WallTime: wallTime, - Logical: int32(logical), - Synthetic: synthetic, + WallTime: wallTime, + Logical: int32(logical), } return t, nil } // AsOfSystemTime returns a string to be used in an AS OF SYSTEM TIME query. func (t Timestamp) AsOfSystemTime() string { - syn := "" - if t.Synthetic { - syn = "?" - } - return fmt.Sprintf("%d.%010d%s", t.WallTime, t.Logical, syn) + return fmt.Sprintf("%d.%010d", t.WallTime, t.Logical) } // IsEmpty returns true if t is an empty Timestamp. @@ -211,39 +201,20 @@ func (t Timestamp) IsSet() bool { return !t.IsEmpty() } -// AddDuration adds a given duration to this Timestamp. The resulting timestamp -// is Synthetic. Normally if you want to bump your clock to the higher of two -// timestamps, use Forward, however this method is here to create a -// hlc.Timestamp in the future (or past). +// AddDuration adds a given duration to this Timestamp. Normally if you want to +// bump your clock to the higher of two timestamps, use Forward, however this +// method is here to create a hlc.Timestamp in the future (or past). func (t Timestamp) AddDuration(duration time.Duration) Timestamp { return t.Add(duration.Nanoseconds(), t.Logical) } // Add returns a timestamp with the WallTime and Logical components increased. // wallTime is expressed in nanos. -// -// TODO(nvanbenschoten): consider an AddNanos method that takes a time.Duration. func (t Timestamp) Add(wallTime int64, logical int32) Timestamp { - s := Timestamp{ - WallTime: t.WallTime + wallTime, - Logical: t.Logical + logical, - Synthetic: t.Synthetic, + return Timestamp{ + WallTime: t.WallTime + wallTime, + Logical: t.Logical + logical, } - // TODO(nvanbenschoten): adding to a timestamp should make it synthetic. - // This breaks a number of tests, so make this change in a separate PR. We - // might also want to wait until we've migrated in the Synthetic flag so we - // don't risk setting it when doing so could cause complications in a mixed - // version cluster. - // - // if t.Less(s) { - // // Adding a positive value to a Timestamp adds the Synthetic flag. - // s.Synthetic = true - // } - // - // When addressing this TODO, remove the hack in - // propBuf.assignClosedTimestampToProposal that manually marks lease - // expirations as synthetic. - return s } // Clone return a new timestamp that has the same contents as the receiver. @@ -258,14 +229,12 @@ func (t Timestamp) Next() Timestamp { panic("cannot take the next value to a max timestamp") } return Timestamp{ - WallTime: t.WallTime + 1, - Synthetic: t.Synthetic, + WallTime: t.WallTime + 1, } } return Timestamp{ - WallTime: t.WallTime, - Logical: t.Logical + 1, - Synthetic: t.Synthetic, + WallTime: t.WallTime, + Logical: t.Logical + 1, } } @@ -273,15 +242,13 @@ func (t Timestamp) Next() Timestamp { func (t Timestamp) Prev() Timestamp { if t.Logical > 0 { return Timestamp{ - WallTime: t.WallTime, - Logical: t.Logical - 1, - Synthetic: t.Synthetic, + WallTime: t.WallTime, + Logical: t.Logical - 1, } } else if t.WallTime > 0 { return Timestamp{ - WallTime: t.WallTime - 1, - Logical: math.MaxInt32, - Synthetic: t.Synthetic, + WallTime: t.WallTime - 1, + Logical: math.MaxInt32, } } panic("cannot take the previous value to a zero timestamp") @@ -293,15 +260,13 @@ func (t Timestamp) Prev() Timestamp { func (t Timestamp) FloorPrev() Timestamp { if t.Logical > 0 { return Timestamp{ - WallTime: t.WallTime, - Logical: t.Logical - 1, - Synthetic: t.Synthetic, + WallTime: t.WallTime, + Logical: t.Logical - 1, } } else if t.WallTime > 0 { return Timestamp{ - WallTime: t.WallTime - 1, - Logical: 0, - Synthetic: t.Synthetic, + WallTime: t.WallTime - 1, + Logical: 0, } } panic("cannot take the previous value to a zero timestamp") @@ -310,18 +275,16 @@ func (t Timestamp) FloorPrev() Timestamp { // WallNext adds 1 to the WallTime and resets Logical. func (t Timestamp) WallNext() Timestamp { return Timestamp{ - WallTime: t.WallTime + 1, - Logical: 0, - Synthetic: t.Synthetic, + WallTime: t.WallTime + 1, + Logical: 0, } } // WallPrev subtracts 1 from the WallTime and resets Logical. func (t Timestamp) WallPrev() Timestamp { return Timestamp{ - WallTime: t.WallTime - 1, - Logical: 0, - Synthetic: t.Synthetic, + WallTime: t.WallTime - 1, + Logical: 0, } } @@ -332,8 +295,6 @@ func (t *Timestamp) Forward(s Timestamp) bool { if t.Less(s) { *t = s return true - } else if t.EqOrdering(s) { - t.Synthetic = bothSynthetic(*t, s) } return false } @@ -341,15 +302,9 @@ func (t *Timestamp) Forward(s Timestamp) bool { // Backward replaces the receiver with the argument, if that moves it backwards // in time. func (t *Timestamp) Backward(s Timestamp) { - syn := bothSynthetic(*t, s) if s.Less(*t) { *t = s } - t.Synthetic = syn -} - -func bothSynthetic(l, r Timestamp) bool { - return l.Synthetic && r.Synthetic } // GoTime converts the timestamp to a time.Time. @@ -357,25 +312,11 @@ func (t Timestamp) GoTime() time.Time { return timeutil.Unix(0, t.WallTime) } -var trueBool = true - // ToLegacyTimestamp converts a Timestamp to a LegacyTimestamp. -func (t Timestamp) ToLegacyTimestamp() LegacyTimestamp { - var synthetic *bool - if t.Synthetic { - synthetic = &trueBool - } - return LegacyTimestamp{WallTime: t.WallTime, Logical: t.Logical, Synthetic: synthetic} -} +func (t Timestamp) ToLegacyTimestamp() LegacyTimestamp { return LegacyTimestamp(t) } // ToTimestamp converts a LegacyTimestamp to a Timestamp. -func (t LegacyTimestamp) ToTimestamp() Timestamp { - var synthetic bool - if t.Synthetic != nil { - synthetic = *t.Synthetic - } - return Timestamp{WallTime: t.WallTime, Logical: t.Logical, Synthetic: synthetic} -} +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 { @@ -400,26 +341,15 @@ func (LegacyTimestamp) SafeValue() {} // timestamp itself is guaranteed to have come from an HLC clock somewhere in // the system. As such, a clock timestamp is a promise that some node in the // system has a clock with a reading equal to or above its value. -// -// ClockTimestamp is the statically typed version of a Timestamp with its -// Synthetic flag set to false. type ClockTimestamp Timestamp // UnsafeToClockTimestamp converts a Timestamp to a ClockTimestamp, regardless -// of whether such a cast would be legal according to the Synthetic flag. The -// method should only be used in tests. -func (t Timestamp) UnsafeToClockTimestamp() ClockTimestamp { - t.Synthetic = false - return ClockTimestamp(t) -} +// of whether the timestamp actually came from a clock. The method should only +// be used in tests. +func (t Timestamp) UnsafeToClockTimestamp() ClockTimestamp { return ClockTimestamp(t) } // ToTimestamp upcasts a ClockTimestamp into a Timestamp. -func (t ClockTimestamp) ToTimestamp() Timestamp { - if t.Synthetic { - panic("ClockTimestamp with Synthetic flag set") - } - return Timestamp(t) -} +func (t ClockTimestamp) ToTimestamp() Timestamp { return Timestamp(t) } // Less returns whether the receiver is less than the parameter. func (t ClockTimestamp) Less(s ClockTimestamp) bool { return Timestamp(t).Less(Timestamp(s)) } diff --git a/pkg/util/hlc/timestamp.proto b/pkg/util/hlc/timestamp.proto index bd3876cfcae4..125a91fad394 100644 --- a/pkg/util/hlc/timestamp.proto +++ b/pkg/util/hlc/timestamp.proto @@ -23,42 +23,10 @@ message Timestamp { // Holds a wall time, typically a unix epoch time expressed in // nanoseconds. - // - // It is not safe to mutate this field directly. Instead, use one of the - // methods on Timestamp, which ensure that the synthetic flag is updated - // appropriately. int64 wall_time = 1; // The logical component captures causality for events whose wall times // are equal. It is effectively bounded by (maximum clock skew)/(minimal // ns between events) and nearly impossible to overflow. - // - // It is not safe to mutate this field directly. Instead, use one of the - // methods on Timestamp, which ensure that the synthetic flag is updated - // appropriately. int32 logical = 2; - // Indicates that the Timestamp did not come from an HLC clock somewhere - // in the system and, therefore, does not have the ability to update a - // peer's HLC clock. If set to true, the "synthetic timestamp" may be - // arbitrarily disconnected from real time. - // - // The flag serves as the dynamically typed version of a ClockTimestamp - // (but inverted). Only Timestamps with this flag set to false can be - // downcast to a ClockTimestamp successfully (see - // DeprecatedTryToClockTimestamp). - // - // Synthetic timestamps with this flag set to true are central to - // non-blocking transactions, which write "into the future". Setting the - // flag to true is also used to disconnect some committed MVCC versions - // from observed timestamps by indicating that those versions were moved - // from the timestamp at which they were originally written. Committed - // MVCC versions with synthetic timestamps require observing the full - // uncertainty interval, whereas readings off the leaseholders's clock - // can tighten the uncertainty interval that is applied to MVCC versions - // with clock timestamp. - // - // This flag does not affect the sort order of Timestamps. However, it - // is considered when performing structural equality checks (e.g. using - // the == operator). Consider use of the EqOrdering method when testing - // for equality. - bool synthetic = 3; + reserved 3; } diff --git a/pkg/util/hlc/timestamp_test.go b/pkg/util/hlc/timestamp_test.go index 23fa3a489a97..1740226b1f00 100644 --- a/pkg/util/hlc/timestamp_test.go +++ b/pkg/util/hlc/timestamp_test.go @@ -23,12 +23,6 @@ func makeTS(walltime int64, logical int32) Timestamp { return Timestamp{WallTime: walltime, Logical: logical} } -func makeSynTS(walltime int64, logical int32) Timestamp { - ts := makeTS(walltime, logical) - ts.Synthetic = true - return ts -} - func TestCompare(t *testing.T) { w0l0 := Timestamp{} w1l1 := Timestamp{WallTime: 1, Logical: 1} @@ -75,10 +69,6 @@ func TestEqOrdering(t *testing.T) { if a.EqOrdering(b) { t.Errorf("expected %+v != %+v", b, a) } - b = makeSynTS(1, 1) - if !a.EqOrdering(b) { - t.Errorf("expected %+v == %+v", b, a) - } } func TestLess(t *testing.T) { @@ -95,10 +85,6 @@ func TestLess(t *testing.T) { if !b.Less(a) { t.Errorf("expected %+v < %+v", b, a) } - b = makeSynTS(1, 1) - if a.Less(b) || b.Less(a) { - t.Errorf("expected %+v == %+v", a, b) - } } func TestLessEq(t *testing.T) { @@ -115,10 +101,6 @@ func TestLessEq(t *testing.T) { if !b.LessEq(a) || a.LessEq(b) { t.Errorf("expected %+v < %+v", b, a) } - b = makeSynTS(1, 1) - if !a.LessEq(b) || !b.LessEq(a) { - t.Errorf("expected %+v == %+v", a, b) - } } func TestIsEmpty(t *testing.T) { @@ -128,11 +110,8 @@ func TestIsEmpty(t *testing.T) { assert.False(t, a.IsEmpty()) a = makeTS(0, 1) assert.False(t, a.IsEmpty()) - a = makeSynTS(0, 0) - assert.False(t, a.IsEmpty()) nonZero := makeTS(1, 1) - nonZero.Synthetic = true require.NoError(t, zerofields.NoZeroField(nonZero), "please update IsEmpty as well") } @@ -144,10 +123,6 @@ func TestTimestampNext(t *testing.T) { {makeTS(1, math.MaxInt32-1), makeTS(1, math.MaxInt32)}, {makeTS(1, math.MaxInt32), makeTS(2, 0)}, {makeTS(math.MaxInt32, math.MaxInt32), makeTS(math.MaxInt32+1, 0)}, - {makeSynTS(1, 2), makeSynTS(1, 3)}, - {makeSynTS(1, math.MaxInt32-1), makeSynTS(1, math.MaxInt32)}, - {makeSynTS(1, math.MaxInt32), makeSynTS(2, 0)}, - {makeSynTS(math.MaxInt32, math.MaxInt32), makeSynTS(math.MaxInt32+1, 0)}, } for _, c := range testCases { assert.Equal(t, c.expNext, c.ts.Next()) @@ -162,7 +137,6 @@ func TestTimestampWallNext(t *testing.T) { {makeTS(1, 2), makeTS(2, 0)}, {makeTS(1, 1), makeTS(2, 0)}, {makeTS(1, 0), makeTS(2, 0)}, - {makeSynTS(1, 2), makeSynTS(2, 0)}, } for _, c := range testCases { assert.Equal(t, c.expWallNext, c.ts.WallNext()) @@ -176,9 +150,6 @@ func TestTimestampPrev(t *testing.T) { {makeTS(1, 2), makeTS(1, 1)}, {makeTS(1, 1), makeTS(1, 0)}, {makeTS(1, 0), makeTS(0, math.MaxInt32)}, - {makeSynTS(1, 2), makeSynTS(1, 1)}, - {makeSynTS(1, 1), makeSynTS(1, 0)}, - {makeSynTS(1, 0), makeSynTS(0, math.MaxInt32)}, } for _, c := range testCases { assert.Equal(t, c.expPrev, c.ts.Prev()) @@ -193,10 +164,6 @@ func TestTimestampFloorPrevWallPrev(t *testing.T) { {makeTS(1, 2), makeTS(1, 1), makeTS(0, 0)}, {makeTS(1, 1), makeTS(1, 0), makeTS(0, 0)}, {makeTS(1, 0), makeTS(0, 0), makeTS(0, 0)}, - {makeSynTS(2, 0), makeSynTS(1, 0), makeSynTS(1, 0)}, - {makeSynTS(1, 2), makeSynTS(1, 1), makeSynTS(0, 0)}, - {makeSynTS(1, 1), makeSynTS(1, 0), makeSynTS(0, 0)}, - {makeSynTS(1, 0), makeSynTS(0, 0), makeSynTS(0, 0)}, } for _, c := range testCases { assert.Equal(t, c.expPrev, c.ts.FloorPrev()) @@ -215,21 +182,6 @@ func TestTimestampForward(t *testing.T) { {makeTS(2, 0), makeTS(2, 0), makeTS(2, 0), false}, {makeTS(2, 0), makeTS(2, 1), makeTS(2, 1), true}, {makeTS(2, 0), makeTS(3, 0), makeTS(3, 0), true}, - {makeSynTS(2, 0), makeTS(1, 0), makeSynTS(2, 0), false}, - {makeSynTS(2, 0), makeTS(1, 1), makeSynTS(2, 0), false}, - {makeSynTS(2, 0), makeTS(2, 0), makeTS(2, 0), false}, - {makeSynTS(2, 0), makeTS(2, 1), makeTS(2, 1), true}, - {makeSynTS(2, 0), makeTS(3, 0), makeTS(3, 0), true}, - {makeTS(2, 0), makeSynTS(1, 0), makeTS(2, 0), false}, - {makeTS(2, 0), makeSynTS(1, 1), makeTS(2, 0), false}, - {makeTS(2, 0), makeSynTS(2, 0), makeTS(2, 0), false}, - {makeTS(2, 0), makeSynTS(2, 1), makeSynTS(2, 1), true}, - {makeTS(2, 0), makeSynTS(3, 0), makeSynTS(3, 0), true}, - {makeSynTS(2, 0), makeSynTS(1, 0), makeSynTS(2, 0), false}, - {makeSynTS(2, 0), makeSynTS(1, 1), makeSynTS(2, 0), false}, - {makeSynTS(2, 0), makeSynTS(2, 0), makeSynTS(2, 0), false}, - {makeSynTS(2, 0), makeSynTS(2, 1), makeSynTS(2, 1), true}, - {makeSynTS(2, 0), makeSynTS(3, 0), makeSynTS(3, 0), true}, } for _, c := range testCases { ts := c.ts @@ -247,21 +199,6 @@ func TestTimestampBackward(t *testing.T) { {makeTS(2, 0), makeTS(2, 0), makeTS(2, 0)}, {makeTS(2, 0), makeTS(2, 1), makeTS(2, 0)}, {makeTS(2, 0), makeTS(3, 0), makeTS(2, 0)}, - {makeSynTS(2, 0), makeTS(1, 0), makeTS(1, 0)}, - {makeSynTS(2, 0), makeTS(1, 1), makeTS(1, 1)}, - {makeSynTS(2, 0), makeTS(2, 0), makeTS(2, 0)}, - {makeSynTS(2, 0), makeTS(2, 1), makeTS(2, 0)}, - {makeSynTS(2, 0), makeTS(3, 0), makeTS(2, 0)}, - {makeTS(2, 0), makeSynTS(1, 0), makeTS(1, 0)}, - {makeTS(2, 0), makeSynTS(1, 1), makeTS(1, 1)}, - {makeTS(2, 0), makeSynTS(2, 0), makeTS(2, 0)}, - {makeTS(2, 0), makeSynTS(2, 1), makeTS(2, 0)}, - {makeTS(2, 0), makeSynTS(3, 0), makeTS(2, 0)}, - {makeSynTS(2, 0), makeSynTS(1, 0), makeSynTS(1, 0)}, - {makeSynTS(2, 0), makeSynTS(1, 1), makeSynTS(1, 1)}, - {makeSynTS(2, 0), makeSynTS(2, 0), makeSynTS(2, 0)}, - {makeSynTS(2, 0), makeSynTS(2, 1), makeSynTS(2, 0)}, - {makeSynTS(2, 0), makeSynTS(3, 0), makeSynTS(2, 0)}, } for _, c := range testCases { ts := c.ts @@ -278,9 +215,6 @@ func TestAsOfSystemTime(t *testing.T) { {makeTS(145, 0), "145.0000000000"}, {makeTS(145, 123), "145.0000000123"}, {makeTS(145, 1123456789), "145.1123456789"}, - {makeSynTS(145, 0), "145.0000000000?"}, - {makeSynTS(145, 123), "145.0000000123?"}, - {makeSynTS(145, 1123456789), "145.1123456789?"}, } for _, c := range testCases { assert.Equal(t, c.exp, c.ts.AsOfSystemTime()) @@ -307,9 +241,6 @@ func TestTimestampFormatParseRoundTrip(t *testing.T) { {makeTS(-1234567890, 0), "-1.234567890,0"}, {makeTS(6661234567890, 0), "6661.234567890,0"}, {makeTS(-6661234567890, 0), "-6661.234567890,0"}, - {makeSynTS(0, 123), "0,123?"}, - {makeSynTS(1, 0), "0.000000001,0?"}, - {makeSynTS(1, 123), "0.000000001,123?"}, } for _, c := range testCases { str := c.ts.String() @@ -345,10 +276,6 @@ func TestTimestampParseFormatNonRoundTrip(t *testing.T) { // Other cases. {"0.000000001", makeTS(1, 0), "0.000000001,0"}, {"99.000000001", makeTS(99000000001, 0), "99.000000001,0"}, - {"0?", makeSynTS(0, 0), "0,0?"}, - {"99?", makeSynTS(99000000000, 0), "99.000000000,0?"}, - {"0.000000001?", makeSynTS(1, 0), "0.000000001,0?"}, - {"99.000000001?", makeSynTS(99000000001, 0), "99.000000001,0?"}, } for _, c := range testCases { parsed, err := ParseTimestamp(c.s) @@ -405,19 +332,11 @@ func BenchmarkTimestampString(b *testing.B) { } } -func BenchmarkTimestampStringSynthetic(b *testing.B) { - ts := makeSynTS(-6661234567890, 0) - - for i := 0; i < b.N; i++ { - _ = ts.String() - } -} - func BenchmarkTimestampIsEmpty(b *testing.B) { cases := map[string]Timestamp{ "empty": {}, "walltime": {WallTime: 1664364012528805328}, - "all": {WallTime: 1664364012528805328, Logical: 65535, Synthetic: true}, + "all": {WallTime: 1664364012528805328, Logical: 65535}, } var result bool