This repository has been archived by the owner on Aug 23, 2023. It is now read-only.
/
meta_record_idx.go
390 lines (338 loc) · 11.3 KB
/
meta_record_idx.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
package cassandra
import (
"encoding/json"
"fmt"
"strings"
"sync"
"time"
"github.com/gocql/gocql"
cassUtils "github.com/grafana/metrictank/cassandra"
"github.com/grafana/metrictank/expr/tagquery"
"github.com/grafana/metrictank/idx"
"github.com/grafana/metrictank/idx/metatags"
"github.com/grafana/metrictank/util"
log "github.com/sirupsen/logrus"
)
var (
// try executing each query 10 times
// when a query fails we retry 9 times with the following sleep times in-between
// 100ms, 200ms, 400ms, 800ms, 1.6s, 3.2s, 6.4s, 12.8s, 20s
// the total time to fail is 45.5s, which is less than the default http timeout
metaRecordRetryPolicy = gocql.ExponentialBackoffRetryPolicy{
NumRetries: 9,
Min: time.Millisecond * time.Duration(100),
Max: time.Second * time.Duration(20),
}
errIdxUpdatesDisabled = fmt.Errorf("Cassandra index updates are disabled")
)
type MetaRecordIdx struct {
wg sync.WaitGroup
shutdown chan struct{}
cfg *Config
status metatags.MetaRecordStatusByOrg
memoryIdx idx.MetaRecordIdx
cluster *gocql.ClusterConfig
session *cassUtils.Session
}
func NewCassandraMetaRecordIdx(cfg *Config, memoryIdx idx.MetaRecordIdx) *MetaRecordIdx {
if err := cfg.Validate(); err != nil {
log.Fatalf("cass-meta-record-idx: %s", err)
}
cluster := gocql.NewCluster(strings.Split(cfg.hosts, ",")...)
cluster.Consistency = gocql.ParseConsistency(cfg.consistency)
cluster.Keyspace = cfg.keyspace
cluster.Timeout = cfg.timeout
cluster.ConnectTimeout = cluster.Timeout
cluster.NumConns = cfg.numConns
cluster.ProtoVersion = cfg.protoVer
cluster.DisableInitialHostLookup = cfg.disableInitialHostLookup
if cfg.ssl {
cluster.SslOpts = &gocql.SslOptions{
CaPath: cfg.caPath,
EnableHostVerification: cfg.hostVerification,
}
}
if cfg.auth {
cluster.Authenticator = gocql.PasswordAuthenticator{
Username: cfg.username,
Password: cfg.password,
}
}
return &MetaRecordIdx{
shutdown: make(chan struct{}),
cfg: cfg,
status: metatags.NewMetaRecordStatusByOrg(),
memoryIdx: memoryIdx,
cluster: cluster,
}
}
func (m *MetaRecordIdx) Init() error {
var err error
m.session, err = cassUtils.NewSession(m.cluster, m.cfg.connectionCheckTimeout, m.cfg.connectionCheckInterval, m.cfg.hosts, "cass-meta-record-idx")
if err != nil {
return fmt.Errorf("cass-meta-record-idx: Failed to create cassandra session: %s", err)
}
schema := fmt.Sprintf(util.ReadEntry(m.cfg.schemaFile, "schema_meta_record_table").(string), m.cfg.keyspace, m.cfg.metaRecordTable)
err = cassUtils.EnsureTableExists(m.session.CurrentSession(), m.cfg.createKeyspace, m.cfg.keyspace, schema, m.cfg.metaRecordTable)
if err != nil {
return err
}
schema = fmt.Sprintf(util.ReadEntry(m.cfg.schemaFile, "schema_meta_record_batch_table").(string), m.cfg.keyspace, m.cfg.metaRecordBatchTable)
err = cassUtils.EnsureTableExists(m.session.CurrentSession(), m.cfg.createKeyspace, m.cfg.keyspace, schema, m.cfg.metaRecordBatchTable)
if err != nil {
return err
}
m.loadMetaRecords()
return nil
}
func (m *MetaRecordIdx) Start() {
m.wg.Add(1)
go m.pollStore()
if m.cfg.updateCassIdx {
m.wg.Add(1)
go m.pruneMetaRecords()
}
}
func (m *MetaRecordIdx) Stop() {
close(m.shutdown)
m.wg.Wait()
m.session.Stop()
}
func (m *MetaRecordIdx) pollStore() {
defer m.wg.Done()
ticker := time.NewTicker(m.cfg.metaRecordPollInterval)
for {
select {
case <-m.shutdown:
ticker.Stop()
return
case <-ticker.C:
m.loadMetaRecords()
}
}
}
func (m *MetaRecordIdx) loadMetaRecords() {
q := fmt.Sprintf("SELECT batchid, orgid, createdat, lastupdate FROM %s", m.cfg.metaRecordBatchTable)
session := m.session.CurrentSession()
iter := session.Query(q).RetryPolicy(&metaRecordRetryPolicy).Iter()
var batchId gocql.UUID
var orgId uint32
var createdAt, lastUpdate uint64
toLoad := make(map[uint32]gocql.UUID)
for iter.Scan(&batchId, &orgId, &createdAt, &lastUpdate) {
load, batchId := m.status.Update(orgId, metatags.UUID(batchId), createdAt, lastUpdate)
if load {
toLoad[orgId] = gocql.UUID(batchId)
}
}
var err error
if err = iter.Close(); err != nil {
log.Errorf("cass-meta-record-idx: Error when loading batches of meta records: %s", err.Error())
return
}
for orgId, batchId := range toLoad {
log.Infof("cass-meta-record-idx: Loading meta record batch %s of org %d", batchId.String(), orgId)
var expressions, metatags string
q = fmt.Sprintf("SELECT expressions, metatags FROM %s WHERE batchid=? AND orgid=?", m.cfg.metaRecordTable)
session := m.session.CurrentSession()
iter = session.Query(q, batchId, orgId).RetryPolicy(&metaRecordRetryPolicy).Iter()
var records []tagquery.MetaTagRecord
for iter.Scan(&expressions, &metatags) {
record := tagquery.MetaTagRecord{}
err = json.Unmarshal([]byte(expressions), &record.Expressions)
if err != nil {
log.Errorf("cass-meta-record-idx: LoadMetaRecords() could not parse stored expressions (%s): %s", expressions, err)
continue
}
err = json.Unmarshal([]byte(metatags), &record.MetaTags)
if err != nil {
log.Errorf("cass-meta-record-idx: LoadMetaRecords() could not parse stored metatags (%s): %s", metatags, err)
continue
}
records = append(records, record)
}
if err = iter.Close(); err != nil {
log.Errorf("cass-meta-record-idx: Error when reading meta records: %s", err.Error())
continue
}
if err = m.memoryIdx.MetaTagRecordSwap(orgId, records); err != nil {
log.Errorf("cass-meta-record-idx: Error when swapping batch of meta records: %s", err.Error())
continue
}
}
}
func (m *MetaRecordIdx) pruneMetaRecords() {
defer m.wg.Done()
ticker := time.NewTicker(m.cfg.metaRecordPruneInterval)
for {
select {
case <-m.shutdown:
ticker.Stop()
return
case <-ticker.C:
q := fmt.Sprintf("SELECT batchid, orgid, createdat FROM %s", m.cfg.metaRecordBatchTable)
session := m.session.CurrentSession()
iter := session.Query(q).RetryPolicy(&metaRecordRetryPolicy).Iter()
var batchId gocql.UUID
var orgId uint32
var createdAt uint64
for iter.Scan(&batchId, &orgId, &createdAt) {
now := time.Now().Unix()
if uint64(now)-uint64(m.cfg.metaRecordPruneAge.Seconds()) <= createdAt/1000 {
continue
}
currentBatchId, _, _ := m.status.GetStatus(orgId)
if batchId != gocql.UUID(currentBatchId) {
err := m.pruneBatch(orgId, batchId)
if err != nil {
log.Errorf("cass-meta-record-idx: Error when pruning batch %d/%s: %s", orgId, batchId.String(), err)
}
}
}
}
}
}
func (m *MetaRecordIdx) pruneBatch(orgId uint32, batchId gocql.UUID) error {
qry := fmt.Sprintf("DELETE FROM %s WHERE orgid=? AND batchid=?", m.cfg.metaRecordTable)
session := m.session.CurrentSession()
err := session.Query(
qry,
orgId,
batchId,
).RetryPolicy(&metaRecordRetryPolicy).Exec()
if err != nil {
return err
}
qry = fmt.Sprintf("DELETE FROM %s WHERE orgid=? AND batchid=?", m.cfg.metaRecordBatchTable)
return session.Query(
qry,
orgId,
batchId,
).RetryPolicy(&metaRecordRetryPolicy).Exec()
}
func (m *MetaRecordIdx) MetaTagRecordList(orgId uint32) []tagquery.MetaTagRecord {
return m.memoryIdx.MetaTagRecordList(orgId)
}
func (m *MetaRecordIdx) MetaTagRecordUpsert(orgId uint32, record tagquery.MetaTagRecord) error {
if !m.cfg.updateCassIdx {
return errIdxUpdatesDisabled
}
var err error
batchId, _, _ := m.status.GetStatus(orgId)
// if a record has no meta tags associated with it, then we delete it
if len(record.MetaTags) > 0 {
err = m.persistMetaRecord(orgId, batchId, record)
} else {
err = m.deleteMetaRecord(orgId, batchId, record)
}
if err != nil {
log.Errorf("cass-meta-record-idx: Failed to update meta records in cassandra: %s", err)
return fmt.Errorf("Failed to update cassandra: %s", err)
}
err = m.markMetaRecordBatchUpdated(orgId, batchId)
if err != nil {
log.Errorf("cass-meta-record-idx: Failed to update meta records in cassandra: %s", err)
return fmt.Errorf("Failed to update cassandra: %s", err)
}
return nil
}
func (m *MetaRecordIdx) markMetaRecordBatchUpdated(orgId uint32, batchId metatags.UUID) error {
session := m.session.CurrentSession()
now := time.Now().UnixNano() / 1000000
if batchId == metatags.DefaultBatchId {
qry := fmt.Sprintf("INSERT INTO %s (orgid, batchid, createdat, lastupdate) VALUES (?, ?, ?, ?)", m.cfg.metaRecordBatchTable)
return session.Query(
qry,
orgId,
gocql.UUID(batchId),
0,
now,
).RetryPolicy(&metaRecordRetryPolicy).Exec()
}
qry := fmt.Sprintf("INSERT INTO %s (orgid, batchid, lastupdate) VALUES (?, ?, ?)", m.cfg.metaRecordBatchTable)
return session.Query(
qry,
orgId,
gocql.UUID(batchId),
now,
).RetryPolicy(&metaRecordRetryPolicy).Exec()
}
func (m *MetaRecordIdx) MetaTagRecordSwap(orgId uint32, records []tagquery.MetaTagRecord) error {
if !m.cfg.updateCassIdx {
return errIdxUpdatesDisabled
}
newBatchId, err := metatags.RandomUUID()
if err != nil {
return fmt.Errorf("Failed to generate new batch id")
}
var expressions, metaTags []byte
var qry string
for _, record := range records {
record.Expressions.Sort()
expressions, err = record.Expressions.MarshalJSON()
if err != nil {
return fmt.Errorf("Failed to marshal expressions: %s", err)
}
metaTags, err = record.MetaTags.MarshalJSON()
if err != nil {
return fmt.Errorf("Failed to marshal meta tags: %s", err)
}
qry = fmt.Sprintf("INSERT INTO %s (batchid, orgid, expressions, metatags) VALUES (?, ?, ?, ?)", m.cfg.metaRecordTable)
session := m.session.CurrentSession()
err = session.Query(
qry,
gocql.UUID(newBatchId),
orgId,
expressions,
metaTags,
).RetryPolicy(&metaRecordRetryPolicy).Exec()
if err != nil {
return fmt.Errorf("Failed to save meta record: %s", err)
}
}
return m.createNewBatch(orgId, newBatchId)
}
func (m *MetaRecordIdx) createNewBatch(orgId uint32, batchId metatags.UUID) error {
session := m.session.CurrentSession()
now := time.Now().UnixNano() / 1000000
qry := fmt.Sprintf("INSERT INTO %s (orgid, batchid, createdat, lastupdate) VALUES (?, ?, ?, ?)", m.cfg.metaRecordBatchTable)
return session.Query(
qry,
orgId,
gocql.UUID(batchId),
now,
now,
).RetryPolicy(&metaRecordRetryPolicy).Exec()
}
func (m *MetaRecordIdx) persistMetaRecord(orgId uint32, batchId metatags.UUID, record tagquery.MetaTagRecord) error {
expressions, err := record.Expressions.MarshalJSON()
if err != nil {
return fmt.Errorf("Failed to marshal expressions: %s", err)
}
metaTags, err := record.MetaTags.MarshalJSON()
if err != nil {
return fmt.Errorf("Failed to marshal meta tags: %s", err)
}
session := m.session.CurrentSession()
qry := fmt.Sprintf("INSERT INTO %s (batchid, orgid, expressions, metatags) VALUES (?, ?, ?, ?)", m.cfg.metaRecordTable)
return session.Query(
qry,
gocql.UUID(batchId),
orgId,
expressions,
metaTags).RetryPolicy(&metaRecordRetryPolicy).Exec()
}
func (m *MetaRecordIdx) deleteMetaRecord(orgId uint32, batchId metatags.UUID, record tagquery.MetaTagRecord) error {
expressions, err := record.Expressions.MarshalJSON()
if err != nil {
return fmt.Errorf("Failed to marshal record expressions: %s", err)
}
session := m.session.CurrentSession()
qry := fmt.Sprintf("DELETE FROM %s WHERE batchid=? AND orgid=? AND expressions=?", m.cfg.metaRecordTable)
return session.Query(
qry,
gocql.UUID(batchId),
orgId,
expressions,
).RetryPolicy(&metaRecordRetryPolicy).Exec()
}