mirror of https://github.com/grafana/loki
commit
274170ffec
@ -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() |
||||
} |
File diff suppressed because it is too large
Load Diff
@ -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() |
||||
} |
Loading…
Reference in new issue