use the defined querypages callback in all the types implementing QueryPages function from IndexClient interface (#5463)

pull/5467/head
Sandeep Sukhani 3 years ago committed by GitHub
parent 56cebfcbae
commit 9f301cf87d
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 4
      pkg/storage/chunk/aws/dynamodb_storage_client.go
  2. 4
      pkg/storage/chunk/cassandra/storage_client.go
  3. 6
      pkg/storage/chunk/gcp/bigtable_index_client.go
  4. 4
      pkg/storage/chunk/grpc/index_client.go
  5. 2
      pkg/storage/chunk/inmemory_storage_client.go
  6. 8
      pkg/storage/chunk/local/boltdb_index_client.go
  7. 8
      pkg/storage/chunk/storage/caching_index_client.go
  8. 2
      pkg/storage/chunk/storage/caching_index_client_test.go
  9. 5
      pkg/storage/chunk/storage_client.go
  10. 9
      pkg/storage/chunk/util/util.go
  11. 2
      pkg/storage/stores/shipper/compactor/deletion/delete_requests_table.go
  12. 4
      pkg/storage/stores/shipper/downloads/index_set.go
  13. 4
      pkg/storage/stores/shipper/downloads/table.go
  14. 4
      pkg/storage/stores/shipper/downloads/table_manager.go
  15. 3
      pkg/storage/stores/shipper/downloads/table_test.go
  16. 5
      pkg/storage/stores/shipper/gateway_client.go
  17. 2
      pkg/storage/stores/shipper/indexgateway/gateway_test.go
  18. 4
      pkg/storage/stores/shipper/shipper_index_client.go
  19. 8
      pkg/storage/stores/shipper/testutil/testutil.go
  20. 4
      pkg/storage/stores/shipper/uploads/table.go
  21. 4
      pkg/storage/stores/shipper/uploads/table_manager.go
  22. 9
      pkg/storage/stores/shipper/util/queries.go
  23. 3
      pkg/storage/stores/shipper/util/queries_test.go

@ -247,11 +247,11 @@ func (a dynamoDBStorageClient) BatchWrite(ctx context.Context, input chunk.Write
}
// QueryPages implements chunk.IndexClient.
func (a dynamoDBStorageClient) QueryPages(ctx context.Context, queries []chunk.IndexQuery, callback func(chunk.IndexQuery, chunk.ReadBatch) bool) error {
func (a dynamoDBStorageClient) QueryPages(ctx context.Context, queries []chunk.IndexQuery, callback chunk.QueryPagesCallback) error {
return chunk_util.DoParallelQueries(ctx, a.query, queries, callback)
}
func (a dynamoDBStorageClient) query(ctx context.Context, query chunk.IndexQuery, callback chunk_util.Callback) error {
func (a dynamoDBStorageClient) query(ctx context.Context, query chunk.IndexQuery, callback chunk.QueryPagesCallback) error {
input := &dynamodb.QueryInput{
TableName: aws.String(query.TableName),
KeyConditions: map[string]*dynamodb.Condition{

@ -345,11 +345,11 @@ func (s *StorageClient) BatchWrite(ctx context.Context, batch chunk.WriteBatch)
}
// QueryPages implement chunk.IndexClient.
func (s *StorageClient) QueryPages(ctx context.Context, queries []chunk.IndexQuery, callback func(chunk.IndexQuery, chunk.ReadBatch) bool) error {
func (s *StorageClient) QueryPages(ctx context.Context, queries []chunk.IndexQuery, callback chunk.QueryPagesCallback) error {
return util.DoParallelQueries(ctx, s.query, queries, callback)
}
func (s *StorageClient) query(ctx context.Context, query chunk.IndexQuery, callback util.Callback) error {
func (s *StorageClient) query(ctx context.Context, query chunk.IndexQuery, callback chunk.QueryPagesCallback) error {
if s.querySemaphore != nil {
if err := s.querySemaphore.Acquire(ctx, 1); err != nil {
return err

@ -215,7 +215,7 @@ func (s *storageClientColumnKey) BatchWrite(ctx context.Context, batch chunk.Wri
return nil
}
func (s *storageClientColumnKey) QueryPages(ctx context.Context, queries []chunk.IndexQuery, callback func(chunk.IndexQuery, chunk.ReadBatch) bool) error {
func (s *storageClientColumnKey) QueryPages(ctx context.Context, queries []chunk.IndexQuery, callback chunk.QueryPagesCallback) error {
sp, ctx := ot.StartSpanFromContext(ctx, "QueryPages")
defer sp.Finish()
@ -323,11 +323,11 @@ func (c *columnKeyIterator) Value() []byte {
return c.items[c.i].Value
}
func (s *storageClientV1) QueryPages(ctx context.Context, queries []chunk.IndexQuery, callback func(chunk.IndexQuery, chunk.ReadBatch) bool) error {
func (s *storageClientV1) QueryPages(ctx context.Context, queries []chunk.IndexQuery, callback chunk.QueryPagesCallback) error {
return chunk_util.DoParallelQueries(ctx, s.query, queries, callback)
}
func (s *storageClientV1) query(ctx context.Context, query chunk.IndexQuery, callback chunk_util.Callback) error {
func (s *storageClientV1) query(ctx context.Context, query chunk.IndexQuery, callback chunk.QueryPagesCallback) error {
const null = string('\xff')
log, ctx := spanlogger.New(ctx, "QueryPages", ot.Tag{Key: "tableName", Value: query.TableName}, ot.Tag{Key: "hashValue", Value: query.HashValue})

@ -48,11 +48,11 @@ func (s *StorageClient) BatchWrite(c context.Context, batch chunk.WriteBatch) er
return nil
}
func (s *StorageClient) QueryPages(ctx context.Context, queries []chunk.IndexQuery, callback func(chunk.IndexQuery, chunk.ReadBatch) (shouldContinue bool)) error {
func (s *StorageClient) QueryPages(ctx context.Context, queries []chunk.IndexQuery, callback chunk.QueryPagesCallback) error {
return util.DoParallelQueries(ctx, s.query, queries, callback)
}
func (s *StorageClient) query(ctx context.Context, query chunk.IndexQuery, callback util.Callback) error {
func (s *StorageClient) query(ctx context.Context, query chunk.IndexQuery, callback chunk.QueryPagesCallback) error {
indexQuery := &QueryIndexRequest{
TableName: query.TableName,
HashValue: query.HashValue,

@ -258,7 +258,7 @@ func (m *MockStorage) BatchWrite(ctx context.Context, batch WriteBatch) error {
}
// QueryPages implements StorageClient.
func (m *MockStorage) QueryPages(ctx context.Context, queries []IndexQuery, callback func(IndexQuery, ReadBatch) (shouldContinue bool)) error {
func (m *MockStorage) QueryPages(ctx context.Context, queries []IndexQuery, callback QueryPagesCallback) error {
m.mtx.RLock()
defer m.mtx.RUnlock()

@ -224,11 +224,11 @@ func (b *BoltIndexClient) BatchWrite(ctx context.Context, batch chunk.WriteBatch
return nil
}
func (b *BoltIndexClient) QueryPages(ctx context.Context, queries []chunk.IndexQuery, callback func(chunk.IndexQuery, chunk.ReadBatch) (shouldContinue bool)) error {
func (b *BoltIndexClient) QueryPages(ctx context.Context, queries []chunk.IndexQuery, callback chunk.QueryPagesCallback) error {
return chunk_util.DoParallelQueries(ctx, b.query, queries, callback)
}
func (b *BoltIndexClient) query(ctx context.Context, query chunk.IndexQuery, callback chunk_util.Callback) error {
func (b *BoltIndexClient) query(ctx context.Context, query chunk.IndexQuery, callback chunk.QueryPagesCallback) error {
db, err := b.GetDB(query.TableName, DBOperationRead)
if err != nil {
if err == ErrUnexistentBoltDB {
@ -242,7 +242,7 @@ func (b *BoltIndexClient) query(ctx context.Context, query chunk.IndexQuery, cal
}
func (b *BoltIndexClient) QueryDB(ctx context.Context, db *bbolt.DB, bucketName []byte, query chunk.IndexQuery,
callback func(chunk.IndexQuery, chunk.ReadBatch) (shouldContinue bool)) error {
callback chunk.QueryPagesCallback) error {
return db.View(func(tx *bbolt.Tx) error {
if len(bucketName) == 0 {
return ErrEmptyIndexBucketName
@ -256,7 +256,7 @@ func (b *BoltIndexClient) QueryDB(ctx context.Context, db *bbolt.DB, bucketName
})
}
func (b *BoltIndexClient) QueryWithCursor(_ context.Context, c *bbolt.Cursor, query chunk.IndexQuery, callback func(chunk.IndexQuery, chunk.ReadBatch) (shouldContinue bool)) error {
func (b *BoltIndexClient) QueryWithCursor(_ context.Context, c *bbolt.Cursor, query chunk.IndexQuery, callback chunk.QueryPagesCallback) error {
var start []byte
if len(query.RangeValuePrefix) > 0 {
start = []byte(query.HashValue + separator + string(query.RangeValuePrefix))

@ -77,7 +77,7 @@ func (s *cachingIndexClient) Stop() {
s.IndexClient.Stop()
}
func (s *cachingIndexClient) QueryPages(ctx context.Context, queries []chunk.IndexQuery, callback func(chunk.IndexQuery, chunk.ReadBatch) (shouldContinue bool)) error {
func (s *cachingIndexClient) QueryPages(ctx context.Context, queries []chunk.IndexQuery, callback chunk.QueryPagesCallback) error {
if len(queries) == 0 {
return nil
}
@ -89,7 +89,7 @@ func (s *cachingIndexClient) QueryPages(ctx context.Context, queries []chunk.Ind
return s.doQueries(ctx, queries, callback)
}
func (s *cachingIndexClient) queryPages(ctx context.Context, queries []chunk.IndexQuery, callback chunk_util.Callback,
func (s *cachingIndexClient) queryPages(ctx context.Context, queries []chunk.IndexQuery, callback chunk.QueryPagesCallback,
buildIndexQuery func(query chunk.IndexQuery) chunk.IndexQuery, buildQueryKey func(query chunk.IndexQuery) string) error {
if len(queries) == 0 {
return nil
@ -210,7 +210,7 @@ func (s *cachingIndexClient) queryPages(ctx context.Context, queries []chunk.Ind
// doBroadQueries does broad queries on the store by using just TableName and HashValue.
// This is useful for chunks queries or when we need to reduce QPS on index store at the expense of higher cache requirement.
// All the results from the index store are cached and the responses are filtered based on the actual queries.
func (s *cachingIndexClient) doBroadQueries(ctx context.Context, queries []chunk.IndexQuery, callback chunk_util.Callback) error {
func (s *cachingIndexClient) doBroadQueries(ctx context.Context, queries []chunk.IndexQuery, callback chunk.QueryPagesCallback) error {
// We cache all the entries for queries looking for Chunk IDs, so filter client side.
callback = chunk_util.QueryFilter(callback)
return s.queryPages(ctx, queries, callback, func(query chunk.IndexQuery) chunk.IndexQuery {
@ -221,7 +221,7 @@ func (s *cachingIndexClient) doBroadQueries(ctx context.Context, queries []chunk
}
// doQueries does the exact same queries as opposed to doBroadQueries doing broad queries with limited query params.
func (s *cachingIndexClient) doQueries(ctx context.Context, queries []chunk.IndexQuery, callback chunk_util.Callback) error {
func (s *cachingIndexClient) doQueries(ctx context.Context, queries []chunk.IndexQuery, callback chunk.QueryPagesCallback) error {
return s.queryPages(ctx, queries, callback, func(query chunk.IndexQuery) chunk.IndexQuery {
return query
}, func(q chunk.IndexQuery) string {

@ -24,7 +24,7 @@ type mockStore struct {
results ReadBatch
}
func (m *mockStore) QueryPages(ctx context.Context, queries []chunk.IndexQuery, callback func(chunk.IndexQuery, chunk.ReadBatch) (shouldContinue bool)) error {
func (m *mockStore) QueryPages(ctx context.Context, queries []chunk.IndexQuery, callback chunk.QueryPagesCallback) error {
for _, query := range queries {
callback(query, m.results)
}

@ -14,6 +14,9 @@ var (
ErrStorageObjectNotFound = errors.New("object not found in storage")
)
// QueryPagesCallback from an IndexQuery.
type QueryPagesCallback func(IndexQuery, ReadBatch) bool
// IndexClient is a client for the storage of the index (e.g. DynamoDB or Bigtable).
type IndexClient interface {
Stop()
@ -23,7 +26,7 @@ type IndexClient interface {
BatchWrite(context.Context, WriteBatch) error
// For the read path.
QueryPages(ctx context.Context, queries []IndexQuery, callback func(IndexQuery, ReadBatch) (shouldContinue bool)) error
QueryPages(ctx context.Context, queries []IndexQuery, callback QueryPagesCallback) error
}
// Client is for storing and retrieving chunks.

@ -13,11 +13,8 @@ import (
"github.com/grafana/loki/pkg/util/math"
)
// Callback from an IndexQuery.
type Callback func(chunk.IndexQuery, chunk.ReadBatch) bool
// DoSingleQuery is the interface for indexes that don't support batching yet.
type DoSingleQuery func(context.Context, chunk.IndexQuery, Callback) error
type DoSingleQuery func(context.Context, chunk.IndexQuery, chunk.QueryPagesCallback) error
// QueryParallelism is the maximum number of subqueries run in
// parallel per higher-level query
@ -27,7 +24,7 @@ var QueryParallelism = 100
// and indexes that don't yet support batching.
func DoParallelQueries(
ctx context.Context, doSingleQuery DoSingleQuery, queries []chunk.IndexQuery,
callback Callback,
callback chunk.QueryPagesCallback,
) error {
if len(queries) == 1 {
return doSingleQuery(ctx, queries[0], callback)
@ -109,7 +106,7 @@ func (f *filteringBatchIter) Next() bool {
// QueryFilter wraps a callback to ensure the results are filtered correctly;
// useful for the cache and Bigtable backend, which only ever fetches the whole
// row.
func QueryFilter(callback Callback) Callback {
func QueryFilter(callback chunk.QueryPagesCallback) chunk.QueryPagesCallback {
return func(query chunk.IndexQuery, batch chunk.ReadBatch) bool {
return callback(query, &filteringBatch{query, batch})
}

@ -180,7 +180,7 @@ func (t *deleteRequestsTable) BatchWrite(ctx context.Context, batch chunk.WriteB
return nil
}
func (t *deleteRequestsTable) QueryPages(ctx context.Context, queries []chunk.IndexQuery, callback func(chunk.IndexQuery, chunk.ReadBatch) (shouldContinue bool)) error {
func (t *deleteRequestsTable) QueryPages(ctx context.Context, queries []chunk.IndexQuery, callback chunk.QueryPagesCallback) error {
for _, query := range queries {
if err := t.boltdbIndexClient.QueryDB(ctx, t.db, local.IndexBucketName, query, callback); err != nil {
return err

@ -28,7 +28,7 @@ import (
type IndexSet interface {
Init() error
Close()
MultiQueries(ctx context.Context, queries []chunk.IndexQuery, callback chunk_util.Callback) error
MultiQueries(ctx context.Context, queries []chunk.IndexQuery, callback chunk.QueryPagesCallback) error
DropAllDBs() error
Err() error
LastUsedAt() time.Time
@ -178,7 +178,7 @@ func (t *indexSet) Close() {
}
// MultiQueries runs multiple queries without having to take lock multiple times for each query.
func (t *indexSet) MultiQueries(ctx context.Context, queries []chunk.IndexQuery, callback chunk_util.Callback) error {
func (t *indexSet) MultiQueries(ctx context.Context, queries []chunk.IndexQuery, callback chunk.QueryPagesCallback) error {
userID, err := tenant.TenantID(ctx)
if err != nil {
return err

@ -29,7 +29,7 @@ const (
)
type BoltDBIndexClient interface {
QueryWithCursor(_ context.Context, c *bbolt.Cursor, query chunk.IndexQuery, callback func(chunk.IndexQuery, chunk.ReadBatch) (shouldContinue bool)) error
QueryWithCursor(_ context.Context, c *bbolt.Cursor, query chunk.IndexQuery, callback chunk.QueryPagesCallback) error
}
type StorageClient interface {
@ -151,7 +151,7 @@ func (t *Table) Close() {
}
// MultiQueries runs multiple queries without having to take lock multiple times for each query.
func (t *Table) MultiQueries(ctx context.Context, queries []chunk.IndexQuery, callback chunk_util.Callback) error {
func (t *Table) MultiQueries(ctx context.Context, queries []chunk.IndexQuery, callback chunk.QueryPagesCallback) error {
userID, err := tenant.TenantID(ctx)
if err != nil {
return err

@ -128,7 +128,7 @@ func (tm *TableManager) Stop() {
}
}
func (tm *TableManager) QueryPages(ctx context.Context, queries []chunk.IndexQuery, callback chunk_util.Callback) error {
func (tm *TableManager) QueryPages(ctx context.Context, queries []chunk.IndexQuery, callback chunk.QueryPagesCallback) error {
queriesByTable := util.QueriesByTable(queries)
for tableName, queries := range queriesByTable {
err := tm.query(ctx, tableName, queries, callback)
@ -140,7 +140,7 @@ func (tm *TableManager) QueryPages(ctx context.Context, queries []chunk.IndexQue
return nil
}
func (tm *TableManager) query(ctx context.Context, tableName string, queries []chunk.IndexQuery, callback chunk_util.Callback) error {
func (tm *TableManager) query(ctx context.Context, tableName string, queries []chunk.IndexQuery, callback chunk.QueryPagesCallback) error {
logger := util_log.WithContext(ctx, util_log.Logger)
level.Debug(logger).Log("table-name", tableName)

@ -17,7 +17,6 @@ import (
"github.com/grafana/loki/pkg/storage/chunk"
"github.com/grafana/loki/pkg/storage/chunk/local"
chunk_util "github.com/grafana/loki/pkg/storage/chunk/util"
"github.com/grafana/loki/pkg/storage/stores/shipper/storage"
"github.com/grafana/loki/pkg/storage/stores/shipper/testutil"
util_log "github.com/grafana/loki/pkg/util/log"
@ -88,7 +87,7 @@ type mockIndexSet struct {
lastUsedAt time.Time
}
func (m *mockIndexSet) MultiQueries(_ context.Context, queries []chunk.IndexQuery, _ chunk_util.Callback) error {
func (m *mockIndexSet) MultiQueries(_ context.Context, queries []chunk.IndexQuery, _ chunk.QueryPagesCallback) error {
m.queriesDone = append(m.queriesDone, queries...)
return nil
}

@ -15,7 +15,6 @@ import (
"google.golang.org/grpc"
"github.com/grafana/loki/pkg/storage/chunk"
"github.com/grafana/loki/pkg/storage/chunk/util"
"github.com/grafana/loki/pkg/storage/stores/shipper/indexgateway/indexgatewaypb"
shipper_util "github.com/grafana/loki/pkg/storage/stores/shipper/util"
util_log "github.com/grafana/loki/pkg/util/log"
@ -78,7 +77,7 @@ func (s *GatewayClient) Stop() {
s.conn.Close()
}
func (s *GatewayClient) QueryPages(ctx context.Context, queries []chunk.IndexQuery, callback func(chunk.IndexQuery, chunk.ReadBatch) (shouldContinue bool)) error {
func (s *GatewayClient) QueryPages(ctx context.Context, queries []chunk.IndexQuery, callback chunk.QueryPagesCallback) error {
errs := make(chan error)
for i := 0; i < len(queries); i += maxQueriesPerGoroutine {
@ -99,7 +98,7 @@ func (s *GatewayClient) QueryPages(ctx context.Context, queries []chunk.IndexQue
return lastErr
}
func (s *GatewayClient) doQueries(ctx context.Context, queries []chunk.IndexQuery, callback util.Callback) error {
func (s *GatewayClient) doQueries(ctx context.Context, queries []chunk.IndexQuery, callback chunk.QueryPagesCallback) error {
queryKeyQueryMap := make(map[string]chunk.IndexQuery, len(queries))
gatewayQueries := make([]*indexgatewaypb.IndexQuery, 0, len(queries))

@ -74,7 +74,7 @@ type mockIndexClient struct {
response *mockBatch
}
func (m mockIndexClient) QueryPages(ctx context.Context, queries []chunk.IndexQuery, callback func(chunk.IndexQuery, chunk.ReadBatch) (shouldContinue bool)) error {
func (m mockIndexClient) QueryPages(ctx context.Context, queries []chunk.IndexQuery, callback chunk.QueryPagesCallback) error {
for _, query := range queries {
callback(query, m.response)
}

@ -47,7 +47,7 @@ const (
)
type boltDBIndexClient interface {
QueryWithCursor(_ context.Context, c *bbolt.Cursor, query chunk.IndexQuery, callback func(chunk.IndexQuery, chunk.ReadBatch) (shouldContinue bool)) error
QueryWithCursor(_ context.Context, c *bbolt.Cursor, query chunk.IndexQuery, callback chunk.QueryPagesCallback) error
NewWriteBatch() chunk.WriteBatch
WriteToDB(ctx context.Context, db *bbolt.DB, bucketName []byte, writes local.TableWrites) error
Stop()
@ -224,7 +224,7 @@ func (s *Shipper) BatchWrite(ctx context.Context, batch chunk.WriteBatch) error
})
}
func (s *Shipper) QueryPages(ctx context.Context, queries []chunk.IndexQuery, callback func(chunk.IndexQuery, chunk.ReadBatch) (shouldContinue bool)) error {
func (s *Shipper) QueryPages(ctx context.Context, queries []chunk.IndexQuery, callback chunk.QueryPagesCallback) error {
return instrument.CollectedRequest(ctx, "Shipper.Query", instrument.NewHistogramCollector(s.metrics.requestDurationSeconds), instrument.ErrorCode, func(ctx context.Context) error {
spanLogger := spanlogger.FromContext(ctx)

@ -46,7 +46,7 @@ func AddRecordsToBatch(batch chunk.WriteBatch, tableName string, start, numRecor
}
type SingleTableQuerier interface {
MultiQueries(ctx context.Context, queries []chunk.IndexQuery, callback chunk_util.Callback) error
MultiQueries(ctx context.Context, queries []chunk.IndexQuery, callback chunk.QueryPagesCallback) error
}
func TestSingleTableQuery(t *testing.T, userID string, queries []chunk.IndexQuery, querier SingleTableQuerier, start, numRecords int) {
@ -62,7 +62,7 @@ func TestSingleTableQuery(t *testing.T, userID string, queries []chunk.IndexQuer
}
type SingleDBQuerier interface {
QueryDB(ctx context.Context, db *bbolt.DB, bucketName []byte, query chunk.IndexQuery, callback func(chunk.IndexQuery, chunk.ReadBatch) (shouldContinue bool)) error
QueryDB(ctx context.Context, db *bbolt.DB, bucketName []byte, query chunk.IndexQuery, callback chunk.QueryPagesCallback) error
}
func TestSingleDBQuery(t *testing.T, query chunk.IndexQuery, db *bbolt.DB, bucketName []byte, querier SingleDBQuerier, start, numRecords int) {
@ -78,7 +78,7 @@ func TestSingleDBQuery(t *testing.T, query chunk.IndexQuery, db *bbolt.DB, bucke
}
type MultiTableQuerier interface {
QueryPages(ctx context.Context, queries []chunk.IndexQuery, callback chunk_util.Callback) error
QueryPages(ctx context.Context, queries []chunk.IndexQuery, callback chunk.QueryPagesCallback) error
}
func TestMultiTableQuery(t *testing.T, userID string, queries []chunk.IndexQuery, querier MultiTableQuerier, start, numRecords int) {
@ -93,7 +93,7 @@ func TestMultiTableQuery(t *testing.T, userID string, queries []chunk.IndexQuery
require.Len(t, fetchedRecords, numRecords)
}
func makeTestCallback(t *testing.T, minValue, maxValue int, records map[string]string) func(query chunk.IndexQuery, batch chunk.ReadBatch) (shouldContinue bool) {
func makeTestCallback(t *testing.T, minValue, maxValue int, records map[string]string) chunk.QueryPagesCallback {
t.Helper()
recordsMtx := sync.Mutex{}
return func(query chunk.IndexQuery, batch chunk.ReadBatch) (shouldContinue bool) {

@ -37,7 +37,7 @@ const (
)
type BoltDBIndexClient interface {
QueryWithCursor(_ context.Context, c *bbolt.Cursor, query chunk.IndexQuery, callback func(chunk.IndexQuery, chunk.ReadBatch) (shouldContinue bool)) error
QueryWithCursor(_ context.Context, c *bbolt.Cursor, query chunk.IndexQuery, callback chunk.QueryPagesCallback) error
WriteToDB(ctx context.Context, db *bbolt.DB, bucketName []byte, writes local.TableWrites) error
}
@ -188,7 +188,7 @@ func (lt *Table) Snapshot() error {
}
// MultiQueries runs multiple queries without having to take lock multiple times for each query.
func (lt *Table) MultiQueries(ctx context.Context, queries []chunk.IndexQuery, callback chunk_util.Callback) error {
func (lt *Table) MultiQueries(ctx context.Context, queries []chunk.IndexQuery, callback chunk.QueryPagesCallback) error {
lt.dbSnapshotsMtx.RLock()
defer lt.dbSnapshotsMtx.RUnlock()

@ -92,7 +92,7 @@ func (tm *TableManager) Stop() {
tm.uploadTables(context.Background(), true)
}
func (tm *TableManager) QueryPages(ctx context.Context, queries []chunk.IndexQuery, callback chunk_util.Callback) error {
func (tm *TableManager) QueryPages(ctx context.Context, queries []chunk.IndexQuery, callback chunk.QueryPagesCallback) error {
queriesByTable := util.QueriesByTable(queries)
for tableName, queries := range queriesByTable {
err := tm.query(ctx, tableName, queries, callback)
@ -104,7 +104,7 @@ func (tm *TableManager) QueryPages(ctx context.Context, queries []chunk.IndexQue
return nil
}
func (tm *TableManager) query(ctx context.Context, tableName string, queries []chunk.IndexQuery, callback chunk_util.Callback) error {
func (tm *TableManager) query(ctx context.Context, tableName string, queries []chunk.IndexQuery, callback chunk.QueryPagesCallback) error {
tm.tablesMtx.RLock()
defer tm.tablesMtx.RUnlock()

@ -5,14 +5,13 @@ import (
"sync"
"github.com/grafana/loki/pkg/storage/chunk"
chunk_util "github.com/grafana/loki/pkg/storage/chunk/util"
util_math "github.com/grafana/loki/pkg/util/math"
)
const maxQueriesPerGoroutine = 100
type TableQuerier interface {
MultiQueries(ctx context.Context, queries []chunk.IndexQuery, callback chunk_util.Callback) error
MultiQueries(ctx context.Context, queries []chunk.IndexQuery, callback chunk.QueryPagesCallback) error
}
// QueriesByTable groups and returns queries by tables.
@ -29,7 +28,7 @@ func QueriesByTable(queries []chunk.IndexQuery) map[string][]chunk.IndexQuery {
return queriesByTable
}
func DoParallelQueries(ctx context.Context, tableQuerier TableQuerier, queries []chunk.IndexQuery, callback chunk_util.Callback) error {
func DoParallelQueries(ctx context.Context, tableQuerier TableQuerier, queries []chunk.IndexQuery, callback chunk.QueryPagesCallback) error {
errs := make(chan error)
id := NewIndexDeduper(callback)
@ -59,12 +58,12 @@ func DoParallelQueries(ctx context.Context, tableQuerier TableQuerier, queries [
// IndexDeduper should always be used on table level not the whole query level because it just looks at range values which can be repeated across tables
// Cortex anyways dedupes entries across tables
type IndexDeduper struct {
callback chunk_util.Callback
callback chunk.QueryPagesCallback
seenRangeValues map[string]map[string]struct{}
mtx sync.RWMutex
}
func NewIndexDeduper(callback chunk_util.Callback) *IndexDeduper {
func NewIndexDeduper(callback chunk.QueryPagesCallback) *IndexDeduper {
return &IndexDeduper{
callback: callback,
seenRangeValues: map[string]map[string]struct{}{},

@ -9,7 +9,6 @@ import (
"github.com/stretchr/testify/require"
"github.com/grafana/loki/pkg/storage/chunk"
chunk_util "github.com/grafana/loki/pkg/storage/chunk/util"
)
type mockTableQuerier struct {
@ -17,7 +16,7 @@ type mockTableQuerier struct {
queries map[string]chunk.IndexQuery
}
func (m *mockTableQuerier) MultiQueries(ctx context.Context, queries []chunk.IndexQuery, callback chunk_util.Callback) error {
func (m *mockTableQuerier) MultiQueries(ctx context.Context, queries []chunk.IndexQuery, callback chunk.QueryPagesCallback) error {
m.Lock()
defer m.Unlock()

Loading…
Cancel
Save