Merge branch 'master' into reduce-config-option

pull/469/head
Xiang Dai 6 years ago
commit 274170ffec
  1. 1
      .circleci/config.yml
  2. 4
      cmd/logcli/client.go
  3. 1
      cmd/logcli/main.go
  4. 6
      cmd/logcli/tail.go
  5. 10
      pkg/distributor/distributor.go
  6. 39
      pkg/ingester/ingester.go
  7. 53
      pkg/ingester/instance.go
  8. 55
      pkg/ingester/stream.go
  9. 243
      pkg/ingester/tailer.go
  10. 28
      pkg/iter/iterator.go
  11. 2
      pkg/iter/iterator_test.go
  12. 1088
      pkg/logproto/logproto.pb.go
  13. 18
      pkg/logproto/logproto.proto
  14. 9
      pkg/logql/ast.go
  15. 19
      pkg/loki/loki.go
  16. 19
      pkg/loki/modules.go
  17. 106
      pkg/querier/http.go
  18. 124
      pkg/querier/querier.go
  19. 346
      pkg/querier/tail.go
  20. 126
      pkg/querier/tail_test.go
  21. 11
      pkg/util/ring.go
  22. 2
      production/helm/promtail/Chart.yaml
  23. 12
      production/loki-mixin/dashboards.libsonnet

@ -188,6 +188,7 @@ jobs:
name: Install k3s
command: |
curl -sfL https://get.k3s.io | sh -
sudo chmod 755 /etc/rancher/k3s/k3s.yaml
mkdir -p ~/.kube
cp /etc/rancher/k3s/k3s.yaml ~/.kube/config
- run:

@ -21,7 +21,7 @@ const (
queryPath = "/api/prom/query?query=%s&limit=%d&start=%d&end=%d&direction=%s&regexp=%s"
labelsPath = "/api/prom/label"
labelValuesPath = "/api/prom/label/%s/values"
tailPath = "/api/prom/tail?query=%s&regexp=%s"
tailPath = "/api/prom/tail?query=%s&regexp=%s&delay_for=%d"
)
func query(from, through time.Time, direction logproto.Direction) (*logproto.QueryResponse, error) {
@ -98,7 +98,7 @@ func doRequest(path string, out interface{}) error {
}
func liveTailQueryConn() (*websocket.Conn, error) {
path := fmt.Sprintf(tailPath, url.QueryEscape(*queryStr), url.QueryEscape(*regexpStr))
path := fmt.Sprintf(tailPath, url.QueryEscape(*queryStr), url.QueryEscape(*regexpStr), *delayFor)
return wsConnect(path)
}

@ -26,6 +26,7 @@ var (
since = queryCmd.Flag("since", "Lookback window.").Default("1h").Duration()
forward = queryCmd.Flag("forward", "Scan forwards through logs.").Default("false").Bool()
tail = queryCmd.Flag("tail", "Tail the logs").Short('t').Default("false").Bool()
delayFor = queryCmd.Flag("delay-for", "Delay in tailing by number of seconds to accumulate logs").Default("0").Int()
noLabels = queryCmd.Flag("no-labels", "Do not print any labels").Default("false").Bool()
ignoreLabelsKey = queryCmd.Flag("exclude-label", "Exclude labels given the provided key during output.").Strings()
showLabelsKey = queryCmd.Flag("include-label", "Include labels given the provided key during output.").Strings()

@ -59,5 +59,11 @@ func tailQuery() {
printLogEntry(entry.Timestamp, labels, entry.Line)
}
}
if len(tailReponse.DroppedEntries) != 0 {
log.Println("Server dropped following entries due to slow client")
for _, d := range tailReponse.DroppedEntries {
log.Println(d.Timestamp, d.Labels)
}
}
}
}

@ -3,7 +3,6 @@ package distributor
import (
"context"
"flag"
"hash/fnv"
"sync/atomic"
"time"
@ -148,7 +147,7 @@ func (d *Distributor) Push(ctx context.Context, req *logproto.PushRequest) (*log
continue
}
stream.Entries = entries
keys = append(keys, tokenFor(userID, stream.Labels))
keys = append(keys, util.TokenFor(userID, stream.Labels))
streams = append(streams, streamTracker{
stream: stream,
})
@ -264,13 +263,6 @@ func (d *Distributor) sendSamplesErr(ctx context.Context, ingester ring.Ingester
return err
}
func tokenFor(userID, labels string) uint32 {
h := fnv.New32()
_, _ = h.Write([]byte(userID))
_, _ = h.Write([]byte(labels))
return h.Sum32()
}
// Check implements the grpc healthcheck
func (*Distributor) Check(_ context.Context, _ *grpc_health_v1.HealthCheckRequest) (*grpc_health_v1.HealthCheckResponse, error) {
return &grpc_health_v1.HealthCheckResponse{Status: grpc_health_v1.HealthCheckResponse_SERVING}, nil

@ -2,6 +2,7 @@ package ingester
import (
"context"
"errors"
"flag"
"net/http"
"sync"
@ -60,8 +61,9 @@ type Ingester struct {
lifecycler *ring.Lifecycler
store ChunkStore
done sync.WaitGroup
quit chan struct{}
done sync.WaitGroup
quit chan struct{}
quitting chan struct{}
// One queue per flush thread. Fingerprint is used to
// pick a queue.
@ -82,6 +84,7 @@ func New(cfg Config, store ChunkStore) (*Ingester, error) {
store: store,
quit: make(chan struct{}),
flushQueues: make([]*util.PriorityQueue, cfg.ConcurrentFlushes),
quitting: make(chan struct{}),
}
i.flushQueuesDone.Add(cfg.ConcurrentFlushes)
@ -127,6 +130,14 @@ func (i *Ingester) Shutdown() {
i.lifecycler.Shutdown()
}
// Stopping helps cleaning up resources before actual shutdown
func (i *Ingester) Stopping() {
close(i.quitting)
for _, instance := range i.getInstances() {
instance.closeTailers()
}
}
// StopIncomingRequests implements ring.Lifecycler.
func (i *Ingester) StopIncomingRequests() {
@ -230,3 +241,27 @@ func (i *Ingester) getInstances() []*instance {
}
return instances
}
// Tail logs matching given query
func (i *Ingester) Tail(req *logproto.TailRequest, queryServer logproto.Querier_TailServer) error {
select {
case <-i.quitting:
return errors.New("Ingester is stopping")
default:
}
instanceID, err := user.ExtractOrgID(queryServer.Context())
if err != nil {
return err
}
instance := i.getOrCreateInstance(instanceID)
tailer, err := newTailer(instanceID, req.Query, req.Regex, queryServer)
if err != nil {
return err
}
instance.addNewTailer(tailer)
tailer.loop()
return nil
}

@ -53,6 +53,8 @@ type instance struct {
streamsRemovedTotal prometheus.Counter
blockSize int
tailers map[uint32]*tailer
tailerMtx sync.RWMutex
}
func newInstance(instanceID string, blockSize int) *instance {
@ -65,6 +67,7 @@ func newInstance(instanceID string, blockSize int) *instance {
streamsRemovedTotal: streamsRemovedTotal.WithLabelValues(instanceID),
blockSize: blockSize,
tailers: map[uint32]*tailer{},
}
}
@ -87,6 +90,7 @@ func (i *instance) Push(ctx context.Context, req *logproto.PushRequest) error {
i.index.Add(labels, fp)
i.streams[fp] = stream
i.streamsCreatedTotal.Inc()
i.addTailersToNewStream(stream)
}
if err := stream.Push(ctx, s.Entries); err != nil {
@ -126,7 +130,7 @@ func (i *instance) Query(req *logproto.QueryRequest, queryServer logproto.Querie
return sendBatches(iter, queryServer, req.Limit)
}
func (i *instance) Label(ctx context.Context, req *logproto.LabelRequest) (*logproto.LabelResponse, error) {
func (i *instance) Label(_ context.Context, req *logproto.LabelRequest) (*logproto.LabelResponse, error) {
var labels []string
if req.Values {
values := i.index.LabelValues(req.Name)
@ -175,6 +179,53 @@ outer:
return iterators, nil
}
func (i *instance) addNewTailer(t *tailer) {
i.streamsMtx.RLock()
for _, stream := range i.streams {
if stream.matchesTailer(t) {
stream.addTailer(t)
}
}
i.streamsMtx.RUnlock()
i.tailerMtx.Lock()
defer i.tailerMtx.Unlock()
i.tailers[t.getID()] = t
}
func (i *instance) addTailersToNewStream(stream *stream) {
closedTailers := []uint32{}
i.tailerMtx.RLock()
for _, t := range i.tailers {
if t.isClosed() {
closedTailers = append(closedTailers, t.getID())
continue
}
if stream.matchesTailer(t) {
stream.addTailer(t)
}
}
i.tailerMtx.RUnlock()
if len(closedTailers) != 0 {
i.tailerMtx.Lock()
defer i.tailerMtx.Unlock()
for _, closedTailer := range closedTailers {
delete(i.tailers, closedTailer)
}
}
}
func (i *instance) closeTailers() {
i.tailerMtx.Lock()
defer i.tailerMtx.Unlock()
for _, t := range i.tailers {
t.close()
}
}
func isDone(ctx context.Context) bool {
select {
case <-ctx.Done():

@ -3,6 +3,7 @@ package ingester
import (
"context"
"net/http"
"sync"
"time"
"github.com/cortexproject/cortex/pkg/ingester/client"
@ -49,6 +50,9 @@ type stream struct {
fp model.Fingerprint
labels []client.LabelAdapter
blockSize int
tailers map[uint32]*tailer
tailerMtx sync.RWMutex
}
type chunkDesc struct {
@ -64,6 +68,7 @@ func newStream(fp model.Fingerprint, labels []client.LabelAdapter, blockSize int
fp: fp,
labels: labels,
blockSize: blockSize,
tailers: map[uint32]*tailer{},
}
}
@ -75,6 +80,8 @@ func (s *stream) Push(_ context.Context, entries []logproto.Entry) error {
chunksCreatedTotal.Inc()
}
storedEntries := []logproto.Entry{}
// Don't fail on the first append error - if samples are sent out of order,
// we still want to append the later ones.
var appendErr error
@ -93,10 +100,40 @@ func (s *stream) Push(_ context.Context, entries []logproto.Entry) error {
}
if err := chunk.chunk.Append(&entries[i]); err != nil {
appendErr = err
} else {
// send only stored entries to tailers
storedEntries = append(storedEntries, entries[i])
}
chunk.lastUpdated = time.Now()
}
if len(storedEntries) != 0 {
go func() {
stream := logproto.Stream{Labels: client.FromLabelAdaptersToLabels(s.labels).String(), Entries: storedEntries}
closedTailers := []uint32{}
s.tailerMtx.RLock()
for _, tailer := range s.tailers {
if tailer.isClosed() {
closedTailers = append(closedTailers, tailer.getID())
continue
}
tailer.send(stream)
}
s.tailerMtx.RUnlock()
if len(closedTailers) != 0 {
s.tailerMtx.Lock()
defer s.tailerMtx.Unlock()
for _, closedTailerID := range closedTailers {
delete(s.tailers, closedTailerID)
}
}
}()
}
if appendErr == chunkenc.ErrOutOfOrder {
return httpgrpc.Errorf(http.StatusBadRequest, "entry out of order for stream: %s", client.FromLabelAdaptersToLabels(s.labels).String())
}
@ -108,12 +145,12 @@ func (s *stream) Push(_ context.Context, entries []logproto.Entry) error {
func (s *stream) Iterator(from, through time.Time, direction logproto.Direction) (iter.EntryIterator, error) {
iterators := make([]iter.EntryIterator, 0, len(s.chunks))
for _, c := range s.chunks {
iter, err := c.chunk.Iterator(from, through, direction)
itr, err := c.chunk.Iterator(from, through, direction)
if err != nil {
return nil, err
}
if iter != nil {
iterators = append(iterators, iter)
if itr != nil {
iterators = append(iterators, itr)
}
}
@ -125,3 +162,15 @@ func (s *stream) Iterator(from, through time.Time, direction logproto.Direction)
return iter.NewNonOverlappingIterator(iterators, client.FromLabelAdaptersToLabels(s.labels).String()), nil
}
func (s *stream) addTailer(t *tailer) {
s.tailerMtx.Lock()
defer s.tailerMtx.Unlock()
s.tailers[t.getID()] = t
}
func (s *stream) matchesTailer(t *tailer) bool {
metric := client.FromLabelAdaptersToMetric(s.labels)
return t.isWatchingLabels(metric)
}

@ -0,0 +1,243 @@
package ingester
import (
"encoding/binary"
"fmt"
"hash/fnv"
"sync"
"time"
"github.com/grafana/loki/pkg/iter"
"github.com/cortexproject/cortex/pkg/util"
"github.com/go-kit/kit/log/level"
"github.com/grafana/loki/pkg/logproto"
"github.com/grafana/loki/pkg/logql"
"github.com/prometheus/common/model"
"github.com/prometheus/prometheus/pkg/labels"
)
const bufferSizeForTailResponse = 5
type tailer struct {
id uint32
orgID string
matchers []*labels.Matcher
expr logql.Expr
sendChan chan *logproto.Stream
done chan struct{}
closeMtx sync.Mutex
blockedAt *time.Time
blockedMtx sync.RWMutex
droppedStreams []*logproto.DroppedStream
conn logproto.Querier_TailServer
}
func newTailer(orgID, query, regex string, conn logproto.Querier_TailServer) (*tailer, error) {
expr, err := logql.ParseExpr(query)
if err != nil {
return nil, err
}
matchers := expr.Matchers()
if regex != "" {
expr = logql.NewFilterExpr(expr, labels.MatchRegexp, regex)
}
return &tailer{
orgID: orgID,
matchers: matchers,
sendChan: make(chan *logproto.Stream, bufferSizeForTailResponse),
conn: conn,
droppedStreams: []*logproto.DroppedStream{},
id: generateUniqueID(orgID, query, regex),
done: make(chan struct{}),
expr: expr,
}, nil
}
func (t *tailer) loop() {
var stream *logproto.Stream
var err error
var ok bool
ticker := time.NewTicker(3 * time.Second)
defer ticker.Stop()
for {
select {
case <-ticker.C:
err := t.conn.Context().Err()
if err != nil {
t.close()
return
}
case <-t.done:
return
case stream, ok = <-t.sendChan:
if !ok {
return
} else if stream == nil {
continue
}
// while sending new stream pop lined up dropped streams metadata for sending to querier
tailResponse := logproto.TailResponse{Stream: stream, DroppedStreams: t.popDroppedStreams()}
err = t.conn.Send(&tailResponse)
if err != nil {
level.Error(util.Logger).Log("Error writing to tail client", fmt.Sprintf("%v", err))
t.close()
return
}
}
}
}
func (t *tailer) send(stream logproto.Stream) {
if t.isClosed() {
return
}
// if we are already dropping streams due to blocked connection, drop new streams directly to save some effort
if blockedSince := t.blockedSince(); blockedSince != nil {
if blockedSince.Before(time.Now().Add(-time.Second * 15)) {
t.close()
return
}
t.dropStream(stream)
return
}
err := t.filterEntriesInStream(&stream)
if err != nil {
t.dropStream(stream)
return
}
if len(stream.Entries) == 0 {
return
}
select {
case t.sendChan <- &stream:
default:
t.dropStream(stream)
}
}
func (t *tailer) filterEntriesInStream(stream *logproto.Stream) error {
querier := logql.QuerierFunc(func(matchers []*labels.Matcher) (iter.EntryIterator, error) {
return iter.NewStreamIterator(stream), nil
})
itr, err := t.expr.Eval(querier)
if err != nil {
return err
}
filteredEntries := new([]logproto.Entry)
for itr.Next() {
*filteredEntries = append(*filteredEntries, itr.Entry())
}
stream.Entries = *filteredEntries
return nil
}
// Returns true if tailer is interested in the passed labelset
func (t *tailer) isWatchingLabels(metric model.Metric) bool {
for _, matcher := range t.matchers {
if !matcher.Matches(string(metric[model.LabelName(matcher.Name)])) {
return false
}
}
return true
}
func (t *tailer) isClosed() bool {
select {
case <-t.done:
return true
default:
return false
}
}
func (t *tailer) close() {
if t.isClosed() {
return
}
t.closeMtx.Lock()
defer t.closeMtx.Unlock()
if t.isClosed() {
return
}
close(t.done)
close(t.sendChan)
}
func (t *tailer) blockedSince() *time.Time {
t.blockedMtx.RLock()
defer t.blockedMtx.RUnlock()
return t.blockedAt
}
func (t *tailer) dropStream(stream logproto.Stream) {
if len(stream.Entries) == 0 {
return
}
t.blockedMtx.Lock()
defer t.blockedMtx.Unlock()
if t.blockedAt == nil {
blockedAt := time.Now()
t.blockedAt = &blockedAt
}
droppedStream := logproto.DroppedStream{
From: stream.Entries[0].Timestamp,
To: stream.Entries[len(stream.Entries)-1].Timestamp,
Labels: stream.Labels,
}
t.droppedStreams = append(t.droppedStreams, &droppedStream)
}
func (t *tailer) popDroppedStreams() []*logproto.DroppedStream {
t.blockedMtx.Lock()
defer t.blockedMtx.Unlock()
if t.blockedAt == nil {
return nil
}
droppedStreams := t.droppedStreams
t.droppedStreams = []*logproto.DroppedStream{}
t.blockedAt = nil
return droppedStreams
}
func (t *tailer) getID() uint32 {
return t.id
}
// An id is useful in managing tailer instances
func generateUniqueID(orgID, query, regex string) uint32 {
uniqueID := fnv.New32()
_, _ = uniqueID.Write([]byte(orgID))
_, _ = uniqueID.Write([]byte(query))
_, _ = uniqueID.Write([]byte(regex))
timeNow := make([]byte, 8)
binary.LittleEndian.PutUint64(timeNow, uint64(time.Now().UnixNano()))
_, _ = uniqueID.Write(timeNow)
return uniqueID.Sum32()
}

@ -27,7 +27,8 @@ type streamIterator struct {
labels string
}
func newStreamIterator(stream *logproto.Stream) EntryIterator {
// NewStreamIterator iterates over entries in a stream.
func NewStreamIterator(stream *logproto.Stream) EntryIterator {
return &streamIterator{
i: -1,
entries: stream.Entries,
@ -97,6 +98,15 @@ func (h iteratorMaxHeap) Less(i, j int) bool {
return h.iteratorHeap[i].Labels() > h.iteratorHeap[j].Labels()
}
// HeapIterator iterates over a heap of iterators with ability to push new iterators and get some properties like time of entry at peek and len
// Not safe for concurrent use
type HeapIterator interface {
EntryIterator
Peek() time.Time
Len() int
Push(EntryIterator)
}
// heapIterator iterates over a heap of iterators.
type heapIterator struct {
heap interface {
@ -110,7 +120,7 @@ type heapIterator struct {
// NewHeapIterator returns a new iterator which uses a heap to merge together
// entries for multiple interators.
func NewHeapIterator(is []EntryIterator, direction logproto.Direction) EntryIterator {
func NewHeapIterator(is []EntryIterator, direction logproto.Direction) HeapIterator {
result := &heapIterator{}
switch direction {
case logproto.BACKWARD:
@ -141,6 +151,10 @@ func (i *heapIterator) requeue(ei EntryIterator, advanced bool) {
helpers.LogError("closing iterator", ei.Close)
}
func (i *heapIterator) Push(ei EntryIterator) {
i.requeue(ei, false)
}
type tuple struct {
logproto.Entry
EntryIterator
@ -236,11 +250,19 @@ func (i *heapIterator) Close() error {
return nil
}
func (i *heapIterator) Peek() time.Time {
return i.heap.Peek().Entry().Timestamp
}
func (i *heapIterator) Len() int {
return i.heap.Len()
}
// NewQueryResponseIterator returns an iterator over a QueryResponse.
func NewQueryResponseIterator(resp *logproto.QueryResponse, direction logproto.Direction) EntryIterator {
is := make([]EntryIterator, 0, len(resp.Streams))
for i := range resp.Streams {
is = append(is, newStreamIterator(resp.Streams[i]))
is = append(is, NewStreamIterator(resp.Streams[i]))
}
return NewHeapIterator(is, direction)
}

@ -159,7 +159,7 @@ func mkStreamIterator(f generator, labels string) EntryIterator {
for i := int64(0); i < testSize; i++ {
entries = append(entries, f(i))
}
return newStreamIterator(&logproto.Stream{
return NewStreamIterator(&logproto.Stream{
Entries: entries,
Labels: labels,
})

File diff suppressed because it is too large Load Diff

@ -12,6 +12,7 @@ service Pusher {
service Querier {
rpc Query(QueryRequest) returns (stream QueryResponse) {};
rpc Label(LabelRequest) returns (LabelResponse) {};
rpc Tail(TailRequest) returns (stream TailResponse) {};
}
message PushRequest {
@ -57,3 +58,20 @@ message Entry {
google.protobuf.Timestamp timestamp = 1 [(gogoproto.stdtime) = true, (gogoproto.nullable) = false, (gogoproto.jsontag) = "ts"];
string line = 2 [(gogoproto.jsontag) = "line"];
}
message TailRequest {
string query = 1;
string regex = 2;
uint32 delayFor = 3;
}
message TailResponse {
Stream stream = 1;
repeated DroppedStream droppedStreams = 2;
}
message DroppedStream {
google.protobuf.Timestamp from = 1 [(gogoproto.stdtime) = true, (gogoproto.nullable) = false];
google.protobuf.Timestamp to = 2 [(gogoproto.stdtime) = true, (gogoproto.nullable) = false];
string labels = 3;
}

@ -26,6 +26,7 @@ type Querier interface {
// Expr is a LogQL expression.
type Expr interface {
Eval(Querier) (iter.EntryIterator, error)
Matchers() []*labels.Matcher
}
type matchersExpr struct {
@ -36,12 +37,20 @@ func (e *matchersExpr) Eval(q Querier) (iter.EntryIterator, error) {
return q.Query(e.matchers)
}
func (e *matchersExpr) Matchers() []*labels.Matcher {
return e.matchers
}
type filterExpr struct {
left Expr
ty labels.MatchType
match string
}
func (e *filterExpr) Matchers() []*labels.Matcher {
return e.left.Matchers()
}
// NewFilterExpr wraps an existing Expr with a next filter expression.
func NewFilterExpr(left Expr, ty labels.MatchType, match string) Expr {
return &filterExpr{

@ -138,6 +138,7 @@ func (t *Loki) Run() error {
// Stop gracefully stops a Loki.
func (t *Loki) Stop() error {
t.stopping(t.cfg.Target)
t.server.Shutdown()
t.stop(t.cfg.Target)
return nil
@ -160,3 +161,21 @@ func (t *Loki) stopModule(m moduleName) {
}
}
}
func (t *Loki) stopping(m moduleName) {
t.stoppingModule(m)
deps := orderedDeps(m)
// iterate over our deps in reverse order and call stoppingModule
for i := len(deps) - 1; i >= 0; i-- {
t.stoppingModule(deps[i])
}
}
func (t *Loki) stoppingModule(m moduleName) {
level.Info(util.Logger).Log("msg", "notifying module about stopping", "module", m)
if modules[m].stopping != nil {
if err := modules[m].stopping(t); err != nil {
level.Error(util.Logger).Log("msg", "error stopping", "module", m, "err", err)
}
}
}

@ -168,6 +168,11 @@ func (t *Loki) stopIngester() error {
return nil
}
func (t *Loki) stoppingIngester() error {
t.ingester.Stopping()
return nil
}
func (t *Loki) initTableManager() error {
err := t.cfg.SchemaConfig.Load()
if err != nil {
@ -269,9 +274,10 @@ func orderedDeps(m moduleName) []moduleName {
}
type module struct {
deps []moduleName
init func(t *Loki) error
stop func(t *Loki) error
deps []moduleName
init func(t *Loki) error
stopping func(t *Loki) error
stop func(t *Loki) error
}
var modules = map[moduleName]module{
@ -300,9 +306,10 @@ var modules = map[moduleName]module{
},
Ingester: {
deps: []moduleName{Store, Server},
init: (*Loki).initIngester,
stop: (*Loki).stopIngester,
deps: []moduleName{Store, Server},
init: (*Loki).initIngester,
stop: (*Loki).stopIngester,
stopping: (*Loki).stoppingIngester,
},
Querier: {

@ -21,15 +21,12 @@ import (
)
const (
defaultQueryLimit = 100
defaulSince = 1 * time.Hour
defaultQueryLimit = 100
defaulSince = 1 * time.Hour
wsPingPeriod = 1 * time.Second
maxDelayForInTailing = 5
)
// TailResponse represents response for tail query
type TailResponse struct {
Streams []*logproto.Stream `json:"streams"`
}
// nolint
func intParam(values url.Values, name string, def int) (int, error) {
value := values.Get(name)
@ -103,6 +100,25 @@ func httpRequestToQueryRequest(httpRequest *http.Request) (*logproto.QueryReques
return &queryRequest, nil
}
func httpRequestToTailRequest(httpRequest *http.Request) (*logproto.TailRequest, error) {
params := httpRequest.URL.Query()
tailRequest := logproto.TailRequest{
Regex: params.Get("regexp"),
Query: params.Get("query"),
}
// delay_for is used to allow server to let slow loggers catch up.
// Entries would be accumulated in a heap until they become older than now()-<delay_for>
delayFor, err := intParam(params, "delay_for", 0)
if err != nil {
return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error())
}
tailRequest.DelayFor = uint32(delayFor)
return &tailRequest, nil
}
// QueryHandler is a http.HandlerFunc for queries.
func (q *Querier) QueryHandler(w http.ResponseWriter, r *http.Request) {
request, err := httpRequestToQueryRequest(r)
@ -148,12 +164,18 @@ func (q *Querier) TailHandler(w http.ResponseWriter, r *http.Request) {
CheckOrigin: func(r *http.Request) bool { return true },
}
queryRequestPtr, err := httpRequestToQueryRequest(r)
tailRequestPtr, err := httpRequestToTailRequest(r)
if err != nil {
server.WriteError(w, err)
return
}
if tailRequestPtr.DelayFor > maxDelayForInTailing {
server.WriteError(w, fmt.Errorf("delay_for can't be greater than %d", maxDelayForInTailing))
level.Error(util.Logger).Log("Error in upgrading websocket", fmt.Sprintf("%v", err))
return
}
conn, err := upgrader.Upgrade(w, r, nil)
if err != nil {
level.Error(util.Logger).Log("Error in upgrading websocket", fmt.Sprintf("%v", err))
@ -161,38 +183,62 @@ func (q *Querier) TailHandler(w http.ResponseWriter, r *http.Request) {
}
defer func() {
err := conn.Close()
level.Error(util.Logger).Log("Error closing websocket", fmt.Sprintf("%v", err))
if err := conn.Close(); err != nil {
level.Error(util.Logger).Log("Error closing websocket", fmt.Sprintf("%v", err))
}
}()
// response from httpRequestToQueryRequest is a ptr, if we keep passing pointer down the call then it would stay on
// heap until connection to websocket stays open
queryRequest := *queryRequestPtr
itr := q.tailQuery(r.Context(), &queryRequest)
tailRequest := *tailRequestPtr
stream := logproto.Stream{}
tailResponse := TailResponse{[]*logproto.Stream{
&stream,
}}
tailer, err := q.Tail(r.Context(), &tailRequest)
if err != nil {
if err := conn.WriteMessage(websocket.CloseMessage, websocket.FormatCloseMessage(websocket.CloseInternalServerErr, err.Error())); err != nil {
level.Error(util.Logger).Log("Error connecting to ingesters for tailing", fmt.Sprintf("%v", err))
}
return
}
defer func() {
if err := tailer.close(); err != nil {
level.Error(util.Logger).Log("Error closing Tailer", fmt.Sprintf("%v", err))
}
}()
for itr.Next() {
stream.Entries = []logproto.Entry{itr.Entry()}
stream.Labels = itr.Labels()
ticker := time.NewTicker(wsPingPeriod)
defer ticker.Stop()
var response *TailResponse
responseChan := tailer.getResponseChan()
closeErrChan := tailer.getCloseErrorChan()
for {
select {
case response = <-responseChan:
err := conn.WriteJSON(*response)
if err != nil {
level.Error(util.Logger).Log("Error writing to websocket", fmt.Sprintf("%v", err))
if err := conn.WriteMessage(websocket.CloseMessage, websocket.FormatCloseMessage(websocket.CloseInternalServerErr, err.Error())); err != nil {
level.Error(util.Logger).Log("Error writing close message to websocket", fmt.Sprintf("%v", err))
}
return
}
err := conn.WriteJSON(tailResponse)
if err != nil {
level.Error(util.Logger).Log("Error writing to websocket", fmt.Sprintf("%v", err))
case err := <-closeErrChan:
level.Error(util.Logger).Log("Error from iterator", fmt.Sprintf("%v", err))
if err := conn.WriteMessage(websocket.CloseMessage, websocket.FormatCloseMessage(websocket.CloseInternalServerErr, err.Error())); err != nil {
level.Error(util.Logger).Log("Error writing close message to websocket", fmt.Sprintf("%v", err))
}
break
}
}
if err := itr.Error(); err != nil {
level.Error(util.Logger).Log("Error from iterator", fmt.Sprintf("%v", err))
if err := conn.WriteMessage(websocket.CloseMessage, websocket.FormatCloseMessage(websocket.CloseInternalServerErr, err.Error())); err != nil {
level.Error(util.Logger).Log("Error writing close message to websocket", fmt.Sprintf("%v", err))
return
case <-ticker.C:
// This is to periodically check whether connection is active, useful to clean up dead connections when there are no entries to send
if err := conn.WriteMessage(websocket.PingMessage, nil); err != nil {
level.Error(util.Logger).Log("Error writing ping message to websocket", fmt.Sprintf("%v", err))
if err := conn.WriteMessage(websocket.CloseMessage, websocket.FormatCloseMessage(websocket.CloseInternalServerErr, err.Error())); err != nil {
level.Error(util.Logger).Log("Error writing close message to websocket", fmt.Sprintf("%v", err))
}
return
}
}
}
}

@ -3,11 +3,14 @@ package querier
import (
"context"
"flag"
"time"
"github.com/cortexproject/cortex/pkg/chunk"
cortex_client "github.com/cortexproject/cortex/pkg/ingester/client"
"github.com/cortexproject/cortex/pkg/ring"
"github.com/cortexproject/cortex/pkg/util"
token_util "github.com/grafana/loki/pkg/util"
"github.com/weaveworks/common/user"
"google.golang.org/grpc/health/grpc_health_v1"
"github.com/grafana/loki/pkg/helpers"
@ -46,15 +49,26 @@ func New(cfg Config, clientCfg client.Config, ring ring.ReadRing, store chunk.St
}, nil
}
type responseFromIngesters struct {
addr string
response interface{}
}
// 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)) ([]interface{}, error) {
func (q *Querier) forAllIngesters(f func(logproto.QuerierClient) (interface{}, error)) ([]responseFromIngesters, error) {
replicationSet, err := q.ring.GetAll()
if err != nil {
return nil, err
}
resps, errs := make(chan interface{}), make(chan error)
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)
@ -67,13 +81,13 @@ func (q *Querier) forAllIngesters(f func(logproto.QuerierClient) (interface{}, e
if err != nil {
errs <- err
} else {
resps <- resp
resps <- responseFromIngesters{ingester.Addr, resp}
}
}(ingester)
}
var lastErr error
result, numErrs := []interface{}{}, 0
result, numErrs := []responseFromIngesters{}, 0
for range replicationSet.Ingesters {
select {
case resp := <-resps:
@ -120,7 +134,7 @@ func (q *Querier) queryIngesters(ctx context.Context, req *logproto.QueryRequest
iterators := make([]iter.EntryIterator, len(clients))
for i := range clients {
iterators[i] = iter.NewQueryClientIterator(clients[i].(logproto.Querier_QueryClient), req.Direction)
iterators[i] = iter.NewQueryClientIterator(clients[i].response.(logproto.Querier_QueryClient), req.Direction)
}
return iterators, nil
}
@ -136,7 +150,7 @@ func (q *Querier) Label(ctx context.Context, req *logproto.LabelRequest) (*logpr
results := make([][]string, 0, len(resps))
for _, resp := range resps {
results = append(results, resp.(*logproto.LabelResponse).Values)
results = append(results, resp.response.(*logproto.LabelResponse).Values)
}
return &logproto.LabelResponse{
@ -212,3 +226,101 @@ func mergePair(s1, s2 []string) []string {
}
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) {
clients, err := q.forAllIngesters(func(client logproto.QuerierClient) (interface{}, error) {
return client.Tail(ctx, 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)
}
return newTailer(time.Duration(req.DelayFor)*time.Second, tailClients, func(from, to time.Time, labels string) (iterator iter.EntryIterator, e error) {
return q.queryDroppedStreams(ctx, req, from, to, labels)
}, func(connectedIngestersAddr []string) (map[string]logproto.Querier_TailClient, error) {
return q.tailDisconnectedIngesters(ctx, req, connectedIngestersAddr)
}), nil
}
// passed to tailer for querying dropped streams
func (q *Querier) queryDroppedStreams(ctx context.Context, req *logproto.TailRequest, start, end time.Time, labels string) (iter.EntryIterator, error) {
userID, err := user.ExtractOrgID(ctx)
if err != nil {
return nil, err
}
key := token_util.TokenFor(userID, labels)
replicationSet, err := q.ring.Get(key, ring.Read)
if err != nil {
return nil, err
}
query := logproto.QueryRequest{
Direction: logproto.FORWARD,
Start: start,
End: end,
Query: req.Query,
Regex: req.Regex,
Limit: 10000,
}
clients, err := q.forGivenIngesters(replicationSet, func(client logproto.QuerierClient) (interface{}, error) {
return client.Query(ctx, &query)
})
if err != nil {
return nil, err
}
ingesterIterators := make([]iter.EntryIterator, len(clients))
for i := range clients {
ingesterIterators[i] = iter.NewQueryClientIterator(clients[i].response.(logproto.Querier_QueryClient), query.Direction)
}
chunkStoreIterators, err := q.queryStore(ctx, &query)
if err != nil {
return nil, err
}
iterators := append(ingesterIterators, chunkStoreIterators)
return iter.NewHeapIterator(iterators, query.Direction), 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
}

@ -1,92 +1,334 @@
package querier
import (
"context"
"fmt"
"sync"
"time"
"github.com/cortexproject/cortex/pkg/util"
"github.com/go-kit/kit/log/level"
"github.com/grafana/loki/pkg/iter"
"github.com/grafana/loki/pkg/logproto"
"github.com/pkg/errors"
)
// tailIteratorIncrement is for defining size of time window we want to query entries for
const tailIteratorIncrement = 10 * time.Second
const (
// if we are not seeing any response from ingester, how long do we want to wait by going into sleep
nextEntryWait = time.Second / 2
// delayQuerying is for delaying querying of logs for specified seconds to not miss any late entries
const delayQuerying = 10 * time.Second
// keep checking connections with ingesters in duration
checkConnectionsWithIngestersPeriod = time.Second * 5
func (q *Querier) tailQuery(ctx context.Context, queryRequest *logproto.QueryRequest) iter.EntryIterator {
queryRequest.Start = time.Now().Add(-(tailIteratorIncrement + delayQuerying))
queryRequest.Direction = logproto.FORWARD
bufferSizeForTailResponse = 10
)
type droppedEntry struct {
Timestamp time.Time
Labels string
}
// TailResponse holds response sent by tailer
type TailResponse struct {
Streams []logproto.Stream `json:"streams"`
DroppedEntries []droppedEntry `json:"dropped_entries"`
}
/*// dropped streams are collected into a heap to quickly find dropped stream which has oldest timestamp
type droppedStreamsIterator []logproto.DroppedStream
func (h droppedStreamsIterator) Len() int { return len(h) }
func (h droppedStreamsIterator) Swap(i, j int) {
h[i], h[j] = h[j], h[i]
}
func (h droppedStreamsIterator) Peek() time.Time {
return h[0].From
}
func (h *droppedStreamsIterator) Push(x interface{}) {
*h = append(*h, x.(logproto.DroppedStream))
}
func (h *droppedStreamsIterator) Pop() interface{} {
old := *h
n := len(old)
x := old[n-1]
*h = old[0 : n-1]
return x
}
func (h droppedStreamsIterator) Less(i, j int) bool {
t1, t2 := h[i].From, h[j].From
if !t1.Equal(t2) {
return t1.Before(t2)
}
return h[i].Labels < h[j].Labels
}*/
// Tailer manages complete lifecycle of a tail request
type Tailer struct {
// openStreamIterator is for streams already open which can be complete streams returned by ingester or
// dropped streams queried from ingester and store
openStreamIterator iter.HeapIterator
/*droppedStreamsIterator interface { // for holding dropped stream metadata
heap.Interface
Peek() time.Time
}*/
streamMtx sync.Mutex // for synchronizing access to openStreamIterator and droppedStreamsIterator
currEntry logproto.Entry
currLabels string
queryDroppedStreams func(from, to time.Time, labels string) (iter.EntryIterator, error)
tailDisconnectedIngesters func([]string) (map[string]logproto.Querier_TailClient, error)
return &tailIterator{
queryRequest: queryRequest,
querier: q,
ctx: ctx,
querierTailClients map[string]logproto.Querier_TailClient // addr -> grpc clients for tailing logs from ingesters
querierTailClientsMtx sync.Mutex
stopped bool
blocked bool
blockedMtx sync.RWMutex
delayFor time.Duration
responseChan chan *TailResponse
closeErrChan chan error
// when tail client is slow, drop entry and store its details in droppedEntries to notify client
droppedEntries []droppedEntry
}
func (t *Tailer) readTailClients() {
for addr, querierTailClient := range t.querierTailClients {
go t.readTailClient(addr, querierTailClient)
}
}
type tailIterator struct {
queryRequest *logproto.QueryRequest
ctx context.Context
querier *Querier
entryIterator iter.EntryIterator
err error
// keeps sending oldest entry to responseChan. If channel is blocked drop the entry
// When channel is unblocked, send details of dropped entries with current entry
func (t *Tailer) loop() {
ticker := time.NewTicker(checkConnectionsWithIngestersPeriod)
defer ticker.Stop()
tailResponse := new(TailResponse)
for {
if t.stopped {
break
}
select {
case <-ticker.C:
// Try to reconnect dropped ingesters and connect to new ingesters
if err := t.checkIngesterConnections(); err != nil {
level.Error(util.Logger).Log("Error reconnecting to disconnected ingesters", fmt.Sprintf("%v", err))
}
default:
}
if !t.next() {
if len(tailResponse.Streams) == 0 {
if len(t.querierTailClients) == 0 {
// All the connections to ingesters are dropped, try reconnecting or return error
if err := t.checkIngesterConnections(); err != nil {
level.Error(util.Logger).Log("Error reconnecting to ingesters", fmt.Sprintf("%v", err))
} else {
continue
}
if err := t.close(); err != nil {
level.Error(util.Logger).Log("Error closing Tailer", fmt.Sprintf("%v", err))
}
t.closeErrChan <- errors.New("All ingesters closed the connection")
break
}
time.Sleep(nextEntryWait)
continue
}
} else {
// If channel is blocked already, drop current entry directly to save the effort
if t.isBlocked() {
t.dropEntry(t.currEntry.Timestamp, t.currLabels, nil)
continue
}
tailResponse.Streams = append(tailResponse.Streams, logproto.Stream{Labels: t.currLabels, Entries: []logproto.Entry{t.currEntry}})
if len(tailResponse.Streams) != 100 {
continue
}
tailResponse.DroppedEntries = t.popDroppedEntries()
}
//response := []tailResponse{{Stream: logproto.Stream{Labels: t.currLabels, Entries: responses[t.currLabels]}, DroppedEntries: t.popDroppedEntries()}}
select {
case t.responseChan <- tailResponse:
default:
t.dropEntry(t.currEntry.Timestamp, t.currLabels, tailResponse.DroppedEntries)
}
tailResponse = new(TailResponse)
}
}
func (t *tailIterator) Next() bool {
var err error
var now time.Time
// Checks whether we are connected to all the ingesters to tail the logs.
// Helps in connecting to disconnected ingesters or connecting to new ingesters
func (t *Tailer) checkIngesterConnections() error {
connectedIngestersAddr := make([]string, 0, len(t.querierTailClients))
for addr := range t.querierTailClients {
connectedIngestersAddr = append(connectedIngestersAddr, addr)
}
for t.entryIterator == nil || !t.entryIterator.Next() {
t.queryRequest.End, now = t.queryRequest.Start.Add(tailIteratorIncrement), time.Now()
if t.queryRequest.End.After(now.Add(-delayQuerying)) {
time.Sleep(t.queryRequest.End.Sub(now.Add(-delayQuerying)))
newConnections, err := t.tailDisconnectedIngesters(connectedIngestersAddr)
if err != nil {
return err
}
if len(newConnections) != 0 {
for addr, tailClient := range newConnections {
t.querierTailClients[addr] = tailClient
go t.readTailClient(addr, tailClient)
}
}
return nil
}
t.entryIterator, err = t.query()
// removes disconnected tail client from map
func (t *Tailer) dropTailClient(addr string) {
t.querierTailClientsMtx.Lock()
defer t.querierTailClientsMtx.Unlock()
delete(t.querierTailClients, addr)
}
// keeps reading streams from grpc connection with ingesters
func (t *Tailer) readTailClient(addr string, querierTailClient logproto.Querier_TailClient) {
var resp *logproto.TailResponse
var err error
defer t.dropTailClient(addr)
for {
if t.stopped {
if err := querierTailClient.CloseSend(); err != nil {
level.Error(util.Logger).Log("Error closing gprc tail client", fmt.Sprintf("%v", err))
}
break
}
resp, err = querierTailClient.Recv()
if err != nil {
t.err = err
return false
level.Error(util.Logger).Log("Error receiving response from gprc tail client", fmt.Sprintf("%v", err))
break
}
t.pushTailResponseFromIngester(resp)
}
}
// pushes new streams from ingesters synchronously
func (t *Tailer) pushTailResponseFromIngester(resp *logproto.TailResponse) {
t.streamMtx.Lock()
defer t.streamMtx.Unlock()
t.openStreamIterator.Push(iter.NewStreamIterator(resp.Stream))
/*if resp.DroppedStreams != nil {
for idx := range resp.DroppedStreams {
heap.Push(t.droppedStreamsIterator, *resp.DroppedStreams[idx])
}
}*/
}
// finds oldest entry by peeking at open stream iterator and dropped stream iterator.
// if open stream iterator has oldest entry then pop it for sending it to tail client
// else pop dropped stream details, to query from ingester and store.
// Response from ingester and store is pushed to open stream for further processing
func (t *Tailer) next() bool {
t.streamMtx.Lock()
defer t.streamMtx.Unlock()
// We store the through time such that if we don't see any entries, we will
// still make forward progress. This is overwritten by any entries we might
// see to ensure pagination works.
t.queryRequest.Start = t.queryRequest.End
if t.openStreamIterator.Len() == 0 || !time.Now().After(t.openStreamIterator.Peek().Add(t.delayFor)) || !t.openStreamIterator.Next() {
return false
}
/*// if we don't have any entries or any of the entries are not older than now()-delay then return false
if !((t.openStreamIterator.Len() != 0 && time.Now().After(t.openStreamIterator.Peek().Add(t.delayFor))) || (t.droppedStreamsIterator.Len() != 0 && time.Now().After(t.droppedStreamsIterator.Peek().Add(t.delayFor)))) {
return false
}
// If any of the dropped streams are older than open streams, pop dropped stream details for querying them
if t.droppedStreamsIterator.Len() != 0 {
oldestTsFromDroppedStreams := t.droppedStreamsIterator.Peek()
if t.droppedStreamsIterator.Len() != 0 && (t.openStreamIterator.Len() == 0 || t.openStreamIterator.Peek().After(t.droppedStreamsIterator.Peek())) {
for t.droppedStreamsIterator.Len() != 0 && t.droppedStreamsIterator.Peek().Equal(oldestTsFromDroppedStreams) {
droppedStream := heap.Pop(t.droppedStreamsIterator).(logproto.DroppedStream)
iterator, err := t.queryDroppedStreams(droppedStream.From, droppedStream.To.Add(1), droppedStream.Labels)
if err != nil {
level.Error(util.Logger).Log("Error querying dropped streams", fmt.Sprintf("%v", err))
continue
}
t.openStreamIterator.Push(iterator)
}
}
}
if !t.openStreamIterator.Next() {
return false
}*/
t.currEntry = t.openStreamIterator.Entry()
t.currLabels = t.openStreamIterator.Labels()
return true
}
func (t *tailIterator) Entry() logproto.Entry {
entry := t.entryIterator.Entry()
t.queryRequest.Start = entry.Timestamp.Add(1 * time.Nanosecond)
return entry
}
func (t *Tailer) close() error {
t.streamMtx.Lock()
defer t.streamMtx.Unlock()
func (t *tailIterator) Error() error {
return t.err
t.stopped = true
return t.openStreamIterator.Close()
}
func (t *tailIterator) Labels() string {
return t.entryIterator.Labels()
func (t *Tailer) dropEntry(timestamp time.Time, labels string, alreadyDroppedEntries []droppedEntry) {
t.blockedMtx.Lock()
defer t.blockedMtx.Unlock()
t.droppedEntries = append(t.droppedEntries, alreadyDroppedEntries...)
t.droppedEntries = append(t.droppedEntries, droppedEntry{timestamp, labels})
}
func (t *tailIterator) Close() error {
return t.entryIterator.Close()
func (t *Tailer) isBlocked() bool {
t.blockedMtx.RLock()
defer t.blockedMtx.RUnlock()
return t.blocked
}
func (t *tailIterator) query() (iter.EntryIterator, error) {
ingesterIterators, err := t.querier.queryIngesters(t.ctx, t.queryRequest)
if err != nil {
return nil, err
func (t *Tailer) popDroppedEntries() []droppedEntry {
t.blockedMtx.Lock()
defer t.blockedMtx.Unlock()
t.blocked = false
if len(t.droppedEntries) == 0 {
return nil
}
droppedEntries := t.droppedEntries
t.droppedEntries = []droppedEntry{}
chunkStoreIterators, err := t.querier.queryStore(t.ctx, t.queryRequest)
if err != nil {
return nil, err
return droppedEntries
}
func (t *Tailer) getResponseChan() <-chan *TailResponse {
return t.responseChan
}
func (t *Tailer) getCloseErrorChan() <-chan error {
return t.closeErrChan
}
func newTailer(delayFor time.Duration, querierTailClients map[string]logproto.Querier_TailClient,
queryDroppedStreams func(from, to time.Time, labels string) (iter.EntryIterator, error),
tailDisconnectedIngesters func([]string) (map[string]logproto.Querier_TailClient, error)) *Tailer {
t := Tailer{
openStreamIterator: iter.NewHeapIterator([]iter.EntryIterator{}, logproto.FORWARD),
//droppedStreamsIterator: &droppedStreamsIterator{},
querierTailClients: querierTailClients,
queryDroppedStreams: queryDroppedStreams,
delayFor: delayFor,
responseChan: make(chan *TailResponse, bufferSizeForTailResponse),
closeErrChan: make(chan error),
tailDisconnectedIngesters: tailDisconnectedIngesters,
}
iterators := append(ingesterIterators, chunkStoreIterators)
return iter.NewHeapIterator(iterators, t.queryRequest.Direction), nil
t.readTailClients()
go t.loop()
return &t
}

@ -0,0 +1,126 @@
package querier
import (
"context"
"sort"
"testing"
"time"
"github.com/grafana/loki/pkg/iter"
"github.com/grafana/loki/pkg/logproto"
"github.com/pkg/errors"
"github.com/stretchr/testify/assert"
"google.golang.org/grpc/metadata"
)
type mockQuerierTailClient struct {
streams []logproto.Stream
index int
}
func (mockQuerierTailClient) Header() (metadata.MD, error) {
return nil, nil
}
func (mockQuerierTailClient) Trailer() metadata.MD {
return nil
}
func (mockQuerierTailClient) CloseSend() error {
return nil
}
func (mockQuerierTailClient) Context() context.Context {
return nil
}
func (mockQuerierTailClient) SendMsg(i interface{}) error {
return nil
}
func (mockQuerierTailClient) RecvMsg(i interface{}) error {
return nil
}
func (m *mockQuerierTailClient) Recv() (*logproto.TailResponse, error) {
if m.index < len(m.streams) {
tailResponse := logproto.TailResponse{
Stream: &m.streams[m.index],
}
m.index++
return &tailResponse, nil
}
return nil, errors.New("No more entries left")
}
func TestQuerier_Tail(t *testing.T) {
testCases := []struct {
tailClients map[string]logproto.Querier_TailClient
}{
{
tailClients: map[string]logproto.Querier_TailClient{
"1": &mockQuerierTailClient{
streams: []logproto.Stream{
{
Labels: "foo=1",
Entries: []logproto.Entry{
{
Timestamp: time.Unix(0, 0),
Line: "foo line 1",
},
{
Timestamp: time.Unix(0, 5),
Line: "foo line 2",
},
},
},
},
},
"2": &mockQuerierTailClient{
streams: []logproto.Stream{
{
Labels: "foo=1&bar=1",
Entries: []logproto.Entry{
{
Timestamp: time.Unix(0, 0),
Line: "foobar line 1",
},
{
Timestamp: time.Unix(0, 1),
Line: "foobar line 2",
},
},
},
},
},
},
},
}
for _, testCase := range testCases {
expected := TailResponse{
Streams: []logproto.Stream{},
}
for i := range testCase.tailClients {
tailClient := testCase.tailClients[i].(*mockQuerierTailClient)
for _, stream := range tailClient.streams {
for _, entry := range stream.Entries {
expected.Streams = append(expected.Streams, logproto.Stream{Labels: stream.Labels, Entries: []logproto.Entry{entry}})
}
}
}
sort.Slice(expected.Streams, func(i, j int) bool {
return expected.Streams[i].Entries[0].Timestamp.Before(expected.Streams[j].Entries[0].Timestamp)
})
tailer := newTailer(0, testCase.tailClients, func(from, to time.Time, labels string) (iterator iter.EntryIterator, e error) {
return nil, nil
}, func(strings []string) (clients map[string]logproto.Querier_TailClient, e error) {
return nil, nil
})
responseChan := tailer.getResponseChan()
response := <-responseChan
assert.Equal(t, expected, *response)
assert.NoError(t, tailer.close())
}
}

@ -0,0 +1,11 @@
package util
import "hash/fnv"
// TokenFor generates a token used for finding ingesters from ring
func TokenFor(userID, labels string) uint32 {
h := fnv.New32()
_, _ = h.Write([]byte(userID))
_, _ = h.Write([]byte(labels))
return h.Sum32()
}

@ -1,5 +1,5 @@
name: promtail
version: 0.8.0
version: 0.8.1
appVersion: 0.0.1
kubeVersion: "^1.10.0-0"
description: "Responsible for gathering logs and sending them to Loki"

@ -11,11 +11,11 @@ local utils = import "mixin-utils/utils.libsonnet";
g.row('Frontend (cortex_gw)')
.addPanel(
g.panel('QPS') +
g.qpsPanel('cortex_gw_request_duration_seconds_count{cluster=~"$cluster", job=~"($namespace)/cortex-gw", route="cortex-write"}')
g.qpsPanel('loki_request_duration_seconds_count{cluster=~"$cluster", job=~"($namespace)/cortex-gw", route="api_prom_push"}')
)
.addPanel(
g.panel('Latency') +
utils.latencyRecordingRulePanel('cortex_gw_request_duration_seconds', [utils.selector.re('job', '($namespace)/cortex-gw'), utils.selector.eq('route', 'cortex-write')], extra_selectors=[utils.selector.re('cluster', '$cluster')])
utils.latencyRecordingRulePanel('loki_request_duration_seconds', [utils.selector.re('job', '($namespace)/cortex-gw'), utils.selector.eq('route', 'api_prom_push')], extra_selectors=[utils.selector.re('cluster', '$cluster')])
)
)
.addRow(
@ -49,11 +49,11 @@ local utils = import "mixin-utils/utils.libsonnet";
g.row('Frontend (cortex_gw)')
.addPanel(
g.panel('QPS') +
g.qpsPanel('cortex_gw_request_duration_seconds_count{cluster="$cluster", job="$namespace/cortex-gw", route="cortex-read"}')
g.qpsPanel('loki_request_duration_seconds_count{cluster="$cluster", job="$namespace/cortex-gw", route="api_prom_query"}')
)
.addPanel(
g.panel('Latency') +
utils.latencyRecordingRulePanel('cortex_gw_request_duration_seconds', [utils.selector.eq('job', '$namespace/cortex-gw'), utils.selector.eq('route', 'cortex-read')], extra_selectors=[utils.selector.eq('cluster', '$cluster')])
utils.latencyRecordingRulePanel('loki_request_duration_seconds', [utils.selector.eq('job', '$namespace/cortex-gw'), utils.selector.eq('route', 'api_prom_query')], extra_selectors=[utils.selector.eq('cluster', '$cluster')])
)
)
.addRow(
@ -139,11 +139,11 @@ local utils = import "mixin-utils/utils.libsonnet";
g.row('loki Reqs (cortex_gw)')
.addPanel(
g.panel('QPS') +
g.qpsPanel('cortex_gw_request_duration_seconds_count{cluster="$cluster", job="$namespace/cortex-gw"}')
g.qpsPanel('loki_request_duration_seconds_count{cluster="$cluster", job="$namespace/cortex-gw"}')
)
.addPanel(
g.panel('Latency') +
utils.latencyRecordingRulePanel('cortex_gw_request_duration_seconds', [utils.selector.eq('job', '$namespace/cortex-gw')], extra_selectors=[utils.selector.eq('cluster', '$cluster')])
utils.latencyRecordingRulePanel('loki_request_duration_seconds', [utils.selector.eq('job', '$namespace/cortex-gw')], extra_selectors=[utils.selector.eq('cluster', '$cluster')])
)
),

Loading…
Cancel
Save