-
Notifications
You must be signed in to change notification settings - Fork 781
/
cassandra_index_reader.go
172 lines (128 loc) · 4.46 KB
/
cassandra_index_reader.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
package scanner
import (
"context"
"fmt"
"math"
"strings"
"github.com/go-kit/kit/log"
"github.com/go-kit/kit/log/level"
"github.com/pkg/errors"
"github.com/prometheus/client_golang/prometheus"
"golang.org/x/sync/errgroup"
"github.com/cortexproject/cortex/pkg/chunk"
"github.com/cortexproject/cortex/pkg/chunk/cassandra"
)
/* Cassandra can easily run out of memory or timeout if we try to SELECT the
* entire table. Splitting into many smaller chunks help a lot. */
const nbTokenRanges = 512
const queryPageSize = 10000
type cassandraIndexReader struct {
log log.Logger
cassandraStorageConfig cassandra.Config
schemaCfg chunk.SchemaConfig
rowsRead prometheus.Counter
parsedIndexEntries prometheus.Counter
currentTableRanges prometheus.Gauge
currentTableScannedRanges prometheus.Gauge
}
func newCassandraIndexReader(cfg cassandra.Config, schemaCfg chunk.SchemaConfig, l log.Logger, rowsRead prometheus.Counter, parsedIndexEntries prometheus.Counter, currentTableRanges, scannedRanges prometheus.Gauge) *cassandraIndexReader {
return &cassandraIndexReader{
log: l,
cassandraStorageConfig: cfg,
rowsRead: rowsRead,
parsedIndexEntries: parsedIndexEntries,
currentTableRanges: currentTableRanges,
currentTableScannedRanges: scannedRanges,
}
}
func (r *cassandraIndexReader) IndexTableNames(ctx context.Context) ([]string, error) {
client, err := cassandra.NewTableClient(ctx, r.cassandraStorageConfig, nil)
if err != nil {
return nil, errors.Wrap(err, "create cassandra client failed")
}
defer client.Stop()
return client.ListTables(ctx)
}
type tokenRange struct {
start int64
end int64
}
func (r *cassandraIndexReader) ReadIndexEntries(ctx context.Context, tableName string, processors []chunk.IndexEntryProcessor) error {
level.Debug(r.log).Log("msg", "scanning table", "table", tableName)
client, err := cassandra.NewStorageClient(r.cassandraStorageConfig, r.schemaCfg, nil)
if err != nil {
return errors.Wrap(err, "create cassandra storage client failed")
}
defer client.Stop()
session := client.GetReadSession()
rangesCh := make(chan tokenRange, nbTokenRanges)
var step, n, start int64
step = int64(math.MaxUint64 / nbTokenRanges)
for n = 0; n < nbTokenRanges; n++ {
start = math.MinInt64 + n*step
end := start + step
if n == (nbTokenRanges - 1) {
end = math.MaxInt64
}
t := tokenRange{start: start, end: end}
rangesCh <- t
}
close(rangesCh)
r.currentTableRanges.Set(float64(len(rangesCh)))
r.currentTableScannedRanges.Set(0)
defer r.currentTableRanges.Set(0)
defer r.currentTableScannedRanges.Set(0)
g, gctx := errgroup.WithContext(ctx)
for ix := range processors {
p := processors[ix]
g.Go(func() error {
for rng := range rangesCh {
level.Debug(r.log).Log("msg", "reading rows", "range_start", rng.start, "range_end", rng.end, "table_name", tableName)
query := fmt.Sprintf("SELECT hash, range, value FROM %s WHERE token(hash) >= %v", tableName, rng.start)
if rng.end < math.MaxInt64 {
query += fmt.Sprintf(" AND token(hash) < %v", rng.end)
}
iter := session.Query(query).WithContext(gctx).PageSize(queryPageSize).Iter()
if len(iter.Warnings()) > 0 {
level.Warn(r.log).Log("msg", "warnings from cassandra", "warnings", strings.Join(iter.Warnings(), " :: "))
}
scanner := iter.Scanner()
oldHash := ""
oldRng := ""
for scanner.Next() {
var hash, rng, value string
err := scanner.Scan(&hash, &rng, &value)
if err != nil {
return errors.Wrap(err, "Cassandra scan error")
}
r.rowsRead.Inc()
r.parsedIndexEntries.Inc()
entry := chunk.IndexEntry{
TableName: tableName,
HashValue: hash,
RangeValue: []byte(rng),
Value: []byte(value),
}
if rng < oldRng && oldHash == hash {
level.Error(r.log).Log("msg", "new rng bad", "rng", rng, "old_rng", oldRng, "hash", hash, "old_hash", oldHash)
return fmt.Errorf("received range row in the wrong order for same hash: %v < %v", rng, oldRng)
}
err = p.ProcessIndexEntry(entry)
if err != nil {
return errors.Wrap(err, "processor error")
}
oldHash = hash
oldRng = rng
}
// This will also close the iterator.
err := scanner.Err()
if err != nil {
return errors.Wrap(err, "Cassandra error during scan")
}
r.currentTableScannedRanges.Inc()
}
return p.Flush()
})
}
return g.Wait()
}