github.com/matrixorigin/matrixone@v1.2.0/pkg/txn/clock/hlc.go (about) 1 // Copyright 2022 Matrix Origin 2 // 3 // Licensed under the Apache License, Version 2.0 (the "License"); 4 // you may not use this file except in compliance with the License. 5 // You may obtain a copy of the License at 6 // 7 // http://www.apache.org/licenses/LICENSE-2.0 8 // 9 // Unless required by applicable law or agreed to in writing, software 10 // distributed under the License is distributed on an "AS IS" BASIS, 11 // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 12 // See the License for the specific language governing permissions and 13 // limitations under the License. 14 15 package clock 16 17 import ( 18 "context" 19 "sync" 20 "time" 21 22 "github.com/matrixorigin/matrixone/pkg/common/stopper" 23 "github.com/matrixorigin/matrixone/pkg/logutil" 24 "github.com/matrixorigin/matrixone/pkg/pb/timestamp" 25 ) 26 27 func toMicrosecond(nanoseconds int64) int64 { 28 return nanoseconds / 1000 29 } 30 31 func physicalClock() int64 { 32 return time.Now().UTC().UnixNano() 33 } 34 35 // SkipClockUncertainityPeriodOnRestart will cause the current goroutine to 36 // sleep to skip clock uncertainty period. This function must be called during 37 // the restart of the node. 38 // 39 // The system assumes that each node keeps synchronizing with accurate NTP 40 // servers to have the clock offset limited under HLCClock.maxOffset. 41 func SkipClockUncertainityPeriodOnRestart(ctx context.Context, clock Clock) { 42 now, sleepUntil := clock.Now() 43 for now.Less(sleepUntil) { 44 time.Sleep(time.Millisecond) 45 select { 46 case <-ctx.Done(): 47 return 48 default: 49 } 50 51 now, _ = clock.Now() 52 } 53 } 54 55 // HLCClock is an implementation of the Hybrid Logical Clock as described in 56 // the paper titled - 57 // 58 // Logical Physical Clocks and Consistent Snapshots in Globally Distributed 59 // Databases 60 type HLCClock struct { 61 nodeID uint16 62 maxOffset time.Duration 63 physicalClock func() int64 64 65 mu struct { 66 sync.Mutex 67 // maxLearnedPhysicalTime is the max learned physical time as defined in 68 // section 3.3 of the HLC paper. 69 maxLearnedPhysicalTime int64 70 // ts records the last HLC timestamp returned by the Now() method. 71 ts timestamp.Timestamp 72 } 73 } 74 75 var _ Clock = (*HLCClock)(nil) 76 77 // NewHLCClock returns a new HLCClock instance. The maxOffset parameter specifies 78 // the max allowed clock offset. The clock readings returned by clock must be 79 // within the maxOffset bound across the entire cluster. 80 func NewHLCClock(clock func() int64, maxOffset time.Duration) *HLCClock { 81 if clock == nil { 82 panic("physical clock source not specified") 83 } 84 85 return &HLCClock{ 86 physicalClock: clock, 87 maxOffset: maxOffset, 88 } 89 } 90 91 // NewUnixNanoHLCClock returns a new HLCClock instance backed by the local wall 92 // clock in Unix epoch nanoseconds. Nodes' clock must be periodically 93 // synchronized, e.g. via NTP, to ensure that wall time readings are within the 94 // maxOffset offset between any two nodes. 95 func NewUnixNanoHLCClock(ctx context.Context, maxOffset time.Duration) *HLCClock { 96 clock := &HLCClock{ 97 physicalClock: physicalClock, 98 maxOffset: maxOffset, 99 } 100 101 go func() { 102 clock.offsetMonitor(ctx) 103 }() 104 105 return clock 106 } 107 108 // NewUnixNanoHLCClockWithStopper is similar to NewUnixNanoHLCClock, but perform 109 // clock check use stopper 110 func NewUnixNanoHLCClockWithStopper(stopper *stopper.Stopper, maxOffset time.Duration) *HLCClock { 111 clock := &HLCClock{ 112 physicalClock: physicalClock, 113 maxOffset: maxOffset, 114 } 115 if maxOffset > 0 { 116 if err := stopper.RunTask(clock.offsetMonitor); err != nil { 117 panic(err) 118 } 119 } 120 return clock 121 } 122 123 // HasNetworkLatency returns a boolean value indicating whether there is network 124 // latency involved when retrieving timestamps. There is no such network latency 125 // in HLCClock. 126 func (c *HLCClock) HasNetworkLatency() bool { 127 return false 128 } 129 130 // MaxOffset returns the max offset of the physical clocks in the cluster. 131 func (c *HLCClock) MaxOffset() time.Duration { 132 return c.maxOffset 133 } 134 135 // Now returns the current HLC timestamp and the upper bound timestamp of the 136 // current time in hlc. 137 func (c *HLCClock) Now() (timestamp.Timestamp, timestamp.Timestamp) { 138 now := c.now() 139 return now, timestamp.Timestamp{PhysicalTime: now.PhysicalTime + int64(c.maxOffset), NodeID: now.NodeID} 140 } 141 142 // Update is called whenever messages are received from other nodes. HLC 143 // timestamp carried by those messages are used to update the local HLC 144 // clock to capture casual relationships. 145 func (c *HLCClock) Update(m timestamp.Timestamp) { 146 c.update(m) 147 } 148 149 func (c *HLCClock) maxClockForwardOffset() time.Duration { 150 return c.maxOffset / 2 151 } 152 153 func (c *HLCClock) clockOffsetMonitoringInterval() time.Duration { 154 return c.maxClockForwardOffset() / 3 155 } 156 157 func (c *HLCClock) offsetMonitor(ctx context.Context) { 158 c.getPhysicalClock() 159 ticker := time.NewTicker(c.clockOffsetMonitoringInterval()) 160 defer ticker.Stop() 161 for { 162 select { 163 case <-ticker.C: 164 c.getPhysicalClock() 165 case <-ctx.Done(): 166 return 167 } 168 } 169 } 170 171 func (c *HLCClock) getPhysicalClock() int64 { 172 newPts := c.physicalClock() 173 oldPts := c.keepPhysicalClock(newPts) 174 if c.maxOffset > 0 { 175 c.handleClockJump(oldPts, newPts) 176 } 177 178 return newPts 179 } 180 181 // keepPhysicalClock updates the c.mu.maxLearnedPhysicalTime field when 182 // necessary and returns the previous maxLearnedPhysicalTime value. 183 func (c *HLCClock) keepPhysicalClock(pts int64) int64 { 184 c.mu.Lock() 185 defer c.mu.Unlock() 186 187 old := c.mu.maxLearnedPhysicalTime 188 if pts > c.mu.maxLearnedPhysicalTime { 189 c.mu.maxLearnedPhysicalTime = pts 190 } 191 192 return old 193 } 194 195 // handleClockJump is called everytime when a physical time is read from the 196 // local wall clock. it logs backward jump whenever it happens, jumps are 197 // compared with maxClockForwardOffset() and cause fatal fail when the 198 // maxClockForwardOffset constrain is violated. 199 // 200 // handleClockJump assumes that there are periodic background probes to the 201 // physical wall clock to monitor and learn its values. the probe interval 202 // is required to be smaller than the maxClockForwardOffset(). 203 func (c *HLCClock) handleClockJump(oldPts int64, newPts int64) { 204 if oldPts == 0 { 205 return 206 } 207 208 jump := int64(0) 209 if oldPts > newPts { 210 jump = oldPts - newPts 211 logutil.Infof("clock backward jump observed, %d microseconds", toMicrosecond(jump)) 212 } else { 213 jump = newPts - oldPts 214 } 215 216 if jump > int64(c.maxClockForwardOffset()+c.clockOffsetMonitoringInterval()) { 217 logutil.Fatalf("big clock jump observed, %d microseconds", toMicrosecond(jump)) 218 } 219 } 220 221 // now returns the current HLC timestamp. it implements the Send or Local event 222 // part of Figure 5 of the HLC paper. 223 func (c *HLCClock) now() timestamp.Timestamp { 224 newPts := c.getPhysicalClock() 225 c.mu.Lock() 226 defer c.mu.Unlock() 227 if c.mu.ts.PhysicalTime >= newPts { 228 c.mu.ts.LogicalTime++ 229 } else { 230 c.mu.ts = timestamp.Timestamp{PhysicalTime: newPts} 231 } 232 c.mu.ts.NodeID = uint32(c.nodeID) 233 return c.mu.ts 234 } 235 236 // update updates the HLCClock based on the received Timestamp, it implements 237 // the Receive Event of message m part of Figure 5 of the HLC paper. 238 func (c *HLCClock) update(m timestamp.Timestamp) { 239 newPts := c.getPhysicalClock() 240 241 c.mu.Lock() 242 defer c.mu.Unlock() 243 if newPts > c.mu.ts.PhysicalTime && newPts > m.PhysicalTime { 244 // local wall time is the max 245 // keep the physical time, reset the logical time 246 c.mu.ts = timestamp.Timestamp{PhysicalTime: newPts} 247 } else if m.PhysicalTime == c.mu.ts.PhysicalTime { 248 if m.LogicalTime > c.mu.ts.LogicalTime { 249 // received physical time is equal to the local physical time, and it 250 // also has a larger logical time keep m's logical time. 251 c.mu.ts.LogicalTime = m.LogicalTime 252 } 253 } else if m.PhysicalTime > c.mu.ts.PhysicalTime { 254 c.mu.ts.PhysicalTime = m.PhysicalTime 255 c.mu.ts.LogicalTime = m.LogicalTime 256 } 257 } 258 259 func (c *HLCClock) SetNodeID(id uint16) { 260 c.nodeID = id 261 }