feat(storage): ReadGroup RPC definitions and storage reader
parent
5f87134ca8
commit
95aa194498
|
@ -117,7 +117,7 @@ func (s *readFilterSource) run(ctx context.Context) error {
|
|||
}
|
||||
|
||||
func createReadFilterSource(s plan.ProcedureSpec, id execute.DatasetID, a execute.Administration) (execute.Source, error) {
|
||||
span, ctx := tracing.StartSpanFromContext(context.TODO())
|
||||
span, ctx := tracing.StartSpanFromContext(a.Context())
|
||||
defer span.Finish()
|
||||
|
||||
spec := s.(*ReadRangePhysSpec)
|
||||
|
@ -190,7 +190,7 @@ func (s *readGroupSource) run(ctx context.Context) error {
|
|||
}
|
||||
|
||||
func createReadGroupSource(s plan.ProcedureSpec, id execute.DatasetID, a execute.Administration) (execute.Source, error) {
|
||||
span, ctx := tracing.StartSpanFromContext(context.TODO())
|
||||
span, ctx := tracing.StartSpanFromContext(a.Context())
|
||||
defer span.Finish()
|
||||
|
||||
spec := s.(*ReadGroupPhysSpec)
|
||||
|
@ -236,7 +236,7 @@ func createReadGroupSource(s plan.ProcedureSpec, id execute.DatasetID, a execute
|
|||
}
|
||||
|
||||
func createReadTagKeysSource(prSpec plan.ProcedureSpec, dsid execute.DatasetID, a execute.Administration) (execute.Source, error) {
|
||||
span, ctx := tracing.StartSpanFromContext(context.TODO())
|
||||
span, ctx := tracing.StartSpanFromContext(a.Context())
|
||||
defer span.Finish()
|
||||
|
||||
spec := prSpec.(*ReadTagKeysPhysSpec)
|
||||
|
@ -300,7 +300,7 @@ func (s *readTagKeysSource) run(ctx context.Context) error {
|
|||
}
|
||||
|
||||
func createReadTagValuesSource(prSpec plan.ProcedureSpec, dsid execute.DatasetID, a execute.Administration) (execute.Source, error) {
|
||||
span, ctx := tracing.StartSpanFromContext(context.TODO())
|
||||
span, ctx := tracing.StartSpanFromContext(a.Context())
|
||||
defer span.Finish()
|
||||
|
||||
spec := prSpec.(*ReadTagValuesPhysSpec)
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -13,12 +13,15 @@ option (gogoproto.unmarshaler_all) = true;
|
|||
option (gogoproto.goproto_getters_all) = false;
|
||||
|
||||
service Storage {
|
||||
// Read performs a read operation using the given ReadRequest
|
||||
// TODO(jlapacik): Remove this unsupported call
|
||||
rpc Read (ReadRequest) returns (stream ReadResponse);
|
||||
|
||||
// ReadFilter performs a filter operation at storage
|
||||
rpc ReadFilter (ReadFilterRequest) returns (stream ReadResponse);
|
||||
|
||||
// ReadGroup performs a group operation at storage
|
||||
rpc ReadGroup (ReadGroupRequest) returns (stream ReadResponse);
|
||||
|
||||
// Capabilities returns a map of keys and values identifying the capabilities supported by the storage engine
|
||||
rpc Capabilities (google.protobuf.Empty) returns (CapabilitiesResponse);
|
||||
|
||||
|
@ -40,8 +43,11 @@ message ReadFilterRequest {
|
|||
Predicate predicate = 3;
|
||||
}
|
||||
|
||||
// Request message for Storage.Read.
|
||||
message ReadRequest {
|
||||
message ReadGroupRequest {
|
||||
google.protobuf.Any read_source = 1 [(gogoproto.customname) = "ReadSource"];
|
||||
TimestampRange range = 2 [(gogoproto.nullable) = false];
|
||||
Predicate predicate = 3;
|
||||
|
||||
enum Group {
|
||||
option (gogoproto.goproto_enum_prefix) = false;
|
||||
|
||||
|
@ -60,6 +66,17 @@ message ReadRequest {
|
|||
GROUP_EXCEPT = 3 [(gogoproto.enumvalue_customname) = "GroupExcept"];
|
||||
}
|
||||
|
||||
// GroupKeys specifies a list of tag keys used to order the data.
|
||||
// It is dependent on the Group property to determine its behavior.
|
||||
repeated string group_keys = 4 [(gogoproto.customname) = "GroupKeys"];
|
||||
|
||||
Group group = 5;
|
||||
Aggregate aggregate = 6;
|
||||
|
||||
// TODO(jlapacik): This field is only used in unit tests.
|
||||
// Specifically the two tests in group_resultset_test.go.
|
||||
// This field should be removed and the tests that depend
|
||||
// on it refactored.
|
||||
enum HintFlags {
|
||||
option (gogoproto.goproto_enum_prefix) = false;
|
||||
|
||||
|
@ -69,45 +86,12 @@ message ReadRequest {
|
|||
// HintSchemaAllTime performs schema queries without using time ranges
|
||||
HINT_SCHEMA_ALL_TIME = 0x04 [(gogoproto.enumvalue_customname) = "HintSchemaAllTime"];
|
||||
}
|
||||
|
||||
google.protobuf.Any read_source = 13 [(gogoproto.customname) = "ReadSource"];
|
||||
|
||||
TimestampRange timestamp_range = 2 [(gogoproto.customname) = "TimestampRange", (gogoproto.nullable) = false];
|
||||
|
||||
// Descending indicates whether points should be returned in descending order.
|
||||
bool descending = 3;
|
||||
|
||||
// GroupKeys specifies a list of tag keys used to order the data. It is dependent on the Group property to determine
|
||||
// its behavior.
|
||||
repeated string group_keys = 4 [(gogoproto.customname) = "GroupKeys"];
|
||||
|
||||
//
|
||||
Group group = 11;
|
||||
|
||||
// Aggregate specifies an optional aggregate to apply to the data.
|
||||
// TODO(sgc): switch to slice for multiple aggregates in a single request
|
||||
Aggregate aggregate = 9;
|
||||
|
||||
Predicate predicate = 5;
|
||||
|
||||
// SeriesLimit determines the maximum number of series to be returned for the request. Specify 0 for no limit.
|
||||
int64 series_limit = 6 [(gogoproto.customname) = "SeriesLimit"];
|
||||
|
||||
// SeriesOffset determines how many series to skip before processing the request.
|
||||
int64 series_offset = 7 [(gogoproto.customname) = "SeriesOffset"];
|
||||
|
||||
// PointsLimit determines the maximum number of values per series to be returned for the request.
|
||||
// Specify 0 for no limit. -1 to return series frames only.
|
||||
int64 points_limit = 8 [(gogoproto.customname) = "PointsLimit"];
|
||||
|
||||
// Trace contains opaque data if a trace is active.
|
||||
map<string, string> trace = 10 [(gogoproto.customname) = "Trace"];
|
||||
|
||||
// Hints is a bitwise OR of HintFlags to control the behavior
|
||||
// of the read request.
|
||||
fixed32 hints = 12 [(gogoproto.customname) = "Hints", (gogoproto.casttype) = "HintFlags"];
|
||||
fixed32 hints = 7 [(gogoproto.customname) = "Hints", (gogoproto.casttype) = "HintFlags"];
|
||||
}
|
||||
|
||||
// TODO(jlapacik): Remove this message
|
||||
message ReadRequest {}
|
||||
|
||||
message Aggregate {
|
||||
enum AggregateType {
|
||||
option (gogoproto.goproto_enum_prefix) = false;
|
||||
|
@ -127,7 +111,7 @@ message Tag {
|
|||
bytes value = 2;
|
||||
}
|
||||
|
||||
// Response message for Storage.Read.
|
||||
// Response message for ReadFilter and ReadGroup
|
||||
message ReadResponse {
|
||||
enum FrameType {
|
||||
option (gogoproto.goproto_enum_prefix) = false;
|
||||
|
@ -161,7 +145,7 @@ message ReadResponse {
|
|||
message GroupFrame {
|
||||
// TagKeys
|
||||
repeated bytes tag_keys = 1 [(gogoproto.customname) = "TagKeys"];
|
||||
// PartitionKeyVals is the values of the partition key for this group, order matching ReadRequest.GroupKeys
|
||||
// PartitionKeyVals is the values of the partition key for this group, order matching ReadGroupRequest.GroupKeys
|
||||
repeated bytes partition_key_vals = 2 [(gogoproto.customname) = "PartitionKeyVals"];
|
||||
}
|
||||
|
||||
|
@ -233,9 +217,3 @@ message TagValuesRequest {
|
|||
message StringValuesResponse {
|
||||
repeated bytes values = 1;
|
||||
}
|
||||
|
||||
//message ExplainRequest {
|
||||
// ReadRequest read_request = 1 [(gogoproto.customname) = "ReadRequest"];
|
||||
//}
|
||||
//
|
||||
//message ExplainResponse {}
|
||||
|
|
|
@ -4,9 +4,10 @@ import (
|
|||
"bytes"
|
||||
"context"
|
||||
"fmt"
|
||||
"github.com/influxdata/influxdb/kit/tracing"
|
||||
"math"
|
||||
"sort"
|
||||
|
||||
"github.com/influxdata/influxdb/kit/tracing"
|
||||
"github.com/influxdata/influxdb/models"
|
||||
"github.com/influxdata/influxdb/storage/reads/datatypes"
|
||||
"github.com/influxdata/influxdb/tsdb/cursors"
|
||||
|
@ -14,7 +15,7 @@ import (
|
|||
|
||||
type groupResultSet struct {
|
||||
ctx context.Context
|
||||
req *datatypes.ReadRequest
|
||||
req *datatypes.ReadGroupRequest
|
||||
agg *datatypes.Aggregate
|
||||
mb multiShardCursors
|
||||
|
||||
|
@ -42,7 +43,7 @@ func GroupOptionNilSortLo() GroupOption {
|
|||
}
|
||||
}
|
||||
|
||||
func NewGroupResultSet(ctx context.Context, req *datatypes.ReadRequest, newCursorFn func() (SeriesCursor, error), opts ...GroupOption) GroupResultSet {
|
||||
func NewGroupResultSet(ctx context.Context, req *datatypes.ReadGroupRequest, newCursorFn func() (SeriesCursor, error), opts ...GroupOption) GroupResultSet {
|
||||
g := &groupResultSet{
|
||||
ctx: ctx,
|
||||
req: req,
|
||||
|
@ -56,7 +57,7 @@ func NewGroupResultSet(ctx context.Context, req *datatypes.ReadRequest, newCurso
|
|||
o(g)
|
||||
}
|
||||
|
||||
g.mb = newMultiShardArrayCursors(ctx, req.TimestampRange.Start, req.TimestampRange.End, !req.Descending, req.PointsLimit)
|
||||
g.mb = newMultiShardArrayCursors(ctx, req.Range.Start, req.Range.End, true, math.MaxInt64)
|
||||
|
||||
for i, k := range req.GroupKeys {
|
||||
g.keys[i] = []byte(k)
|
||||
|
|
|
@ -6,6 +6,7 @@ import (
|
|||
"testing"
|
||||
|
||||
"github.com/google/go-cmp/cmp"
|
||||
|
||||
"github.com/influxdata/influxdb/models"
|
||||
"github.com/influxdata/influxdb/pkg/data/gen"
|
||||
"github.com/influxdata/influxdb/storage/reads"
|
||||
|
@ -16,7 +17,7 @@ func TestGroupGroupResultSetSorting(t *testing.T) {
|
|||
tests := []struct {
|
||||
name string
|
||||
cur reads.SeriesCursor
|
||||
group datatypes.ReadRequest_Group
|
||||
group datatypes.ReadGroupRequest_Group
|
||||
keys []string
|
||||
exp string
|
||||
}{
|
||||
|
@ -181,7 +182,14 @@ group:
|
|||
|
||||
var hints datatypes.HintFlags
|
||||
hints.SetHintSchemaAllTime()
|
||||
rs := reads.NewGroupResultSet(context.Background(), &datatypes.ReadRequest{Group: tt.group, GroupKeys: tt.keys, Hints: hints}, newCursor)
|
||||
rs := reads.NewGroupResultSet(context.Background(), &datatypes.ReadGroupRequest{
|
||||
Group: tt.group,
|
||||
GroupKeys: tt.keys,
|
||||
// TODO(jlapacik):
|
||||
// Hints is not used except for the tests in this file.
|
||||
// Eventually this field should be removed entirely.
|
||||
Hints: hints,
|
||||
}, newCursor)
|
||||
|
||||
sb := new(strings.Builder)
|
||||
GroupResultSetToString(sb, rs, SkipNilCursor())
|
||||
|
@ -202,7 +210,7 @@ func TestNewGroupResultSet_GroupNone_NoDataReturnsNil(t *testing.T) {
|
|||
)}, nil
|
||||
}
|
||||
|
||||
rs := reads.NewGroupResultSet(context.Background(), &datatypes.ReadRequest{Group: datatypes.GroupNone}, newCursor)
|
||||
rs := reads.NewGroupResultSet(context.Background(), &datatypes.ReadGroupRequest{Group: datatypes.GroupNone}, newCursor)
|
||||
if rs != nil {
|
||||
t.Errorf("expected nil cursor")
|
||||
}
|
||||
|
@ -217,7 +225,7 @@ func TestNewGroupResultSet_GroupBy_NoDataReturnsNil(t *testing.T) {
|
|||
)}, nil
|
||||
}
|
||||
|
||||
rs := reads.NewGroupResultSet(context.Background(), &datatypes.ReadRequest{Group: datatypes.GroupBy, GroupKeys: []string{"tag0"}}, newCursor)
|
||||
rs := reads.NewGroupResultSet(context.Background(), &datatypes.ReadGroupRequest{Group: datatypes.GroupBy, GroupKeys: []string{"tag0"}}, newCursor)
|
||||
if rs != nil {
|
||||
t.Errorf("expected nil cursor")
|
||||
}
|
||||
|
@ -300,7 +308,14 @@ group:
|
|||
|
||||
var hints datatypes.HintFlags
|
||||
hints.SetHintSchemaAllTime()
|
||||
rs := reads.NewGroupResultSet(context.Background(), &datatypes.ReadRequest{Group: datatypes.GroupBy, GroupKeys: tt.keys, Hints: hints}, newCursor, tt.opts...)
|
||||
rs := reads.NewGroupResultSet(context.Background(), &datatypes.ReadGroupRequest{
|
||||
Group: datatypes.GroupBy,
|
||||
GroupKeys: tt.keys,
|
||||
// TODO(jlapacik):
|
||||
// Hints is not used except for the tests in this file.
|
||||
// Eventually this field should be removed entirely.
|
||||
Hints: hints,
|
||||
}, newCursor, tt.opts...)
|
||||
|
||||
sb := new(strings.Builder)
|
||||
GroupResultSetToString(sb, rs, SkipNilCursor())
|
||||
|
@ -364,9 +379,7 @@ func BenchmarkNewGroupResultSet_GroupBy(b *testing.B) {
|
|||
b.ResetTimer()
|
||||
b.ReportAllocs()
|
||||
for i := 0; i < b.N; i++ {
|
||||
var hints datatypes.HintFlags
|
||||
hints.SetHintSchemaAllTime()
|
||||
rs := reads.NewGroupResultSet(context.Background(), &datatypes.ReadRequest{Group: datatypes.GroupBy, GroupKeys: []string{"tag2"}, Hints: hints}, newCursor)
|
||||
rs := reads.NewGroupResultSet(context.Background(), &datatypes.ReadGroupRequest{Group: datatypes.GroupBy, GroupKeys: []string{"tag2"}}, newCursor)
|
||||
rs.Close()
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,7 +1,6 @@
|
|||
package reads
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
"fmt"
|
||||
"strings"
|
||||
|
@ -33,27 +32,20 @@ func NewReader(s Store) influxdb.Reader {
|
|||
}
|
||||
|
||||
func (r *storeReader) Read(ctx context.Context, rs influxdb.ReadSpec, start, stop execute.Time, alloc *memory.Allocator) (influxdb.TableIterator, error) {
|
||||
var predicate *datatypes.Predicate
|
||||
if rs.Predicate != nil {
|
||||
p, err := toStoragePredicate(rs.Predicate)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
predicate = p
|
||||
}
|
||||
|
||||
return &tableIterator{
|
||||
ctx: ctx,
|
||||
bounds: execute.Bounds{Start: start, Stop: stop},
|
||||
s: r.s,
|
||||
readSpec: rs,
|
||||
predicate: predicate,
|
||||
alloc: alloc,
|
||||
}, nil
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
func (r *storeReader) ReadFilter(ctx context.Context, spec influxdb.ReadFilterSpec, alloc *memory.Allocator) (influxdb.TableIterator, error) {
|
||||
return &simpleTableIterator{
|
||||
return &filterIterator{
|
||||
ctx: ctx,
|
||||
s: r.s,
|
||||
spec: spec,
|
||||
alloc: alloc,
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (r *storeReader) ReadGroup(ctx context.Context, spec influxdb.ReadGroupSpec, alloc *memory.Allocator) (influxdb.TableIterator, error) {
|
||||
return &groupIterator{
|
||||
ctx: ctx,
|
||||
s: r.s,
|
||||
spec: spec,
|
||||
|
@ -103,7 +95,7 @@ func (r *storeReader) ReadTagValues(ctx context.Context, spec influxdb.ReadTagVa
|
|||
|
||||
func (r *storeReader) Close() {}
|
||||
|
||||
type simpleTableIterator struct {
|
||||
type filterIterator struct {
|
||||
ctx context.Context
|
||||
s Store
|
||||
spec influxdb.ReadFilterSpec
|
||||
|
@ -111,12 +103,12 @@ type simpleTableIterator struct {
|
|||
alloc *memory.Allocator
|
||||
}
|
||||
|
||||
func (bi *simpleTableIterator) Statistics() cursors.CursorStats { return bi.stats }
|
||||
func (fi *filterIterator) Statistics() cursors.CursorStats { return fi.stats }
|
||||
|
||||
func (bi *simpleTableIterator) Do(f func(flux.Table) error) error {
|
||||
src := bi.s.GetSource(
|
||||
uint64(bi.spec.OrganizationID),
|
||||
uint64(bi.spec.BucketID),
|
||||
func (fi *filterIterator) Do(f func(flux.Table) error) error {
|
||||
src := fi.s.GetSource(
|
||||
uint64(fi.spec.OrganizationID),
|
||||
uint64(fi.spec.BucketID),
|
||||
)
|
||||
|
||||
// Setup read request
|
||||
|
@ -126,8 +118,8 @@ func (bi *simpleTableIterator) Do(f func(flux.Table) error) error {
|
|||
}
|
||||
|
||||
var predicate *datatypes.Predicate
|
||||
if bi.spec.Predicate != nil {
|
||||
p, err := toStoragePredicate(bi.spec.Predicate)
|
||||
if fi.spec.Predicate != nil {
|
||||
p, err := toStoragePredicate(fi.spec.Predicate)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -137,10 +129,10 @@ func (bi *simpleTableIterator) Do(f func(flux.Table) error) error {
|
|||
var req datatypes.ReadFilterRequest
|
||||
req.ReadSource = any
|
||||
req.Predicate = predicate
|
||||
req.Range.Start = int64(bi.spec.Bounds.Start)
|
||||
req.Range.End = int64(bi.spec.Bounds.Stop)
|
||||
req.Range.Start = int64(fi.spec.Bounds.Start)
|
||||
req.Range.End = int64(fi.spec.Bounds.Stop)
|
||||
|
||||
rs, err := bi.s.ReadFilter(bi.ctx, &req)
|
||||
rs, err := fi.s.ReadFilter(fi.ctx, &req)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -149,10 +141,10 @@ func (bi *simpleTableIterator) Do(f func(flux.Table) error) error {
|
|||
return nil
|
||||
}
|
||||
|
||||
return bi.handleRead(f, rs)
|
||||
return fi.handleRead(f, rs)
|
||||
}
|
||||
|
||||
func (bi *simpleTableIterator) handleRead(f func(flux.Table) error, rs ResultSet) error {
|
||||
func (fi *filterIterator) handleRead(f func(flux.Table) error, rs ResultSet) error {
|
||||
// these resources must be closed if not nil on return
|
||||
var (
|
||||
cur cursors.Cursor
|
||||
|
@ -177,25 +169,25 @@ READ:
|
|||
continue
|
||||
}
|
||||
|
||||
bnds := bi.spec.Bounds
|
||||
bnds := fi.spec.Bounds
|
||||
key := defaultGroupKeyForSeries(rs.Tags(), bnds)
|
||||
done := make(chan struct{})
|
||||
switch typedCur := cur.(type) {
|
||||
case cursors.IntegerArrayCursor:
|
||||
cols, defs := determineTableColsForSeries(rs.Tags(), flux.TInt)
|
||||
table = newIntegerTable(done, typedCur, bnds, key, cols, rs.Tags(), defs, bi.alloc)
|
||||
table = newIntegerTable(done, typedCur, bnds, key, cols, rs.Tags(), defs, fi.alloc)
|
||||
case cursors.FloatArrayCursor:
|
||||
cols, defs := determineTableColsForSeries(rs.Tags(), flux.TFloat)
|
||||
table = newFloatTable(done, typedCur, bnds, key, cols, rs.Tags(), defs, bi.alloc)
|
||||
table = newFloatTable(done, typedCur, bnds, key, cols, rs.Tags(), defs, fi.alloc)
|
||||
case cursors.UnsignedArrayCursor:
|
||||
cols, defs := determineTableColsForSeries(rs.Tags(), flux.TUInt)
|
||||
table = newUnsignedTable(done, typedCur, bnds, key, cols, rs.Tags(), defs, bi.alloc)
|
||||
table = newUnsignedTable(done, typedCur, bnds, key, cols, rs.Tags(), defs, fi.alloc)
|
||||
case cursors.BooleanArrayCursor:
|
||||
cols, defs := determineTableColsForSeries(rs.Tags(), flux.TBool)
|
||||
table = newBooleanTable(done, typedCur, bnds, key, cols, rs.Tags(), defs, bi.alloc)
|
||||
table = newBooleanTable(done, typedCur, bnds, key, cols, rs.Tags(), defs, fi.alloc)
|
||||
case cursors.StringArrayCursor:
|
||||
cols, defs := determineTableColsForSeries(rs.Tags(), flux.TString)
|
||||
table = newStringTable(done, typedCur, bnds, key, cols, rs.Tags(), defs, bi.alloc)
|
||||
table = newStringTable(done, typedCur, bnds, key, cols, rs.Tags(), defs, fi.alloc)
|
||||
default:
|
||||
panic(fmt.Sprintf("unreachable: %T", typedCur))
|
||||
}
|
||||
|
@ -210,214 +202,79 @@ READ:
|
|||
}
|
||||
select {
|
||||
case <-done:
|
||||
case <-bi.ctx.Done():
|
||||
case <-fi.ctx.Done():
|
||||
table.Cancel()
|
||||
break READ
|
||||
}
|
||||
}
|
||||
|
||||
stats := table.Statistics()
|
||||
bi.stats.ScannedValues += stats.ScannedValues
|
||||
bi.stats.ScannedBytes += stats.ScannedBytes
|
||||
fi.stats.ScannedValues += stats.ScannedValues
|
||||
fi.stats.ScannedBytes += stats.ScannedBytes
|
||||
table.Close()
|
||||
table = nil
|
||||
}
|
||||
return rs.Err()
|
||||
}
|
||||
|
||||
type tableIterator struct {
|
||||
ctx context.Context
|
||||
bounds execute.Bounds
|
||||
s Store
|
||||
readSpec influxdb.ReadSpec
|
||||
predicate *datatypes.Predicate
|
||||
stats cursors.CursorStats
|
||||
alloc *memory.Allocator
|
||||
type groupIterator struct {
|
||||
ctx context.Context
|
||||
s Store
|
||||
spec influxdb.ReadGroupSpec
|
||||
stats cursors.CursorStats
|
||||
alloc *memory.Allocator
|
||||
}
|
||||
|
||||
func (bi *tableIterator) Statistics() cursors.CursorStats { return bi.stats }
|
||||
func (gi *groupIterator) Statistics() cursors.CursorStats { return gi.stats }
|
||||
|
||||
func (bi *tableIterator) Do(f func(flux.Table) error) error {
|
||||
src := bi.s.GetSource(
|
||||
uint64(bi.readSpec.OrganizationID),
|
||||
uint64(bi.readSpec.BucketID),
|
||||
func (gi *groupIterator) Do(f func(flux.Table) error) error {
|
||||
src := gi.s.GetSource(
|
||||
uint64(gi.spec.OrganizationID),
|
||||
uint64(gi.spec.BucketID),
|
||||
)
|
||||
|
||||
// Setup read request
|
||||
var req datatypes.ReadRequest
|
||||
if any, err := types.MarshalAny(src); err != nil {
|
||||
any, err := types.MarshalAny(src)
|
||||
if err != nil {
|
||||
return err
|
||||
} else {
|
||||
req.ReadSource = any
|
||||
}
|
||||
req.Predicate = bi.predicate
|
||||
req.Descending = bi.readSpec.Descending
|
||||
req.TimestampRange.Start = int64(bi.bounds.Start)
|
||||
req.TimestampRange.End = int64(bi.bounds.Stop)
|
||||
req.Group = convertGroupMode(bi.readSpec.GroupMode)
|
||||
req.GroupKeys = bi.readSpec.GroupKeys
|
||||
req.SeriesLimit = bi.readSpec.SeriesLimit
|
||||
req.PointsLimit = bi.readSpec.PointsLimit
|
||||
req.SeriesOffset = bi.readSpec.SeriesOffset
|
||||
|
||||
if req.PointsLimit == -1 {
|
||||
req.Hints.SetNoPoints()
|
||||
}
|
||||
|
||||
if agg, err := determineAggregateMethod(bi.readSpec.AggregateMethod); err != nil {
|
||||
var predicate *datatypes.Predicate
|
||||
if gi.spec.Predicate != nil {
|
||||
p, err := toStoragePredicate(gi.spec.Predicate)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
predicate = p
|
||||
}
|
||||
|
||||
var req datatypes.ReadGroupRequest
|
||||
req.ReadSource = any
|
||||
req.Predicate = predicate
|
||||
req.Range.Start = int64(gi.spec.Bounds.Start)
|
||||
req.Range.End = int64(gi.spec.Bounds.Stop)
|
||||
|
||||
req.Group = convertGroupMode(gi.spec.GroupMode)
|
||||
req.GroupKeys = gi.spec.GroupKeys
|
||||
|
||||
if agg, err := determineAggregateMethod(gi.spec.AggregateMethod); err != nil {
|
||||
return err
|
||||
} else if agg != datatypes.AggregateTypeNone {
|
||||
req.Aggregate = &datatypes.Aggregate{Type: agg}
|
||||
}
|
||||
|
||||
switch {
|
||||
case req.Group != datatypes.GroupAll:
|
||||
rs, err := bi.s.GroupRead(bi.ctx, &req)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if rs == nil {
|
||||
return nil
|
||||
}
|
||||
|
||||
if req.Hints.NoPoints() {
|
||||
return bi.handleGroupReadNoPoints(f, rs)
|
||||
}
|
||||
return bi.handleGroupRead(f, rs)
|
||||
|
||||
default:
|
||||
rs, err := bi.s.Read(bi.ctx, &req)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if rs == nil {
|
||||
return nil
|
||||
}
|
||||
|
||||
if req.Hints.NoPoints() {
|
||||
return bi.handleReadNoPoints(f, rs)
|
||||
}
|
||||
return bi.handleRead(f, rs)
|
||||
rs, err := gi.s.ReadGroup(gi.ctx, &req)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if rs == nil {
|
||||
return nil
|
||||
}
|
||||
return gi.handleRead(f, rs)
|
||||
}
|
||||
|
||||
func (bi *tableIterator) handleRead(f func(flux.Table) error, rs ResultSet) error {
|
||||
// these resources must be closed if not nil on return
|
||||
var (
|
||||
cur cursors.Cursor
|
||||
table storageTable
|
||||
)
|
||||
|
||||
defer func() {
|
||||
if table != nil {
|
||||
table.Close()
|
||||
}
|
||||
if cur != nil {
|
||||
cur.Close()
|
||||
}
|
||||
rs.Close()
|
||||
}()
|
||||
|
||||
READ:
|
||||
for rs.Next() {
|
||||
cur = rs.Cursor()
|
||||
if cur == nil {
|
||||
// no data for series key + field combination
|
||||
continue
|
||||
}
|
||||
|
||||
key := groupKeyForSeries(rs.Tags(), &bi.readSpec, bi.bounds)
|
||||
done := make(chan struct{})
|
||||
switch typedCur := cur.(type) {
|
||||
case cursors.IntegerArrayCursor:
|
||||
cols, defs := determineTableColsForSeries(rs.Tags(), flux.TInt)
|
||||
table = newIntegerTable(done, typedCur, bi.bounds, key, cols, rs.Tags(), defs, bi.alloc)
|
||||
case cursors.FloatArrayCursor:
|
||||
cols, defs := determineTableColsForSeries(rs.Tags(), flux.TFloat)
|
||||
table = newFloatTable(done, typedCur, bi.bounds, key, cols, rs.Tags(), defs, bi.alloc)
|
||||
case cursors.UnsignedArrayCursor:
|
||||
cols, defs := determineTableColsForSeries(rs.Tags(), flux.TUInt)
|
||||
table = newUnsignedTable(done, typedCur, bi.bounds, key, cols, rs.Tags(), defs, bi.alloc)
|
||||
case cursors.BooleanArrayCursor:
|
||||
cols, defs := determineTableColsForSeries(rs.Tags(), flux.TBool)
|
||||
table = newBooleanTable(done, typedCur, bi.bounds, key, cols, rs.Tags(), defs, bi.alloc)
|
||||
case cursors.StringArrayCursor:
|
||||
cols, defs := determineTableColsForSeries(rs.Tags(), flux.TString)
|
||||
table = newStringTable(done, typedCur, bi.bounds, key, cols, rs.Tags(), defs, bi.alloc)
|
||||
default:
|
||||
panic(fmt.Sprintf("unreachable: %T", typedCur))
|
||||
}
|
||||
|
||||
cur = nil
|
||||
|
||||
if !table.Empty() {
|
||||
if err := f(table); err != nil {
|
||||
table.Close()
|
||||
table = nil
|
||||
return err
|
||||
}
|
||||
select {
|
||||
case <-done:
|
||||
case <-bi.ctx.Done():
|
||||
table.Cancel()
|
||||
break READ
|
||||
}
|
||||
}
|
||||
|
||||
stats := table.Statistics()
|
||||
bi.stats.ScannedValues += stats.ScannedValues
|
||||
bi.stats.ScannedBytes += stats.ScannedBytes
|
||||
table.Close()
|
||||
table = nil
|
||||
}
|
||||
return rs.Err()
|
||||
}
|
||||
|
||||
func (bi *tableIterator) handleReadNoPoints(f func(flux.Table) error, rs ResultSet) error {
|
||||
// these resources must be closed if not nil on return
|
||||
var table storageTable
|
||||
|
||||
defer func() {
|
||||
if table != nil {
|
||||
table.Close()
|
||||
}
|
||||
rs.Close()
|
||||
}()
|
||||
|
||||
READ:
|
||||
for rs.Next() {
|
||||
cur := rs.Cursor()
|
||||
if !hasPoints(cur) {
|
||||
// no data for series key + field combination
|
||||
continue
|
||||
}
|
||||
|
||||
key := groupKeyForSeries(rs.Tags(), &bi.readSpec, bi.bounds)
|
||||
done := make(chan struct{})
|
||||
cols, defs := determineTableColsForSeries(rs.Tags(), flux.TString)
|
||||
table = newTableNoPoints(done, bi.bounds, key, cols, rs.Tags(), defs, bi.alloc)
|
||||
|
||||
if err := f(table); err != nil {
|
||||
table.Close()
|
||||
table = nil
|
||||
return err
|
||||
}
|
||||
select {
|
||||
case <-done:
|
||||
case <-bi.ctx.Done():
|
||||
table.Cancel()
|
||||
break READ
|
||||
}
|
||||
|
||||
table.Close()
|
||||
table = nil
|
||||
}
|
||||
return rs.Err()
|
||||
}
|
||||
|
||||
func (bi *tableIterator) handleGroupRead(f func(flux.Table) error, rs GroupResultSet) error {
|
||||
func (gi *groupIterator) handleRead(f func(flux.Table) error, rs GroupResultSet) error {
|
||||
// these resources must be closed if not nil on return
|
||||
var (
|
||||
gc GroupCursor
|
||||
|
@ -454,24 +311,25 @@ READ:
|
|||
continue
|
||||
}
|
||||
|
||||
key := groupKeyForGroup(gc.PartitionKeyVals(), &bi.readSpec, bi.bounds)
|
||||
bnds := gi.spec.Bounds
|
||||
key := groupKeyForGroup(gc.PartitionKeyVals(), &gi.spec, bnds)
|
||||
done := make(chan struct{})
|
||||
switch typedCur := cur.(type) {
|
||||
case cursors.IntegerArrayCursor:
|
||||
cols, defs := determineTableColsForGroup(gc.Keys(), flux.TInt)
|
||||
table = newIntegerGroupTable(done, gc, typedCur, bi.bounds, key, cols, gc.Tags(), defs, bi.alloc)
|
||||
table = newIntegerGroupTable(done, gc, typedCur, bnds, key, cols, gc.Tags(), defs, gi.alloc)
|
||||
case cursors.FloatArrayCursor:
|
||||
cols, defs := determineTableColsForGroup(gc.Keys(), flux.TFloat)
|
||||
table = newFloatGroupTable(done, gc, typedCur, bi.bounds, key, cols, gc.Tags(), defs, bi.alloc)
|
||||
table = newFloatGroupTable(done, gc, typedCur, bnds, key, cols, gc.Tags(), defs, gi.alloc)
|
||||
case cursors.UnsignedArrayCursor:
|
||||
cols, defs := determineTableColsForGroup(gc.Keys(), flux.TUInt)
|
||||
table = newUnsignedGroupTable(done, gc, typedCur, bi.bounds, key, cols, gc.Tags(), defs, bi.alloc)
|
||||
table = newUnsignedGroupTable(done, gc, typedCur, bnds, key, cols, gc.Tags(), defs, gi.alloc)
|
||||
case cursors.BooleanArrayCursor:
|
||||
cols, defs := determineTableColsForGroup(gc.Keys(), flux.TBool)
|
||||
table = newBooleanGroupTable(done, gc, typedCur, bi.bounds, key, cols, gc.Tags(), defs, bi.alloc)
|
||||
table = newBooleanGroupTable(done, gc, typedCur, bnds, key, cols, gc.Tags(), defs, gi.alloc)
|
||||
case cursors.StringArrayCursor:
|
||||
cols, defs := determineTableColsForGroup(gc.Keys(), flux.TString)
|
||||
table = newStringGroupTable(done, gc, typedCur, bi.bounds, key, cols, gc.Tags(), defs, bi.alloc)
|
||||
table = newStringGroupTable(done, gc, typedCur, bnds, key, cols, gc.Tags(), defs, gi.alloc)
|
||||
default:
|
||||
panic(fmt.Sprintf("unreachable: %T", typedCur))
|
||||
}
|
||||
|
@ -487,61 +345,14 @@ READ:
|
|||
}
|
||||
select {
|
||||
case <-done:
|
||||
case <-bi.ctx.Done():
|
||||
case <-gi.ctx.Done():
|
||||
table.Cancel()
|
||||
break READ
|
||||
}
|
||||
|
||||
stats := table.Statistics()
|
||||
bi.stats.ScannedValues += stats.ScannedValues
|
||||
bi.stats.ScannedBytes += stats.ScannedBytes
|
||||
table.Close()
|
||||
table = nil
|
||||
|
||||
gc = rs.Next()
|
||||
}
|
||||
return rs.Err()
|
||||
}
|
||||
|
||||
func (bi *tableIterator) handleGroupReadNoPoints(f func(flux.Table) error, rs GroupResultSet) error {
|
||||
// these resources must be closed if not nil on return
|
||||
var (
|
||||
gc GroupCursor
|
||||
table storageTable
|
||||
)
|
||||
|
||||
defer func() {
|
||||
if table != nil {
|
||||
table.Close()
|
||||
}
|
||||
if gc != nil {
|
||||
gc.Close()
|
||||
}
|
||||
rs.Close()
|
||||
}()
|
||||
|
||||
gc = rs.Next()
|
||||
READ:
|
||||
for gc != nil {
|
||||
key := groupKeyForGroup(gc.PartitionKeyVals(), &bi.readSpec, bi.bounds)
|
||||
done := make(chan struct{})
|
||||
cols, defs := determineTableColsForGroup(gc.Keys(), flux.TString)
|
||||
table = newGroupTableNoPoints(done, bi.bounds, key, cols, defs, bi.alloc)
|
||||
gc.Close()
|
||||
gc = nil
|
||||
|
||||
if err := f(table); err != nil {
|
||||
table.Close()
|
||||
table = nil
|
||||
return err
|
||||
}
|
||||
select {
|
||||
case <-done:
|
||||
case <-bi.ctx.Done():
|
||||
table.Cancel()
|
||||
break READ
|
||||
}
|
||||
|
||||
gi.stats.ScannedValues += stats.ScannedValues
|
||||
gi.stats.ScannedBytes += stats.ScannedBytes
|
||||
table.Close()
|
||||
table = nil
|
||||
|
||||
|
@ -561,7 +372,7 @@ func determineAggregateMethod(agg string) (datatypes.Aggregate_AggregateType, er
|
|||
return 0, fmt.Errorf("unknown aggregate type %q", agg)
|
||||
}
|
||||
|
||||
func convertGroupMode(m influxdb.GroupMode) datatypes.ReadRequest_Group {
|
||||
func convertGroupMode(m influxdb.GroupMode) datatypes.ReadGroupRequest_Group {
|
||||
switch m {
|
||||
case influxdb.GroupModeNone:
|
||||
return datatypes.GroupNone
|
||||
|
@ -636,49 +447,6 @@ func defaultGroupKeyForSeries(tags models.Tags, bnds execute.Bounds) flux.GroupK
|
|||
return execute.NewGroupKey(cols, vs)
|
||||
}
|
||||
|
||||
func groupKeyForSeries(tags models.Tags, readSpec *influxdb.ReadSpec, bnds execute.Bounds) flux.GroupKey {
|
||||
cols := make([]flux.ColMeta, 2, len(tags))
|
||||
vs := make([]values.Value, 2, len(tags))
|
||||
cols[0] = flux.ColMeta{
|
||||
Label: execute.DefaultStartColLabel,
|
||||
Type: flux.TTime,
|
||||
}
|
||||
vs[0] = values.NewTime(bnds.Start)
|
||||
cols[1] = flux.ColMeta{
|
||||
Label: execute.DefaultStopColLabel,
|
||||
Type: flux.TTime,
|
||||
}
|
||||
vs[1] = values.NewTime(bnds.Stop)
|
||||
switch readSpec.GroupMode {
|
||||
case influxdb.GroupModeBy:
|
||||
// group key in GroupKeys order, including tags in the GroupKeys slice
|
||||
for _, k := range readSpec.GroupKeys {
|
||||
bk := []byte(k)
|
||||
for _, t := range tags {
|
||||
if bytes.Equal(t.Key, bk) && len(t.Value) > 0 {
|
||||
cols = append(cols, flux.ColMeta{
|
||||
Label: k,
|
||||
Type: flux.TString,
|
||||
})
|
||||
vs = append(vs, values.NewString(string(t.Value)))
|
||||
}
|
||||
}
|
||||
}
|
||||
case influxdb.GroupModeExcept:
|
||||
// group key in GroupKeys order, skipping tags in the GroupKeys slice
|
||||
panic("not implemented")
|
||||
case influxdb.GroupModeDefault, influxdb.GroupModeAll:
|
||||
for i := range tags {
|
||||
cols = append(cols, flux.ColMeta{
|
||||
Label: string(tags[i].Key),
|
||||
Type: flux.TString,
|
||||
})
|
||||
vs = append(vs, values.NewString(string(tags[i].Value)))
|
||||
}
|
||||
}
|
||||
return execute.NewGroupKey(cols, vs)
|
||||
}
|
||||
|
||||
func determineTableColsForGroup(tagKeys [][]byte, typ flux.ColType) ([]flux.ColMeta, [][]byte) {
|
||||
cols := make([]flux.ColMeta, 4+len(tagKeys))
|
||||
defs := make([][]byte, 4+len(tagKeys))
|
||||
|
@ -709,9 +477,9 @@ func determineTableColsForGroup(tagKeys [][]byte, typ flux.ColType) ([]flux.ColM
|
|||
return cols, defs
|
||||
}
|
||||
|
||||
func groupKeyForGroup(kv [][]byte, readSpec *influxdb.ReadSpec, bnds execute.Bounds) flux.GroupKey {
|
||||
cols := make([]flux.ColMeta, 2, len(readSpec.GroupKeys)+2)
|
||||
vs := make([]values.Value, 2, len(readSpec.GroupKeys)+2)
|
||||
func groupKeyForGroup(kv [][]byte, spec *influxdb.ReadGroupSpec, bnds execute.Bounds) flux.GroupKey {
|
||||
cols := make([]flux.ColMeta, 2, len(spec.GroupKeys)+2)
|
||||
vs := make([]values.Value, 2, len(spec.GroupKeys)+2)
|
||||
cols[0] = flux.ColMeta{
|
||||
Label: execute.DefaultStartColLabel,
|
||||
Type: flux.TTime,
|
||||
|
@ -722,12 +490,12 @@ func groupKeyForGroup(kv [][]byte, readSpec *influxdb.ReadSpec, bnds execute.Bou
|
|||
Type: flux.TTime,
|
||||
}
|
||||
vs[1] = values.NewTime(bnds.Stop)
|
||||
for i := range readSpec.GroupKeys {
|
||||
if readSpec.GroupKeys[i] == execute.DefaultStartColLabel || readSpec.GroupKeys[i] == execute.DefaultStopColLabel {
|
||||
for i := range spec.GroupKeys {
|
||||
if spec.GroupKeys[i] == execute.DefaultStartColLabel || spec.GroupKeys[i] == execute.DefaultStopColLabel {
|
||||
continue
|
||||
}
|
||||
cols = append(cols, flux.ColMeta{
|
||||
Label: readSpec.GroupKeys[i],
|
||||
Label: spec.GroupKeys[i],
|
||||
Type: flux.TString,
|
||||
})
|
||||
vs = append(vs, values.NewString(string(kv[i])))
|
||||
|
|
|
@ -22,16 +22,7 @@ type resultSet struct {
|
|||
mb multiShardCursors
|
||||
}
|
||||
|
||||
func NewResultSet(ctx context.Context, req *datatypes.ReadRequest, cur SeriesCursor) ResultSet {
|
||||
return &resultSet{
|
||||
ctx: ctx,
|
||||
agg: req.Aggregate,
|
||||
cur: cur,
|
||||
mb: newMultiShardArrayCursors(ctx, req.TimestampRange.Start, req.TimestampRange.End, !req.Descending, req.PointsLimit),
|
||||
}
|
||||
}
|
||||
|
||||
func NewResultSetFromFilter(ctx context.Context, req *datatypes.ReadFilterRequest, cur SeriesCursor) ResultSet {
|
||||
func NewFilteredResultSet(ctx context.Context, req *datatypes.ReadFilterRequest, cur SeriesCursor) ResultSet {
|
||||
return &resultSet{
|
||||
ctx: ctx,
|
||||
cur: cur,
|
||||
|
|
|
@ -74,10 +74,14 @@ type GroupCursor interface {
|
|||
}
|
||||
|
||||
type Store interface {
|
||||
Read(ctx context.Context, req *datatypes.ReadRequest) (ResultSet, error)
|
||||
ReadFilter(ctx context.Context, req *datatypes.ReadFilterRequest) (ResultSet, error)
|
||||
GroupRead(ctx context.Context, req *datatypes.ReadRequest) (GroupResultSet, error)
|
||||
GetSource(orgID, bucketID uint64) proto.Message
|
||||
ReadGroup(ctx context.Context, req *datatypes.ReadGroupRequest) (GroupResultSet, error)
|
||||
|
||||
TagKeys(ctx context.Context, req *datatypes.TagKeysRequest) (cursors.StringIterator, error)
|
||||
TagValues(ctx context.Context, req *datatypes.TagValuesRequest) (cursors.StringIterator, error)
|
||||
|
||||
// Deprecated method; should use ReadFilter and ReadGroup instead.
|
||||
Read(ctx context.Context, req *datatypes.ReadRequest) (ResultSet, error)
|
||||
|
||||
GetSource(orgID, bucketID uint64) proto.Message
|
||||
}
|
||||
|
|
|
@ -3,7 +3,6 @@ package readservice
|
|||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"math"
|
||||
|
||||
"github.com/gogo/protobuf/proto"
|
||||
"github.com/gogo/protobuf/types"
|
||||
|
@ -44,22 +43,10 @@ func (s *store) ReadFilter(ctx context.Context, req *datatypes.ReadFilterRequest
|
|||
cur = ic
|
||||
}
|
||||
|
||||
return reads.NewResultSetFromFilter(ctx, req, cur), nil
|
||||
return reads.NewFilteredResultSet(ctx, req, cur), nil
|
||||
}
|
||||
|
||||
func (s *store) Read(ctx context.Context, req *datatypes.ReadRequest) (reads.ResultSet, error) {
|
||||
if len(req.GroupKeys) > 0 {
|
||||
panic("Read: len(Grouping) > 0")
|
||||
}
|
||||
|
||||
if req.Hints.NoPoints() {
|
||||
req.PointsLimit = -1
|
||||
}
|
||||
|
||||
if req.PointsLimit == 0 {
|
||||
req.PointsLimit = math.MaxInt64
|
||||
}
|
||||
|
||||
func (s *store) ReadGroup(ctx context.Context, req *datatypes.ReadGroupRequest) (reads.GroupResultSet, error) {
|
||||
if req.ReadSource == nil {
|
||||
return nil, errors.New("missing read source")
|
||||
}
|
||||
|
@ -69,60 +56,6 @@ func (s *store) Read(ctx context.Context, req *datatypes.ReadRequest) (reads.Res
|
|||
return nil, err
|
||||
}
|
||||
|
||||
if req.TimestampRange.Start == 0 {
|
||||
req.TimestampRange.Start = math.MaxInt64
|
||||
}
|
||||
|
||||
if req.TimestampRange.End == 0 {
|
||||
req.TimestampRange.End = math.MaxInt64
|
||||
}
|
||||
|
||||
var cur reads.SeriesCursor
|
||||
if ic, err := newIndexSeriesCursor(ctx, &source, req.Predicate, s.engine); err != nil {
|
||||
return nil, err
|
||||
} else if ic == nil {
|
||||
return nil, nil
|
||||
} else {
|
||||
cur = ic
|
||||
}
|
||||
|
||||
if req.SeriesLimit > 0 || req.SeriesOffset > 0 {
|
||||
cur = reads.NewLimitSeriesCursor(ctx, cur, req.SeriesLimit, req.SeriesOffset)
|
||||
}
|
||||
|
||||
return reads.NewResultSet(ctx, req, cur), nil
|
||||
}
|
||||
|
||||
func (s *store) GroupRead(ctx context.Context, req *datatypes.ReadRequest) (reads.GroupResultSet, error) {
|
||||
if req.SeriesLimit > 0 || req.SeriesOffset > 0 {
|
||||
return nil, errors.New("groupRead: SeriesLimit and SeriesOffset not supported when Grouping")
|
||||
}
|
||||
|
||||
if req.Hints.NoPoints() {
|
||||
req.PointsLimit = -1
|
||||
}
|
||||
|
||||
if req.PointsLimit == 0 {
|
||||
req.PointsLimit = math.MaxInt64
|
||||
}
|
||||
|
||||
if req.ReadSource == nil {
|
||||
return nil, errors.New("missing read source")
|
||||
}
|
||||
|
||||
source, err := getReadSource(*req.ReadSource)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
if req.TimestampRange.Start <= 0 {
|
||||
req.TimestampRange.Start = math.MinInt64
|
||||
}
|
||||
|
||||
if req.TimestampRange.End <= 0 {
|
||||
req.TimestampRange.End = math.MaxInt64
|
||||
}
|
||||
|
||||
newCursor := func() (reads.SeriesCursor, error) {
|
||||
cur, err := newIndexSeriesCursor(ctx, &source, req.Predicate, s.engine)
|
||||
if cur == nil || err != nil {
|
||||
|
@ -134,6 +67,10 @@ func (s *store) GroupRead(ctx context.Context, req *datatypes.ReadRequest) (read
|
|||
return reads.NewGroupResultSet(ctx, req, newCursor), nil
|
||||
}
|
||||
|
||||
func (s *store) Read(ctx context.Context, req *datatypes.ReadRequest) (reads.ResultSet, error) {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
func (s *store) TagKeys(ctx context.Context, req *datatypes.TagKeysRequest) (cursors.StringIterator, error) {
|
||||
span, _ := tracing.StartSpanFromContext(ctx)
|
||||
defer span.Finish()
|
||||
|
|
Loading…
Reference in New Issue