From 65281a1e9105f5d8df579987187dbb423b530993 Mon Sep 17 00:00:00 2001 From: Tom Wilkie Date: Mon, 3 Sep 2018 13:02:56 +0100 Subject: [PATCH] Revert "Revert "Merge pull request #971 from grafana/batch-index-lookups"" This reverts commit 8b74f9053037c4590ec1512b65288e4869d5e748. Signed-off-by: Tom Wilkie --- aws/storage_client.go | 32 +++++-- cassandra/storage_client.go | 28 +++--- chunk_store.go | 45 ++------- gcp/storage_client.go | 149 ++++++++++++++++++++++++------ inmemory_storage_client.go | 47 +++++++--- storage/caching_storage_client.go | 127 ++++++++----------------- storage/index_test.go | 22 +++-- storage_client.go | 8 +- util/util.go | 77 +++++++++++++++ 9 files changed, 329 insertions(+), 206 deletions(-) create mode 100644 util/util.go diff --git a/aws/storage_client.go b/aws/storage_client.go index 9353280ead53..b8d5f7b3f92c 100644 --- a/aws/storage_client.go +++ b/aws/storage_client.go @@ -30,6 +30,7 @@ import ( "github.com/weaveworks/common/instrument" "github.com/weaveworks/common/user" "github.com/weaveworks/cortex/pkg/chunk" + chunk_util "github.com/weaveworks/cortex/pkg/chunk/util" "github.com/weaveworks/cortex/pkg/util" ) @@ -301,7 +302,11 @@ func (a storageClient) BatchWrite(ctx context.Context, input chunk.WriteBatch) e return backoff.Err() } -func (a storageClient) QueryPages(ctx context.Context, query chunk.IndexQuery, callback func(result chunk.ReadBatch) (shouldContinue bool)) error { +func (a storageClient) QueryPages(ctx context.Context, queries []chunk.IndexQuery, callback func(chunk.IndexQuery, chunk.ReadBatch) bool) error { + return chunk_util.DoParallelQueries(ctx, a.query, queries, callback) +} + +func (a storageClient) query(ctx context.Context, query chunk.IndexQuery, callback func(result chunk.ReadBatch) (shouldContinue bool)) error { sp, ctx := ot.StartSpanFromContext(ctx, "QueryPages", ot.Tag{Key: "tableName", Value: query.TableName}, ot.Tag{Key: "hashValue", Value: query.HashValue}) defer sp.Finish() @@ -371,7 +376,7 @@ func (a storageClient) QueryPages(ctx context.Context, query chunk.IndexQuery, c return nil } -func (a storageClient) queryPage(ctx context.Context, input *dynamodb.QueryInput, page dynamoDBRequest) (dynamoDBReadResponse, error) { +func (a storageClient) queryPage(ctx context.Context, input *dynamodb.QueryInput, page dynamoDBRequest) (*dynamoDBReadResponse, error) { backoff := util.NewBackoff(ctx, a.cfg.backoffConfig) defer func() { dynamoQueryRetryCount.WithLabelValues("queryPage").Observe(float64(backoff.NumRetries())) @@ -401,7 +406,10 @@ func (a storageClient) queryPage(ctx context.Context, input *dynamodb.QueryInput } queryOutput := page.Data().(*dynamodb.QueryOutput) - return dynamoDBReadResponse(queryOutput.Items), nil + return &dynamoDBReadResponse{ + i: -1, + items: queryOutput.Items, + }, nil } return nil, fmt.Errorf("QueryPage error: %s for table %v, last error %v", backoff.Err(), *input.TableName, err) } @@ -785,18 +793,22 @@ func (a storageClient) putS3Chunk(ctx context.Context, key string, buf []byte) e } // Slice of values returned; map key is attribute name -type dynamoDBReadResponse []map[string]*dynamodb.AttributeValue +type dynamoDBReadResponse struct { + i int + items []map[string]*dynamodb.AttributeValue +} -func (b dynamoDBReadResponse) Len() int { - return len(b) +func (b *dynamoDBReadResponse) Next() bool { + b.i++ + return b.i < len(b.items) } -func (b dynamoDBReadResponse) RangeValue(i int) []byte { - return b[i][rangeKey].B +func (b *dynamoDBReadResponse) RangeValue() []byte { + return b.items[b.i][rangeKey].B } -func (b dynamoDBReadResponse) Value(i int) []byte { - chunkValue, ok := b[i][valueKey] +func (b *dynamoDBReadResponse) Value() []byte { + chunkValue, ok := b.items[b.i][valueKey] if !ok { return nil } diff --git a/cassandra/storage_client.go b/cassandra/storage_client.go index 88d443db8bf5..13078b2e227a 100644 --- a/cassandra/storage_client.go +++ b/cassandra/storage_client.go @@ -12,6 +12,7 @@ import ( "github.com/prometheus/common/model" "github.com/weaveworks/cortex/pkg/chunk" + "github.com/weaveworks/cortex/pkg/chunk/util" ) const ( @@ -185,7 +186,11 @@ func (s *storageClient) BatchWrite(ctx context.Context, batch chunk.WriteBatch) return nil } -func (s *storageClient) QueryPages(ctx context.Context, query chunk.IndexQuery, callback func(result chunk.ReadBatch) (shouldContinue bool)) error { +func (s *storageClient) QueryPages(ctx context.Context, queries []chunk.IndexQuery, callback func(chunk.IndexQuery, chunk.ReadBatch) bool) error { + return util.DoParallelQueries(ctx, s.query, queries, callback) +} + +func (s *storageClient) query(ctx context.Context, query chunk.IndexQuery, callback func(result chunk.ReadBatch) (shouldContinue bool)) error { var q *gocql.Query switch { @@ -218,7 +223,7 @@ func (s *storageClient) QueryPages(ctx context.Context, query chunk.IndexQuery, defer iter.Close() scanner := iter.Scanner() for scanner.Next() { - var b readBatch + b := &readBatch{} if err := scanner.Scan(&b.rangeValue, &b.value); err != nil { return errors.WithStack(err) } @@ -231,27 +236,26 @@ func (s *storageClient) QueryPages(ctx context.Context, query chunk.IndexQuery, // readBatch represents a batch of rows read from Cassandra. type readBatch struct { + consumed bool rangeValue []byte value []byte } // Len implements chunk.ReadBatch; in Cassandra we 'stream' results back // one-by-one, so this always returns 1. -func (readBatch) Len() int { - return 1 +func (b *readBatch) Next() bool { + if b.consumed { + return false + } + b.consumed = true + return true } -func (b readBatch) RangeValue(index int) []byte { - if index != 0 { - panic("index != 0") - } +func (b *readBatch) RangeValue() []byte { return b.rangeValue } -func (b readBatch) Value(index int) []byte { - if index != 0 { - panic("index != 0") - } +func (b *readBatch) Value() []byte { return b.value } diff --git a/chunk_store.go b/chunk_store.go index c8abb23441c1..f811a9cf7b8b 100644 --- a/chunk_store.go +++ b/chunk_store.go @@ -347,53 +347,22 @@ func (c *store) lookupChunksByMetricName(ctx context.Context, from, through mode } func (c *store) lookupEntriesByQueries(ctx context.Context, queries []IndexQuery) ([]IndexEntry, error) { - incomingEntries := make(chan []IndexEntry) - incomingErrors := make(chan error) - for _, query := range queries { - go func(query IndexQuery) { - entries, err := c.lookupEntriesByQuery(ctx, query) - if err != nil { - incomingErrors <- err - } else { - incomingEntries <- entries - } - }(query) - } - - // Combine the results into one slice - var entries []IndexEntry - var lastErr error - for i := 0; i < len(queries); i++ { - select { - case incoming := <-incomingEntries: - entries = append(entries, incoming...) - case err := <-incomingErrors: - lastErr = err - } - } - - return entries, lastErr -} - -func (c *store) lookupEntriesByQuery(ctx context.Context, query IndexQuery) ([]IndexEntry, error) { var entries []IndexEntry - - if err := c.storage.QueryPages(ctx, query, func(resp ReadBatch) (shouldContinue bool) { - for i := 0; i < resp.Len(); i++ { + err := c.storage.QueryPages(ctx, queries, func(query IndexQuery, resp ReadBatch) bool { + for resp.Next() { entries = append(entries, IndexEntry{ TableName: query.TableName, HashValue: query.HashValue, - RangeValue: resp.RangeValue(i), - Value: resp.Value(i), + RangeValue: resp.RangeValue(), + Value: resp.Value(), }) } return true - }); err != nil { + }) + if err != nil { level.Error(util.WithContext(ctx, util.Logger)).Log("msg", "error querying storage", "err", err) - return nil, err } - - return entries, nil + return entries, err } func (c *store) parseIndexEntries(ctx context.Context, entries []IndexEntry, matcher *labels.Matcher) ([]string, error) { diff --git a/gcp/storage_client.go b/gcp/storage_client.go index 9135236acc75..ec0597bc22a9 100644 --- a/gcp/storage_client.go +++ b/gcp/storage_client.go @@ -14,11 +14,13 @@ import ( "github.com/pkg/errors" "github.com/weaveworks/cortex/pkg/chunk" + chunk_util "github.com/weaveworks/cortex/pkg/chunk/util" "github.com/weaveworks/cortex/pkg/util" ) const ( columnFamily = "f" + columnPrefix = columnFamily + ":" column = "c" separator = "\000" maxRowReads = 100 @@ -187,7 +189,88 @@ func (s *storageClientColumnKey) BatchWrite(ctx context.Context, batch chunk.Wri return nil } -func (s *storageClientColumnKey) QueryPages(ctx context.Context, query chunk.IndexQuery, callback func(result chunk.ReadBatch) (shouldContinue bool)) error { +func (s *storageClientColumnKey) QueryPages(ctx context.Context, queries []chunk.IndexQuery, callback func(chunk.IndexQuery, chunk.ReadBatch) bool) error { + sp, ctx := ot.StartSpanFromContext(ctx, "QueryPages") + defer sp.Finish() + + // A limitation of this approach is that this only fetches whole rows; but + // whatever, we filter them in the cache on the client. But for unit tests to + // pass, we must do this. + callback = chunk_util.QueryFilter(callback) + + type tableQuery struct { + name string + queries map[string]chunk.IndexQuery + rows bigtable.RowList + } + + tableQueries := map[string]tableQuery{} + for _, query := range queries { + tq, ok := tableQueries[query.TableName] + if !ok { + tq = tableQuery{ + name: query.TableName, + queries: map[string]chunk.IndexQuery{}, + } + } + tq.queries[query.HashValue] = query + tq.rows = append(tq.rows, query.HashValue) + tableQueries[query.TableName] = tq + } + + errs := make(chan error) + + for _, tq := range tableQueries { + + table := s.client.Open(tq.name) + for i := 0; i < len(tq.rows); i += maxRowReads { + + page := tq.rows[i:util.Min(i+maxRowReads, len(tq.rows))] + go func(page bigtable.RowList, tq tableQuery) { + var processingErr error + // rows are returned in key order, not order in row list + err := table.ReadRows(ctx, page, func(row bigtable.Row) bool { + + query, ok := tq.queries[row.Key()] + if !ok { + processingErr = errors.WithStack(fmt.Errorf("Got row for unknown chunk: %s", row.Key())) + return false + } + + val, ok := row[columnFamily] + if !ok { + // There are no matching rows. + return true + } + + return callback(query, &bigtableReadBatchColumnKey{ + i: -1, + items: val, + }) + }) + + if processingErr != nil { + errs <- processingErr + } else { + errs <- err + } + }(page, tq) + } + } + + var lastErr error + for _, tq := range tableQueries { + for i := 0; i < len(tq.rows); i += maxRowReads { + err := <-errs + if err != nil { + lastErr = err + } + } + } + return lastErr +} + +func (s *storageClientColumnKey) query(ctx context.Context, query chunk.IndexQuery, callback func(result chunk.ReadBatch) (shouldContinue bool)) error { const null = string('\xff') sp, ctx := ot.StartSpanFromContext(ctx, "QueryPages", ot.Tag{Key: "tableName", Value: query.TableName}, ot.Tag{Key: "hashValue", Value: query.HashValue}) @@ -227,31 +310,30 @@ func (s *storageClientColumnKey) QueryPages(ctx context.Context, query chunk.Ind val = filteredItems } - callback(bigtableReadBatchColumnKey{ - items: val, - columnPrefix: columnFamily + ":", + callback(&bigtableReadBatchColumnKey{ + i: -1, + items: val, }) return nil } // bigtableReadBatchColumnKey represents a batch of values read from Bigtable. type bigtableReadBatchColumnKey struct { - items []bigtable.ReadItem - columnPrefix string + i int + items []bigtable.ReadItem } -func (b bigtableReadBatchColumnKey) Len() int { - return len(b.items) +func (b *bigtableReadBatchColumnKey) Next() bool { + b.i++ + return b.i < len(b.items) } -func (b bigtableReadBatchColumnKey) RangeValue(index int) []byte { - return []byte( - strings.TrimPrefix(b.items[index].Column, b.columnPrefix), - ) +func (b *bigtableReadBatchColumnKey) RangeValue() []byte { + return []byte(strings.TrimPrefix(b.items[b.i].Column, columnPrefix)) } -func (b bigtableReadBatchColumnKey) Value(index int) []byte { - return b.items[index].Value +func (b *bigtableReadBatchColumnKey) Value() []byte { + return b.items[b.i].Value } func (s *storageClientColumnKey) PutChunks(ctx context.Context, chunks []chunk.Chunk) error { @@ -368,7 +450,11 @@ func (s *storageClientColumnKey) GetChunks(ctx context.Context, input []chunk.Ch return output, nil } -func (s *storageClientV1) QueryPages(ctx context.Context, query chunk.IndexQuery, callback func(result chunk.ReadBatch) (shouldContinue bool)) error { +func (s *storageClientV1) QueryPages(ctx context.Context, queries []chunk.IndexQuery, callback func(chunk.IndexQuery, chunk.ReadBatch) bool) error { + return chunk_util.DoParallelQueries(ctx, s.query, queries, callback) +} + +func (s *storageClientV1) query(ctx context.Context, query chunk.IndexQuery, callback func(result chunk.ReadBatch) (shouldContinue bool)) error { const null = string('\xff') sp, ctx := ot.StartSpanFromContext(ctx, "QueryPages", ot.Tag{Key: "tableName", Value: query.TableName}, ot.Tag{Key: "hashValue", Value: query.HashValue}) @@ -398,7 +484,9 @@ func (s *storageClientV1) QueryPages(ctx context.Context, query chunk.IndexQuery err := table.ReadRows(ctx, rowRange, func(r bigtable.Row) bool { if query.ValueEqual == nil || bytes.Equal(r[columnFamily][0].Value, query.ValueEqual) { - return callback(bigtableReadBatchV1(r)) + return callback(&bigtableReadBatchV1{ + row: r, + }) } return true @@ -413,24 +501,27 @@ func (s *storageClientV1) QueryPages(ctx context.Context, query chunk.IndexQuery // bigtableReadBatchV1 represents a batch of rows read from Bigtable. As the // bigtable interface gives us rows one-by-one, a batch always only contains // a single row. -type bigtableReadBatchV1 bigtable.Row - -func (bigtableReadBatchV1) Len() int { - return 1 +type bigtableReadBatchV1 struct { + consumed bool + row bigtable.Row } -func (b bigtableReadBatchV1) RangeValue(index int) []byte { - if index != 0 { - panic("index != 0") + +func (b *bigtableReadBatchV1) Next() bool { + if b.consumed { + return false } + b.consumed = true + return true +} + +func (b *bigtableReadBatchV1) RangeValue() []byte { // String before the first separator is the hashkey - parts := strings.SplitN(bigtable.Row(b).Key(), separator, 2) + parts := strings.SplitN(b.row.Key(), separator, 2) return []byte(parts[1]) } -func (b bigtableReadBatchV1) Value(index int) []byte { - if index != 0 { - panic("index != 0") - } - cf, ok := b[columnFamily] + +func (b *bigtableReadBatchV1) Value() []byte { + cf, ok := b.row[columnFamily] if !ok || len(cf) != 1 { panic("bad response from bigtable") } diff --git a/inmemory_storage_client.go b/inmemory_storage_client.go index 0add546b2a4b..a47c32601a9c 100644 --- a/inmemory_storage_client.go +++ b/inmemory_storage_client.go @@ -169,13 +169,26 @@ func (m *MockStorage) BatchWrite(ctx context.Context, batch WriteBatch) error { } // QueryPages implements StorageClient. -func (m *MockStorage) QueryPages(ctx context.Context, query IndexQuery, callback func(result ReadBatch) (shouldContinue bool)) error { - logger := util.WithContext(ctx, util.Logger) - level.Debug(logger).Log("msg", "QueryPages", "query", query.HashValue) - +func (m *MockStorage) QueryPages(ctx context.Context, queries []IndexQuery, callback func(IndexQuery, ReadBatch) (shouldContinue bool)) error { m.mtx.RLock() defer m.mtx.RUnlock() + for _, query := range queries { + err := m.query(ctx, query, func(b ReadBatch) bool { + return callback(query, b) + }) + if err != nil { + return err + } + } + + return nil +} + +func (m *MockStorage) query(ctx context.Context, query IndexQuery, callback func(ReadBatch) (shouldContinue bool)) error { + logger := util.WithContext(ctx, util.Logger) + level.Debug(logger).Log("msg", "QueryPages", "query", query.HashValue) + table, ok := m.tables[query.TableName] if !ok { return fmt.Errorf("table not found") @@ -241,12 +254,14 @@ func (m *MockStorage) QueryPages(ctx context.Context, query IndexQuery, callback items = filtered } - result := mockReadBatch{} + result := mockReadBatch{ + index: -1, + } for _, item := range items { - result = append(result, item) + result.items = append(result.items, item) } - callback(result) + callback(&result) return nil } @@ -300,16 +315,20 @@ func (b *mockWriteBatch) Add(tableName, hashValue string, rangeValue []byte, val }{tableName, hashValue, rangeValue, value}) } -type mockReadBatch []mockItem +type mockReadBatch struct { + index int + items []mockItem +} -func (b mockReadBatch) Len() int { - return len(b) +func (b *mockReadBatch) Next() bool { + b.index++ + return b.index < len(b.items) } -func (b mockReadBatch) RangeValue(i int) []byte { - return b[i].rangeValue +func (b *mockReadBatch) RangeValue() []byte { + return b.items[b.index].rangeValue } -func (b mockReadBatch) Value(i int) []byte { - return b[i].value +func (b *mockReadBatch) Value() []byte { + return b.items[b.index].value } diff --git a/storage/caching_storage_client.go b/storage/caching_storage_client.go index 00e9ed922e4d..c7f7ecb3b375 100644 --- a/storage/caching_storage_client.go +++ b/storage/caching_storage_client.go @@ -1,11 +1,8 @@ package storage import ( - "bytes" "context" - "encoding/hex" - "hash/fnv" - "strings" + "sync" "time" proto "github.com/golang/protobuf/proto" @@ -14,6 +11,7 @@ import ( "github.com/weaveworks/cortex/pkg/chunk" "github.com/weaveworks/cortex/pkg/chunk/cache" + chunk_util "github.com/weaveworks/cortex/pkg/chunk/util" ) var ( @@ -104,102 +102,53 @@ func newCachingStorageClient(client chunk.StorageClient, cache cache.Cache, vali } } -func (s *cachingStorageClient) QueryPages(ctx context.Context, query chunk.IndexQuery, callback func(result chunk.ReadBatch) (shouldContinue bool)) error { - value, ok, err := s.cache.Fetch(ctx, queryKey(query)) - if err != nil { - cacheCorruptErrs.Inc() - } - - if ok && err == nil { - filteredBatch, _ := filterBatchByQuery(query, []chunk.ReadBatch{value}) - callback(filteredBatch) +func (s *cachingStorageClient) QueryPages(ctx context.Context, queries []chunk.IndexQuery, callback func(chunk.IndexQuery, chunk.ReadBatch) (shouldContinue bool)) error { + // We cache the entire row, so filter client side. + callback = chunk_util.QueryFilter(callback) + cacheableMissed := []chunk.IndexQuery{} + missed := map[string]chunk.IndexQuery{} + + for _, query := range queries { + value, ok := s.cache.Get(ctx, queryKey(query)) + if !ok { + cacheableMissed = append(cacheableMissed, chunk.IndexQuery{ + TableName: query.TableName, + HashValue: query.HashValue, + }) + missed[queryKey(query)] = query + continue + } - return nil + for _, batch := range value.([]chunk.ReadBatch) { + callback(query, batch) + } } - batches := []chunk.ReadBatch{} - cacheableQuery := chunk.IndexQuery{ - TableName: query.TableName, - HashValue: query.HashValue, - } // Just reads the entire row and caches it. - - expiryTime := time.Now().Add(s.validity) - err = s.StorageClient.QueryPages(ctx, cacheableQuery, copyingCallback(&batches)) + var resultsMtx sync.Mutex + results := map[string][]chunk.ReadBatch{} + err := s.StorageClient.QueryPages(ctx, cacheableMissed, func(cacheableQuery chunk.IndexQuery, r chunk.ReadBatch) bool { + resultsMtx.Lock() + defer resultsMtx.Unlock() + key := queryKey(cacheableQuery) + results[key] = append(results[key], r) + return true + }) if err != nil { return err } - filteredBatch, totalBatches := filterBatchByQuery(query, batches) - callback(filteredBatch) - - totalBatches.Key = queryKey(query) - totalBatches.Expiry = expiryTime.UnixNano() - - s.cache.Store(ctx, totalBatches.Key, totalBatches) - return nil -} - -// Len implements chunk.ReadBatch. -func (b ReadBatch) Len() int { return len(b.Entries) } - -// RangeValue implements chunk.ReadBatch. -func (b ReadBatch) RangeValue(i int) []byte { return b.Entries[i].Column } - -// Value implements chunk.ReadBatch. -func (b ReadBatch) Value(i int) []byte { return b.Entries[i].Value } - -func copyingCallback(readBatches *[]chunk.ReadBatch) func(chunk.ReadBatch) bool { - return func(result chunk.ReadBatch) bool { - *readBatches = append(*readBatches, result) - return true + resultsMtx.Lock() + defer resultsMtx.Unlock() + for key, batches := range results { + query := missed[key] + for _, batch := range batches { + callback(query, batch) + } } + return nil } func queryKey(q chunk.IndexQuery) string { const sep = "\xff" return q.TableName + sep + q.HashValue } - -func filterBatchByQuery(query chunk.IndexQuery, batches []chunk.ReadBatch) (filteredBatch, totalBatch ReadBatch) { - filter := func([]byte, []byte) bool { return true } - - if len(query.RangeValuePrefix) != 0 { - filter = func(rangeValue []byte, value []byte) bool { - return strings.HasPrefix(string(rangeValue), string(query.RangeValuePrefix)) - } - } - if len(query.RangeValueStart) != 0 { - filter = func(rangeValue []byte, value []byte) bool { - return string(rangeValue) >= string(query.RangeValueStart) - } - } - if len(query.ValueEqual) != 0 { - // This is on top of the existing filters. - existingFilter := filter - filter = func(rangeValue []byte, value []byte) bool { - return existingFilter(rangeValue, value) && bytes.Equal(value, query.ValueEqual) - } - } - - filteredBatch.Entries = make([]*Entry, 0, len(batches)) // On the higher side for most queries. On the lower side for column key schema. - totalBatch.Entries = make([]*Entry, 0, len(batches)) - for _, batch := range batches { - for i := 0; i < batch.Len(); i++ { - totalBatch.Entries = append(totalBatch.Entries, &Entry{Column: batch.RangeValue(i), Value: batch.Value(i)}) - - if filter(batch.RangeValue(i), batch.Value(i)) { - filteredBatch.Entries = append(filteredBatch.Entries, &Entry{Column: batch.RangeValue(i), Value: batch.Value(i)}) - } - } - } - - return -} - -func hashKey(key string) string { - hasher := fnv.New64a() - hasher.Write([]byte(key)) // This'll never error. - - // Hex because memcache errors for the bytes produced by the hash. - return hex.EncodeToString(hasher.Sum(nil)) -} diff --git a/storage/index_test.go b/storage/index_test.go index 91d8532bb96b..260d0fb56325 100644 --- a/storage/index_test.go +++ b/storage/index_test.go @@ -21,15 +21,17 @@ func TestIndexBasic(t *testing.T) { // Make sure we get back the correct entries by hash value. for i := 0; i < 30; i++ { - entry := chunk.IndexQuery{ - TableName: tableName, - HashValue: fmt.Sprintf("hash%d", i), + entries := []chunk.IndexQuery{ + { + TableName: tableName, + HashValue: fmt.Sprintf("hash%d", i), + }, } var have []chunk.IndexEntry - err := client.QueryPages(context.Background(), entry, func(read chunk.ReadBatch) bool { - for j := 0; j < read.Len(); j++ { + err := client.QueryPages(context.Background(), entries, func(_ chunk.IndexQuery, read chunk.ReadBatch) bool { + for read.Next() { have = append(have, chunk.IndexEntry{ - RangeValue: read.RangeValue(j), + RangeValue: read.RangeValue(), }) } return true @@ -167,13 +169,13 @@ func TestQueryPages(t *testing.T) { run := true for run { var have []chunk.IndexEntry - err = client.QueryPages(context.Background(), tt.query, func(read chunk.ReadBatch) bool { - for i := 0; i < read.Len(); i++ { + err = client.QueryPages(context.Background(), []chunk.IndexQuery{tt.query}, func(_ chunk.IndexQuery, read chunk.ReadBatch) bool { + for read.Next() { have = append(have, chunk.IndexEntry{ TableName: tt.query.TableName, HashValue: tt.query.HashValue, - RangeValue: read.RangeValue(i), - Value: read.Value(i), + RangeValue: read.RangeValue(), + Value: read.Value(), }) } return true diff --git a/storage_client.go b/storage_client.go index c86f573b77bc..d867def5ab31 100644 --- a/storage_client.go +++ b/storage_client.go @@ -9,7 +9,7 @@ type StorageClient interface { BatchWrite(context.Context, WriteBatch) error // For the read path. - QueryPages(ctx context.Context, query IndexQuery, callback func(result ReadBatch) (shouldContinue bool)) error + QueryPages(ctx context.Context, queries []IndexQuery, callback func(IndexQuery, ReadBatch) (shouldContinue bool)) error // For storing and retrieving chunks. PutChunks(ctx context.Context, chunks []Chunk) error @@ -23,7 +23,7 @@ type WriteBatch interface { // ReadBatch represents the results of a QueryPages. type ReadBatch interface { - Len() int - RangeValue(index int) []byte - Value(index int) []byte + Next() bool + RangeValue() []byte + Value() []byte } diff --git a/util/util.go b/util/util.go new file mode 100644 index 000000000000..6a6c412d3031 --- /dev/null +++ b/util/util.go @@ -0,0 +1,77 @@ +package util + +import ( + "bytes" + "context" + "strings" + + "github.com/weaveworks/cortex/pkg/chunk" +) + +// DoSingleQuery is the interface for indexes that don't support batching yet. +type DoSingleQuery func( + ctx context.Context, query chunk.IndexQuery, + callback func(chunk.ReadBatch) bool, +) error + +// DoParallelQueries translates between our interface for query batching, +// and indexes that don't yet support batching. +func DoParallelQueries( + ctx context.Context, doSingleQuery DoSingleQuery, queries []chunk.IndexQuery, + callback func(chunk.IndexQuery, chunk.ReadBatch) bool, +) error { + incomingErrors := make(chan error) + for _, query := range queries { + go func(query chunk.IndexQuery) { + incomingErrors <- doSingleQuery(ctx, query, func(r chunk.ReadBatch) bool { + return callback(query, r) + }) + }(query) + } + var lastErr error + for i := 0; i < len(queries); i++ { + err := <-incomingErrors + if err != nil { + + lastErr = err + } + } + return lastErr +} + +// Callback from an IndexQuery. +type Callback func(chunk.IndexQuery, chunk.ReadBatch) bool + +type filteringBatch struct { + query chunk.IndexQuery + chunk.ReadBatch +} + +func (f *filteringBatch) Next() bool { + for f.ReadBatch.Next() { + rangeValue, value := f.ReadBatch.RangeValue(), f.ReadBatch.Value() + + if len(f.query.RangeValuePrefix) != 0 && !strings.HasPrefix(string(rangeValue), string(f.query.RangeValuePrefix)) { + continue + } + if len(f.query.RangeValueStart) != 0 && string(rangeValue) < string(f.query.RangeValueStart) { + continue + } + if len(f.query.ValueEqual) != 0 && !bytes.Equal(value, f.query.ValueEqual) { + continue + } + + return true + } + + return false +} + +// 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 { + return func(query chunk.IndexQuery, batch chunk.ReadBatch) bool { + return callback(query, &filteringBatch{query, batch}) + } +}