-
Notifications
You must be signed in to change notification settings - Fork 3.3k
/
querier.go
346 lines (298 loc) · 10.1 KB
/
querier.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
package querier
import (
"context"
"flag"
"net/http"
"time"
cortex_client "github.com/cortexproject/cortex/pkg/ingester/client"
"github.com/cortexproject/cortex/pkg/ring"
"github.com/cortexproject/cortex/pkg/util"
"github.com/go-kit/kit/log/level"
"github.com/prometheus/common/model"
"google.golang.org/grpc/health/grpc_health_v1"
"github.com/grafana/loki/pkg/helpers"
"github.com/grafana/loki/pkg/ingester/client"
"github.com/grafana/loki/pkg/iter"
"github.com/grafana/loki/pkg/logproto"
"github.com/grafana/loki/pkg/storage"
)
var readinessProbeSuccess = []byte("Ready")
// Config for a querier.
type Config struct {
TailMaxDuration time.Duration `yaml:"tail_max_duration"`
QueryTimeout time.Duration `yaml:"query_timeout"`
}
// RegisterFlags register flags.
func (cfg *Config) RegisterFlags(f *flag.FlagSet) {
f.DurationVar(&cfg.TailMaxDuration, "querier.tail-max-duration", 1*time.Hour, "Limit the duration for which live tailing request would be served")
f.DurationVar(&cfg.QueryTimeout, "querier.query_timeout", 1*time.Minute, "Timeout when querying backends (ingesters or storage) during the execution of a query request")
}
// Querier handlers queries.
type Querier struct {
cfg Config
ring ring.ReadRing
pool *cortex_client.Pool
store storage.Store
}
// New makes a new Querier.
func New(cfg Config, clientCfg client.Config, ring ring.ReadRing, store storage.Store) (*Querier, error) {
factory := func(addr string) (grpc_health_v1.HealthClient, error) {
return client.New(clientCfg, addr)
}
return newQuerier(cfg, clientCfg, factory, ring, store)
}
// newQuerier creates a new Querier and allows to pass a custom ingester client factory
// used for testing purposes
func newQuerier(cfg Config, clientCfg client.Config, clientFactory cortex_client.Factory, ring ring.ReadRing, store storage.Store) (*Querier, error) {
return &Querier{
cfg: cfg,
ring: ring,
pool: cortex_client.NewPool(clientCfg.PoolConfig, ring, clientFactory, util.Logger),
store: store,
}, nil
}
type responseFromIngesters struct {
addr string
response interface{}
}
// ReadinessHandler is used to indicate to k8s when the querier is ready.
// Returns 200 when the querier is ready, 500 otherwise.
func (q *Querier) ReadinessHandler(w http.ResponseWriter, r *http.Request) {
_, err := q.ring.GetAll()
if err != nil {
http.Error(w, "Not ready: "+err.Error(), http.StatusInternalServerError)
return
}
w.WriteHeader(http.StatusOK)
if _, err := w.Write(readinessProbeSuccess); err != nil {
level.Error(util.Logger).Log("msg", "error writing success message", "error", err)
}
}
// forAllIngesters runs f, in parallel, for all ingesters
// TODO taken from Cortex, see if we can refactor out an usable interface.
func (q *Querier) forAllIngesters(f func(logproto.QuerierClient) (interface{}, error)) ([]responseFromIngesters, error) {
replicationSet, err := q.ring.GetAll()
if err != nil {
return nil, err
}
return q.forGivenIngesters(replicationSet, f)
}
// forGivenIngesters runs f, in parallel, for given ingesters
// TODO taken from Cortex, see if we can refactor out an usable interface.
func (q *Querier) forGivenIngesters(replicationSet ring.ReplicationSet, f func(logproto.QuerierClient) (interface{}, error)) ([]responseFromIngesters, error) {
resps, errs := make(chan responseFromIngesters), make(chan error)
for _, ingester := range replicationSet.Ingesters {
go func(ingester ring.IngesterDesc) {
client, err := q.pool.GetClientFor(ingester.Addr)
if err != nil {
errs <- err
return
}
resp, err := f(client.(logproto.QuerierClient))
if err != nil {
errs <- err
} else {
resps <- responseFromIngesters{ingester.Addr, resp}
}
}(ingester)
}
var lastErr error
result, numErrs := []responseFromIngesters{}, 0
for range replicationSet.Ingesters {
select {
case resp := <-resps:
result = append(result, resp)
case lastErr = <-errs:
numErrs++
}
}
if numErrs > replicationSet.MaxErrors {
return nil, lastErr
}
return result, nil
}
// Query does the heavy lifting for an actual query.
func (q *Querier) Query(ctx context.Context, req *logproto.QueryRequest) (*logproto.QueryResponse, error) {
// Enforce the query timeout while querying backends
ctx, cancel := context.WithDeadline(ctx, time.Now().Add(q.cfg.QueryTimeout))
defer cancel()
iterators, err := q.getQueryIterators(ctx, req)
if err != nil {
return nil, err
}
iterator := iter.NewHeapIterator(iterators, req.Direction)
defer helpers.LogError("closing iterator", iterator.Close)
resp, _, err := iter.ReadBatch(iterator, req.Limit)
return resp, err
}
func (q *Querier) getQueryIterators(ctx context.Context, req *logproto.QueryRequest) ([]iter.EntryIterator, error) {
ingesterIterators, err := q.queryIngesters(ctx, req)
if err != nil {
return nil, err
}
chunkStoreIterators, err := q.store.LazyQuery(ctx, req)
if err != nil {
return nil, err
}
iterators := append(ingesterIterators, chunkStoreIterators)
return iterators, nil
}
func (q *Querier) queryIngesters(ctx context.Context, req *logproto.QueryRequest) ([]iter.EntryIterator, error) {
clients, err := q.forAllIngesters(func(client logproto.QuerierClient) (interface{}, error) {
return client.Query(ctx, req)
})
if err != nil {
return nil, err
}
iterators := make([]iter.EntryIterator, len(clients))
for i := range clients {
iterators[i] = iter.NewQueryClientIterator(clients[i].response.(logproto.Querier_QueryClient), req.Direction)
}
return iterators, nil
}
// Label does the heavy lifting for a Label query.
func (q *Querier) Label(ctx context.Context, req *logproto.LabelRequest) (*logproto.LabelResponse, error) {
// Enforce the query timeout while querying backends
ctx, cancel := context.WithDeadline(ctx, time.Now().Add(q.cfg.QueryTimeout))
defer cancel()
resps, err := q.forAllIngesters(func(client logproto.QuerierClient) (interface{}, error) {
return client.Label(ctx, req)
})
if err != nil {
return nil, err
}
from, through := model.TimeFromUnixNano(req.Start.UnixNano()), model.TimeFromUnixNano(req.End.UnixNano())
var storeValues []string
if req.Values {
storeValues, err = q.store.LabelValuesForMetricName(ctx, from, through, "logs", req.Name)
if err != nil {
return nil, err
}
} else {
storeValues, err = q.store.LabelNamesForMetricName(ctx, from, through, "logs")
if err != nil {
return nil, err
}
}
results := make([][]string, 0, len(resps))
for _, resp := range resps {
results = append(results, resp.response.(*logproto.LabelResponse).Values)
}
results = append(results, storeValues)
return &logproto.LabelResponse{
Values: mergeLists(results...),
}, nil
}
// Check implements the grpc healthcheck
func (*Querier) Check(_ context.Context, _ *grpc_health_v1.HealthCheckRequest) (*grpc_health_v1.HealthCheckResponse, error) {
return &grpc_health_v1.HealthCheckResponse{Status: grpc_health_v1.HealthCheckResponse_SERVING}, nil
}
func mergeLists(ss ...[]string) []string {
switch len(ss) {
case 0:
return nil
case 1:
return ss[0]
case 2:
return mergePair(ss[0], ss[1])
default:
n := len(ss) / 2
return mergePair(mergeLists(ss[:n]...), mergeLists(ss[n:]...))
}
}
func mergePair(s1, s2 []string) []string {
i, j := 0, 0
result := make([]string, 0, len(s1)+len(s2))
for i < len(s1) && j < len(s2) {
if s1[i] < s2[j] {
result = append(result, s1[i])
i++
} else if s1[i] > s2[j] {
result = append(result, s2[j])
j++
} else {
result = append(result, s1[i])
i++
j++
}
}
for ; i < len(s1); i++ {
result = append(result, s1[i])
}
for ; j < len(s2); j++ {
result = append(result, s2[j])
}
return result
}
// Tail keeps getting matching logs from all ingesters for given query
func (q *Querier) Tail(ctx context.Context, req *logproto.TailRequest) (*Tailer, error) {
// Enforce the query timeout except when tailing, otherwise the tailing
// will be terminated once the query timeout is reached
tailCtx := ctx
queryCtx, cancelQuery := context.WithDeadline(ctx, time.Now().Add(q.cfg.QueryTimeout))
defer cancelQuery()
clients, err := q.forAllIngesters(func(client logproto.QuerierClient) (interface{}, error) {
return client.Tail(tailCtx, req)
})
if err != nil {
return nil, err
}
tailClients := make(map[string]logproto.Querier_TailClient)
for i := range clients {
tailClients[clients[i].addr] = clients[i].response.(logproto.Querier_TailClient)
}
histReq := logproto.QueryRequest{
Query: req.Query,
Start: req.Start,
End: time.Now(),
Limit: req.Limit,
Direction: logproto.BACKWARD,
Regex: req.Regex,
}
histIterators, err := q.getQueryIterators(queryCtx, &histReq)
if err != nil {
return nil, err
}
reversedIterator, err := iter.NewEntryIteratorForward(iter.NewHeapIterator(histIterators, logproto.BACKWARD), req.Limit,true)
if err != nil {
return nil, err
}
return newTailer(
time.Duration(req.DelayFor)*time.Second,
tailClients,
reversedIterator,
func(connectedIngestersAddr []string) (map[string]logproto.Querier_TailClient, error) {
return q.tailDisconnectedIngesters(tailCtx, req, connectedIngestersAddr)
},
q.cfg.TailMaxDuration,
), nil
}
// passed to tailer for (re)connecting to new or disconnected ingesters
func (q *Querier) tailDisconnectedIngesters(ctx context.Context, req *logproto.TailRequest, connectedIngestersAddr []string) (map[string]logproto.Querier_TailClient, error) {
tailClients := make(map[string]logproto.Querier_TailClient)
for i := range connectedIngestersAddr {
tailClients[connectedIngestersAddr[i]] = nil
}
disconnectedIngesters := []ring.IngesterDesc{}
replicationSet, err := q.ring.GetAll()
if err != nil {
return nil, err
}
for _, ingester := range replicationSet.Ingesters {
if _, isOk := tailClients[ingester.Addr]; isOk {
delete(tailClients, ingester.Addr)
} else {
disconnectedIngesters = append(disconnectedIngesters, ingester)
}
}
clients, err := q.forGivenIngesters(ring.ReplicationSet{Ingesters: disconnectedIngesters}, func(client logproto.QuerierClient) (interface{}, error) {
return client.Tail(ctx, req)
})
if err != nil {
return nil, err
}
for i := range clients {
tailClients[clients[i].addr] = clients[i].response.(logproto.Querier_TailClient)
}
return tailClients, nil
}