feat(storage): ReadGroup RPC definitions and storage reader

pull/13735/head
jlapacik 2019-04-24 16:18:33 -07:00
parent 5f87134ca8
commit 95aa194498
9 changed files with 716 additions and 1132 deletions

View File

@ -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

View File

@ -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 {}

View File

@ -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)

View File

@ -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()
}
}

View File

@ -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])))

View File

@ -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,

View File

@ -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
}

View File

@ -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()