-
Notifications
You must be signed in to change notification settings - Fork 784
/
storage_client.go
526 lines (459 loc) · 18.5 KB
/
storage_client.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
package cassandra
import (
"bytes"
"context"
"crypto/tls"
"flag"
"fmt"
"io/ioutil"
"strings"
"time"
"github.com/go-kit/kit/log"
"github.com/go-kit/kit/log/level"
"github.com/gocql/gocql"
"github.com/pkg/errors"
"github.com/prometheus/client_golang/prometheus"
"golang.org/x/sync/semaphore"
"github.com/cortexproject/cortex/pkg/chunk"
"github.com/cortexproject/cortex/pkg/chunk/util"
pkgutil "github.com/cortexproject/cortex/pkg/util"
"github.com/cortexproject/cortex/pkg/util/flagext"
)
// Config for a StorageClient
type Config struct {
Addresses string `yaml:"addresses"`
Port int `yaml:"port"`
Keyspace string `yaml:"keyspace"`
Consistency string `yaml:"consistency"`
ReplicationFactor int `yaml:"replication_factor"`
DisableInitialHostLookup bool `yaml:"disable_initial_host_lookup"`
SSL bool `yaml:"SSL"`
HostVerification bool `yaml:"host_verification"`
CAPath string `yaml:"CA_path"`
Auth bool `yaml:"auth"`
Username string `yaml:"username"`
Password flagext.Secret `yaml:"password"`
PasswordFile string `yaml:"password_file"`
CustomAuthenticators flagext.StringSlice `yaml:"custom_authenticators"`
Timeout time.Duration `yaml:"timeout"`
ConnectTimeout time.Duration `yaml:"connect_timeout"`
ReconnectInterval time.Duration `yaml:"reconnect_interval"`
Retries int `yaml:"max_retries"`
MaxBackoff time.Duration `yaml:"retry_max_backoff"`
MinBackoff time.Duration `yaml:"retry_min_backoff"`
QueryConcurrency int `yaml:"query_concurrency"`
NumConnections int `yaml:"num_connections"`
ConvictHosts bool `yaml:"convict_hosts_on_failure"`
TableOptions string `yaml:"table_options"`
}
// RegisterFlags adds the flags required to config this to the given FlagSet
func (cfg *Config) RegisterFlags(f *flag.FlagSet) {
f.StringVar(&cfg.Addresses, "cassandra.addresses", "", "Comma-separated hostnames or IPs of Cassandra instances.")
f.IntVar(&cfg.Port, "cassandra.port", 9042, "Port that Cassandra is running on")
f.StringVar(&cfg.Keyspace, "cassandra.keyspace", "", "Keyspace to use in Cassandra.")
f.StringVar(&cfg.Consistency, "cassandra.consistency", "QUORUM", "Consistency level for Cassandra.")
f.IntVar(&cfg.ReplicationFactor, "cassandra.replication-factor", 1, "Replication factor to use in Cassandra.")
f.BoolVar(&cfg.DisableInitialHostLookup, "cassandra.disable-initial-host-lookup", false, "Instruct the cassandra driver to not attempt to get host info from the system.peers table.")
f.BoolVar(&cfg.SSL, "cassandra.ssl", false, "Use SSL when connecting to cassandra instances.")
f.BoolVar(&cfg.HostVerification, "cassandra.host-verification", true, "Require SSL certificate validation.")
f.StringVar(&cfg.CAPath, "cassandra.ca-path", "", "Path to certificate file to verify the peer.")
f.BoolVar(&cfg.Auth, "cassandra.auth", false, "Enable password authentication when connecting to cassandra.")
f.StringVar(&cfg.Username, "cassandra.username", "", "Username to use when connecting to cassandra.")
f.Var(&cfg.Password, "cassandra.password", "Password to use when connecting to cassandra.")
f.StringVar(&cfg.PasswordFile, "cassandra.password-file", "", "File containing password to use when connecting to cassandra.")
f.Var(&cfg.CustomAuthenticators, "cassandra.custom-authenticator", "If set, when authenticating with cassandra a custom authenticator will be expected during the handshake. This flag can be set multiple times.")
f.DurationVar(&cfg.Timeout, "cassandra.timeout", 2*time.Second, "Timeout when connecting to cassandra.")
f.DurationVar(&cfg.ConnectTimeout, "cassandra.connect-timeout", 5*time.Second, "Initial connection timeout, used during initial dial to server.")
f.DurationVar(&cfg.ReconnectInterval, "cassandra.reconnent-interval", 1*time.Second, "Interval to retry connecting to cassandra nodes marked as DOWN.")
f.IntVar(&cfg.Retries, "cassandra.max-retries", 0, "Number of retries to perform on a request. Set to 0 to disable retries.")
f.DurationVar(&cfg.MinBackoff, "cassandra.retry-min-backoff", 100*time.Millisecond, "Minimum time to wait before retrying a failed request.")
f.DurationVar(&cfg.MaxBackoff, "cassandra.retry-max-backoff", 10*time.Second, "Maximum time to wait before retrying a failed request.")
f.IntVar(&cfg.QueryConcurrency, "cassandra.query-concurrency", 0, "Limit number of concurrent queries to Cassandra. Set to 0 to disable the limit.")
f.IntVar(&cfg.NumConnections, "cassandra.num-connections", 2, "Number of TCP connections per host.")
f.BoolVar(&cfg.ConvictHosts, "cassandra.convict-hosts-on-failure", true, "Convict hosts of being down on failure.")
f.StringVar(&cfg.TableOptions, "cassandra.table-options", "", "Table options used to create index or chunk tables. This value is used as plain text in the table `WITH` like this, \"CREATE TABLE <generated_by_cortex> (...) WITH <cassandra.table-options>\". For details, see https://cortexmetrics.io/docs/production/cassandra. By default it will use the default table options of your Cassandra cluster.")
}
func (cfg *Config) Validate() error {
if cfg.Password.Value != "" && cfg.PasswordFile != "" {
return errors.Errorf("The password and password_file config options are mutually exclusive.")
}
if cfg.SSL && cfg.HostVerification && len(strings.Split(cfg.Addresses, ",")) != 1 {
return errors.Errorf("Host verification is only possible for a single host.")
}
return nil
}
func (cfg *Config) session(name string, reg prometheus.Registerer) (*gocql.Session, error) {
consistency, err := gocql.ParseConsistencyWrapper(cfg.Consistency)
if err != nil {
return nil, errors.WithStack(err)
}
cluster := gocql.NewCluster(strings.Split(cfg.Addresses, ",")...)
cluster.Port = cfg.Port
cluster.Keyspace = cfg.Keyspace
cluster.Consistency = consistency
cluster.BatchObserver = observer{}
cluster.QueryObserver = observer{}
cluster.Timeout = cfg.Timeout
cluster.ConnectTimeout = cfg.ConnectTimeout
cluster.ReconnectInterval = cfg.ReconnectInterval
cluster.NumConns = cfg.NumConnections
cluster.Logger = log.With(pkgutil.Logger, "module", "gocql", "client", name)
cluster.Registerer = prometheus.WrapRegistererWith(
prometheus.Labels{"client": name}, reg)
if cfg.Retries > 0 {
cluster.RetryPolicy = &gocql.ExponentialBackoffRetryPolicy{
NumRetries: cfg.Retries,
Min: cfg.MinBackoff,
Max: cfg.MaxBackoff,
}
}
if !cfg.ConvictHosts {
cluster.ConvictionPolicy = noopConvictionPolicy{}
}
if err = cfg.setClusterConfig(cluster); err != nil {
return nil, errors.WithStack(err)
}
session, err := cluster.CreateSession()
if err == nil {
return session, nil
}
// ErrNoConnectionsStarted will be returned if keyspace don't exist or is invalid.
// ref. https://github.com/gocql/gocql/blob/07ace3bab0f84bb88477bab5d79ba1f7e1da0169/cassandra_test.go#L85-L97
if err != gocql.ErrNoConnectionsStarted {
return nil, errors.WithStack(err)
}
// keyspace not exist
if err := cfg.createKeyspace(); err != nil {
return nil, errors.WithStack(err)
}
session, err = cluster.CreateSession()
return session, errors.WithStack(err)
}
// apply config settings to a cassandra ClusterConfig
func (cfg *Config) setClusterConfig(cluster *gocql.ClusterConfig) error {
cluster.DisableInitialHostLookup = cfg.DisableInitialHostLookup
if cfg.SSL {
if cfg.HostVerification {
cluster.SslOpts = &gocql.SslOptions{
CaPath: cfg.CAPath,
EnableHostVerification: true,
Config: &tls.Config{
ServerName: strings.Split(cfg.Addresses, ",")[0],
},
}
} else {
cluster.SslOpts = &gocql.SslOptions{
EnableHostVerification: false,
}
}
}
if cfg.Auth {
password := cfg.Password.Value
if cfg.PasswordFile != "" {
passwordBytes, err := ioutil.ReadFile(cfg.PasswordFile)
if err != nil {
return errors.Errorf("Could not read Cassandra password file: %v", err)
}
passwordBytes = bytes.TrimRight(passwordBytes, "\n")
password = string(passwordBytes)
}
if len(cfg.CustomAuthenticators) != 0 {
cluster.Authenticator = CustomPasswordAuthenticator{
ApprovedAuthenticators: cfg.CustomAuthenticators,
Username: cfg.Username,
Password: password,
}
return nil
}
cluster.Authenticator = gocql.PasswordAuthenticator{
Username: cfg.Username,
Password: password,
}
}
return nil
}
// createKeyspace will create the desired keyspace if it doesn't exist.
func (cfg *Config) createKeyspace() error {
cluster := gocql.NewCluster(strings.Split(cfg.Addresses, ",")...)
cluster.Port = cfg.Port
cluster.Keyspace = "system"
cluster.Timeout = 20 * time.Second
cluster.ConnectTimeout = 20 * time.Second
if err := cfg.setClusterConfig(cluster); err != nil {
return errors.WithStack(err)
}
session, err := cluster.CreateSession()
if err != nil {
return errors.WithStack(err)
}
defer session.Close()
err = session.Query(fmt.Sprintf(
`CREATE KEYSPACE IF NOT EXISTS %s
WITH replication = {
'class' : 'SimpleStrategy',
'replication_factor' : %d
}`,
cfg.Keyspace, cfg.ReplicationFactor)).Exec()
return errors.WithStack(err)
}
// StorageClient implements chunk.IndexClient and chunk.ObjectClient for Cassandra.
type StorageClient struct {
cfg Config
schemaCfg chunk.SchemaConfig
readSession *gocql.Session
writeSession *gocql.Session
querySemaphore *semaphore.Weighted
}
// NewStorageClient returns a new StorageClient.
func NewStorageClient(cfg Config, schemaCfg chunk.SchemaConfig, registerer prometheus.Registerer) (*StorageClient, error) {
pkgutil.WarnExperimentalUse("Cassandra Backend")
readSession, err := cfg.session("index-read", registerer)
if err != nil {
return nil, errors.WithStack(err)
}
writeSession, err := cfg.session("index-write", registerer)
if err != nil {
return nil, errors.WithStack(err)
}
var querySemaphore *semaphore.Weighted
if cfg.QueryConcurrency > 0 {
querySemaphore = semaphore.NewWeighted(int64(cfg.QueryConcurrency))
}
client := &StorageClient{
cfg: cfg,
schemaCfg: schemaCfg,
readSession: readSession,
writeSession: writeSession,
querySemaphore: querySemaphore,
}
return client, nil
}
// Stop implement chunk.IndexClient.
func (s *StorageClient) Stop() {
s.readSession.Close()
s.writeSession.Close()
}
// Cassandra batching isn't really useful in this case, its more to do multiple
// atomic writes. Therefore we just do a bunch of writes in parallel.
type writeBatch struct {
entries []chunk.IndexEntry
deletes []chunk.IndexEntry
}
// NewWriteBatch implement chunk.IndexClient.
func (s *StorageClient) NewWriteBatch() chunk.WriteBatch {
return &writeBatch{}
}
func (b *writeBatch) Add(tableName, hashValue string, rangeValue []byte, value []byte) {
b.entries = append(b.entries, chunk.IndexEntry{
TableName: tableName,
HashValue: hashValue,
RangeValue: rangeValue,
Value: value,
})
}
func (b *writeBatch) Delete(tableName, hashValue string, rangeValue []byte) {
b.deletes = append(b.deletes, chunk.IndexEntry{
TableName: tableName,
HashValue: hashValue,
RangeValue: rangeValue,
})
}
// BatchWrite implement chunk.IndexClient.
func (s *StorageClient) BatchWrite(ctx context.Context, batch chunk.WriteBatch) error {
b := batch.(*writeBatch)
for _, entry := range b.entries {
err := s.writeSession.Query(fmt.Sprintf("INSERT INTO %s (hash, range, value) VALUES (?, ?, ?)",
entry.TableName), entry.HashValue, entry.RangeValue, entry.Value).WithContext(ctx).Exec()
if err != nil {
return errors.WithStack(err)
}
}
for _, entry := range b.deletes {
err := s.writeSession.Query(fmt.Sprintf("DELETE FROM %s WHERE hash = ? and range = ?",
entry.TableName), entry.HashValue, entry.RangeValue).WithContext(ctx).Exec()
if err != nil {
return errors.WithStack(err)
}
}
return nil
}
// QueryPages implement chunk.IndexClient.
func (s *StorageClient) QueryPages(ctx context.Context, queries []chunk.IndexQuery, callback func(chunk.IndexQuery, chunk.ReadBatch) bool) error {
return util.DoParallelQueries(ctx, s.query, queries, callback)
}
func (s *StorageClient) query(ctx context.Context, query chunk.IndexQuery, callback util.Callback) error {
if s.querySemaphore != nil {
if err := s.querySemaphore.Acquire(ctx, 1); err != nil {
return err
}
defer s.querySemaphore.Release(1)
}
var q *gocql.Query
switch {
case len(query.RangeValuePrefix) > 0 && query.ValueEqual == nil:
q = s.readSession.Query(fmt.Sprintf("SELECT range, value FROM %s WHERE hash = ? AND range >= ? AND range < ?",
query.TableName), query.HashValue, query.RangeValuePrefix, append(query.RangeValuePrefix, '\xff'))
case len(query.RangeValuePrefix) > 0 && query.ValueEqual != nil:
q = s.readSession.Query(fmt.Sprintf("SELECT range, value FROM %s WHERE hash = ? AND range >= ? AND range < ? AND value = ? ALLOW FILTERING",
query.TableName), query.HashValue, query.RangeValuePrefix, append(query.RangeValuePrefix, '\xff'), query.ValueEqual)
case len(query.RangeValueStart) > 0 && query.ValueEqual == nil:
q = s.readSession.Query(fmt.Sprintf("SELECT range, value FROM %s WHERE hash = ? AND range >= ?",
query.TableName), query.HashValue, query.RangeValueStart)
case len(query.RangeValueStart) > 0 && query.ValueEqual != nil:
q = s.readSession.Query(fmt.Sprintf("SELECT range, value FROM %s WHERE hash = ? AND range >= ? AND value = ? ALLOW FILTERING",
query.TableName), query.HashValue, query.RangeValueStart, query.ValueEqual)
case query.ValueEqual == nil:
q = s.readSession.Query(fmt.Sprintf("SELECT range, value FROM %s WHERE hash = ?",
query.TableName), query.HashValue)
case query.ValueEqual != nil:
q = s.readSession.Query(fmt.Sprintf("SELECT range, value FROM %s WHERE hash = ? AND value = ? ALLOW FILTERING",
query.TableName), query.HashValue, query.ValueEqual)
}
iter := q.WithContext(ctx).Iter()
defer iter.Close()
scanner := iter.Scanner()
for scanner.Next() {
b := &readBatch{}
if err := scanner.Scan(&b.rangeValue, &b.value); err != nil {
return errors.WithStack(err)
}
if !callback(query, b) {
return nil
}
}
return errors.WithStack(scanner.Err())
}
// readBatch represents a batch of rows read from Cassandra.
type readBatch struct {
rangeValue []byte
value []byte
}
func (r *readBatch) Iterator() chunk.ReadBatchIterator {
return &readBatchIter{
readBatch: r,
}
}
type readBatchIter struct {
consumed bool
*readBatch
}
func (b *readBatchIter) Next() bool {
if b.consumed {
return false
}
b.consumed = true
return true
}
func (b *readBatchIter) RangeValue() []byte {
return b.rangeValue
}
func (b *readBatchIter) Value() []byte {
return b.value
}
// ObjectClient implements chunk.ObjectClient for Cassandra.
type ObjectClient struct {
cfg Config
schemaCfg chunk.SchemaConfig
readSession *gocql.Session
writeSession *gocql.Session
querySemaphore *semaphore.Weighted
}
// NewObjectClient returns a new ObjectClient.
func NewObjectClient(cfg Config, schemaCfg chunk.SchemaConfig, registerer prometheus.Registerer) (*ObjectClient, error) {
pkgutil.WarnExperimentalUse("Cassandra Backend")
readSession, err := cfg.session("chunks-read", registerer)
if err != nil {
return nil, errors.WithStack(err)
}
writeSession, err := cfg.session("chunks-write", registerer)
if err != nil {
return nil, errors.WithStack(err)
}
var querySemaphore *semaphore.Weighted
if cfg.QueryConcurrency > 0 {
querySemaphore = semaphore.NewWeighted(int64(cfg.QueryConcurrency))
}
client := &ObjectClient{
cfg: cfg,
schemaCfg: schemaCfg,
readSession: readSession,
writeSession: writeSession,
querySemaphore: querySemaphore,
}
return client, nil
}
// PutChunks implements chunk.ObjectClient.
func (s *ObjectClient) PutChunks(ctx context.Context, chunks []chunk.Chunk) error {
for i := range chunks {
buf, err := chunks[i].Encoded()
if err != nil {
return errors.WithStack(err)
}
key := chunks[i].ExternalKey()
tableName, err := s.schemaCfg.ChunkTableFor(chunks[i].From)
if err != nil {
return err
}
// Must provide a range key, even though its not useds - hence 0x00.
q := s.writeSession.Query(fmt.Sprintf("INSERT INTO %s (hash, range, value) VALUES (?, 0x00, ?)",
tableName), key, buf)
if err := q.WithContext(ctx).Exec(); err != nil {
return errors.WithStack(err)
}
}
return nil
}
// GetChunks implements chunk.ObjectClient.
func (s *ObjectClient) GetChunks(ctx context.Context, input []chunk.Chunk) ([]chunk.Chunk, error) {
return util.GetParallelChunks(ctx, input, s.getChunk)
}
func (s *ObjectClient) getChunk(ctx context.Context, decodeContext *chunk.DecodeContext, input chunk.Chunk) (chunk.Chunk, error) {
if s.querySemaphore != nil {
if err := s.querySemaphore.Acquire(ctx, 1); err != nil {
return input, err
}
defer s.querySemaphore.Release(1)
}
tableName, err := s.schemaCfg.ChunkTableFor(input.From)
if err != nil {
return input, err
}
var buf []byte
if err := s.readSession.Query(fmt.Sprintf("SELECT value FROM %s WHERE hash = ?", tableName), input.ExternalKey()).
WithContext(ctx).Scan(&buf); err != nil {
return input, errors.WithStack(err)
}
err = input.Decode(decodeContext, buf)
return input, err
}
func (s *ObjectClient) DeleteChunk(ctx context.Context, userID, chunkID string) error {
chunkRef, err := chunk.ParseExternalKey(userID, chunkID)
if err != nil {
return err
}
tableName, err := s.schemaCfg.ChunkTableFor(chunkRef.From)
if err != nil {
return err
}
q := s.writeSession.Query(fmt.Sprintf("DELETE FROM %s WHERE hash = ?",
tableName), chunkID)
if err := q.WithContext(ctx).Exec(); err != nil {
return errors.WithStack(err)
}
return nil
}
// Stop implement chunk.ObjectClient.
func (s *ObjectClient) Stop() {
s.readSession.Close()
s.writeSession.Close()
}
type noopConvictionPolicy struct{}
// AddFailure should return `true` if the host should be convicted, `false` otherwise.
// Convicted means connections are removed - we don't want that.
// Implementats gocql.ConvictionPolicy.
func (noopConvictionPolicy) AddFailure(err error, host *gocql.HostInfo) bool {
level.Error(pkgutil.Logger).Log("msg", "Cassandra host failure", "err", err, "host", host.String())
return false
}
// Implementats gocql.ConvictionPolicy.
func (noopConvictionPolicy) Reset(host *gocql.HostInfo) {}