forked from thingio/kapacitor
-
Notifications
You must be signed in to change notification settings - Fork 0
/
stats.go
244 lines (218 loc) · 5.53 KB
/
stats.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
package edge
import (
"sync"
expvar "github.com/influxdata/kapacitor/expvar"
"github.com/influxdata/kapacitor/models"
"github.com/influxdata/kapacitor/pipeline"
)
// StatsEdge is an edge that tracks various statistics about message passing through the edge.
type StatsEdge interface {
Edge
// Collected returns the number of messages collected by this edge.
Collected() int64
// Emitted returns the number of messages emitted by this edge.
Emitted() int64
// CollectedVar is an exported var the represents the number of messages collected by this edge.
CollectedVar() expvar.IntVar
// EmittedVar is an exported var the represents the number of messages emitted by this edge.
EmittedVar() expvar.IntVar
// ReadGroupStats allows for the reading of the current statistics by group.
ReadGroupStats(func(*GroupStats))
}
// GroupStats represents the statistics for a specific group.
type GroupStats struct {
GroupInfo GroupInfo
Collected int64
Emitted int64
}
// NewStatsEdge creates an edge that tracks statistics about the message passing through the edge.
func NewStatsEdge(e Edge) StatsEdge {
switch e.Type() {
case pipeline.StreamEdge:
return &streamStatsEdge{
statsEdge: statsEdge{
edge: e,
groupStats: make(map[models.GroupID]*GroupStats),
collected: new(expvar.Int),
emitted: new(expvar.Int),
},
}
case pipeline.BatchEdge:
return &batchStatsEdge{
statsEdge: statsEdge{
edge: e,
groupStats: make(map[models.GroupID]*GroupStats),
collected: new(expvar.Int),
emitted: new(expvar.Int),
},
}
}
return nil
}
type statsEdge struct {
edge Edge
collected *expvar.Int
emitted *expvar.Int
mu sync.RWMutex
groupStats map[models.GroupID]*GroupStats
}
func (e *statsEdge) Collected() int64 {
return e.collected.IntValue()
}
func (e *statsEdge) Emitted() int64 {
return e.emitted.IntValue()
}
func (e *statsEdge) CollectedVar() expvar.IntVar {
return e.collected
}
func (e *statsEdge) EmittedVar() expvar.IntVar {
return e.emitted
}
func (e *statsEdge) Close() error {
return e.edge.Close()
}
func (e *statsEdge) Abort() {
e.edge.Abort()
}
// ReadGroupStats calls f for each of the group stats.
func (e *statsEdge) ReadGroupStats(f func(groupStat *GroupStats)) {
e.mu.RLock()
defer e.mu.RUnlock()
for _, stats := range e.groupStats {
f(stats)
}
}
func (e *statsEdge) incCollected(group models.GroupID, infoF func() GroupInfo, count int64) {
// Manually unlock below as defer was too much of a performance hit
e.mu.Lock()
if stats, ok := e.groupStats[group]; ok {
stats.Collected += count
} else {
stats = &GroupStats{
Collected: count,
GroupInfo: infoF(),
}
e.groupStats[group] = stats
}
e.mu.Unlock()
}
// Increment the emitted count of the group for this edge.
func (e *statsEdge) incEmitted(group models.GroupID, infoF func() GroupInfo, count int64) {
// Manually unlock below as defer was too much of a performance hit
e.mu.Lock()
if stats, ok := e.groupStats[group]; ok {
stats.Emitted += count
} else {
stats = &GroupStats{
Emitted: count,
GroupInfo: infoF(),
}
e.groupStats[group] = stats
}
e.mu.Unlock()
}
// deleteGroup removes a group's stats
func (e *statsEdge) deleteGroup(group models.GroupID) {
// Manually unlock below as defer was too much of a performance hit
e.mu.Lock()
delete(e.groupStats, group)
e.mu.Unlock()
}
type batchStatsEdge struct {
statsEdge
currentCollectGroup GroupInfo
currentEmitGroup GroupInfo
collectSize int64
emitSize int64
}
func (e *batchStatsEdge) Collect(m Message) error {
if err := e.edge.Collect(m); err != nil {
return err
}
switch b := m.(type) {
case BeginBatchMessage:
g := b.GroupInfo()
e.currentCollectGroup = g
e.collectSize = 0
case BatchPointMessage:
e.collectSize++
case EndBatchMessage:
e.collected.Add(1)
e.incCollected(
e.currentCollectGroup.ID,
func() GroupInfo { return e.currentCollectGroup },
e.collectSize,
)
case BufferedBatchMessage:
e.collected.Add(1)
begin := b.Begin()
e.incCollected(begin.GroupID(), begin.GroupInfo, int64(len(b.Points())))
default:
// Do not count other messages
}
return nil
}
func (e *batchStatsEdge) Emit() (m Message, ok bool) {
m, ok = e.edge.Emit()
if ok {
switch b := m.(type) {
case BeginBatchMessage:
g := b.GroupInfo()
e.currentEmitGroup = g
e.emitSize = 0
case BatchPointMessage:
e.emitSize++
case EndBatchMessage:
e.emitted.Add(1)
e.incEmitted(
e.currentEmitGroup.ID,
func() GroupInfo { return e.currentEmitGroup },
e.emitSize,
)
case BufferedBatchMessage:
e.emitted.Add(1)
begin := b.Begin()
e.incEmitted(begin.GroupID(), begin.GroupInfo, int64(len(b.Points())))
case DeleteGroupMessage:
e.deleteGroup(b.GroupID())
default:
// Do not count other messages
}
}
return
}
func (e *batchStatsEdge) Type() pipeline.EdgeType {
return e.edge.Type()
}
type streamStatsEdge struct {
statsEdge
}
func (e *streamStatsEdge) Collect(m Message) error {
if err := e.edge.Collect(m); err != nil {
return err
}
switch m := m.(type) {
case PointMessage:
e.collected.Add(1)
e.incCollected(m.GroupID(), m.GroupInfo, 1)
default:
// Do not count other messages
}
return nil
}
func (e *streamStatsEdge) Emit() (m Message, ok bool) {
m, ok = e.edge.Emit()
switch m := m.(type) {
case PointMessage:
e.emitted.Add(1)
e.incEmitted(m.GroupID(), m.GroupInfo, 1)
case DeleteGroupMessage:
e.deleteGroup(m.GroupID())
default:
// Do not count other messages
}
return
}
func (e *streamStatsEdge) Type() pipeline.EdgeType {
return e.edge.Type()
}