-
Notifications
You must be signed in to change notification settings - Fork 3.9k
/
Copy pathclock_offset.go
280 lines (251 loc) · 9.33 KB
/
clock_offset.go
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
// Copyright 2014 The Cockroach Authors.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
// implied. See the License for the specific language governing
// permissions and limitations under the License.
package rpc
import (
"context"
"math"
"time"
"github.com/VividCortex/ewma"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/metric"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/montanaflynn/stats"
"github.com/pkg/errors"
)
// RemoteClockMetrics is the collection of metrics for the clock monitor.
type RemoteClockMetrics struct {
ClockOffsetMeanNanos *metric.Gauge
ClockOffsetStdDevNanos *metric.Gauge
LatencyHistogramNanos *metric.Histogram
}
// avgLatencyMeasurementAge determines how to exponentially weight the
// moving average of latency measurements. This means that the weight
// will center around the 20th oldest measurement, such that for measurements
// that are made every 3 seconds, the average measurement will be about one
// minute old.
const avgLatencyMeasurementAge = 20.0
var (
metaClockOffsetMeanNanos = metric.Metadata{
Name: "clock-offset.meannanos",
Help: "Mean clock offset with other nodes",
Measurement: "Clock Offset",
Unit: metric.Unit_NANOSECONDS,
}
metaClockOffsetStdDevNanos = metric.Metadata{
Name: "clock-offset.stddevnanos",
Help: "Stdddev clock offset with other nodes",
Measurement: "Clock Offset",
Unit: metric.Unit_NANOSECONDS,
}
metaLatencyHistogramNanos = metric.Metadata{
Name: "round-trip-latency",
Help: "Distribution of round-trip latencies with other nodes",
Measurement: "Roundtrip Latency",
Unit: metric.Unit_NANOSECONDS,
}
)
// RemoteClockMonitor keeps track of the most recent measurements of remote
// offsets and round-trip latency from this node to connected nodes.
type RemoteClockMonitor struct {
clock *hlc.Clock
offsetTTL time.Duration
mu struct {
syncutil.Mutex
offsets map[string]RemoteOffset
latenciesNanos map[string]ewma.MovingAverage
}
metrics RemoteClockMetrics
}
// newRemoteClockMonitor returns a monitor with the given server clock.
func newRemoteClockMonitor(
clock *hlc.Clock, offsetTTL time.Duration, histogramWindowInterval time.Duration,
) *RemoteClockMonitor {
r := RemoteClockMonitor{
clock: clock,
offsetTTL: offsetTTL,
}
r.mu.offsets = make(map[string]RemoteOffset)
r.mu.latenciesNanos = make(map[string]ewma.MovingAverage)
if histogramWindowInterval == 0 {
histogramWindowInterval = time.Duration(math.MaxInt64)
}
r.metrics = RemoteClockMetrics{
ClockOffsetMeanNanos: metric.NewGauge(metaClockOffsetMeanNanos),
ClockOffsetStdDevNanos: metric.NewGauge(metaClockOffsetStdDevNanos),
LatencyHistogramNanos: metric.NewLatency(metaLatencyHistogramNanos, histogramWindowInterval),
}
return &r
}
// Metrics returns the metrics struct. Useful to examine individual metrics,
// or to add to the registry.
func (r *RemoteClockMonitor) Metrics() *RemoteClockMetrics {
return &r.metrics
}
// Latency returns the exponentially weighted moving average latency to the
// given node address. Returns true if the measurement is valid, or false if
// we don't have enough samples to compute a reliable average.
func (r *RemoteClockMonitor) Latency(addr string) (time.Duration, bool) {
r.mu.Lock()
defer r.mu.Unlock()
if avg, ok := r.mu.latenciesNanos[addr]; ok && avg.Value() != 0.0 {
return time.Duration(int64(avg.Value())), true
}
return 0, false
}
// AllLatencies returns a map of all currently valid latency measurements.
func (r *RemoteClockMonitor) AllLatencies() map[string]time.Duration {
r.mu.Lock()
defer r.mu.Unlock()
result := make(map[string]time.Duration)
for addr, avg := range r.mu.latenciesNanos {
if avg.Value() != 0.0 {
result[addr] = time.Duration(int64(avg.Value()))
}
}
return result
}
// UpdateOffset is a thread-safe way to update the remote clock and latency
// measurements.
//
// It only updates the offset for addr if one of the following cases holds:
// 1. There is no prior offset for that address.
// 2. The old offset for addr was measured long enough ago to be considered
// stale.
// 3. The new offset's error is smaller than the old offset's error.
//
// Pass a roundTripLatency of 0 or less to avoid recording the latency.
func (r *RemoteClockMonitor) UpdateOffset(
ctx context.Context, addr string, offset RemoteOffset, roundTripLatency time.Duration,
) {
emptyOffset := offset == RemoteOffset{}
r.mu.Lock()
defer r.mu.Unlock()
if oldOffset, ok := r.mu.offsets[addr]; !ok {
// We don't have a measurement - if the incoming measurement is not empty,
// set it.
if !emptyOffset {
r.mu.offsets[addr] = offset
}
} else if oldOffset.isStale(r.offsetTTL, r.clock.PhysicalTime()) {
// We have a measurement but it's old - if the incoming measurement is not empty,
// set it, otherwise delete the old measurement.
if !emptyOffset {
r.mu.offsets[addr] = offset
} else {
delete(r.mu.offsets, addr)
}
} else if offset.Uncertainty < oldOffset.Uncertainty {
// We have a measurement but its uncertainty is greater than that of the
// incoming measurement - if the incoming measurement is not empty, set it.
if !emptyOffset {
r.mu.offsets[addr] = offset
}
}
if roundTripLatency > 0 {
latencyAvg, ok := r.mu.latenciesNanos[addr]
if !ok {
latencyAvg = ewma.NewMovingAverage(avgLatencyMeasurementAge)
r.mu.latenciesNanos[addr] = latencyAvg
}
latencyAvg.Add(float64(roundTripLatency.Nanoseconds()))
r.metrics.LatencyHistogramNanos.RecordValue(roundTripLatency.Nanoseconds())
}
if log.V(2) {
log.Infof(ctx, "update offset: %s %v", addr, r.mu.offsets[addr])
}
}
// VerifyClockOffset calculates the number of nodes to which the known offset
// is healthy (as defined by RemoteOffset.isHealthy). It returns nil iff more
// than half the known offsets are healthy, and an error otherwise. A non-nil
// return indicates that this node's clock is unreliable, and that the node
// should terminate.
func (r *RemoteClockMonitor) VerifyClockOffset(ctx context.Context) error {
// By the contract of the hlc, if the value is 0, then safety checking of
// the max offset is disabled. However we may still want to propagate the
// information to a status node.
//
// TODO(tschottdorf): disallow maxOffset == 0 but probably lots of tests to
// fix.
if maxOffset := r.clock.MaxOffset(); maxOffset != 0 && maxOffset != timeutil.ClocklessMaxOffset {
now := r.clock.PhysicalTime()
healthyOffsetCount := 0
r.mu.Lock()
// Each measurement is recorded as its minimum and maximum value.
offsets := make(stats.Float64Data, 0, 2*len(r.mu.offsets))
for addr, offset := range r.mu.offsets {
if offset.isStale(r.offsetTTL, now) {
delete(r.mu.offsets, addr)
continue
}
offsets = append(offsets, float64(offset.Offset+offset.Uncertainty))
offsets = append(offsets, float64(offset.Offset-offset.Uncertainty))
if offset.isHealthy(ctx, maxOffset) {
healthyOffsetCount++
}
}
numClocks := len(r.mu.offsets)
r.mu.Unlock()
mean, err := offsets.Mean()
if err != nil && err != stats.EmptyInput {
return err
}
stdDev, err := offsets.StandardDeviation()
if err != nil && err != stats.EmptyInput {
return err
}
r.metrics.ClockOffsetMeanNanos.Update(int64(mean))
r.metrics.ClockOffsetStdDevNanos.Update(int64(stdDev))
if numClocks > 0 && healthyOffsetCount <= numClocks/2 {
return errors.Errorf(
"clock synchronization error: this node is more than %s away from at least half of the known nodes (%d of %d are within the offset)",
maxOffset, healthyOffsetCount, numClocks)
}
if log.V(1) {
log.Infof(ctx, "%d of %d nodes are within the maximum clock offset of %s", healthyOffsetCount, numClocks, maxOffset)
}
}
return nil
}
func (r RemoteOffset) isHealthy(ctx context.Context, maxOffset time.Duration) bool {
// Tolerate up to 80% of the maximum offset.
toleratedOffset := maxOffset * 4 / 5
// Offset may be negative, but Uncertainty is always positive.
absOffset := r.Offset
if absOffset < 0 {
absOffset = -absOffset
}
switch {
case time.Duration(absOffset-r.Uncertainty)*time.Nanosecond > toleratedOffset:
// The minimum possible true offset exceeds the maximum offset; definitely
// unhealthy.
return false
case time.Duration(absOffset+r.Uncertainty)*time.Nanosecond < toleratedOffset:
// The maximum possible true offset does not exceed the maximum offset;
// definitely healthy.
return true
default:
// The maximum offset is in the uncertainty window of the measured offset;
// health is ambiguous. For now, we err on the side of not spuriously
// killing nodes.
if log.V(1) {
log.Infof(ctx, "uncertain remote offset %s for maximum tolerated offset %s, treating as healthy", r, toleratedOffset)
}
return true
}
}
func (r RemoteOffset) isStale(ttl time.Duration, now time.Time) bool {
return r.measuredAt().Add(ttl).Before(now)
}