-
Notifications
You must be signed in to change notification settings - Fork 451
/
aggregator.go
1091 lines (984 loc) · 31.8 KB
/
aggregator.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
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
// Copyright (c) 2016 Uber Technologies, Inc.
//
// Permission is hereby granted, free of charge, to any person obtaining a copy
// of this software and associated documentation files (the "Software"), to deal
// in the Software without restriction, including without limitation the rights
// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
// copies of the Software, and to permit persons to whom the Software is
// furnished to do so, subject to the following conditions:
//
// The above copyright notice and this permission notice shall be included in
// all copies or substantial portions of the Software.
//
// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
// THE SOFTWARE.
package aggregator
import (
"context"
"errors"
"math"
"strconv"
"sync"
"sync/atomic"
"time"
"github.com/m3db/m3/src/aggregator/aggregator/handler"
"github.com/m3db/m3/src/aggregator/aggregator/handler/writer"
"github.com/m3db/m3/src/aggregator/client"
"github.com/m3db/m3/src/aggregator/sharding"
"github.com/m3db/m3/src/cluster/placement"
"github.com/m3db/m3/src/cluster/shard"
"github.com/m3db/m3/src/metrics/metadata"
"github.com/m3db/m3/src/metrics/metric"
"github.com/m3db/m3/src/metrics/metric/aggregated"
"github.com/m3db/m3/src/metrics/metric/id"
"github.com/m3db/m3/src/metrics/metric/unaggregated"
"github.com/m3db/m3/src/metrics/policy"
"github.com/m3db/m3/src/x/clock"
xerrors "github.com/m3db/m3/src/x/errors"
"github.com/m3db/m3/src/x/instrument"
"github.com/uber-go/tally"
"go.uber.org/zap"
)
const (
uninitializedCutoverNanos = math.MinInt64
uninitializedShardSetID = 0
)
var (
errAggregatorNotOpenOrClosed = errors.New("aggregator is not open or closed")
errAggregatorAlreadyOpenOrClosed = errors.New("aggregator is already open or closed")
errInvalidMetricType = errors.New("invalid metric type")
errActivePlacementChanged = errors.New("active placement has changed")
errShardNotOwned = errors.New("aggregator shard is not owned")
)
// Aggregator aggregates different types of metrics.
type Aggregator interface {
// Open opens the aggregator.
Open() error
// AddUntimed adds an untimed metric with staged metadatas.
AddUntimed(metric unaggregated.MetricUnion, metas metadata.StagedMetadatas) error
// AddTimed adds a timed metric with metadata.
AddTimed(metric aggregated.Metric, metadata metadata.TimedMetadata) error
// AddTimedWithStagedMetadatas adds a timed metric with staged metadatas.
AddTimedWithStagedMetadatas(metric aggregated.Metric, metas metadata.StagedMetadatas) error
// AddForwarded adds a forwarded metric with metadata.
AddForwarded(metric aggregated.ForwardedMetric, metadata metadata.ForwardMetadata) error
// AddPassthrough adds a passthrough metric with storage policy.
AddPassthrough(metric aggregated.Metric, storagePolicy policy.StoragePolicy) error
// Resign stops the aggregator from participating in leader election and resigns
// from ongoing campaign if any.
Resign() error
// Status returns the run-time status of the aggregator.
Status() RuntimeStatus
// Close closes the aggregator.
Close() error
}
// aggregator stores aggregations of different types of metrics (e.g., counter,
// timer, gauges) and periodically flushes them out.
type aggregator struct {
sync.RWMutex
opts Options
nowFn clock.NowFn
shardFn sharding.ShardFn
checkInterval time.Duration
placementManager PlacementManager
flushTimesManager FlushTimesManager
flushTimesChecker flushTimesChecker
electionManager ElectionManager
flushManager FlushManager
flushHandler handler.Handler
passthroughWriter writer.Writer
adminClient client.AdminClient
resignTimeout time.Duration
shardSetID uint32
shardSetOpen bool
shardIDs []uint32
shards []*aggregatorShard
currStagedPlacement placement.ActiveStagedPlacement
currPlacement placement.Placement
state aggregatorState
doneCh chan struct{}
wg sync.WaitGroup
sleepFn sleepFn
shardsPendingClose int32
metrics aggregatorMetrics
logger *zap.Logger
}
// NewAggregator creates a new aggregator.
func NewAggregator(opts Options) Aggregator {
iOpts := opts.InstrumentOptions()
scope := iOpts.MetricsScope()
timerOpts := iOpts.TimerOptions()
return &aggregator{
opts: opts,
nowFn: opts.ClockOptions().NowFn(),
shardFn: opts.ShardFn(),
checkInterval: opts.EntryCheckInterval(),
placementManager: opts.PlacementManager(),
flushTimesManager: opts.FlushTimesManager(),
flushTimesChecker: newFlushTimesChecker(scope.SubScope("tick.shard-check")),
electionManager: opts.ElectionManager(),
flushManager: opts.FlushManager(),
flushHandler: opts.FlushHandler(),
passthroughWriter: opts.PassthroughWriter(),
adminClient: opts.AdminClient(),
resignTimeout: opts.ResignTimeout(),
doneCh: make(chan struct{}),
sleepFn: time.Sleep,
metrics: newAggregatorMetrics(scope, timerOpts, opts.MaxAllowedForwardingDelayFn()),
logger: iOpts.Logger(),
}
}
func (agg *aggregator) Open() error {
agg.Lock()
defer agg.Unlock()
if agg.state != aggregatorNotOpen {
return errAggregatorAlreadyOpenOrClosed
}
if err := agg.placementManager.Open(); err != nil {
return err
}
stagedPlacement, placement, err := agg.placementManager.Placement()
if err != nil {
return err
}
if err := agg.processPlacementWithLock(stagedPlacement, placement); err != nil {
return err
}
if agg.checkInterval > 0 {
agg.wg.Add(1)
go agg.tick()
}
agg.state = aggregatorOpen
return nil
}
func (agg *aggregator) AddUntimed(
metric unaggregated.MetricUnion,
metadatas metadata.StagedMetadatas,
) error {
callStart := agg.nowFn()
if err := agg.checkMetricType(metric); err != nil {
agg.metrics.addUntimed.ReportError(err)
return err
}
shard, err := agg.shardFor(metric.ID)
if err != nil {
agg.metrics.addUntimed.ReportError(err)
return err
}
if err = shard.AddUntimed(metric, metadatas); err != nil {
agg.metrics.addUntimed.ReportError(err)
return err
}
agg.metrics.addUntimed.ReportSuccess(agg.nowFn().Sub(callStart))
return nil
}
func (agg *aggregator) AddTimed(
metric aggregated.Metric,
metadata metadata.TimedMetadata,
) error {
callStart := agg.nowFn()
agg.metrics.timed.Inc(1)
shard, err := agg.shardFor(metric.ID)
if err != nil {
agg.metrics.addTimed.ReportError(err)
return err
}
if err = shard.AddTimed(metric, metadata); err != nil {
agg.metrics.addTimed.ReportError(err)
return err
}
agg.metrics.addTimed.ReportSuccess(agg.nowFn().Sub(callStart))
return nil
}
func (agg *aggregator) AddTimedWithStagedMetadatas(
metric aggregated.Metric,
metas metadata.StagedMetadatas,
) error {
callStart := agg.nowFn()
agg.metrics.timed.Inc(1)
shard, err := agg.shardFor(metric.ID)
if err != nil {
agg.metrics.addTimed.ReportError(err)
return err
}
if err = shard.AddTimedWithStagedMetadatas(metric, metas); err != nil {
agg.metrics.addTimed.ReportError(err)
return err
}
agg.metrics.addTimed.ReportSuccess(agg.nowFn().Sub(callStart))
return nil
}
func (agg *aggregator) AddForwarded(
metric aggregated.ForwardedMetric,
metadata metadata.ForwardMetadata,
) error {
callStart := agg.nowFn()
agg.metrics.forwarded.Inc(1)
shard, err := agg.shardFor(metric.ID)
if err != nil {
agg.metrics.addForwarded.ReportError(err)
return err
}
if err = shard.AddForwarded(metric, metadata); err != nil {
agg.metrics.addForwarded.ReportError(err)
return err
}
callEnd := agg.nowFn()
agg.metrics.addForwarded.ReportSuccess(callEnd.Sub(callStart))
forwardingDelay := time.Duration(callEnd.UnixNano() - metric.TimeNanos)
agg.metrics.addForwarded.ReportForwardingLatency(
metadata.StoragePolicy.Resolution().Window,
metadata.NumForwardedTimes,
forwardingDelay,
)
return nil
}
func (agg *aggregator) AddPassthrough(
metric aggregated.Metric,
storagePolicy policy.StoragePolicy,
) error {
callStart := agg.nowFn()
agg.metrics.passthrough.Inc(1)
if agg.electionManager.ElectionState() == FollowerState {
agg.metrics.addPassthrough.ReportFollowerNoop()
return nil
}
pw, err := agg.passWriter()
if err != nil {
agg.metrics.addPassthrough.ReportError(err)
return err
}
mp := aggregated.ChunkedMetricWithStoragePolicy{
ChunkedMetric: aggregated.ChunkedMetric{
ChunkedID: id.ChunkedID{
Data: []byte(metric.ID),
},
TimeNanos: metric.TimeNanos,
Value: metric.Value,
},
StoragePolicy: storagePolicy,
}
if err := pw.Write(mp); err != nil {
agg.metrics.addPassthrough.ReportError(err)
return err
}
agg.metrics.addPassthrough.ReportSuccess(agg.nowFn().Sub(callStart))
return nil
}
func (agg *aggregator) Resign() error {
ctx, cancel := context.WithTimeout(context.Background(), agg.resignTimeout)
defer cancel()
return agg.electionManager.Resign(ctx)
}
func (agg *aggregator) Status() RuntimeStatus {
return RuntimeStatus{
FlushStatus: agg.flushManager.Status(),
}
}
func (agg *aggregator) Close() error {
agg.Lock()
defer agg.Unlock()
if agg.state != aggregatorOpen {
return errAggregatorNotOpenOrClosed
}
close(agg.doneCh)
for _, shardID := range agg.shardIDs {
agg.shards[shardID].Close()
}
if agg.shardSetOpen {
agg.closeShardSetWithLock()
}
agg.flushHandler.Close()
agg.passthroughWriter.Close()
if agg.adminClient != nil {
agg.adminClient.Close()
}
agg.state = aggregatorClosed
return nil
}
func (agg *aggregator) passWriter() (writer.Writer, error) {
agg.RLock()
defer agg.RUnlock()
if agg.state != aggregatorOpen {
return nil, errAggregatorNotOpenOrClosed
}
if agg.electionManager.ElectionState() == FollowerState {
return writer.NewBlackholeWriter(), nil
}
return agg.passthroughWriter, nil
}
func (agg *aggregator) shardFor(id id.RawID) (*aggregatorShard, error) {
agg.RLock()
shard, err := agg.shardForWithLock(id, noUpdateShards)
if err == nil || err != errActivePlacementChanged {
agg.RUnlock()
return shard, err
}
agg.RUnlock()
agg.Lock()
shard, err = agg.shardForWithLock(id, updateShards)
agg.Unlock()
return shard, err
}
func (agg *aggregator) shardForWithLock(id id.RawID, updateShardsType updateShardsType) (*aggregatorShard, error) {
if agg.state != aggregatorOpen {
return nil, errAggregatorNotOpenOrClosed
}
stagedPlacement, placement, err := agg.placementManager.Placement()
if err != nil {
return nil, err
}
if agg.shouldProcessPlacementWithLock(stagedPlacement, placement) {
if updateShardsType == noUpdateShards {
return nil, errActivePlacementChanged
}
if err := agg.processPlacementWithLock(stagedPlacement, placement); err != nil {
return nil, err
}
}
shardID := agg.shardFn([]byte(id), uint32(placement.NumShards()))
if int(shardID) >= len(agg.shards) || agg.shards[shardID] == nil {
return nil, errShardNotOwned
}
return agg.shards[shardID], nil
}
func (agg *aggregator) processPlacementWithLock(
newStagedPlacement placement.ActiveStagedPlacement,
newPlacement placement.Placement,
) error {
// If someone has already processed the placement ahead of us, do nothing.
if !agg.shouldProcessPlacementWithLock(newStagedPlacement, newPlacement) {
return nil
}
var newShardSet shard.Shards
instance, err := agg.placementManager.InstanceFrom(newPlacement)
if err == nil {
newShardSet = instance.Shards()
} else if err == ErrInstanceNotFoundInPlacement {
// NB(r): Without this log message it's hard for operators to debug
// logs about receiving metrics that the aggregator does not own.
placementInstances := newPlacement.Instances()
placementInstanceIDs := make([]string, 0, len(placementInstances))
for _, instance := range placementInstances {
placementInstanceIDs = append(placementInstanceIDs, instance.ID())
}
msg := "aggregator instance ID must appear in placement: " +
"no shards assigned since not found with current instance ID"
agg.logger.Error(msg,
zap.String("currInstanceID", agg.placementManager.InstanceID()),
zap.Strings("placementInstanceIDs", placementInstanceIDs))
newShardSet = shard.NewShards(nil)
} else {
return err
}
agg.updateShardsWithLock(newStagedPlacement, newPlacement, newShardSet)
if err := agg.updateShardSetIDWithLock(instance); err != nil {
return err
}
agg.metrics.placement.updated.Inc(1)
return nil
}
func (agg *aggregator) shouldProcessPlacementWithLock(
newStagedPlacement placement.ActiveStagedPlacement,
newPlacement placement.Placement,
) bool {
// If there is no staged placement yet, or the staged placement has been updated,
// process this placement.
if agg.currStagedPlacement == nil || agg.currStagedPlacement != newStagedPlacement {
agg.metrics.placement.stagedPlacementChanged.Inc(1)
return true
}
// If there is no placement yet, or the new placement has a later cutover time,
// process this placement.
if agg.currPlacement == nil || agg.currPlacement.CutoverNanos() < newPlacement.CutoverNanos() {
agg.metrics.placement.cutoverChanged.Inc(1)
return true
}
return false
}
// updateShardSetWithLock resets the instance's shard set id given the instance from
// the latest placement, or clears it if the instance is nil (i.e., instance not found).
func (agg *aggregator) updateShardSetIDWithLock(instance placement.Instance) error {
if instance == nil {
return agg.clearShardSetIDWithLock()
}
return agg.resetShardSetIDWithLock(instance)
}
// clearShardSetIDWithLock clears the instance's shard set id.
func (agg *aggregator) clearShardSetIDWithLock() error {
agg.metrics.shardSetID.clear.Inc(1)
if !agg.shardSetOpen {
return nil
}
if err := agg.closeShardSetWithLock(); err != nil {
return err
}
agg.shardSetID = uninitializedShardSetID
agg.shardSetOpen = false
return nil
}
// resetShardSetIDWithLock resets the instance's shard set id given the instance from
// the latest placement.
func (agg *aggregator) resetShardSetIDWithLock(instance placement.Instance) error {
agg.metrics.shardSetID.reset.Inc(1)
if !agg.shardSetOpen {
shardSetID := instance.ShardSetID()
if err := agg.openShardSetWithLock(shardSetID); err != nil {
return err
}
agg.shardSetID = shardSetID
agg.shardSetOpen = true
return nil
}
if instance.ShardSetID() == agg.shardSetID {
agg.metrics.shardSetID.same.Inc(1)
return nil
}
agg.metrics.shardSetID.changed.Inc(1)
if err := agg.closeShardSetWithLock(); err != nil {
return err
}
newShardSetID := instance.ShardSetID()
if err := agg.openShardSetWithLock(newShardSetID); err != nil {
return err
}
agg.shardSetID = newShardSetID
agg.shardSetOpen = true
return nil
}
func (agg *aggregator) openShardSetWithLock(shardSetID uint32) error {
agg.metrics.shardSetID.open.Inc(1)
if err := agg.flushTimesManager.Open(shardSetID); err != nil {
return err
}
if err := agg.electionManager.Open(shardSetID); err != nil {
return err
}
return agg.flushManager.Open()
}
func (agg *aggregator) closeShardSetWithLock() error {
agg.metrics.shardSetID.close.Inc(1)
if err := agg.flushManager.Close(); err != nil {
return err
}
if err := agg.flushManager.Reset(); err != nil {
return err
}
if err := agg.electionManager.Close(); err != nil {
return err
}
if err := agg.electionManager.Reset(); err != nil {
return err
}
if err := agg.flushTimesManager.Close(); err != nil {
return err
}
return agg.flushTimesManager.Reset()
}
func (agg *aggregator) updateShardsWithLock(
newStagedPlacement placement.ActiveStagedPlacement,
newPlacement placement.Placement,
newShardSet shard.Shards,
) {
var (
incoming []*aggregatorShard
closing = make([]*aggregatorShard, 0, len(agg.shardIDs))
)
for _, shard := range agg.shards {
if shard == nil {
continue
}
if !newShardSet.Contains(shard.ID()) {
closing = append(closing, shard)
}
}
// NB(xichen): shards are guaranteed to be sorted by their ids in ascending order.
var (
newShards = newShardSet.All()
newShardIDs []uint32
)
if numShards := len(newShards); numShards > 0 {
newShardIDs = make([]uint32, 0, numShards)
maxShardID := newShards[numShards-1].ID()
incoming = make([]*aggregatorShard, maxShardID+1)
}
for _, shard := range newShards {
shardID := shard.ID()
newShardIDs = append(newShardIDs, shardID)
if int(shardID) < len(agg.shards) && agg.shards[shardID] != nil {
incoming[shardID] = agg.shards[shardID]
} else {
incoming[shardID] = newAggregatorShard(shardID, agg.opts)
agg.metrics.shards.add.Inc(1)
}
shardTimeRange := timeRange{
cutoverNanos: shard.CutoverNanos(),
cutoffNanos: shard.CutoffNanos(),
}
incoming[shardID].SetWriteableRange(shardTimeRange)
}
agg.shardIDs = newShardIDs
agg.shards = incoming
agg.currStagedPlacement = newStagedPlacement
agg.currPlacement = newPlacement
agg.closeShardsAsync(closing)
}
func (agg *aggregator) checkMetricType(mu unaggregated.MetricUnion) error {
switch mu.Type {
case metric.CounterType:
agg.metrics.counters.Inc(1)
return nil
case metric.TimerType:
agg.metrics.timerBatches.Inc(1)
agg.metrics.timers.Inc(int64(len(mu.BatchTimerVal)))
return nil
case metric.GaugeType:
agg.metrics.gauges.Inc(1)
return nil
default:
return errInvalidMetricType
}
}
func (agg *aggregator) ownedShards() (owned, toClose []*aggregatorShard) {
agg.Lock()
defer agg.Unlock()
if len(agg.shardIDs) == 0 {
return nil, nil
}
flushTimes, err := agg.flushTimesManager.Get()
if err != nil {
agg.metrics.tick.flushTimesErrors.Inc(1)
}
owned = make([]*aggregatorShard, 0, len(agg.shardIDs))
for i := 0; i < len(agg.shardIDs); i++ {
shardID := agg.shardIDs[i]
shard := agg.shards[shardID]
// NB(xichen): a shard can be closed when all of the following conditions are met:
// * The shard is not writeable.
// * The shard has been cut off (we do not want to close a shard that has not been
// cut over in that it may be warming up).
// * All of the shard's data has been flushed up until the shard's cutoff time.
canCloseShard := !shard.IsWritable() &&
shard.IsCutoff() &&
agg.flushTimesChecker.HasFlushed(
shard.ID(),
shard.CutoffNanos(),
flushTimes,
)
if !canCloseShard {
owned = append(owned, shard)
} else {
lastIdx := len(agg.shardIDs) - 1
agg.shardIDs[i], agg.shardIDs[lastIdx] = agg.shardIDs[lastIdx], agg.shardIDs[i]
agg.shardIDs = agg.shardIDs[:lastIdx]
i--
agg.shards[shardID] = nil
toClose = append(toClose, shard)
}
}
return owned, toClose
}
// closeShardsAsync asynchronously closes the shards to avoid blocking writes.
// Because each shard write happens while holding the shard read lock, the shard
// may only close itself after all its pending writes are finished.
func (agg *aggregator) closeShardsAsync(shards []*aggregatorShard) {
pendingClose := atomic.AddInt32(&agg.shardsPendingClose, int32(len(shards)))
agg.metrics.shards.pendingClose.Update(float64(pendingClose))
for _, shard := range shards {
shard := shard
go func() {
shard.Close()
pendingClose := atomic.AddInt32(&agg.shardsPendingClose, -1)
agg.metrics.shards.pendingClose.Update(float64(pendingClose))
agg.metrics.shards.close.Inc(1)
}()
}
}
func (agg *aggregator) tick() {
defer agg.wg.Done()
for {
select {
case <-agg.doneCh:
return
default:
agg.tickInternal()
}
}
}
func (agg *aggregator) tickInternal() {
ownedShards, closingShards := agg.ownedShards()
agg.closeShardsAsync(closingShards)
numShards := len(ownedShards)
agg.metrics.shards.owned.Update(float64(numShards))
agg.metrics.shards.pendingClose.Update(float64(atomic.LoadInt32(&agg.shardsPendingClose)))
if numShards == 0 {
agg.sleepFn(agg.checkInterval)
return
}
var (
start = agg.nowFn()
perShardTickDuration = agg.checkInterval / time.Duration(numShards)
tickResult tickResult
)
for _, shard := range ownedShards {
shardTickResult := shard.Tick(perShardTickDuration)
tickResult = tickResult.merge(shardTickResult)
}
tickDuration := agg.nowFn().Sub(start)
agg.metrics.tick.Report(tickResult, tickDuration)
if tickDuration < agg.checkInterval {
agg.sleepFn(agg.checkInterval - tickDuration)
}
}
type aggregatorAddMetricMetrics struct {
success tally.Counter
successLatency tally.Timer
shardNotOwned tally.Counter
shardNotWriteable tally.Counter
valueRateLimitExceeded tally.Counter
newMetricRateLimitExceeded tally.Counter
uncategorizedErrors tally.Counter
}
func newAggregatorAddMetricMetrics(
scope tally.Scope,
opts instrument.TimerOptions,
) aggregatorAddMetricMetrics {
return aggregatorAddMetricMetrics{
success: scope.Counter("success"),
successLatency: instrument.NewTimer(scope, "success-latency", opts),
shardNotOwned: scope.Tagged(map[string]string{
"reason": "shard-not-owned",
}).Counter("errors"),
shardNotWriteable: scope.Tagged(map[string]string{
"reason": "shard-not-writeable",
}).Counter("errors"),
valueRateLimitExceeded: scope.Tagged(map[string]string{
"reason": "value-rate-limit-exceeded",
}).Counter("errors"),
newMetricRateLimitExceeded: scope.Tagged(map[string]string{
"reason": "new-metric-rate-limit-exceeded",
}).Counter("errors"),
uncategorizedErrors: scope.Tagged(map[string]string{
"reason": "not-categorized",
}).Counter("errors"),
}
}
func (m *aggregatorAddMetricMetrics) ReportSuccess(d time.Duration) {
m.success.Inc(1)
m.successLatency.Record(d)
}
func (m *aggregatorAddMetricMetrics) ReportError(err error) {
if err == nil {
return
}
switch err {
case errShardNotOwned:
m.shardNotOwned.Inc(1)
case errAggregatorShardNotWriteable:
m.shardNotWriteable.Inc(1)
case errWriteNewMetricRateLimitExceeded:
m.newMetricRateLimitExceeded.Inc(1)
case errWriteValueRateLimitExceeded:
m.valueRateLimitExceeded.Inc(1)
default:
m.uncategorizedErrors.Inc(1)
}
}
type aggregatorAddUntimedMetrics struct {
aggregatorAddMetricMetrics
invalidMetricTypes tally.Counter
}
func newAggregatorAddUntimedMetrics(
scope tally.Scope,
opts instrument.TimerOptions,
) aggregatorAddUntimedMetrics {
return aggregatorAddUntimedMetrics{
aggregatorAddMetricMetrics: newAggregatorAddMetricMetrics(scope, opts),
invalidMetricTypes: scope.Tagged(map[string]string{
"reason": "invalid-metric-types",
}).Counter("errors"),
}
}
func (m *aggregatorAddUntimedMetrics) ReportError(err error) {
if err == errInvalidMetricType {
m.invalidMetricTypes.Inc(1)
return
}
m.aggregatorAddMetricMetrics.ReportError(err)
}
type aggregatorAddTimedMetrics struct {
aggregatorAddMetricMetrics
tooFarInTheFuture tally.Counter
tooFarInThePast tally.Counter
}
func newAggregatorAddTimedMetrics(
scope tally.Scope,
opts instrument.TimerOptions,
) aggregatorAddTimedMetrics {
return aggregatorAddTimedMetrics{
aggregatorAddMetricMetrics: newAggregatorAddMetricMetrics(scope, opts),
tooFarInTheFuture: scope.Tagged(map[string]string{
"reason": "too-far-in-the-future",
}).Counter("errors"),
tooFarInThePast: scope.Tagged(map[string]string{
"reason": "too-far-in-the-past",
}).Counter("errors"),
}
}
func (m *aggregatorAddTimedMetrics) ReportError(err error) {
switch {
case err == errTooFarInTheFuture ||
xerrors.InnerError(err) == errTooFarInTheFuture:
m.tooFarInTheFuture.Inc(1)
case err == errTooFarInThePast ||
xerrors.InnerError(err) == errTooFarInThePast:
m.tooFarInThePast.Inc(1)
default:
m.aggregatorAddMetricMetrics.ReportError(err)
}
}
type aggregatorAddPassthroughMetrics struct {
aggregatorAddMetricMetrics
followerNoop tally.Counter
}
func newAggregatorAddPassthroughMetrics(
scope tally.Scope,
opts instrument.TimerOptions,
) aggregatorAddPassthroughMetrics {
return aggregatorAddPassthroughMetrics{
aggregatorAddMetricMetrics: newAggregatorAddMetricMetrics(scope, opts),
followerNoop: scope.Counter("follower-noop"),
}
}
func (m *aggregatorAddPassthroughMetrics) ReportError(err error) {
m.aggregatorAddMetricMetrics.ReportError(err)
}
func (m *aggregatorAddPassthroughMetrics) ReportFollowerNoop() {
m.followerNoop.Inc(1)
}
type latencyBucketKey struct {
resolution time.Duration
numForwardedTimes int
}
type aggregatorAddForwardedMetrics struct {
sync.RWMutex
aggregatorAddMetricMetrics
scope tally.Scope
maxAllowedForwardingDelayFn MaxAllowedForwardingDelayFn
forwardingLatency map[latencyBucketKey]tally.Histogram
}
func newAggregatorAddForwardedMetrics(
scope tally.Scope,
opts instrument.TimerOptions,
maxAllowedForwardingDelayFn MaxAllowedForwardingDelayFn,
) aggregatorAddForwardedMetrics {
return aggregatorAddForwardedMetrics{
aggregatorAddMetricMetrics: newAggregatorAddMetricMetrics(scope, opts),
scope: scope,
maxAllowedForwardingDelayFn: maxAllowedForwardingDelayFn,
forwardingLatency: make(map[latencyBucketKey]tally.Histogram),
}
}
func (m *aggregatorAddForwardedMetrics) ReportForwardingLatency(
resolution time.Duration,
numForwardedTimes int,
duration time.Duration,
) {
key := latencyBucketKey{
resolution: resolution,
numForwardedTimes: numForwardedTimes,
}
m.RLock()
histogram, exists := m.forwardingLatency[key]
m.RUnlock()
if exists {
histogram.RecordDuration(duration)
return
}
m.Lock()
histogram, exists = m.forwardingLatency[key]
if exists {
m.Unlock()
histogram.RecordDuration(duration)
return
}
maxForwardingDelayAllowed := m.maxAllowedForwardingDelayFn(resolution, numForwardedTimes)
maxLatencyBucketLimit := maxForwardingDelayAllowed * maxLatencyBucketLimitScaleFactor
latencyBucketSize := maxLatencyBucketLimit / time.Duration(numLatencyBuckets)
latencyBuckets := tally.MustMakeLinearDurationBuckets(0, latencyBucketSize, numLatencyBuckets)
histogram = m.scope.Tagged(map[string]string{
"bucket-version": strconv.Itoa(latencyBucketVersion),
"resolution": resolution.String(),
"num-forwarded-times": strconv.Itoa(numForwardedTimes),
}).Histogram("forwarding-latency", latencyBuckets)
m.forwardingLatency[key] = histogram
m.Unlock()
histogram.RecordDuration(duration)
}
type tickMetricsForMetricCategory struct {
scope tally.Scope
activeEntries tally.Gauge
expiredEntries tally.Counter
activeElems map[time.Duration]tally.Gauge
}
func newTickMetricsForMetricCategory(scope tally.Scope) tickMetricsForMetricCategory {
return tickMetricsForMetricCategory{
scope: scope,
activeEntries: scope.Gauge("active-entries"),
expiredEntries: scope.Counter("expired-entries"),
activeElems: make(map[time.Duration]tally.Gauge),
}
}
func (m tickMetricsForMetricCategory) Report(tickResult tickResultForMetricCategory) {
m.activeEntries.Update(float64(tickResult.activeEntries))
m.expiredEntries.Inc(int64(tickResult.expiredEntries))
for dur, val := range tickResult.activeElems {
gauge, exists := m.activeElems[dur]
if !exists {
gauge = m.scope.Tagged(
map[string]string{"resolution": dur.String()},
).Gauge("active-elems")
m.activeElems[dur] = gauge
}
gauge.Update(float64(val))
}
}
type aggregatorTickMetrics struct {
flushTimesErrors tally.Counter
duration tally.Timer
standard tickMetricsForMetricCategory
forwarded tickMetricsForMetricCategory
}
func newAggregatorTickMetrics(scope tally.Scope) aggregatorTickMetrics {
standardScope := scope.Tagged(map[string]string{"metric-type": "standard"})
forwardedScope := scope.Tagged(map[string]string{"metric-type": "forwarded"})
return aggregatorTickMetrics{
flushTimesErrors: scope.Counter("flush-times-errors"),
duration: scope.Timer("duration"),
standard: newTickMetricsForMetricCategory(standardScope),
forwarded: newTickMetricsForMetricCategory(forwardedScope),
}
}
func (m aggregatorTickMetrics) Report(tickResult tickResult, duration time.Duration) {
m.duration.Record(duration)
m.standard.Report(tickResult.standard)
m.forwarded.Report(tickResult.forwarded)
}
type aggregatorShardsMetrics struct {
add tally.Counter
close tally.Counter
owned tally.Gauge
pendingClose tally.Gauge
}
func newAggregatorShardsMetrics(scope tally.Scope) aggregatorShardsMetrics {
return aggregatorShardsMetrics{
add: scope.Counter("add"),
close: scope.Counter("close"),
owned: scope.Gauge("owned"),
pendingClose: scope.Gauge("pending-close"),
}
}
type aggregatorPlacementMetrics struct {
stagedPlacementChanged tally.Counter
cutoverChanged tally.Counter
updated tally.Counter
}
func newAggregatorPlacementMetrics(scope tally.Scope) aggregatorPlacementMetrics {
return aggregatorPlacementMetrics{
stagedPlacementChanged: scope.Counter("staged-placement-changed"),
cutoverChanged: scope.Counter("cutover-changed"),
updated: scope.Counter("updated"),
}
}
type aggregatorShardSetIDMetrics struct {
open tally.Counter
close tally.Counter
clear tally.Counter
reset tally.Counter
same tally.Counter