refactor(services/storage): Enhanced group support for Read RPC

pull/9921/head
Stuart Carnie 2018-05-23 15:26:07 -07:00 committed by Stuart Carnie
parent 0253f6fe05
commit d42abde836
23 changed files with 2075 additions and 700 deletions

View File

@ -31,20 +31,23 @@ type Command struct {
addr string
cpuProfile string
memProfile string
orgID string
database string
retentionPolicy string
startTime int64
endTime int64
limit uint64
slimit uint64
soffset uint64
limit int64
slimit int64
soffset int64
desc bool
silent bool
expr string
agg string
grouping string
groupArg string
group storage.ReadRequest_Group
groupKeys string
keys []string
hintsArg string
hints storage.HintFlags
aggType storage.Aggregate_AggregateType
@ -82,19 +85,20 @@ func (cmd *Command) Run(args ...string) error {
fs.StringVar(&cmd.cpuProfile, "cpuprofile", "", "CPU profile name")
fs.StringVar(&cmd.memProfile, "memprofile", "", "memory profile name")
fs.StringVar(&cmd.addr, "addr", ":8082", "the RPC address")
fs.StringVar(&cmd.orgID, "org-id", "", "Optional: org identifier when querying multi-tenant store")
fs.StringVar(&cmd.database, "database", "", "the database to query")
fs.StringVar(&cmd.retentionPolicy, "retention", "", "Optional: the retention policy to query")
fs.StringVar(&start, "start", "", "Optional: the start time to query (RFC3339 format)")
fs.StringVar(&end, "end", "", "Optional: the end time to query (RFC3339 format)")
fs.Uint64Var(&cmd.slimit, "slimit", 0, "Optional: limit number of series")
fs.Uint64Var(&cmd.soffset, "soffset", 0, "Optional: start offset for series")
fs.Uint64Var(&cmd.limit, "limit", 0, "Optional: limit number of values per series")
fs.Int64Var(&cmd.slimit, "slimit", 0, "Optional: limit number of series")
fs.Int64Var(&cmd.soffset, "soffset", 0, "Optional: start offset for series")
fs.Int64Var(&cmd.limit, "limit", 0, "Optional: limit number of values per series (-1 to return series only)")
fs.BoolVar(&cmd.desc, "desc", false, "Optional: return results in descending order")
fs.BoolVar(&cmd.silent, "silent", false, "silence output")
fs.StringVar(&cmd.expr, "expr", "", "InfluxQL conditional expression")
fs.StringVar(&cmd.agg, "agg", "", "aggregate functions (sum, count)")
fs.StringVar(&cmd.grouping, "grouping", "", "comma-separated list of tags to specify series order")
fs.StringVar(&cmd.groupArg, "group", "none", "group operation (none,all,by,except,disable)")
fs.StringVar(&cmd.groupKeys, "group-keys", "", "comma-separated list of tags to specify series order")
fs.StringVar(&cmd.hintsArg, "hints", "none", "comma-separated list of read hints (none,no_points,no_series)")
fs.SetOutput(cmd.Stdout)
fs.Usage = func() {
@ -130,17 +134,8 @@ func (cmd *Command) Run(args ...string) error {
cmd.endTime = models.MaxNanoTime
}
if cmd.agg != "" {
tm := proto.EnumValueMap("storage.Aggregate_AggregateType")
agg, ok := tm[strings.ToUpper(cmd.agg)]
if !ok {
return errors.New("invalid aggregate function: " + cmd.agg)
}
cmd.aggType = storage.Aggregate_AggregateType(agg)
}
if cmd.grouping != "" {
cmd.keys = strings.Split(cmd.grouping, ",")
if cmd.groupKeys != "" {
cmd.keys = strings.Split(cmd.groupKeys, ",")
}
if err := cmd.validate(); err != nil {
@ -157,25 +152,41 @@ func (cmd *Command) Run(args ...string) error {
}
func (cmd *Command) validate() error {
if cmd.orgID != "" && cmd.retentionPolicy != "" {
return fmt.Errorf("omit retention policy for multi-tenant request")
}
if cmd.database == "" {
return fmt.Errorf("must specify a database")
}
if cmd.startTime != 0 && cmd.endTime != 0 && cmd.endTime < cmd.startTime {
return fmt.Errorf("end time before start time")
}
if cmd.agg != "" {
tm := proto.EnumValueMap("com.github.influxdata.influxdb.services.storage.Aggregate_AggregateType")
agg, ok := tm[strings.ToUpper(cmd.agg)]
if !ok {
return errors.New("invalid aggregate function: " + cmd.agg)
}
cmd.aggType = storage.Aggregate_AggregateType(agg)
}
enums := proto.EnumValueMap("com.github.influxdata.influxdb.services.storage.ReadRequest_Group")
group, ok := enums["GROUP_"+strings.ToUpper(cmd.groupArg)]
if !ok {
return errors.New("invalid group type: " + cmd.groupArg)
}
cmd.group = storage.ReadRequest_Group(group)
enums = proto.EnumValueMap("com.github.influxdata.influxdb.services.storage.ReadRequest_HintFlags")
for _, h := range strings.Split(cmd.hintsArg, ",") {
cmd.hints |= storage.HintFlags(enums["HINT_"+strings.ToUpper(h)])
}
return nil
}
func (cmd *Command) query(c storage.StorageClient) error {
var req storage.ReadRequest
req.Database = cmd.database
if cmd.orgID != "" {
req.RequestType = storage.ReadRequestTypeMultiTenant
req.OrgID = cmd.orgID
} else if cmd.retentionPolicy != "" {
if cmd.retentionPolicy != "" {
req.Database += "/" + cmd.retentionPolicy
}
@ -185,7 +196,9 @@ func (cmd *Command) query(c storage.StorageClient) error {
req.SeriesOffset = cmd.soffset
req.PointsLimit = cmd.limit
req.Descending = cmd.desc
req.Grouping = cmd.keys
req.Group = cmd.group
req.GroupKeys = cmd.keys
req.Hints = cmd.hints
if cmd.aggType != storage.AggregateTypeNone {
req.Aggregate = &storage.Aggregate{Type: cmd.aggType}
@ -274,12 +287,34 @@ func (cmd *Command) processFramesSilent(frames []storage.ReadResponse_Frame) {
}
}
func printByteSlice(wr *bufio.Writer, v [][]byte) {
wr.WriteString("[\033[36m")
first := true
for _, t := range v {
if !first {
wr.WriteByte(',')
} else {
first = false
}
wr.Write(t)
}
wr.WriteString("\033[0m]\n")
}
func (cmd *Command) processFrames(wr *bufio.Writer, frames []storage.ReadResponse_Frame) {
var buf [1024]byte
var line []byte
for _, frame := range frames {
switch f := frame.Data.(type) {
case *storage.ReadResponse_Frame_Group:
g := f.Group
wr.WriteString("partition values")
printByteSlice(wr, g.PartitionKeyVals)
wr.WriteString("group keys")
printByteSlice(wr, g.TagKeys)
wr.Flush()
case *storage.ReadResponse_Frame_Series:
s := f.Series
wr.WriteString("\033[36m")

24
logger/context.go Normal file
View File

@ -0,0 +1,24 @@
package logger
import (
"context"
"go.uber.org/zap"
)
type key int
const (
loggerKey key = iota
)
// NewContextWithLogger returns a new context with log added.
func NewContextWithLogger(ctx context.Context, log *zap.Logger) context.Context {
return context.WithValue(ctx, loggerKey, log)
}
// LoggerFromContext returns the zap.Logger associated with ctx or nil if no logger has been assigned.
func LoggerFromContext(ctx context.Context) *zap.Logger {
l, _ := ctx.Value(loggerKey).(*zap.Logger)
return l
}

View File

@ -7,7 +7,6 @@
package storage
import (
"context"
"errors"
"github.com/influxdata/influxdb/tsdb"
@ -72,33 +71,23 @@ func (c *floatFilterBatchCursor) Next() (key []int64, value []float64) {
type floatMultiShardBatchCursor struct {
tsdb.FloatBatchCursor
ctx context.Context
cursorContext
filter *floatFilterBatchCursor
itrs tsdb.CursorIterators
req *tsdb.CursorRequest
err error
limit uint64
count uint64
}
func newFloatMultiShardBatchCursor(ctx context.Context, cur tsdb.FloatBatchCursor, rr *readRequest, req *tsdb.CursorRequest, itrs tsdb.CursorIterators, cond expression) *floatMultiShardBatchCursor {
var filter *floatFilterBatchCursor
func (c *floatMultiShardBatchCursor) reset(cur tsdb.FloatBatchCursor, itrs tsdb.CursorIterators, cond expression) {
if cond != nil {
filter = newFloatFilterBatchCursor(cond)
filter.reset(cur)
cur = filter
if c.filter == nil {
c.filter = newFloatFilterBatchCursor(cond)
}
c.filter.reset(cur)
cur = c.filter
}
c := &floatMultiShardBatchCursor{
FloatBatchCursor: cur,
ctx: ctx,
filter: filter,
req: req,
itrs: itrs,
limit: rr.limit,
}
return c
c.FloatBatchCursor = cur
c.itrs = itrs
c.err = nil
c.count = 0
}
func (c *floatMultiShardBatchCursor) Err() error { return c.err }
@ -111,11 +100,11 @@ func (c *floatMultiShardBatchCursor) Next() (key []int64, value []float64) {
continue
}
}
c.count += uint64(len(ks))
c.count += int64(len(ks))
if c.count > c.limit {
diff := c.count - c.limit
c.count -= diff
rem := uint64(len(ks)) - diff
rem := int64(len(ks)) - diff
ks = ks[:rem]
vs = vs[:rem]
}
@ -276,33 +265,23 @@ func (c *integerFilterBatchCursor) Next() (key []int64, value []int64) {
type integerMultiShardBatchCursor struct {
tsdb.IntegerBatchCursor
ctx context.Context
cursorContext
filter *integerFilterBatchCursor
itrs tsdb.CursorIterators
req *tsdb.CursorRequest
err error
limit uint64
count uint64
}
func newIntegerMultiShardBatchCursor(ctx context.Context, cur tsdb.IntegerBatchCursor, rr *readRequest, req *tsdb.CursorRequest, itrs tsdb.CursorIterators, cond expression) *integerMultiShardBatchCursor {
var filter *integerFilterBatchCursor
func (c *integerMultiShardBatchCursor) reset(cur tsdb.IntegerBatchCursor, itrs tsdb.CursorIterators, cond expression) {
if cond != nil {
filter = newIntegerFilterBatchCursor(cond)
filter.reset(cur)
cur = filter
if c.filter == nil {
c.filter = newIntegerFilterBatchCursor(cond)
}
c.filter.reset(cur)
cur = c.filter
}
c := &integerMultiShardBatchCursor{
IntegerBatchCursor: cur,
ctx: ctx,
filter: filter,
req: req,
itrs: itrs,
limit: rr.limit,
}
return c
c.IntegerBatchCursor = cur
c.itrs = itrs
c.err = nil
c.count = 0
}
func (c *integerMultiShardBatchCursor) Err() error { return c.err }
@ -315,11 +294,11 @@ func (c *integerMultiShardBatchCursor) Next() (key []int64, value []int64) {
continue
}
}
c.count += uint64(len(ks))
c.count += int64(len(ks))
if c.count > c.limit {
diff := c.count - c.limit
c.count -= diff
rem := uint64(len(ks)) - diff
rem := int64(len(ks)) - diff
ks = ks[:rem]
vs = vs[:rem]
}
@ -480,33 +459,23 @@ func (c *unsignedFilterBatchCursor) Next() (key []int64, value []uint64) {
type unsignedMultiShardBatchCursor struct {
tsdb.UnsignedBatchCursor
ctx context.Context
cursorContext
filter *unsignedFilterBatchCursor
itrs tsdb.CursorIterators
req *tsdb.CursorRequest
err error
limit uint64
count uint64
}
func newUnsignedMultiShardBatchCursor(ctx context.Context, cur tsdb.UnsignedBatchCursor, rr *readRequest, req *tsdb.CursorRequest, itrs tsdb.CursorIterators, cond expression) *unsignedMultiShardBatchCursor {
var filter *unsignedFilterBatchCursor
func (c *unsignedMultiShardBatchCursor) reset(cur tsdb.UnsignedBatchCursor, itrs tsdb.CursorIterators, cond expression) {
if cond != nil {
filter = newUnsignedFilterBatchCursor(cond)
filter.reset(cur)
cur = filter
if c.filter == nil {
c.filter = newUnsignedFilterBatchCursor(cond)
}
c.filter.reset(cur)
cur = c.filter
}
c := &unsignedMultiShardBatchCursor{
UnsignedBatchCursor: cur,
ctx: ctx,
filter: filter,
req: req,
itrs: itrs,
limit: rr.limit,
}
return c
c.UnsignedBatchCursor = cur
c.itrs = itrs
c.err = nil
c.count = 0
}
func (c *unsignedMultiShardBatchCursor) Err() error { return c.err }
@ -519,11 +488,11 @@ func (c *unsignedMultiShardBatchCursor) Next() (key []int64, value []uint64) {
continue
}
}
c.count += uint64(len(ks))
c.count += int64(len(ks))
if c.count > c.limit {
diff := c.count - c.limit
c.count -= diff
rem := uint64(len(ks)) - diff
rem := int64(len(ks)) - diff
ks = ks[:rem]
vs = vs[:rem]
}
@ -684,33 +653,23 @@ func (c *stringFilterBatchCursor) Next() (key []int64, value []string) {
type stringMultiShardBatchCursor struct {
tsdb.StringBatchCursor
ctx context.Context
cursorContext
filter *stringFilterBatchCursor
itrs tsdb.CursorIterators
req *tsdb.CursorRequest
err error
limit uint64
count uint64
}
func newStringMultiShardBatchCursor(ctx context.Context, cur tsdb.StringBatchCursor, rr *readRequest, req *tsdb.CursorRequest, itrs tsdb.CursorIterators, cond expression) *stringMultiShardBatchCursor {
var filter *stringFilterBatchCursor
func (c *stringMultiShardBatchCursor) reset(cur tsdb.StringBatchCursor, itrs tsdb.CursorIterators, cond expression) {
if cond != nil {
filter = newStringFilterBatchCursor(cond)
filter.reset(cur)
cur = filter
if c.filter == nil {
c.filter = newStringFilterBatchCursor(cond)
}
c.filter.reset(cur)
cur = c.filter
}
c := &stringMultiShardBatchCursor{
StringBatchCursor: cur,
ctx: ctx,
filter: filter,
req: req,
itrs: itrs,
limit: rr.limit,
}
return c
c.StringBatchCursor = cur
c.itrs = itrs
c.err = nil
c.count = 0
}
func (c *stringMultiShardBatchCursor) Err() error { return c.err }
@ -723,11 +682,11 @@ func (c *stringMultiShardBatchCursor) Next() (key []int64, value []string) {
continue
}
}
c.count += uint64(len(ks))
c.count += int64(len(ks))
if c.count > c.limit {
diff := c.count - c.limit
c.count -= diff
rem := uint64(len(ks)) - diff
rem := int64(len(ks)) - diff
ks = ks[:rem]
vs = vs[:rem]
}
@ -860,33 +819,23 @@ func (c *booleanFilterBatchCursor) Next() (key []int64, value []bool) {
type booleanMultiShardBatchCursor struct {
tsdb.BooleanBatchCursor
ctx context.Context
cursorContext
filter *booleanFilterBatchCursor
itrs tsdb.CursorIterators
req *tsdb.CursorRequest
err error
limit uint64
count uint64
}
func newBooleanMultiShardBatchCursor(ctx context.Context, cur tsdb.BooleanBatchCursor, rr *readRequest, req *tsdb.CursorRequest, itrs tsdb.CursorIterators, cond expression) *booleanMultiShardBatchCursor {
var filter *booleanFilterBatchCursor
func (c *booleanMultiShardBatchCursor) reset(cur tsdb.BooleanBatchCursor, itrs tsdb.CursorIterators, cond expression) {
if cond != nil {
filter = newBooleanFilterBatchCursor(cond)
filter.reset(cur)
cur = filter
if c.filter == nil {
c.filter = newBooleanFilterBatchCursor(cond)
}
c.filter.reset(cur)
cur = c.filter
}
c := &booleanMultiShardBatchCursor{
BooleanBatchCursor: cur,
ctx: ctx,
filter: filter,
req: req,
itrs: itrs,
limit: rr.limit,
}
return c
c.BooleanBatchCursor = cur
c.itrs = itrs
c.err = nil
c.count = 0
}
func (c *booleanMultiShardBatchCursor) Err() error { return c.err }
@ -899,11 +848,11 @@ func (c *booleanMultiShardBatchCursor) Next() (key []int64, value []bool) {
continue
}
}
c.count += uint64(len(ks))
c.count += int64(len(ks))
if c.count > c.limit {
diff := c.count - c.limit
c.count -= diff
rem := uint64(len(ks)) - diff
rem := int64(len(ks)) - diff
ks = ks[:rem]
vs = vs[:rem]
}

View File

@ -1,7 +1,6 @@
package storage
import (
"context"
"errors"
"github.com/influxdata/influxdb/tsdb"
@ -68,35 +67,26 @@ func (c *{{.name}}FilterBatchCursor) Next() (key []int64, value []{{.Type}}) {
type {{.name}}MultiShardBatchCursor struct {
tsdb.{{.Name}}BatchCursor
ctx context.Context
cursorContext
filter *{{.name}}FilterBatchCursor
itrs tsdb.CursorIterators
req *tsdb.CursorRequest
err error
limit uint64
count uint64
}
func new{{.Name}}MultiShardBatchCursor(ctx context.Context, cur tsdb.{{.Name}}BatchCursor, rr *readRequest, req *tsdb.CursorRequest, itrs tsdb.CursorIterators, cond expression) *{{.name}}MultiShardBatchCursor {
var filter *{{.name}}FilterBatchCursor
func (c *{{.name}}MultiShardBatchCursor) reset(cur tsdb.{{.Name}}BatchCursor, itrs tsdb.CursorIterators, cond expression) {
if cond != nil {
filter = new{{.Name}}FilterBatchCursor(cond)
filter.reset(cur)
cur = filter
if c.filter == nil {
c.filter = new{{.Name}}FilterBatchCursor(cond)
}
c.filter.reset(cur)
cur = c.filter
}
c := &{{.name}}MultiShardBatchCursor{
{{.Name}}BatchCursor: cur,
ctx: ctx,
filter: filter,
req: req,
itrs: itrs,
limit: rr.limit,
}
return c
c.{{.Name}}BatchCursor = cur
c.itrs = itrs
c.err = nil
c.count = 0
}
func (c *{{.name}}MultiShardBatchCursor) Err() error { return c.err }
func (c *{{.name}}MultiShardBatchCursor) Next() (key []int64, value []{{.Type}}) {
@ -107,11 +97,11 @@ func (c *{{.name}}MultiShardBatchCursor) Next() (key []int64, value []{{.Type}})
continue
}
}
c.count += uint64(len(ks))
c.count += int64(len(ks))
if c.count > c.limit {
diff := c.count - c.limit
c.count -= diff
rem := uint64(len(ks)) - diff
rem := int64(len(ks)) - diff
ks = ks[:rem]
vs = vs[:rem]
}

View File

@ -62,15 +62,64 @@ func newCountBatchCursor(cur tsdb.Cursor) tsdb.Cursor {
}
}
func newMultiShardBatchCursor(ctx context.Context, row seriesRow, rr *readRequest) tsdb.Cursor {
req := &tsdb.CursorRequest{
Name: row.name,
Tags: row.stags,
Field: row.field,
Ascending: rr.asc,
StartTime: rr.start,
EndTime: rr.end,
type cursorContext struct {
ctx context.Context
req *tsdb.CursorRequest
itrs tsdb.CursorIterators
limit int64
count int64
err error
}
type multiShardBatchCursors struct {
ctx context.Context
limit int64
req tsdb.CursorRequest
cursors struct {
i integerMultiShardBatchCursor
f floatMultiShardBatchCursor
u unsignedMultiShardBatchCursor
b booleanMultiShardBatchCursor
s stringMultiShardBatchCursor
}
}
func newMultiShardBatchCursors(ctx context.Context, rr *readRequest) *multiShardBatchCursors {
lim := rr.limit
if lim < 0 {
lim = 1
}
m := &multiShardBatchCursors{
ctx: ctx,
limit: lim,
req: tsdb.CursorRequest{
Ascending: rr.asc,
StartTime: rr.start,
EndTime: rr.end,
},
}
cc := cursorContext{
ctx: ctx,
limit: lim,
req: &m.req,
}
m.cursors.i.cursorContext = cc
m.cursors.f.cursorContext = cc
m.cursors.u.cursorContext = cc
m.cursors.b.cursorContext = cc
m.cursors.s.cursorContext = cc
return m
}
func (m *multiShardBatchCursors) createCursor(row seriesRow) tsdb.Cursor {
m.req.Name = row.name
m.req.Tags = row.stags
m.req.Field = row.field.n
var cond expression
if row.valueCond != nil {
@ -81,7 +130,7 @@ func newMultiShardBatchCursor(ctx context.Context, row seriesRow, rr *readReques
var cur tsdb.Cursor
for cur == nil && len(row.query) > 0 {
shard, row.query = row.query[0], row.query[1:]
cur, _ = shard.Next(ctx, req)
cur, _ = shard.Next(m.ctx, &m.req)
}
if cur == nil {
@ -90,15 +139,20 @@ func newMultiShardBatchCursor(ctx context.Context, row seriesRow, rr *readReques
switch c := cur.(type) {
case tsdb.IntegerBatchCursor:
return newIntegerMultiShardBatchCursor(ctx, c, rr, req, row.query, cond)
m.cursors.i.reset(c, row.query, cond)
return &m.cursors.i
case tsdb.FloatBatchCursor:
return newFloatMultiShardBatchCursor(ctx, c, rr, req, row.query, cond)
m.cursors.f.reset(c, row.query, cond)
return &m.cursors.f
case tsdb.UnsignedBatchCursor:
return newUnsignedMultiShardBatchCursor(ctx, c, rr, req, row.query, cond)
m.cursors.u.reset(c, row.query, cond)
return &m.cursors.u
case tsdb.StringBatchCursor:
return newStringMultiShardBatchCursor(ctx, c, rr, req, row.query, cond)
m.cursors.s.reset(c, row.query, cond)
return &m.cursors.s
case tsdb.BooleanBatchCursor:
return newBooleanMultiShardBatchCursor(ctx, c, rr, req, row.query, cond)
m.cursors.b.reset(c, row.query, cond)
return &m.cursors.b
default:
panic(fmt.Sprintf("unreachable: %T", cur))
}

View File

@ -247,6 +247,33 @@ func evalBinaryExpr(expr *influxql.BinaryExpr, m valuer) interface{} {
}
return !rhs.MatchString(lhs)
}
case []byte:
switch expr.Op {
case influxql.EQ:
rhs, ok := rhs.(string)
if !ok {
return nil
}
return string(lhs) == rhs
case influxql.NEQ:
rhs, ok := rhs.(string)
if !ok {
return nil
}
return string(lhs) != rhs
case influxql.EQREGEX:
rhs, ok := rhs.(*regexp.Regexp)
if !ok {
return nil
}
return rhs.Match(lhs)
case influxql.NEQREGEX:
rhs, ok := rhs.(*regexp.Regexp)
if !ok {
return nil
}
return !rhs.Match(lhs)
}
}
return nil
}

View File

@ -0,0 +1,436 @@
package storage
import (
"bytes"
"context"
"fmt"
"sort"
"strings"
"github.com/influxdata/influxdb/logger"
"github.com/influxdata/influxdb/models"
"github.com/influxdata/influxdb/tsdb"
"github.com/opentracing/opentracing-go"
"go.uber.org/zap"
)
type GroupCursor interface {
Tags() models.Tags
Keys() [][]byte
PartitionKeyVals() [][]byte
Next() bool
Cursor() tsdb.Cursor
Close()
}
type groupResultSet struct {
ctx context.Context
req *ReadRequest
rr readRequest
mb *multiShardBatchCursors
i int
rows []*seriesRow
keys [][]byte
rgc groupByCursor
km keyMerger
newCursorFn func() (seriesCursor, error)
nextGroupFn func(c *groupResultSet) GroupCursor
sortFn func(c *groupResultSet) (int, error)
eof bool
}
func newGroupResultSet(ctx context.Context, req *ReadRequest, newCursorFn func() (seriesCursor, error)) *groupResultSet {
g := &groupResultSet{
ctx: ctx,
req: req,
rr: readRequest{
ctx: ctx,
start: req.TimestampRange.Start,
end: req.TimestampRange.End,
asc: !req.Descending,
limit: req.PointsLimit,
aggregate: req.Aggregate,
},
keys: make([][]byte, len(req.GroupKeys)),
newCursorFn: newCursorFn,
}
g.mb = newMultiShardBatchCursors(ctx, &g.rr)
for i, k := range req.GroupKeys {
g.keys[i] = []byte(k)
}
switch req.Group {
case GroupBy:
g.sortFn = groupBySort
g.nextGroupFn = groupByNextGroup
g.rgc = groupByCursor{
mb: g.mb,
req: &g.rr,
vals: make([][]byte, len(req.GroupKeys)),
}
case GroupNone:
g.sortFn = groupNoneSort
g.nextGroupFn = groupNoneNextGroup
default:
panic("not implemented")
}
n, err := g.sort()
if n == 0 || err != nil {
return nil
}
return g
}
var nilKey = [...]byte{0xff}
func (g *groupResultSet) Close() {}
func (g *groupResultSet) Next() GroupCursor {
if g.eof {
return nil
}
return g.nextGroupFn(g)
}
func (g *groupResultSet) sort() (int, error) {
log := logger.LoggerFromContext(g.ctx)
if log != nil {
var f func()
log, f = logger.NewOperation(log, "Sort", "group.sort", zap.String("group_type", g.req.Group.String()))
defer f()
}
span := opentracing.SpanFromContext(g.ctx)
if span != nil {
span = opentracing.StartSpan(
"group.sort",
opentracing.ChildOf(span.Context()),
opentracing.Tag{Key: "group_type", Value: g.req.Group.String()})
defer span.Finish()
}
n, err := g.sortFn(g)
if span != nil {
span.SetTag("rows", n)
}
if log != nil {
log.Info("Sort completed", zap.Int("rows", n))
}
return n, err
}
// seriesHasPoints reads the first block of TSM data to verify the series has points for
// the time range of the query.
func (g *groupResultSet) seriesHasPoints(row *seriesRow) bool {
// TODO(sgc): this is expensive. Storage engine must provide efficient time range queries of series keys.
cur := g.mb.createCursor(*row)
var ts []int64
switch c := cur.(type) {
case tsdb.IntegerBatchCursor:
ts, _ = c.Next()
case tsdb.FloatBatchCursor:
ts, _ = c.Next()
case tsdb.UnsignedBatchCursor:
ts, _ = c.Next()
case tsdb.BooleanBatchCursor:
ts, _ = c.Next()
case tsdb.StringBatchCursor:
ts, _ = c.Next()
case nil:
return false
default:
panic(fmt.Sprintf("unreachable: %T", c))
}
cur.Close()
return len(ts) > 0
}
func groupNoneNextGroup(g *groupResultSet) GroupCursor {
cur, err := g.newCursorFn()
if err != nil {
// TODO(sgc): store error
return nil
} else if cur == nil {
return nil
}
g.eof = true
return &groupNoneCursor{
mb: g.mb,
req: &g.rr,
cur: cur,
keys: g.km.get(),
}
}
func groupNoneSort(g *groupResultSet) (int, error) {
cur, err := g.newCursorFn()
if err != nil {
return 0, err
} else if cur == nil {
return 0, nil
}
n := 0
row := cur.Next()
g.km.setTags(row.tags)
allTime := g.req.Hints.HintSchemaAllTime()
for {
n++
row = cur.Next()
if row == nil {
break
}
if allTime || g.seriesHasPoints(row) {
g.km.mergeTagKeys(row.tags)
}
}
cur.Close()
return n, nil
}
func groupByNextGroup(g *groupResultSet) GroupCursor {
row := g.rows[g.i]
for i := range g.keys {
g.rgc.vals[i] = row.tags.Get(g.keys[i])
}
allTime := g.req.Hints.HintSchemaAllTime()
rowKey := row.sortKey
g.km.setTags(row.tags)
j := g.i + 1
for j < len(g.rows) && bytes.Equal(rowKey, g.rows[j].sortKey) {
if allTime || g.seriesHasPoints(g.rows[j]) {
g.km.mergeTagKeys(g.rows[j].tags)
}
j++
}
g.rgc.reset(g.rows[g.i:j])
g.rgc.keys = g.km.get()
g.i = j
if j == len(g.rows) {
g.eof = true
}
return &g.rgc
}
func groupBySort(g *groupResultSet) (int, error) {
cur, err := g.newCursorFn()
if err != nil {
return 0, err
} else if cur == nil {
return 0, nil
}
var rows []*seriesRow
vals := make([][]byte, len(g.keys))
tagsBuf := &tagsBuffer{sz: 4096}
row := cur.Next()
for row != nil {
nr := *row
nr.stags = tagsBuf.copyTags(nr.stags)
nr.tags = tagsBuf.copyTags(nr.tags)
l := 0
for i, k := range g.keys {
vals[i] = nr.tags.Get(k)
if len(vals[i]) == 0 {
vals[i] = nilKey[:] // if there was no value, ensure it sorts last
}
l += len(vals[i])
}
nr.sortKey = make([]byte, 0, l)
for _, v := range vals {
nr.sortKey = append(nr.sortKey, v...)
}
rows = append(rows, &nr)
row = cur.Next()
}
sort.Slice(rows, func(i, j int) bool {
return bytes.Compare(rows[i].sortKey, rows[j].sortKey) == -1
})
g.rows = rows
cur.Close()
return len(rows), nil
}
type groupNoneCursor struct {
mb *multiShardBatchCursors
req *readRequest
cur seriesCursor
row seriesRow
keys [][]byte
}
func (c *groupNoneCursor) Tags() models.Tags { return c.row.tags }
func (c *groupNoneCursor) Keys() [][]byte { return c.keys }
func (c *groupNoneCursor) PartitionKeyVals() [][]byte { return nil }
func (c *groupNoneCursor) Close() { c.cur.Close() }
func (c *groupNoneCursor) Next() bool {
row := c.cur.Next()
if row == nil {
return false
}
c.row = *row
return true
}
func (c *groupNoneCursor) Cursor() tsdb.Cursor {
cur := c.mb.createCursor(c.row)
if c.req.aggregate != nil {
cur = newAggregateBatchCursor(c.req.ctx, c.req.aggregate, cur)
}
return cur
}
type groupByCursor struct {
mb *multiShardBatchCursors
req *readRequest
i int
rows []*seriesRow
keys [][]byte
vals [][]byte
}
func (c *groupByCursor) reset(rows []*seriesRow) {
c.i = 0
c.rows = rows
}
func (c *groupByCursor) Keys() [][]byte { return c.keys }
func (c *groupByCursor) PartitionKeyVals() [][]byte { return c.vals }
func (c *groupByCursor) Tags() models.Tags { return c.rows[c.i-1].tags }
func (c *groupByCursor) Close() {}
func (c *groupByCursor) Next() bool {
if c.i < len(c.rows) {
c.i++
return true
}
return false
}
func (c *groupByCursor) Cursor() tsdb.Cursor {
cur := c.mb.createCursor(*c.rows[c.i-1])
if c.req.aggregate != nil {
cur = newAggregateBatchCursor(c.req.ctx, c.req.aggregate, cur)
}
return cur
}
// keyMerger is responsible for determining a merged set of tag keys
type keyMerger struct {
i int
keys [2][][]byte
}
func (km *keyMerger) setTags(tags models.Tags) {
km.i = 0
if cap(km.keys[0]) < len(tags) {
km.keys[0] = make([][]byte, len(tags))
} else {
km.keys[0] = km.keys[0][:len(tags)]
}
for i := range tags {
km.keys[0][i] = tags[i].Key
}
}
func (km *keyMerger) get() [][]byte { return km.keys[km.i&1] }
func (km *keyMerger) String() string {
var s []string
for _, k := range km.get() {
s = append(s, string(k))
}
return strings.Join(s, ",")
}
func (km *keyMerger) mergeTagKeys(tags models.Tags) {
keys := km.keys[km.i&1]
i, j := 0, 0
for i < len(keys) && j < len(tags) && bytes.Equal(keys[i], tags[j].Key) {
i++
j++
}
if j == len(tags) {
// no new tags
return
}
km.i = (km.i + 1) & 1
l := len(keys) + len(tags)
if cap(km.keys[km.i]) < l {
km.keys[km.i] = make([][]byte, l)
} else {
km.keys[km.i] = km.keys[km.i][:l]
}
keya := km.keys[km.i]
// back up the pointers
if i > 0 {
i--
j--
}
k := i
copy(keya[:k], keys[:k])
for i < len(keys) && j < len(tags) {
cmp := bytes.Compare(keys[i], tags[j].Key)
if cmp < 0 {
keya[k] = keys[i]
i++
} else if cmp > 0 {
keya[k] = tags[j].Key
j++
} else {
keya[k] = keys[i]
i++
j++
}
k++
}
if i < len(keys) {
k += copy(keya[k:], keys[i:])
}
for j < len(tags) {
keya[k] = tags[j].Key
j++
k++
}
km.keys[km.i] = keya[:k]
}

View File

@ -0,0 +1,213 @@
package storage
import (
"context"
"strings"
"testing"
"github.com/google/go-cmp/cmp"
"github.com/influxdata/influxdb/models"
)
func TestGroupGroupResultSetSorting(t *testing.T) {
tests := []struct {
name string
cur seriesCursor
group ReadRequest_Group
keys []string
exp []seriesRow
}{
{
name: "group by tag1 in all series",
cur: &sliceSeriesCursor{
rows: newSeriesRows(
"cpu,tag0=val0,tag1=val0",
"cpu,tag0=val0,tag1=val1",
"cpu,tag0=val0,tag1=val2",
"cpu,tag0=val1,tag1=val0",
"cpu,tag0=val1,tag1=val1",
"cpu,tag0=val1,tag1=val2",
)},
group: GroupBy,
keys: []string{"tag1"},
exp: newSeriesRows(
"cpu,tag0=val0,tag1=val0",
"cpu,tag0=val1,tag1=val0",
"cpu,tag0=val0,tag1=val1",
"cpu,tag0=val1,tag1=val1",
"cpu,tag0=val0,tag1=val2",
"cpu,tag0=val1,tag1=val2",
),
},
{
name: "group by tag1 in partial series",
cur: &sliceSeriesCursor{
rows: newSeriesRows(
"aaa,tag0=val0",
"aaa,tag0=val1",
"cpu,tag0=val0,tag1=val0",
"cpu,tag0=val0,tag1=val1",
"cpu,tag0=val0,tag1=val2",
"cpu,tag0=val1,tag1=val0",
"cpu,tag0=val1,tag1=val1",
"cpu,tag0=val1,tag1=val2",
)},
group: GroupBy,
keys: []string{"tag1"},
exp: newSeriesRows(
"cpu,tag0=val0,tag1=val0",
"cpu,tag0=val1,tag1=val0",
"cpu,tag0=val0,tag1=val1",
"cpu,tag0=val1,tag1=val1",
"cpu,tag0=val0,tag1=val2",
"cpu,tag0=val1,tag1=val2",
"aaa,tag0=val0",
"aaa,tag0=val1",
),
},
{
name: "group by tag2,tag1 with partial series",
cur: &sliceSeriesCursor{
rows: newSeriesRows(
"aaa,tag0=val0",
"aaa,tag0=val1",
"cpu,tag0=val0,tag1=val0",
"cpu,tag0=val0,tag1=val1",
"cpu,tag0=val0,tag1=val2",
"mem,tag1=val0,tag2=val0",
"mem,tag1=val1,tag2=val0",
"mem,tag1=val1,tag2=val1",
)},
group: GroupBy,
keys: []string{"tag2,tag1"},
exp: newSeriesRows(
"mem,tag1=val0,tag2=val0",
"mem,tag1=val1,tag2=val0",
"mem,tag1=val1,tag2=val1",
"cpu,tag0=val0,tag1=val0",
"cpu,tag0=val0,tag1=val1",
"cpu,tag0=val0,tag1=val2",
"aaa,tag0=val0",
"aaa,tag0=val1",
),
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
newCursor := func() (seriesCursor, error) {
return tt.cur, nil
}
rs := newGroupResultSet(context.Background(), &ReadRequest{Group: tt.group, GroupKeys: tt.keys}, newCursor)
var rows []seriesRow
for i := range rs.rows {
rows = append(rows, *rs.rows[i])
}
got := selectTags(rows, tt.keys)
exp := selectTags(tt.exp, tt.keys)
if !cmp.Equal(got, exp) {
t.Errorf("unexpected rows -got/+exp\n%s", cmp.Diff(got, exp))
}
})
}
}
func TestKeyMerger(t *testing.T) {
tests := []struct {
name string
tags []models.Tags
exp string
}{
{
name: "mixed",
tags: []models.Tags{
models.ParseTags([]byte("foo,tag0=v0,tag1=v0,tag2=v0")),
models.ParseTags([]byte("foo,tag0=v0,tag1=v0,tag2=v1")),
models.ParseTags([]byte("foo,tag0=v0")),
models.ParseTags([]byte("foo,tag0=v0,tag3=v0")),
},
exp: "tag0,tag1,tag2,tag3",
},
{
name: "mixed 2",
tags: []models.Tags{
models.ParseTags([]byte("foo,tag0=v0")),
models.ParseTags([]byte("foo,tag0=v0,tag3=v0")),
models.ParseTags([]byte("foo,tag0=v0,tag1=v0,tag2=v0")),
models.ParseTags([]byte("foo,tag0=v0,tag1=v0,tag2=v1")),
},
exp: "tag0,tag1,tag2,tag3",
},
{
name: "all different",
tags: []models.Tags{
models.ParseTags([]byte("foo,tag0=v0")),
models.ParseTags([]byte("foo,tag1=v0")),
models.ParseTags([]byte("foo,tag2=v1")),
models.ParseTags([]byte("foo,tag3=v0")),
},
exp: "tag0,tag1,tag2,tag3",
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
var km keyMerger
km.setTags(tt.tags[0])
for _, tags := range tt.tags[1:] {
km.mergeTagKeys(tags)
}
if got := km.String(); !cmp.Equal(got, tt.exp) {
t.Errorf("unexpected keys -got/+exp\n%s", cmp.Diff(got, tt.exp))
}
})
}
}
func selectTags(rows []seriesRow, keys []string) string {
var srows []string
for _, row := range rows {
var ss []string
for _, key := range keys {
for _, tag := range row.tags {
if key == string(tag.Key) {
ss = append(ss, string(tag.Key)+"="+string(tag.Value))
}
}
}
srows = append(srows, strings.Join(ss, ","))
}
return strings.Join(srows, "\n")
}
type sliceSeriesCursor struct {
rows []seriesRow
i int
}
func newSeriesRows(keys ...string) []seriesRow {
rows := make([]seriesRow, len(keys))
for i := range keys {
rows[i].name, rows[i].stags = models.ParseKeyBytes([]byte(keys[i]))
rows[i].tags = rows[i].stags.Clone()
rows[i].tags.Set([]byte("_m"), rows[i].name)
}
return rows
}
func (s *sliceSeriesCursor) Close() {}
func (s *sliceSeriesCursor) Err() error { return nil }
func (s *sliceSeriesCursor) Next() *seriesRow {
if s.i < len(s.rows) {
s.i++
return &s.rows[s.i-1]
}
return nil
}

View File

@ -113,7 +113,7 @@ func (x Node_Logical) String() string {
func (Node_Logical) EnumDescriptor() ([]byte, []int) { return fileDescriptorPredicate, []int{0, 2} }
type Node struct {
NodeType Node_Type `protobuf:"varint,1,opt,name=node_type,json=nodeType,proto3,enum=storage.Node_Type" json:"nodeType"`
NodeType Node_Type `protobuf:"varint,1,opt,name=node_type,json=nodeType,proto3,enum=com.github.influxdata.influxdb.services.storage.Node_Type" json:"nodeType"`
Children []*Node `protobuf:"bytes,2,rep,name=children" json:"children,omitempty"`
// Types that are valid to be assigned to Value:
// *Node_StringValue
@ -165,10 +165,10 @@ type Node_FieldRefValue struct {
FieldRefValue string `protobuf:"bytes,10,opt,name=field_ref_value,json=fieldRefValue,proto3,oneof"`
}
type Node_Logical_ struct {
Logical Node_Logical `protobuf:"varint,11,opt,name=logical,proto3,enum=storage.Node_Logical,oneof"`
Logical Node_Logical `protobuf:"varint,11,opt,name=logical,proto3,enum=com.github.influxdata.influxdb.services.storage.Node_Logical,oneof"`
}
type Node_Comparison_ struct {
Comparison Node_Comparison `protobuf:"varint,12,opt,name=comparison,proto3,enum=storage.Node_Comparison,oneof"`
Comparison Node_Comparison `protobuf:"varint,12,opt,name=comparison,proto3,enum=com.github.influxdata.influxdb.services.storage.Node_Comparison,oneof"`
}
func (*Node_StringValue) isNode_Value() {}
@ -474,11 +474,11 @@ func (m *Predicate) GetRoot() *Node {
}
func init() {
proto.RegisterType((*Node)(nil), "storage.Node")
proto.RegisterType((*Predicate)(nil), "storage.Predicate")
proto.RegisterEnum("storage.Node_Type", Node_Type_name, Node_Type_value)
proto.RegisterEnum("storage.Node_Comparison", Node_Comparison_name, Node_Comparison_value)
proto.RegisterEnum("storage.Node_Logical", Node_Logical_name, Node_Logical_value)
proto.RegisterType((*Node)(nil), "com.github.influxdata.influxdb.services.storage.Node")
proto.RegisterType((*Predicate)(nil), "com.github.influxdata.influxdb.services.storage.Predicate")
proto.RegisterEnum("com.github.influxdata.influxdb.services.storage.Node_Type", Node_Type_name, Node_Type_value)
proto.RegisterEnum("com.github.influxdata.influxdb.services.storage.Node_Comparison", Node_Comparison_name, Node_Comparison_value)
proto.RegisterEnum("com.github.influxdata.influxdb.services.storage.Node_Logical", Node_Logical_name, Node_Logical_value)
}
func (m *Node) Marshal() (dAtA []byte, err error) {
size := m.Size()
@ -1287,58 +1287,61 @@ var (
func init() { proto.RegisterFile("predicate.proto", fileDescriptorPredicate) }
var fileDescriptorPredicate = []byte{
// 845 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x6c, 0x94, 0xcf, 0x6e, 0xdb, 0x46,
0x10, 0xc6, 0x45, 0x49, 0xb6, 0xc4, 0x91, 0x65, 0x33, 0x9b, 0x38, 0x56, 0xd9, 0x46, 0xda, 0x3a,
0x28, 0xa0, 0x1c, 0x2a, 0xc3, 0x6e, 0x73, 0x69, 0x0e, 0x05, 0xe5, 0xd0, 0xb2, 0x00, 0x56, 0x52,
0x29, 0xa6, 0xc9, 0x4d, 0xa0, 0xa5, 0x15, 0x4d, 0x80, 0xe1, 0xaa, 0xcb, 0x55, 0x91, 0xbc, 0x41,
0xc1, 0x53, 0xef, 0x05, 0x4f, 0x7d, 0x99, 0x02, 0x45, 0x81, 0x3e, 0x81, 0x50, 0xa8, 0xb7, 0x3e,
0x45, 0xc1, 0xe5, 0x3f, 0xa9, 0xc9, 0x6d, 0x67, 0xbe, 0xef, 0x37, 0xb3, 0xbb, 0x1c, 0x2e, 0x9c,
0xac, 0x18, 0x59, 0xb8, 0x73, 0x9b, 0x93, 0xde, 0x8a, 0x51, 0x4e, 0x51, 0x2d, 0xe0, 0x94, 0xd9,
0x0e, 0x51, 0xbf, 0x74, 0x5c, 0x7e, 0xbf, 0xbe, 0xeb, 0xcd, 0xe9, 0xdb, 0x0b, 0x87, 0x3a, 0xf4,
0x42, 0xe8, 0x77, 0xeb, 0xa5, 0x88, 0x44, 0x20, 0x56, 0x09, 0x77, 0xfe, 0x07, 0x40, 0x75, 0x44,
0x17, 0x04, 0x0d, 0x41, 0xf6, 0xe9, 0x82, 0xcc, 0xf8, 0xfb, 0x15, 0x69, 0x49, 0x58, 0xea, 0x1e,
0x5f, 0xa1, 0x5e, 0x5a, 0xb4, 0x17, 0x3b, 0x7a, 0xd6, 0xfb, 0x15, 0xe9, 0xb7, 0xb6, 0x9b, 0x4e,
0x3d, 0x0e, 0xe3, 0xe8, 0xdf, 0x4d, 0xa7, 0xee, 0xa7, 0x6b, 0x33, 0x5f, 0xa1, 0x67, 0x50, 0x9f,
0xdf, 0xbb, 0xde, 0x82, 0x11, 0xbf, 0x55, 0xc6, 0x95, 0x6e, 0xe3, 0xaa, 0xb9, 0x57, 0xc9, 0xcc,
0x65, 0xf4, 0x35, 0x1c, 0x05, 0x9c, 0xb9, 0xbe, 0x33, 0xfb, 0xc9, 0xf6, 0xd6, 0xa4, 0x55, 0xc1,
0x52, 0x57, 0xee, 0x9f, 0x6c, 0x37, 0x9d, 0xc6, 0x54, 0xe4, 0x7f, 0x88, 0xd3, 0xb7, 0x25, 0xb3,
0x11, 0x14, 0x21, 0xba, 0x04, 0xb8, 0xa3, 0xd4, 0x4b, 0x99, 0x2a, 0x96, 0xba, 0xf5, 0xbe, 0xb2,
0xdd, 0x74, 0x8e, 0xfa, 0x94, 0x7a, 0xc4, 0xf6, 0x33, 0x48, 0x8e, 0x5d, 0x09, 0x72, 0x01, 0xb2,
0xeb, 0xf3, 0x94, 0x38, 0xc0, 0x52, 0xb7, 0x92, 0x10, 0x43, 0x9f, 0x13, 0x87, 0xb0, 0x8c, 0xa8,
0xbb, 0x3e, 0x4f, 0x80, 0x2b, 0x80, 0x75, 0x41, 0x1c, 0x62, 0xa9, 0x5b, 0xed, 0x3f, 0xd8, 0x6e,
0x3a, 0xcd, 0x57, 0x7e, 0xe0, 0x3a, 0x3e, 0x59, 0xe4, 0x4d, 0xd6, 0x39, 0x73, 0x09, 0x8d, 0xa5,
0x47, 0xed, 0x0c, 0xaa, 0x61, 0xa9, 0x2b, 0xf5, 0x8f, 0xb7, 0x9b, 0x0e, 0xdc, 0xc4, 0xe9, 0x8c,
0x80, 0x65, 0x1e, 0xc5, 0x08, 0x23, 0x0e, 0x79, 0x97, 0x22, 0x75, 0x71, 0x7e, 0x81, 0x98, 0x71,
0x3a, 0x47, 0x58, 0x1e, 0xa1, 0xe7, 0xd0, 0xe4, 0xb6, 0x33, 0x63, 0x64, 0x99, 0x42, 0x72, 0x71,
0x69, 0x96, 0xed, 0x98, 0x64, 0x99, 0x5f, 0x1a, 0x2f, 0x42, 0xf4, 0x02, 0x4e, 0x96, 0x2e, 0xf1,
0x16, 0x3b, 0x20, 0x08, 0x50, 0x9c, 0xea, 0x26, 0x96, 0x76, 0xd0, 0xe6, 0x72, 0x37, 0x81, 0x2e,
0xa1, 0xe6, 0x51, 0xc7, 0x9d, 0xdb, 0x5e, 0xab, 0x21, 0x66, 0xe3, 0x74, 0x7f, 0x36, 0x8c, 0x44,
0xbc, 0x2d, 0x99, 0x99, 0x0f, 0x7d, 0x03, 0x30, 0xa7, 0x6f, 0x57, 0x36, 0x73, 0x03, 0xea, 0xb7,
0x8e, 0x04, 0xd5, 0xda, 0xa7, 0xae, 0x73, 0x3d, 0x3e, 0x62, 0xe1, 0x3e, 0xff, 0xb5, 0x0c, 0x55,
0x31, 0x4a, 0xcf, 0x01, 0x19, 0xe3, 0xc1, 0xf0, 0x5a, 0x33, 0x66, 0xfa, 0x9b, 0x89, 0xa9, 0x4f,
0xa7, 0xc3, 0xf1, 0x48, 0x29, 0xa9, 0x4f, 0xc2, 0x08, 0x7f, 0x92, 0x8d, 0x61, 0xda, 0x5c, 0x7f,
0xb7, 0x62, 0x24, 0x08, 0x5c, 0xea, 0xa3, 0x17, 0x70, 0x7a, 0x3d, 0xfe, 0x6e, 0xa2, 0x99, 0xc3,
0xe9, 0x78, 0xb4, 0x4b, 0x4a, 0x2a, 0x0e, 0x23, 0xfc, 0x59, 0x46, 0x16, 0x1b, 0xd8, 0x81, 0x2f,
0x41, 0x99, 0x68, 0xa6, 0xbe, 0xc7, 0x95, 0xd5, 0x4f, 0xc3, 0x08, 0x9f, 0x65, 0xdc, 0xc4, 0x66,
0x64, 0x17, 0xe9, 0x40, 0xcd, 0xd2, 0x06, 0x33, 0x53, 0xbf, 0x51, 0x2a, 0x2a, 0x0a, 0x23, 0x7c,
0x9c, 0x39, 0x93, 0x0f, 0x82, 0x30, 0xd4, 0x8c, 0xa1, 0xa5, 0x9b, 0x9a, 0xa1, 0x54, 0xd5, 0x87,
0x61, 0x84, 0x4f, 0xf2, 0xcd, 0xbb, 0x9c, 0x30, 0xdb, 0x43, 0x4f, 0x41, 0xbe, 0x19, 0xea, 0xc6,
0x4b, 0x51, 0xe4, 0x40, 0x7d, 0x14, 0x46, 0x58, 0xc9, 0x3c, 0xd9, 0xc7, 0x51, 0xab, 0x3f, 0xff,
0xd6, 0x2e, 0x9d, 0xff, 0x59, 0x06, 0x28, 0x76, 0x8e, 0xda, 0x70, 0xa0, 0x7f, 0xff, 0x4a, 0x33,
0x94, 0x52, 0x52, 0x79, 0xe7, 0x50, 0x3f, 0xae, 0x6d, 0x0f, 0x7d, 0x01, 0xf2, 0x68, 0x6c, 0xcd,
0x12, 0x8f, 0xa4, 0x3e, 0x0e, 0x23, 0x8c, 0x0a, 0xcf, 0x88, 0xf2, 0xc4, 0xf6, 0x0c, 0x1a, 0x53,
0x4b, 0x33, 0xad, 0xe9, 0xec, 0xf5, 0xd0, 0xba, 0x55, 0xca, 0x6a, 0x2b, 0x8c, 0xf0, 0xa3, 0xc2,
0x38, 0xe5, 0x36, 0xe3, 0xc1, 0x6b, 0x97, 0xdf, 0xc7, 0x1d, 0x4d, 0x7d, 0xa0, 0xbf, 0x51, 0x2a,
0xff, 0xef, 0x28, 0x86, 0x36, 0xeb, 0x98, 0x78, 0xaa, 0x1f, 0xe9, 0x98, 0xd8, 0x54, 0x28, 0x1b,
0x96, 0x72, 0x90, 0x5c, 0x58, 0xa1, 0x1b, 0x24, 0x08, 0x10, 0x86, 0x8a, 0x61, 0xe9, 0xca, 0xa1,
0x7a, 0x16, 0x46, 0xf8, 0xe1, 0xbe, 0x98, 0xec, 0xf7, 0x09, 0x94, 0x07, 0x96, 0x52, 0x53, 0x4f,
0xc3, 0x08, 0x3f, 0x28, 0x0c, 0x03, 0x46, 0x6c, 0x4e, 0x18, 0x7a, 0x0a, 0x95, 0x81, 0xa5, 0x2b,
0x75, 0x55, 0x0d, 0x23, 0xfc, 0xf8, 0x03, 0x5d, 0xd4, 0x48, 0xef, 0xf3, 0x5b, 0xa8, 0xa5, 0x23,
0x84, 0xce, 0xa0, 0xa2, 0x8d, 0x5e, 0x2a, 0x25, 0xf5, 0x38, 0x8c, 0x30, 0xa4, 0x59, 0xcd, 0x5f,
0xa0, 0x53, 0x28, 0x8f, 0x4d, 0x45, 0x52, 0x9b, 0x61, 0x84, 0xe5, 0x34, 0x3f, 0x66, 0x49, 0x81,
0x7e, 0x0d, 0x0e, 0xc4, 0x0f, 0x75, 0xde, 0x03, 0x79, 0x92, 0x3d, 0xcc, 0xe8, 0x73, 0xa8, 0x32,
0x4a, 0xb9, 0x78, 0x4c, 0x3f, 0x78, 0x02, 0x85, 0xd4, 0x57, 0x7e, 0xdf, 0xb6, 0xa5, 0xbf, 0xb6,
0x6d, 0xe9, 0xef, 0x6d, 0x5b, 0xfa, 0xe5, 0x9f, 0x76, 0xe9, 0xee, 0x50, 0x3c, 0xcb, 0x5f, 0xfd,
0x17, 0x00, 0x00, 0xff, 0xff, 0xfb, 0xde, 0x9f, 0x18, 0xe1, 0x05, 0x00, 0x00,
// 883 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x95, 0xcb, 0x6e, 0xdb, 0x46,
0x14, 0x86, 0x45, 0x5d, 0x2c, 0xe9, 0xc8, 0x17, 0x66, 0x12, 0xc7, 0x0a, 0xdb, 0x48, 0x03, 0x07,
0x05, 0xd4, 0x45, 0x65, 0xd8, 0xad, 0x37, 0x0d, 0x8a, 0x56, 0x72, 0x68, 0x59, 0x00, 0x2b, 0x29,
0x14, 0x73, 0x69, 0x37, 0x02, 0x25, 0x8d, 0x68, 0x02, 0x34, 0x47, 0x19, 0x8e, 0x02, 0xe7, 0x0d,
0x0a, 0xae, 0xba, 0x2f, 0xb8, 0xea, 0xcb, 0x74, 0x53, 0xa0, 0x4f, 0x20, 0x14, 0xea, 0xae, 0x7d,
0x89, 0x82, 0xc3, 0x9b, 0xdc, 0x76, 0x53, 0xef, 0xe6, 0x9c, 0xf9, 0xbf, 0xff, 0xcc, 0x1c, 0x1e,
0x92, 0x70, 0xb0, 0x64, 0x64, 0x6e, 0xcf, 0x4c, 0x4e, 0xda, 0x4b, 0x46, 0x39, 0x45, 0x27, 0x33,
0x7a, 0xd3, 0xb6, 0x6c, 0x7e, 0xbd, 0x9a, 0xb6, 0x6d, 0x77, 0xe1, 0xac, 0x6e, 0xe7, 0x26, 0x37,
0x93, 0xe5, 0xb4, 0xed, 0x11, 0xf6, 0xde, 0x9e, 0x11, 0xaf, 0xed, 0x71, 0xca, 0x4c, 0x8b, 0x28,
0x9f, 0xc5, 0xe2, 0x19, 0xbd, 0x39, 0xb1, 0xa8, 0x45, 0x4f, 0x84, 0xcf, 0x74, 0xb5, 0x10, 0x91,
0x08, 0xc4, 0x2a, 0xf2, 0x3f, 0xfe, 0xab, 0x06, 0xc5, 0x01, 0x9d, 0x13, 0xf4, 0x0e, 0xaa, 0x2e,
0x9d, 0x93, 0x09, 0xff, 0xb0, 0x24, 0x75, 0x09, 0x4b, 0xad, 0xfd, 0xb3, 0x2f, 0xdb, 0xff, 0xb3,
0x78, 0x3b, 0x74, 0x6a, 0x1b, 0x1f, 0x96, 0xa4, 0x5b, 0xdf, 0xac, 0x9b, 0x95, 0x30, 0x0c, 0xa3,
0x3f, 0xd7, 0xcd, 0x8a, 0x1b, 0xaf, 0xf5, 0x74, 0x85, 0x5e, 0x42, 0x65, 0x76, 0x6d, 0x3b, 0x73,
0x46, 0xdc, 0x7a, 0x1e, 0x17, 0x5a, 0xb5, 0xb3, 0xf3, 0x7b, 0x55, 0xd4, 0x53, 0x1b, 0xf4, 0x05,
0xec, 0x7a, 0x9c, 0xd9, 0xae, 0x35, 0x79, 0x6f, 0x3a, 0x2b, 0x52, 0x2f, 0x60, 0xa9, 0x55, 0xed,
0x1e, 0x6c, 0xd6, 0xcd, 0xda, 0x58, 0xe4, 0x5f, 0x87, 0xe9, 0xab, 0x9c, 0x5e, 0xf3, 0xb2, 0x10,
0x9d, 0x02, 0x4c, 0x29, 0x75, 0x62, 0xa6, 0x88, 0xa5, 0x56, 0xa5, 0x2b, 0x6f, 0xd6, 0xcd, 0xdd,
0x2e, 0xa5, 0x0e, 0x31, 0xdd, 0x04, 0xaa, 0x86, 0xaa, 0x08, 0x39, 0x81, 0xaa, 0xed, 0xf2, 0x98,
0x28, 0x61, 0xa9, 0x55, 0x88, 0x88, 0xbe, 0xcb, 0x89, 0x45, 0x58, 0x42, 0x54, 0x6c, 0x97, 0x47,
0xc0, 0x19, 0xc0, 0x2a, 0x23, 0x76, 0xb0, 0xd4, 0x2a, 0x76, 0x1f, 0x6c, 0xd6, 0xcd, 0xbd, 0x57,
0xae, 0x67, 0x5b, 0x2e, 0x99, 0xa7, 0x45, 0x56, 0x29, 0x73, 0x0a, 0xb5, 0x85, 0x43, 0xcd, 0x04,
0x2a, 0x63, 0xa9, 0x25, 0x75, 0xf7, 0x37, 0xeb, 0x26, 0x5c, 0x86, 0xe9, 0x84, 0x80, 0x45, 0x1a,
0x85, 0x08, 0x23, 0x16, 0xb9, 0x8d, 0x91, 0x8a, 0xb8, 0xbf, 0x40, 0xf4, 0x30, 0x9d, 0x22, 0x2c,
0x8d, 0xd0, 0x39, 0xec, 0x71, 0xd3, 0x9a, 0x30, 0xb2, 0x88, 0xa1, 0x6a, 0xd6, 0x34, 0xc3, 0xb4,
0x74, 0xb2, 0x48, 0x9b, 0xc6, 0xb3, 0x10, 0x3d, 0x87, 0x83, 0x85, 0x4d, 0x9c, 0xf9, 0x16, 0x08,
0x02, 0x14, 0xb7, 0xba, 0x0c, 0xb7, 0xb6, 0xd0, 0xbd, 0xc5, 0x76, 0x02, 0x7d, 0x07, 0x65, 0x87,
0x5a, 0xf6, 0xcc, 0x74, 0xea, 0x35, 0x31, 0x6b, 0x5f, 0xdd, 0x6f, 0xd6, 0xb4, 0xc8, 0xe4, 0x2a,
0xa7, 0x27, 0x7e, 0x68, 0x0a, 0x30, 0xa3, 0x37, 0x4b, 0x93, 0xd9, 0x1e, 0x75, 0xeb, 0xbb, 0xc2,
0xfd, 0x9b, 0xfb, 0xb9, 0x5f, 0xa4, 0x3e, 0x61, 0xcb, 0x32, 0xd7, 0xe3, 0x9f, 0xf2, 0x50, 0x14,
0x23, 0x7c, 0x0e, 0x48, 0x1b, 0xf6, 0xfa, 0x17, 0x1d, 0x6d, 0xa2, 0xbe, 0x1d, 0xe9, 0xea, 0x78,
0xdc, 0x1f, 0x0e, 0xe4, 0x9c, 0xf2, 0xd4, 0x0f, 0xf0, 0x93, 0x64, 0xfc, 0xe3, 0x43, 0xaa, 0xb7,
0x4b, 0x46, 0x3c, 0xcf, 0xa6, 0x2e, 0x7a, 0x0e, 0x87, 0x17, 0xc3, 0x6f, 0x47, 0x1d, 0xbd, 0x3f,
0x1e, 0x0e, 0xb6, 0x49, 0x49, 0xc1, 0x7e, 0x80, 0x3f, 0x4e, 0xc8, 0xec, 0x00, 0x5b, 0xf0, 0x29,
0xc8, 0xa3, 0x8e, 0xae, 0xde, 0xe1, 0xf2, 0xca, 0x47, 0x7e, 0x80, 0x8f, 0x12, 0x6e, 0x64, 0x32,
0xb2, 0x8d, 0x34, 0xa1, 0x6c, 0x74, 0x7a, 0x13, 0x5d, 0xbd, 0x94, 0x0b, 0x0a, 0xf2, 0x03, 0xbc,
0x9f, 0x28, 0xa3, 0x07, 0x8c, 0x30, 0x94, 0xb5, 0xbe, 0xa1, 0xea, 0x1d, 0x4d, 0x2e, 0x2a, 0x0f,
0xfd, 0x00, 0x1f, 0xa4, 0x87, 0xb7, 0x39, 0x61, 0xa6, 0x83, 0x9e, 0x41, 0xf5, 0xb2, 0xaf, 0x6a,
0x2f, 0x84, 0x49, 0x49, 0x79, 0xe4, 0x07, 0x58, 0x4e, 0x34, 0xc9, 0xc3, 0x56, 0x8a, 0x3f, 0xfc,
0xdc, 0xc8, 0x1d, 0xff, 0x9a, 0x07, 0xc8, 0x4e, 0x8e, 0x1a, 0x50, 0x52, 0x5f, 0xbe, 0xea, 0x68,
0x72, 0x2e, 0x72, 0xde, 0xba, 0xd4, 0xbb, 0x95, 0xe9, 0xa0, 0x4f, 0xa0, 0x3a, 0x18, 0x1a, 0x93,
0x48, 0x23, 0x29, 0x8f, 0xfd, 0x00, 0xa3, 0x4c, 0x33, 0xa0, 0x3c, 0x92, 0x7d, 0x0a, 0xb5, 0xb1,
0xd1, 0xd1, 0x8d, 0xf1, 0xe4, 0x4d, 0xdf, 0xb8, 0x92, 0xf3, 0x4a, 0xdd, 0x0f, 0xf0, 0xa3, 0x4c,
0x38, 0xe6, 0x26, 0xe3, 0xde, 0x1b, 0x9b, 0x5f, 0x87, 0x15, 0x75, 0xb5, 0xa7, 0xbe, 0x95, 0x0b,
0xff, 0xac, 0x28, 0x5e, 0x82, 0xa4, 0x62, 0xa4, 0x29, 0xfe, 0x47, 0xc5, 0x48, 0xa6, 0x40, 0x5e,
0x33, 0xe4, 0x52, 0xd4, 0xb0, 0x6c, 0x5f, 0x23, 0x9e, 0x87, 0x30, 0x14, 0x34, 0x43, 0x95, 0x77,
0x94, 0x23, 0x3f, 0xc0, 0x0f, 0xef, 0x6e, 0x46, 0xe7, 0x7d, 0x0a, 0xf9, 0x9e, 0x21, 0x97, 0x95,
0x43, 0x3f, 0xc0, 0x0f, 0x32, 0x41, 0x8f, 0x11, 0x93, 0x13, 0x86, 0x9e, 0x41, 0xa1, 0x67, 0xa8,
0x72, 0x45, 0x51, 0xfc, 0x00, 0x3f, 0xfe, 0xd7, 0xbe, 0xf0, 0x88, 0xfb, 0xf9, 0x35, 0x94, 0xe3,
0x11, 0x42, 0x47, 0x50, 0xe8, 0x0c, 0x5e, 0xc8, 0x39, 0x65, 0xdf, 0x0f, 0x30, 0xc4, 0xd9, 0x8e,
0x3b, 0x47, 0x87, 0x90, 0x1f, 0xea, 0xb2, 0xa4, 0xec, 0xf9, 0x01, 0xae, 0xc6, 0xf9, 0x21, 0x8b,
0x0c, 0xba, 0x65, 0x28, 0x89, 0x17, 0xf4, 0xf8, 0x35, 0x54, 0x47, 0xc9, 0x0f, 0x06, 0xf5, 0xa1,
0xc8, 0x28, 0xe5, 0xe2, 0x63, 0x7f, 0xef, 0x4f, 0xaf, 0xb0, 0xe8, 0x3e, 0xf9, 0x65, 0xd3, 0x90,
0x7e, 0xdb, 0x34, 0xa4, 0xdf, 0x37, 0x0d, 0xe9, 0xc7, 0x3f, 0x1a, 0xb9, 0xef, 0xcb, 0xb1, 0x6a,
0xba, 0x23, 0xfe, 0x33, 0x9f, 0xff, 0x1d, 0x00, 0x00, 0xff, 0xff, 0x37, 0x3b, 0xbc, 0x43, 0xda,
0x06, 0x00, 0x00,
}

View File

@ -1,5 +1,6 @@
syntax = "proto3";
package storage;
package com.github.influxdata.influxdb.services.storage;
option go_package = "storage";
import "github.com/gogo/protobuf/gogoproto/gogo.proto";

View File

@ -30,6 +30,19 @@ func (w *responseWriter) putFloatPointsFrame(f *ReadResponse_Frame_FloatPoints)
w.buffer.Float = append(w.buffer.Float, f)
}
func (w *responseWriter) streamFloatSeries(cur tsdb.FloatBatchCursor) {
w.sf.DataType = DataTypeFloat
ss := len(w.res.Frames) - 1
ts, _ := cur.Next()
if len(ts) == 0 {
w.sz -= w.sf.Size()
w.putSeriesFrame(w.res.Frames[ss].Data.(*ReadResponse_Frame_Series))
w.res.Frames = w.res.Frames[:ss]
} else if w.sz > writeSize {
w.flushFrames()
}
}
func (w *responseWriter) streamFloatPoints(cur tsdb.FloatBatchCursor) {
w.sf.DataType = DataTypeFloat
ss := len(w.res.Frames) - 1
@ -69,7 +82,6 @@ func (w *responseWriter) streamFloatPoints(cur tsdb.FloatBatchCursor) {
w.res.Frames = append(w.res.Frames, ReadResponse_Frame{p})
}
}
cur.Close()
seriesValueCount += b
w.vc += seriesValueCount
@ -102,6 +114,19 @@ func (w *responseWriter) putIntegerPointsFrame(f *ReadResponse_Frame_IntegerPoin
w.buffer.Integer = append(w.buffer.Integer, f)
}
func (w *responseWriter) streamIntegerSeries(cur tsdb.IntegerBatchCursor) {
w.sf.DataType = DataTypeInteger
ss := len(w.res.Frames) - 1
ts, _ := cur.Next()
if len(ts) == 0 {
w.sz -= w.sf.Size()
w.putSeriesFrame(w.res.Frames[ss].Data.(*ReadResponse_Frame_Series))
w.res.Frames = w.res.Frames[:ss]
} else if w.sz > writeSize {
w.flushFrames()
}
}
func (w *responseWriter) streamIntegerPoints(cur tsdb.IntegerBatchCursor) {
w.sf.DataType = DataTypeInteger
ss := len(w.res.Frames) - 1
@ -141,7 +166,6 @@ func (w *responseWriter) streamIntegerPoints(cur tsdb.IntegerBatchCursor) {
w.res.Frames = append(w.res.Frames, ReadResponse_Frame{p})
}
}
cur.Close()
seriesValueCount += b
w.vc += seriesValueCount
@ -174,6 +198,19 @@ func (w *responseWriter) putUnsignedPointsFrame(f *ReadResponse_Frame_UnsignedPo
w.buffer.Unsigned = append(w.buffer.Unsigned, f)
}
func (w *responseWriter) streamUnsignedSeries(cur tsdb.UnsignedBatchCursor) {
w.sf.DataType = DataTypeUnsigned
ss := len(w.res.Frames) - 1
ts, _ := cur.Next()
if len(ts) == 0 {
w.sz -= w.sf.Size()
w.putSeriesFrame(w.res.Frames[ss].Data.(*ReadResponse_Frame_Series))
w.res.Frames = w.res.Frames[:ss]
} else if w.sz > writeSize {
w.flushFrames()
}
}
func (w *responseWriter) streamUnsignedPoints(cur tsdb.UnsignedBatchCursor) {
w.sf.DataType = DataTypeUnsigned
ss := len(w.res.Frames) - 1
@ -213,7 +250,6 @@ func (w *responseWriter) streamUnsignedPoints(cur tsdb.UnsignedBatchCursor) {
w.res.Frames = append(w.res.Frames, ReadResponse_Frame{p})
}
}
cur.Close()
seriesValueCount += b
w.vc += seriesValueCount
@ -246,6 +282,19 @@ func (w *responseWriter) putStringPointsFrame(f *ReadResponse_Frame_StringPoints
w.buffer.String = append(w.buffer.String, f)
}
func (w *responseWriter) streamStringSeries(cur tsdb.StringBatchCursor) {
w.sf.DataType = DataTypeString
ss := len(w.res.Frames) - 1
ts, _ := cur.Next()
if len(ts) == 0 {
w.sz -= w.sf.Size()
w.putSeriesFrame(w.res.Frames[ss].Data.(*ReadResponse_Frame_Series))
w.res.Frames = w.res.Frames[:ss]
} else if w.sz > writeSize {
w.flushFrames()
}
}
func (w *responseWriter) streamStringPoints(cur tsdb.StringBatchCursor) {
w.sf.DataType = DataTypeString
ss := len(w.res.Frames) - 1
@ -285,7 +334,6 @@ func (w *responseWriter) streamStringPoints(cur tsdb.StringBatchCursor) {
w.res.Frames = append(w.res.Frames, ReadResponse_Frame{p})
}
}
cur.Close()
seriesValueCount += b
w.vc += seriesValueCount
@ -318,6 +366,19 @@ func (w *responseWriter) putBooleanPointsFrame(f *ReadResponse_Frame_BooleanPoin
w.buffer.Boolean = append(w.buffer.Boolean, f)
}
func (w *responseWriter) streamBooleanSeries(cur tsdb.BooleanBatchCursor) {
w.sf.DataType = DataTypeBoolean
ss := len(w.res.Frames) - 1
ts, _ := cur.Next()
if len(ts) == 0 {
w.sz -= w.sf.Size()
w.putSeriesFrame(w.res.Frames[ss].Data.(*ReadResponse_Frame_Series))
w.res.Frames = w.res.Frames[:ss]
} else if w.sz > writeSize {
w.flushFrames()
}
}
func (w *responseWriter) streamBooleanPoints(cur tsdb.BooleanBatchCursor) {
w.sf.DataType = DataTypeBoolean
ss := len(w.res.Frames) - 1
@ -357,7 +418,6 @@ func (w *responseWriter) streamBooleanPoints(cur tsdb.BooleanBatchCursor) {
w.res.Frames = append(w.res.Frames, ReadResponse_Frame{p})
}
}
cur.Close()
seriesValueCount += b
w.vc += seriesValueCount

View File

@ -26,6 +26,19 @@ func (w *responseWriter) put{{.Name}}PointsFrame(f *ReadResponse_Frame_{{.Name}}
w.buffer.{{.Name}} = append(w.buffer.{{.Name}}, f)
}
func (w *responseWriter) stream{{.Name}}Series(cur tsdb.{{.Name}}BatchCursor) {
w.sf.DataType = DataType{{.Name}}
ss := len(w.res.Frames) - 1
ts, _ := cur.Next()
if len(ts) == 0 {
w.sz -= w.sf.Size()
w.putSeriesFrame(w.res.Frames[ss].Data.(*ReadResponse_Frame_Series))
w.res.Frames = w.res.Frames[:ss]
} else if w.sz > writeSize {
w.flushFrames()
}
}
func (w *responseWriter) stream{{.Name}}Points(cur tsdb.{{.Name}}BatchCursor) {
w.sf.DataType = DataType{{.Name}}
ss := len(w.res.Frames) - 1
@ -65,7 +78,6 @@ func (w *responseWriter) stream{{.Name}}Points(cur tsdb.{{.Name}}BatchCursor) {
w.res.Frames = append(w.res.Frames, ReadResponse_Frame{p})
}
}
cur.Close()
seriesValueCount += b
w.vc += seriesValueCount

View File

@ -1,7 +1,10 @@
package storage
import (
"fmt"
"github.com/influxdata/influxdb/models"
"github.com/influxdata/influxdb/tsdb"
"go.uber.org/zap"
)
@ -25,7 +28,46 @@ type responseWriter struct {
Boolean []*ReadResponse_Frame_BooleanPoints
String []*ReadResponse_Frame_StringPoints
Series []*ReadResponse_Frame_Series
Group []*ReadResponse_Frame_Group
}
hints HintFlags
}
func (w *responseWriter) getGroupFrame(keys, partitionKey [][]byte) *ReadResponse_Frame_Group {
var res *ReadResponse_Frame_Group
if len(w.buffer.Group) > 0 {
i := len(w.buffer.Group) - 1
res = w.buffer.Group[i]
w.buffer.Group[i] = nil
w.buffer.Group = w.buffer.Group[:i]
} else {
res = &ReadResponse_Frame_Group{&ReadResponse_GroupFrame{}}
}
if cap(res.Group.TagKeys) < len(keys) {
res.Group.TagKeys = make([][]byte, len(keys))
} else if len(res.Group.TagKeys) != len(keys) {
res.Group.TagKeys = res.Group.TagKeys[:len(keys)]
}
if cap(res.Group.PartitionKeyVals) < len(partitionKey) {
res.Group.PartitionKeyVals = make([][]byte, len(partitionKey))
} else if len(res.Group.PartitionKeyVals) != len(partitionKey) {
res.Group.PartitionKeyVals = res.Group.PartitionKeyVals[:len(partitionKey)]
}
return res
}
func (w *responseWriter) putGroupFrame(f *ReadResponse_Frame_Group) {
for i := range f.Group.TagKeys {
f.Group.TagKeys[i] = nil
}
for i := range f.Group.PartitionKeyVals {
f.Group.PartitionKeyVals[i] = nil
}
w.buffer.Group = append(w.buffer.Group, f)
}
func (w *responseWriter) getSeriesFrame(next models.Tags) *ReadResponse_Frame_Series {
@ -57,7 +99,19 @@ func (w *responseWriter) putSeriesFrame(f *ReadResponse_Frame_Series) {
w.buffer.Series = append(w.buffer.Series, f)
}
func (w *responseWriter) startGroup(keys, partitionKey [][]byte) {
f := w.getGroupFrame(keys, partitionKey)
copy(f.Group.TagKeys, keys)
copy(f.Group.PartitionKeyVals, partitionKey)
w.res.Frames = append(w.res.Frames, ReadResponse_Frame{f})
w.sz += f.Size()
}
func (w *responseWriter) startSeries(next models.Tags) {
if w.hints.NoSeries() {
return
}
w.ss = len(w.res.Frames)
f := w.getSeriesFrame(next)
@ -69,6 +123,45 @@ func (w *responseWriter) startSeries(next models.Tags) {
w.sz += w.sf.Size()
}
func (w *responseWriter) streamCursor(cur tsdb.Cursor) {
switch {
case w.hints.NoSeries():
// skip
case w.hints.NoPoints():
switch cur := cur.(type) {
case tsdb.IntegerBatchCursor:
w.streamIntegerSeries(cur)
case tsdb.FloatBatchCursor:
w.streamFloatSeries(cur)
case tsdb.UnsignedBatchCursor:
w.streamUnsignedSeries(cur)
case tsdb.BooleanBatchCursor:
w.streamBooleanSeries(cur)
case tsdb.StringBatchCursor:
w.streamStringSeries(cur)
default:
panic(fmt.Sprintf("unreachable: %T", cur))
}
default:
switch cur := cur.(type) {
case tsdb.IntegerBatchCursor:
w.streamIntegerPoints(cur)
case tsdb.FloatBatchCursor:
w.streamFloatPoints(cur)
case tsdb.UnsignedBatchCursor:
w.streamUnsignedPoints(cur)
case tsdb.BooleanBatchCursor:
w.streamBooleanPoints(cur)
case tsdb.StringBatchCursor:
w.streamStringPoints(cur)
default:
panic(fmt.Sprintf("unreachable: %T", cur))
}
}
cur.Close()
}
func (w *responseWriter) flushFrames() {
if w.err != nil || w.sz == 0 {
return
@ -97,6 +190,8 @@ func (w *responseWriter) flushFrames() {
w.putStringPointsFrame(p)
case *ReadResponse_Frame_Series:
w.putSeriesFrame(p)
case *ReadResponse_Frame_Group:
w.putGroupFrame(p)
}
}
w.res.Frames = w.res.Frames[:0]

View File

@ -11,7 +11,7 @@ type readRequest struct {
ctx context.Context
start, end int64
asc bool
limit uint64
limit int64
aggregate *Aggregate
}
@ -19,6 +19,7 @@ type ResultSet struct {
req readRequest
cur seriesCursor
row seriesRow
mb *multiShardBatchCursors
}
func (r *ResultSet) Close() {
@ -38,7 +39,7 @@ func (r *ResultSet) Next() bool {
}
func (r *ResultSet) Cursor() tsdb.Cursor {
cur := newMultiShardBatchCursor(r.req.ctx, r.row, &r.req)
cur := r.mb.createCursor(r.row)
if r.req.aggregate != nil {
cur = newAggregateBatchCursor(r.req.ctx, r.req.aggregate, cur)
}

View File

@ -3,13 +3,12 @@ package storage
import (
"context"
"errors"
"fmt"
"math"
"strings"
"github.com/gogo/protobuf/types"
"github.com/influxdata/influxdb/logger"
"github.com/influxdata/influxdb/pkg/metrics"
"github.com/influxdata/influxdb/tsdb"
"github.com/influxdata/influxdb/tsdb/engine/tsm1"
"github.com/opentracing/opentracing-go"
"github.com/opentracing/opentracing-go/ext"
@ -67,7 +66,7 @@ func (r *rpcService) Read(req *ReadRequest, stream Storage_ReadServer) error {
agg = req.Aggregate.Type
}
pred := truncateString(PredicateToExprString(req.Predicate))
groupKeys := truncateString(strings.Join(req.Grouping, ","))
groupKeys := truncateString(strings.Join(req.GroupKeys, ","))
span.
SetTag("predicate", pred).
SetTag("series_limit", req.SeriesLimit).
@ -76,76 +75,75 @@ func (r *rpcService) Read(req *ReadRequest, stream Storage_ReadServer) error {
SetTag("start", req.TimestampRange.Start).
SetTag("end", req.TimestampRange.End).
SetTag("desc", req.Descending).
SetTag("group", req.Group.String()).
SetTag("group_keys", groupKeys).
SetTag("aggregate", agg.String())
log, logEnd := logger.NewOperation(r.Logger, "Read", "storage_read")
defer logEnd()
ctx = logger.NewContextWithLogger(ctx, log)
if r.loggingEnabled {
r.Logger.Info("request",
log.Info("Read request info",
zap.String("database", req.Database),
zap.String("predicate", pred),
zap.Uint64("series_limit", req.SeriesLimit),
zap.Uint64("series_offset", req.SeriesOffset),
zap.Uint64("points_limit", req.PointsLimit),
zap.String("hints", req.Hints.String()),
zap.Int64("series_limit", req.SeriesLimit),
zap.Int64("series_offset", req.SeriesOffset),
zap.Int64("points_limit", req.PointsLimit),
zap.Int64("start", req.TimestampRange.Start),
zap.Int64("end", req.TimestampRange.End),
zap.Bool("desc", req.Descending),
zap.String("group", req.Group.String()),
zap.String("group_keys", groupKeys),
zap.String("aggregate", agg.String()),
)
}
if req.Hints.NoPoints() {
req.PointsLimit = -1
}
if req.PointsLimit == 0 {
req.PointsLimit = math.MaxUint64
req.PointsLimit = math.MaxInt64
}
rs, err := r.Store.Read(ctx, req)
if err != nil {
r.Logger.Error("Store.Read failed", zap.Error(err))
return err
}
if rs == nil {
return nil
}
defer rs.Close()
w := &responseWriter{
stream: stream,
res: &ReadResponse{Frames: make([]ReadResponse_Frame, 0, frameCount)},
logger: r.Logger,
logger: log,
hints: req.Hints,
}
for rs.Next() {
cur := rs.Cursor()
if cur == nil {
// no data for series key + field combination
continue
switch req.Group {
case GroupBy, GroupExcept:
if len(req.GroupKeys) == 0 {
return errors.New("read: GroupKeys must not be empty when GroupBy or GroupExcept specified")
}
w.startSeries(rs.Tags())
switch cur := cur.(type) {
case tsdb.IntegerBatchCursor:
w.streamIntegerPoints(cur)
case tsdb.FloatBatchCursor:
w.streamFloatPoints(cur)
case tsdb.UnsignedBatchCursor:
w.streamUnsignedPoints(cur)
case tsdb.BooleanBatchCursor:
w.streamBooleanPoints(cur)
case tsdb.StringBatchCursor:
w.streamStringPoints(cur)
default:
panic(fmt.Sprintf("unreachable: %T", cur))
case GroupNone, GroupAll:
if len(req.GroupKeys) > 0 {
return errors.New("read: GroupKeys must be empty when GroupNone or GroupAll specified")
}
default:
return errors.New("read: unexpected value for Group")
}
if w.err != nil {
return w.err
}
if req.Group == GroupAll {
err = r.handleRead(ctx, req, w)
} else {
err = r.handleGroupRead(ctx, req, w)
}
if err != nil {
log.Error("Read failed", zap.Error(err))
}
w.flushFrames()
if r.loggingEnabled {
log.Info("Read completed", zap.Int("num_values", w.vc))
}
span.SetTag("num_values", w.vc)
grp := tsm1.MetricsGroupFromContext(ctx)
grp.ForEach(func(v metrics.Metric) {
@ -157,3 +155,68 @@ func (r *rpcService) Read(req *ReadRequest, stream Storage_ReadServer) error {
return nil
}
func (r *rpcService) handleRead(ctx context.Context, req *ReadRequest, w *responseWriter) error {
rs, err := r.Store.Read(ctx, req)
if err != nil {
return err
}
if rs == nil {
return nil
}
defer rs.Close()
for rs.Next() {
cur := rs.Cursor()
if cur == nil {
// no data for series key + field combination
continue
}
w.startSeries(rs.Tags())
w.streamCursor(cur)
if w.err != nil {
cur.Close()
return w.err
}
}
return nil
}
func (r *rpcService) handleGroupRead(ctx context.Context, req *ReadRequest, w *responseWriter) error {
rs, err := r.Store.GroupRead(ctx, req)
if err != nil {
return err
}
if rs == nil {
return nil
}
defer rs.Close()
gc := rs.Next()
for gc != nil {
w.startGroup(gc.Keys(), gc.PartitionKeyVals())
if !req.Hints.HintSchemaAllTime() {
for gc.Next() {
cur := gc.Cursor()
if cur == nil {
// no data for series key + field combination
continue
}
w.startSeries(gc.Tags())
w.streamCursor(cur)
if w.err != nil {
gc.Close()
return w.err
}
}
}
gc.Close()
gc = rs.Next()
}
return nil
}

View File

@ -1,7 +1,6 @@
package storage
import (
"bytes"
"context"
"errors"
"sort"
@ -25,9 +24,10 @@ type seriesCursor interface {
}
type seriesRow struct {
sortKey []byte
name []byte // measurement name
stags models.Tags // unmodified series tags
field string
field field
tags models.Tags
query tsdb.CursorIterators
valueCond influxql.Expr
@ -44,21 +44,19 @@ func (vs mapValuer) Value(key string) (interface{}, bool) {
type indexSeriesCursor struct {
sqry tsdb.SeriesCursor
fields []string
nf []string
fields measurementFields
nf []field
err error
tags models.Tags
filterset mapValuer
cond influxql.Expr
measurementCond influxql.Expr
row seriesRow
eof bool
hasFieldExpr bool
hasValueExpr bool
multiTenant bool
}
func newIndexSeriesCursor(ctx context.Context, req *ReadRequest, shards []*tsdb.Shard) (*indexSeriesCursor, error) {
func newIndexSeriesCursor(ctx context.Context, predicate *Predicate, shards []*tsdb.Shard) (*indexSeriesCursor, error) {
queries, err := tsdb.CreateCursorIterators(ctx, shards)
if err != nil {
return nil, err
@ -82,22 +80,8 @@ func newIndexSeriesCursor(ctx context.Context, req *ReadRequest, shards []*tsdb.
}
p := &indexSeriesCursor{row: seriesRow{query: queries}}
var (
remap map[string]string
mi tsdb.MeasurementIterator
)
if req.RequestType == ReadRequestTypeMultiTenant {
p.multiTenant = true
m := []byte(req.OrgID)
m = append(m, 0, 0)
m = append(m, req.Database...)
mi = tsdb.NewMeasurementSliceIterator([][]byte{m})
} else {
remap = measurementRemap
}
if root := req.Predicate.GetRoot(); root != nil {
if p.cond, err = NodeToExpr(root, remap); err != nil {
if root := predicate.GetRoot(); root != nil {
if p.cond, err = NodeToExpr(root, measurementRemap); err != nil {
return nil, err
}
@ -119,7 +103,7 @@ func newIndexSeriesCursor(ctx context.Context, req *ReadRequest, shards []*tsdb.
}
sg := tsdb.Shards(shards)
p.sqry, err = sg.CreateSeriesCursor(ctx, tsdb.SeriesCursorRequest{Measurements: mi}, opt.Condition)
p.sqry, err = sg.CreateSeriesCursor(ctx, tsdb.SeriesCursorRequest{}, opt.Condition)
if p.sqry != nil && err == nil {
var (
itr query.Iterator
@ -182,32 +166,22 @@ RETRY:
c.row.name = sr.Name
c.row.stags = sr.Tags
c.tags = copyTags(c.tags, sr.Tags)
c.tags.Set(measurementKey, sr.Name)
c.filterset = make(mapValuer)
for _, tag := range c.tags {
c.filterset[string(tag.Key)] = string(tag.Value)
}
if !c.multiTenant {
c.filterset["_name"] = string(sr.Name)
c.tags.Set(measurementKey, sr.Name)
}
c.nf = c.fields
c.nf = c.fields[string(sr.Name)]
}
c.row.field, c.nf = c.nf[0], c.nf[1:]
c.filterset["_field"] = c.row.field
if c.measurementCond != nil && !evalExprBool(c.measurementCond, c.filterset) {
if c.measurementCond != nil && !evalExprBool(c.measurementCond, c) {
goto RETRY
}
c.tags.Set(fieldKey, []byte(c.row.field))
c.tags.Set(fieldKey, c.row.field.nb)
if c.cond != nil && c.hasValueExpr {
// TODO(sgc): lazily evaluate valueCond
c.row.valueCond = influxql.Reduce(c.cond, c.filterset)
c.row.valueCond = influxql.Reduce(c.cond, c)
if isBooleanLiteral(c.row.valueCond) {
// we've reduced the expression to "true"
c.row.valueCond = nil
@ -219,22 +193,34 @@ RETRY:
return &c.row
}
func (c *indexSeriesCursor) Value(key string) (interface{}, bool) {
switch key {
case "_name":
return c.row.name, true
case "_field":
return c.row.field.n, true
default:
res := c.row.stags.Get([]byte(key))
return res, res != nil
}
}
func (c *indexSeriesCursor) Err() error {
return c.err
}
type limitSeriesCursor struct {
seriesCursor
n, o, c uint64
n, o, c int64
}
func newLimitSeriesCursor(ctx context.Context, cur seriesCursor, n, o uint64) *limitSeriesCursor {
func newLimitSeriesCursor(ctx context.Context, cur seriesCursor, n, o int64) *limitSeriesCursor {
return &limitSeriesCursor{seriesCursor: cur, o: o, n: n}
}
func (c *limitSeriesCursor) Next() *seriesRow {
if c.o > 0 {
for i := uint64(0); i < c.o; i++ {
for i := int64(0); i < c.o; i++ {
if c.seriesCursor.Next() == nil {
break
}
@ -249,78 +235,6 @@ func (c *limitSeriesCursor) Next() *seriesRow {
return c.seriesCursor.Next()
}
type groupSeriesCursor struct {
seriesCursor
ctx context.Context
rows []seriesRow
keys [][]byte
f bool
}
func newGroupSeriesCursor(ctx context.Context, cur seriesCursor, keys []string) *groupSeriesCursor {
g := &groupSeriesCursor{seriesCursor: cur, ctx: ctx}
g.keys = make([][]byte, 0, len(keys))
for _, k := range keys {
g.keys = append(g.keys, []byte(k))
}
return g
}
func (c *groupSeriesCursor) Next() *seriesRow {
if !c.f {
c.sort()
}
if len(c.rows) > 0 {
row := &c.rows[0]
c.rows = c.rows[1:]
return row
}
return nil
}
func (c *groupSeriesCursor) sort() {
span := opentracing.SpanFromContext(c.ctx)
if span != nil {
span = opentracing.StartSpan("group_series_cursor.sort", opentracing.ChildOf(span.Context()))
defer span.Finish()
}
var rows []seriesRow
row := c.seriesCursor.Next()
for row != nil {
rows = append(rows, *row)
row = c.seriesCursor.Next()
}
sort.Slice(rows, func(i, j int) bool {
for _, k := range c.keys {
ik := rows[i].tags.Get(k)
jk := rows[j].tags.Get(k)
cmp := bytes.Compare(ik, jk)
if cmp == 0 {
continue
}
return cmp == -1
}
return false
})
if span != nil {
span.SetTag("rows", len(rows))
}
c.rows = rows
// free early
c.seriesCursor.Close()
c.f = true
}
func isBooleanLiteral(expr influxql.Expr) bool {
_, ok := expr.(*influxql.BooleanLiteral)
return ok
@ -335,31 +249,56 @@ func toFloatIterator(iter query.Iterator) (query.FloatIterator, error) {
return sitr, nil
}
func extractFields(itr query.FloatIterator) []string {
var a []string
type measurementFields map[string][]field
type field struct {
n string
nb []byte
d influxql.DataType
}
func extractFields(itr query.FloatIterator) measurementFields {
mf := make(measurementFields)
for {
p, err := itr.Next()
if err != nil {
return nil
} else if p == nil {
break
} else if f, ok := p.Aux[0].(string); ok {
a = append(a, f)
}
// Aux is populated by `fieldKeysIterator#Next`
fields := append(mf[p.Name], field{
n: p.Aux[0].(string),
d: influxql.DataTypeFromString(p.Aux[1].(string)),
})
mf[p.Name] = fields
}
if len(a) == 0 {
return a
if len(mf) == 0 {
return nil
}
sort.Strings(a)
i := 1
for j := 1; j < len(a); j++ {
if a[j] != a[j-1] {
a[i] = a[j]
i++
for k, fields := range mf {
sort.Slice(fields, func(i, j int) bool {
return fields[i].n < fields[j].n
})
// deduplicate
i := 1
fields[0].nb = []byte(fields[0].n)
for j := 1; j < len(fields); j++ {
if fields[j].n != fields[j-1].n {
fields[i] = fields[j]
fields[i].nb = []byte(fields[i].n)
i++
}
}
mf[k] = fields[:i]
}
return a[:i]
return mf
}

View File

@ -22,9 +22,18 @@ func TestPlannerCondition(t *testing.T) {
t.Fatal("ParseExpr", err)
}
makeFields := func(s ...string) []field {
f := make([]field, len(s))
for i := range s {
f[i].n = s[i]
f[i].nb = []byte(s[i])
}
return f
}
p := &indexSeriesCursor{
sqry: sqry,
fields: []string{"user", "system", "val"},
fields: measurementFields{"cpu": makeFields("system", "user"), "mem": makeFields("val")},
cond: cond,
measurementCond: influxql.Reduce(RewriteExprRemoveFieldValue(influxql.CloneExpr(cond)), nil),
}
@ -32,13 +41,13 @@ func TestPlannerCondition(t *testing.T) {
var keys []string
row := p.Next()
for row != nil {
keys = append(keys, string(models.MakeKey(row.name, row.stags))+" "+row.field)
keys = append(keys, string(models.MakeKey(row.name, row.stags))+" "+row.field.n)
row = p.Next()
}
exp := []string{"cpu,host=host1 user", "cpu,host=host1 system", "mem,host=host1 user", "mem,host=host1 system", "mem,host=host1 val"}
if !cmp.Equal(exp, keys) {
t.Errorf("unexpected, %s", cmp.Diff(exp, keys))
exp := []string{"cpu,host=host1 system", "cpu,host=host1 user", "mem,host=host1 val"}
if !cmp.Equal(keys, exp) {
t.Errorf("unexpected -got/+want\n%s", cmp.Diff(keys, exp))
}
}

View File

@ -0,0 +1,55 @@
package storage
import (
"strings"
"github.com/gogo/protobuf/proto"
)
type HintFlags uint32
func (h HintFlags) NoPoints() bool {
return uint32(h)&uint32(HintNoPoints) != 0
}
func (h *HintFlags) SetNoPoints() {
*h |= HintFlags(HintNoPoints)
}
func (h HintFlags) NoSeries() bool {
return uint32(h)&uint32(HintNoSeries) != 0
}
func (h *HintFlags) SetNoSeries() {
*h |= HintFlags(HintNoSeries)
}
func (h HintFlags) HintSchemaAllTime() bool {
return uint32(h)&uint32(HintSchemaAllTime) != 0
}
func (h *HintFlags) SetHintSchemaAllTime() {
*h |= HintFlags(HintSchemaAllTime)
}
func (h HintFlags) String() string {
f := uint32(h)
var s []string
enums := proto.EnumValueMap("com.github.influxdata.influxdb.services.storage.ReadRequest_HintFlags")
if h == 0 {
return "HINT_NONE"
}
for k, v := range enums {
if v == 0 {
continue
}
v := uint32(v)
if f&v == v {
s = append(s, k)
}
}
return strings.Join(s, ",")
}

View File

@ -41,26 +41,68 @@ var _ = math.Inf
// proto package needs to be updated.
const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package
type ReadRequest_Type int32
type ReadRequest_Group int32
const (
ReadRequestTypeSingleTenant ReadRequest_Type = 0
ReadRequestTypeMultiTenant ReadRequest_Type = 1
// GroupNone returns all series as a single group.
// The single GroupFrame.TagKeys will be the union of all tag keys.
GroupNone ReadRequest_Group = 0
// GroupAll returns a unique group for each series.
// As an optimization, no GroupFrames will be generated.
GroupAll ReadRequest_Group = 1
// GroupBy returns a group for each unique value of the specified GroupKeys.
GroupBy ReadRequest_Group = 2
// GroupExcept in not implemented.
GroupExcept ReadRequest_Group = 3
)
var ReadRequest_Type_name = map[int32]string{
0: "SINGLE_TENANT",
1: "MULTI_TENANT",
var ReadRequest_Group_name = map[int32]string{
0: "GROUP_NONE",
1: "GROUP_ALL",
2: "GROUP_BY",
3: "GROUP_EXCEPT",
}
var ReadRequest_Type_value = map[string]int32{
"SINGLE_TENANT": 0,
"MULTI_TENANT": 1,
var ReadRequest_Group_value = map[string]int32{
"GROUP_NONE": 0,
"GROUP_ALL": 1,
"GROUP_BY": 2,
"GROUP_EXCEPT": 3,
}
func (x ReadRequest_Type) String() string {
return proto.EnumName(ReadRequest_Type_name, int32(x))
func (x ReadRequest_Group) String() string {
return proto.EnumName(ReadRequest_Group_name, int32(x))
}
func (ReadRequest_Group) EnumDescriptor() ([]byte, []int) { return fileDescriptorStorage, []int{0, 0} }
type ReadRequest_HintFlags int32
const (
HintNone ReadRequest_HintFlags = 0
HintNoPoints ReadRequest_HintFlags = 1
HintNoSeries ReadRequest_HintFlags = 2
// HintSchemaAllTime performs schema queries without using time ranges
HintSchemaAllTime ReadRequest_HintFlags = 4
)
var ReadRequest_HintFlags_name = map[int32]string{
0: "HINT_NONE",
1: "HINT_NO_POINTS",
2: "HINT_NO_SERIES",
4: "HINT_SCHEMA_ALL_TIME",
}
var ReadRequest_HintFlags_value = map[string]int32{
"HINT_NONE": 0,
"HINT_NO_POINTS": 1,
"HINT_NO_SERIES": 2,
"HINT_SCHEMA_ALL_TIME": 4,
}
func (x ReadRequest_HintFlags) String() string {
return proto.EnumName(ReadRequest_HintFlags_name, int32(x))
}
func (ReadRequest_HintFlags) EnumDescriptor() ([]byte, []int) {
return fileDescriptorStorage, []int{0, 1}
}
func (ReadRequest_Type) EnumDescriptor() ([]byte, []int) { return fileDescriptorStorage, []int{0, 0} }
type Aggregate_AggregateType int32
@ -145,30 +187,32 @@ func (ReadResponse_DataType) EnumDescriptor() ([]byte, []int) {
// Request message for Storage.Read.
type ReadRequest struct {
// RequestType specifies the request type as either single or multi tenant.
RequestType ReadRequest_Type `protobuf:"varint,11,opt,name=request_type,json=requestType,proto3,enum=storage.ReadRequest_Type" json:"request_type,omitempty"`
// OrgID specifies the organization identifier for this request.
OrgID string `protobuf:"bytes,12,opt,name=org_id,json=orgId,proto3" json:"org_id,omitempty"`
// Database specifies the database name (single tenant) or bucket identifier (multi tenant).
Database string `protobuf:"bytes,1,opt,name=database,proto3" json:"database,omitempty"`
TimestampRange TimestampRange `protobuf:"bytes,2,opt,name=timestamp_range,json=timestampRange" json:"timestamp_range"`
// Descending indicates whether points should be returned in descending order.
Descending bool `protobuf:"varint,3,opt,name=descending,proto3" json:"descending,omitempty"`
// Grouping specifies a list of tags used to order the data
Grouping []string `protobuf:"bytes,4,rep,name=grouping" json:"grouping,omitempty"`
// GroupKeys specifies a list of tag keys used to order the data. It is dependent on the Group property to determine
// its behavior.
GroupKeys []string `protobuf:"bytes,4,rep,name=group_keys,json=groupKeys" json:"group_keys,omitempty"`
//
Group ReadRequest_Group `protobuf:"varint,11,opt,name=group,proto3,enum=com.github.influxdata.influxdb.services.storage.ReadRequest_Group" json:"group,omitempty"`
// Aggregate specifies an optional aggregate to apply to the data.
// TODO(sgc): switch to slice for multiple aggregates in a single request
Aggregate *Aggregate `protobuf:"bytes,9,opt,name=aggregate" json:"aggregate,omitempty"`
Predicate *Predicate `protobuf:"bytes,5,opt,name=predicate" json:"predicate,omitempty"`
// SeriesLimit determines the maximum number of series to be returned for the request. Specify 0 for no limit.
SeriesLimit uint64 `protobuf:"varint,6,opt,name=series_limit,json=seriesLimit,proto3" json:"series_limit,omitempty"`
SeriesLimit int64 `protobuf:"varint,6,opt,name=series_limit,json=seriesLimit,proto3" json:"series_limit,omitempty"`
// SeriesOffset determines how many series to skip before processing the request.
SeriesOffset uint64 `protobuf:"varint,7,opt,name=series_offset,json=seriesOffset,proto3" json:"series_offset,omitempty"`
SeriesOffset int64 `protobuf:"varint,7,opt,name=series_offset,json=seriesOffset,proto3" json:"series_offset,omitempty"`
// PointsLimit determines the maximum number of values per series to be returned for the request.
// Specify 0 for no limit.
PointsLimit uint64 `protobuf:"varint,8,opt,name=points_limit,json=pointsLimit,proto3" json:"points_limit,omitempty"`
// Specify 0 for no limit. -1 to return series frames only.
PointsLimit int64 `protobuf:"varint,8,opt,name=points_limit,json=pointsLimit,proto3" json:"points_limit,omitempty"`
// Trace contains opaque data if a trace is active.
Trace map[string]string `protobuf:"bytes,10,rep,name=trace" json:"trace,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"`
// Hints is a bitwise OR of HintFlags to control the behavior
// of the read request.
Hints HintFlags `protobuf:"fixed32,12,opt,name=hints,proto3,casttype=HintFlags" json:"hints,omitempty"`
}
func (m *ReadRequest) Reset() { *m = ReadRequest{} }
@ -177,7 +221,7 @@ func (*ReadRequest) ProtoMessage() {}
func (*ReadRequest) Descriptor() ([]byte, []int) { return fileDescriptorStorage, []int{0} }
type Aggregate struct {
Type Aggregate_AggregateType `protobuf:"varint,1,opt,name=type,proto3,enum=storage.Aggregate_AggregateType" json:"type,omitempty"`
Type Aggregate_AggregateType `protobuf:"varint,1,opt,name=type,proto3,enum=com.github.influxdata.influxdb.services.storage.Aggregate_AggregateType" json:"type,omitempty"`
}
func (m *Aggregate) Reset() { *m = Aggregate{} }
@ -207,6 +251,7 @@ func (*ReadResponse) Descriptor() ([]byte, []int) { return fileDescriptorStorage
type ReadResponse_Frame struct {
// Types that are valid to be assigned to Data:
// *ReadResponse_Frame_Group
// *ReadResponse_Frame_Series
// *ReadResponse_Frame_FloatPoints
// *ReadResponse_Frame_IntegerPoints
@ -227,6 +272,9 @@ type isReadResponse_Frame_Data interface {
Size() int
}
type ReadResponse_Frame_Group struct {
Group *ReadResponse_GroupFrame `protobuf:"bytes,7,opt,name=group,oneof"`
}
type ReadResponse_Frame_Series struct {
Series *ReadResponse_SeriesFrame `protobuf:"bytes,1,opt,name=series,oneof"`
}
@ -246,6 +294,7 @@ type ReadResponse_Frame_StringPoints struct {
StringPoints *ReadResponse_StringPointsFrame `protobuf:"bytes,6,opt,name=string_points,json=stringPoints,oneof"`
}
func (*ReadResponse_Frame_Group) isReadResponse_Frame_Data() {}
func (*ReadResponse_Frame_Series) isReadResponse_Frame_Data() {}
func (*ReadResponse_Frame_FloatPoints) isReadResponse_Frame_Data() {}
func (*ReadResponse_Frame_IntegerPoints) isReadResponse_Frame_Data() {}
@ -260,6 +309,13 @@ func (m *ReadResponse_Frame) GetData() isReadResponse_Frame_Data {
return nil
}
func (m *ReadResponse_Frame) GetGroup() *ReadResponse_GroupFrame {
if x, ok := m.GetData().(*ReadResponse_Frame_Group); ok {
return x.Group
}
return nil
}
func (m *ReadResponse_Frame) GetSeries() *ReadResponse_SeriesFrame {
if x, ok := m.GetData().(*ReadResponse_Frame_Series); ok {
return x.Series
@ -305,6 +361,7 @@ func (m *ReadResponse_Frame) GetStringPoints() *ReadResponse_StringPointsFrame {
// XXX_OneofFuncs is for the internal use of the proto package.
func (*ReadResponse_Frame) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) error, func(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error), func(msg proto.Message) (n int), []interface{}) {
return _ReadResponse_Frame_OneofMarshaler, _ReadResponse_Frame_OneofUnmarshaler, _ReadResponse_Frame_OneofSizer, []interface{}{
(*ReadResponse_Frame_Group)(nil),
(*ReadResponse_Frame_Series)(nil),
(*ReadResponse_Frame_FloatPoints)(nil),
(*ReadResponse_Frame_IntegerPoints)(nil),
@ -318,6 +375,11 @@ func _ReadResponse_Frame_OneofMarshaler(msg proto.Message, b *proto.Buffer) erro
m := msg.(*ReadResponse_Frame)
// data
switch x := m.Data.(type) {
case *ReadResponse_Frame_Group:
_ = b.EncodeVarint(7<<3 | proto.WireBytes)
if err := b.EncodeMessage(x.Group); err != nil {
return err
}
case *ReadResponse_Frame_Series:
_ = b.EncodeVarint(1<<3 | proto.WireBytes)
if err := b.EncodeMessage(x.Series); err != nil {
@ -358,6 +420,14 @@ func _ReadResponse_Frame_OneofMarshaler(msg proto.Message, b *proto.Buffer) erro
func _ReadResponse_Frame_OneofUnmarshaler(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error) {
m := msg.(*ReadResponse_Frame)
switch tag {
case 7: // data.group
if wire != proto.WireBytes {
return true, proto.ErrInternalBadWireType
}
msg := new(ReadResponse_GroupFrame)
err := b.DecodeMessage(msg)
m.Data = &ReadResponse_Frame_Group{msg}
return true, err
case 1: // data.series
if wire != proto.WireBytes {
return true, proto.ErrInternalBadWireType
@ -415,6 +485,11 @@ func _ReadResponse_Frame_OneofSizer(msg proto.Message) (n int) {
m := msg.(*ReadResponse_Frame)
// data
switch x := m.Data.(type) {
case *ReadResponse_Frame_Group:
s := proto.Size(x.Group)
n += proto.SizeVarint(7<<3 | proto.WireBytes)
n += proto.SizeVarint(uint64(s))
n += s
case *ReadResponse_Frame_Series:
s := proto.Size(x.Series)
n += proto.SizeVarint(1<<3 | proto.WireBytes)
@ -452,16 +527,30 @@ func _ReadResponse_Frame_OneofSizer(msg proto.Message) (n int) {
return n
}
type ReadResponse_GroupFrame struct {
// TagKeys
TagKeys [][]byte `protobuf:"bytes,1,rep,name=tag_keys,json=tagKeys" json:"tag_keys,omitempty"`
// PartitionKeyVals is the values of the partition key for this group, order matching ReadRequest.GroupKeys
PartitionKeyVals [][]byte `protobuf:"bytes,2,rep,name=partition_key_vals,json=partitionKeyVals" json:"partition_key_vals,omitempty"`
}
func (m *ReadResponse_GroupFrame) Reset() { *m = ReadResponse_GroupFrame{} }
func (m *ReadResponse_GroupFrame) String() string { return proto.CompactTextString(m) }
func (*ReadResponse_GroupFrame) ProtoMessage() {}
func (*ReadResponse_GroupFrame) Descriptor() ([]byte, []int) {
return fileDescriptorStorage, []int{3, 1}
}
type ReadResponse_SeriesFrame struct {
Tags []Tag `protobuf:"bytes,1,rep,name=tags" json:"tags"`
DataType ReadResponse_DataType `protobuf:"varint,2,opt,name=data_type,json=dataType,proto3,enum=storage.ReadResponse_DataType" json:"data_type,omitempty"`
DataType ReadResponse_DataType `protobuf:"varint,2,opt,name=data_type,json=dataType,proto3,enum=com.github.influxdata.influxdb.services.storage.ReadResponse_DataType" json:"data_type,omitempty"`
}
func (m *ReadResponse_SeriesFrame) Reset() { *m = ReadResponse_SeriesFrame{} }
func (m *ReadResponse_SeriesFrame) String() string { return proto.CompactTextString(m) }
func (*ReadResponse_SeriesFrame) ProtoMessage() {}
func (*ReadResponse_SeriesFrame) Descriptor() ([]byte, []int) {
return fileDescriptorStorage, []int{3, 1}
return fileDescriptorStorage, []int{3, 2}
}
type ReadResponse_FloatPointsFrame struct {
@ -473,7 +562,7 @@ func (m *ReadResponse_FloatPointsFrame) Reset() { *m = ReadResponse_Floa
func (m *ReadResponse_FloatPointsFrame) String() string { return proto.CompactTextString(m) }
func (*ReadResponse_FloatPointsFrame) ProtoMessage() {}
func (*ReadResponse_FloatPointsFrame) Descriptor() ([]byte, []int) {
return fileDescriptorStorage, []int{3, 2}
return fileDescriptorStorage, []int{3, 3}
}
type ReadResponse_IntegerPointsFrame struct {
@ -485,7 +574,7 @@ func (m *ReadResponse_IntegerPointsFrame) Reset() { *m = ReadResponse_In
func (m *ReadResponse_IntegerPointsFrame) String() string { return proto.CompactTextString(m) }
func (*ReadResponse_IntegerPointsFrame) ProtoMessage() {}
func (*ReadResponse_IntegerPointsFrame) Descriptor() ([]byte, []int) {
return fileDescriptorStorage, []int{3, 3}
return fileDescriptorStorage, []int{3, 4}
}
type ReadResponse_UnsignedPointsFrame struct {
@ -497,7 +586,7 @@ func (m *ReadResponse_UnsignedPointsFrame) Reset() { *m = ReadResponse_U
func (m *ReadResponse_UnsignedPointsFrame) String() string { return proto.CompactTextString(m) }
func (*ReadResponse_UnsignedPointsFrame) ProtoMessage() {}
func (*ReadResponse_UnsignedPointsFrame) Descriptor() ([]byte, []int) {
return fileDescriptorStorage, []int{3, 4}
return fileDescriptorStorage, []int{3, 5}
}
type ReadResponse_BooleanPointsFrame struct {
@ -509,7 +598,7 @@ func (m *ReadResponse_BooleanPointsFrame) Reset() { *m = ReadResponse_Bo
func (m *ReadResponse_BooleanPointsFrame) String() string { return proto.CompactTextString(m) }
func (*ReadResponse_BooleanPointsFrame) ProtoMessage() {}
func (*ReadResponse_BooleanPointsFrame) Descriptor() ([]byte, []int) {
return fileDescriptorStorage, []int{3, 5}
return fileDescriptorStorage, []int{3, 6}
}
type ReadResponse_StringPointsFrame struct {
@ -521,7 +610,7 @@ func (m *ReadResponse_StringPointsFrame) Reset() { *m = ReadResponse_Str
func (m *ReadResponse_StringPointsFrame) String() string { return proto.CompactTextString(m) }
func (*ReadResponse_StringPointsFrame) ProtoMessage() {}
func (*ReadResponse_StringPointsFrame) Descriptor() ([]byte, []int) {
return fileDescriptorStorage, []int{3, 6}
return fileDescriptorStorage, []int{3, 7}
}
type CapabilitiesResponse struct {
@ -555,24 +644,26 @@ func (*TimestampRange) ProtoMessage() {}
func (*TimestampRange) Descriptor() ([]byte, []int) { return fileDescriptorStorage, []int{6} }
func init() {
proto.RegisterType((*ReadRequest)(nil), "storage.ReadRequest")
proto.RegisterType((*Aggregate)(nil), "storage.Aggregate")
proto.RegisterType((*Tag)(nil), "storage.Tag")
proto.RegisterType((*ReadResponse)(nil), "storage.ReadResponse")
proto.RegisterType((*ReadResponse_Frame)(nil), "storage.ReadResponse.Frame")
proto.RegisterType((*ReadResponse_SeriesFrame)(nil), "storage.ReadResponse.SeriesFrame")
proto.RegisterType((*ReadResponse_FloatPointsFrame)(nil), "storage.ReadResponse.FloatPointsFrame")
proto.RegisterType((*ReadResponse_IntegerPointsFrame)(nil), "storage.ReadResponse.IntegerPointsFrame")
proto.RegisterType((*ReadResponse_UnsignedPointsFrame)(nil), "storage.ReadResponse.UnsignedPointsFrame")
proto.RegisterType((*ReadResponse_BooleanPointsFrame)(nil), "storage.ReadResponse.BooleanPointsFrame")
proto.RegisterType((*ReadResponse_StringPointsFrame)(nil), "storage.ReadResponse.StringPointsFrame")
proto.RegisterType((*CapabilitiesResponse)(nil), "storage.CapabilitiesResponse")
proto.RegisterType((*HintsResponse)(nil), "storage.HintsResponse")
proto.RegisterType((*TimestampRange)(nil), "storage.TimestampRange")
proto.RegisterEnum("storage.ReadRequest_Type", ReadRequest_Type_name, ReadRequest_Type_value)
proto.RegisterEnum("storage.Aggregate_AggregateType", Aggregate_AggregateType_name, Aggregate_AggregateType_value)
proto.RegisterEnum("storage.ReadResponse_FrameType", ReadResponse_FrameType_name, ReadResponse_FrameType_value)
proto.RegisterEnum("storage.ReadResponse_DataType", ReadResponse_DataType_name, ReadResponse_DataType_value)
proto.RegisterType((*ReadRequest)(nil), "com.github.influxdata.influxdb.services.storage.ReadRequest")
proto.RegisterType((*Aggregate)(nil), "com.github.influxdata.influxdb.services.storage.Aggregate")
proto.RegisterType((*Tag)(nil), "com.github.influxdata.influxdb.services.storage.Tag")
proto.RegisterType((*ReadResponse)(nil), "com.github.influxdata.influxdb.services.storage.ReadResponse")
proto.RegisterType((*ReadResponse_Frame)(nil), "com.github.influxdata.influxdb.services.storage.ReadResponse.Frame")
proto.RegisterType((*ReadResponse_GroupFrame)(nil), "com.github.influxdata.influxdb.services.storage.ReadResponse.GroupFrame")
proto.RegisterType((*ReadResponse_SeriesFrame)(nil), "com.github.influxdata.influxdb.services.storage.ReadResponse.SeriesFrame")
proto.RegisterType((*ReadResponse_FloatPointsFrame)(nil), "com.github.influxdata.influxdb.services.storage.ReadResponse.FloatPointsFrame")
proto.RegisterType((*ReadResponse_IntegerPointsFrame)(nil), "com.github.influxdata.influxdb.services.storage.ReadResponse.IntegerPointsFrame")
proto.RegisterType((*ReadResponse_UnsignedPointsFrame)(nil), "com.github.influxdata.influxdb.services.storage.ReadResponse.UnsignedPointsFrame")
proto.RegisterType((*ReadResponse_BooleanPointsFrame)(nil), "com.github.influxdata.influxdb.services.storage.ReadResponse.BooleanPointsFrame")
proto.RegisterType((*ReadResponse_StringPointsFrame)(nil), "com.github.influxdata.influxdb.services.storage.ReadResponse.StringPointsFrame")
proto.RegisterType((*CapabilitiesResponse)(nil), "com.github.influxdata.influxdb.services.storage.CapabilitiesResponse")
proto.RegisterType((*HintsResponse)(nil), "com.github.influxdata.influxdb.services.storage.HintsResponse")
proto.RegisterType((*TimestampRange)(nil), "com.github.influxdata.influxdb.services.storage.TimestampRange")
proto.RegisterEnum("com.github.influxdata.influxdb.services.storage.ReadRequest_Group", ReadRequest_Group_name, ReadRequest_Group_value)
proto.RegisterEnum("com.github.influxdata.influxdb.services.storage.ReadRequest_HintFlags", ReadRequest_HintFlags_name, ReadRequest_HintFlags_value)
proto.RegisterEnum("com.github.influxdata.influxdb.services.storage.Aggregate_AggregateType", Aggregate_AggregateType_name, Aggregate_AggregateType_value)
proto.RegisterEnum("com.github.influxdata.influxdb.services.storage.ReadResponse_FrameType", ReadResponse_FrameType_name, ReadResponse_FrameType_value)
proto.RegisterEnum("com.github.influxdata.influxdb.services.storage.ReadResponse_DataType", ReadResponse_DataType_name, ReadResponse_DataType_value)
}
func (m *ReadRequest) Marshal() (dAtA []byte, err error) {
size := m.Size()
@ -613,8 +704,8 @@ func (m *ReadRequest) MarshalTo(dAtA []byte) (int, error) {
}
i++
}
if len(m.Grouping) > 0 {
for _, s := range m.Grouping {
if len(m.GroupKeys) > 0 {
for _, s := range m.GroupKeys {
dAtA[i] = 0x22
i++
l = len(s)
@ -680,16 +771,15 @@ func (m *ReadRequest) MarshalTo(dAtA []byte) (int, error) {
i += copy(dAtA[i:], v)
}
}
if m.RequestType != 0 {
if m.Group != 0 {
dAtA[i] = 0x58
i++
i = encodeVarintStorage(dAtA, i, uint64(m.RequestType))
i = encodeVarintStorage(dAtA, i, uint64(m.Group))
}
if len(m.OrgID) > 0 {
dAtA[i] = 0x62
if m.Hints != 0 {
dAtA[i] = 0x65
i++
i = encodeVarintStorage(dAtA, i, uint64(len(m.OrgID)))
i += copy(dAtA[i:], m.OrgID)
i = encodeFixed32Storage(dAtA, i, uint32(m.Hints))
}
return i, nil
}
@ -886,6 +976,54 @@ func (m *ReadResponse_Frame_StringPoints) MarshalTo(dAtA []byte) (int, error) {
}
return i, nil
}
func (m *ReadResponse_Frame_Group) MarshalTo(dAtA []byte) (int, error) {
i := 0
if m.Group != nil {
dAtA[i] = 0x3a
i++
i = encodeVarintStorage(dAtA, i, uint64(m.Group.Size()))
n11, err := m.Group.MarshalTo(dAtA[i:])
if err != nil {
return 0, err
}
i += n11
}
return i, nil
}
func (m *ReadResponse_GroupFrame) Marshal() (dAtA []byte, err error) {
size := m.Size()
dAtA = make([]byte, size)
n, err := m.MarshalTo(dAtA)
if err != nil {
return nil, err
}
return dAtA[:n], nil
}
func (m *ReadResponse_GroupFrame) MarshalTo(dAtA []byte) (int, error) {
var i int
_ = i
var l int
_ = l
if len(m.TagKeys) > 0 {
for _, b := range m.TagKeys {
dAtA[i] = 0xa
i++
i = encodeVarintStorage(dAtA, i, uint64(len(b)))
i += copy(dAtA[i:], b)
}
}
if len(m.PartitionKeyVals) > 0 {
for _, b := range m.PartitionKeyVals {
dAtA[i] = 0x12
i++
i = encodeVarintStorage(dAtA, i, uint64(len(b)))
i += copy(dAtA[i:], b)
}
}
return i, nil
}
func (m *ReadResponse_SeriesFrame) Marshal() (dAtA []byte, err error) {
size := m.Size()
dAtA = make([]byte, size)
@ -964,22 +1102,22 @@ func (m *ReadResponse_FloatPointsFrame) MarshalTo(dAtA []byte) (int, error) {
i++
i = encodeVarintStorage(dAtA, i, uint64(len(m.Values)*8))
for _, num := range m.Values {
f11 := math.Float64bits(float64(num))
dAtA[i] = uint8(f11)
f12 := math.Float64bits(float64(num))
dAtA[i] = uint8(f12)
i++
dAtA[i] = uint8(f11 >> 8)
dAtA[i] = uint8(f12 >> 8)
i++
dAtA[i] = uint8(f11 >> 16)
dAtA[i] = uint8(f12 >> 16)
i++
dAtA[i] = uint8(f11 >> 24)
dAtA[i] = uint8(f12 >> 24)
i++
dAtA[i] = uint8(f11 >> 32)
dAtA[i] = uint8(f12 >> 32)
i++
dAtA[i] = uint8(f11 >> 40)
dAtA[i] = uint8(f12 >> 40)
i++
dAtA[i] = uint8(f11 >> 48)
dAtA[i] = uint8(f12 >> 48)
i++
dAtA[i] = uint8(f11 >> 56)
dAtA[i] = uint8(f12 >> 56)
i++
}
}
@ -1025,22 +1163,22 @@ func (m *ReadResponse_IntegerPointsFrame) MarshalTo(dAtA []byte) (int, error) {
}
}
if len(m.Values) > 0 {
dAtA13 := make([]byte, len(m.Values)*10)
var j12 int
dAtA14 := make([]byte, len(m.Values)*10)
var j13 int
for _, num1 := range m.Values {
num := uint64(num1)
for num >= 1<<7 {
dAtA13[j12] = uint8(uint64(num)&0x7f | 0x80)
dAtA14[j13] = uint8(uint64(num)&0x7f | 0x80)
num >>= 7
j12++
j13++
}
dAtA13[j12] = uint8(num)
j12++
dAtA14[j13] = uint8(num)
j13++
}
dAtA[i] = 0x12
i++
i = encodeVarintStorage(dAtA, i, uint64(j12))
i += copy(dAtA[i:], dAtA13[:j12])
i = encodeVarintStorage(dAtA, i, uint64(j13))
i += copy(dAtA[i:], dAtA14[:j13])
}
return i, nil
}
@ -1084,21 +1222,21 @@ func (m *ReadResponse_UnsignedPointsFrame) MarshalTo(dAtA []byte) (int, error) {
}
}
if len(m.Values) > 0 {
dAtA15 := make([]byte, len(m.Values)*10)
var j14 int
dAtA16 := make([]byte, len(m.Values)*10)
var j15 int
for _, num := range m.Values {
for num >= 1<<7 {
dAtA15[j14] = uint8(uint64(num)&0x7f | 0x80)
dAtA16[j15] = uint8(uint64(num)&0x7f | 0x80)
num >>= 7
j14++
j15++
}
dAtA15[j14] = uint8(num)
j14++
dAtA16[j15] = uint8(num)
j15++
}
dAtA[i] = 0x12
i++
i = encodeVarintStorage(dAtA, i, uint64(j14))
i += copy(dAtA[i:], dAtA15[:j14])
i = encodeVarintStorage(dAtA, i, uint64(j15))
i += copy(dAtA[i:], dAtA16[:j15])
}
return i, nil
}
@ -1333,8 +1471,8 @@ func (m *ReadRequest) Size() (n int) {
if m.Descending {
n += 2
}
if len(m.Grouping) > 0 {
for _, s := range m.Grouping {
if len(m.GroupKeys) > 0 {
for _, s := range m.GroupKeys {
l = len(s)
n += 1 + l + sovStorage(uint64(l))
}
@ -1364,12 +1502,11 @@ func (m *ReadRequest) Size() (n int) {
n += mapEntrySize + 1 + sovStorage(uint64(mapEntrySize))
}
}
if m.RequestType != 0 {
n += 1 + sovStorage(uint64(m.RequestType))
if m.Group != 0 {
n += 1 + sovStorage(uint64(m.Group))
}
l = len(m.OrgID)
if l > 0 {
n += 1 + l + sovStorage(uint64(l))
if m.Hints != 0 {
n += 5
}
return n
}
@ -1472,6 +1609,33 @@ func (m *ReadResponse_Frame_StringPoints) Size() (n int) {
}
return n
}
func (m *ReadResponse_Frame_Group) Size() (n int) {
var l int
_ = l
if m.Group != nil {
l = m.Group.Size()
n += 1 + l + sovStorage(uint64(l))
}
return n
}
func (m *ReadResponse_GroupFrame) Size() (n int) {
var l int
_ = l
if len(m.TagKeys) > 0 {
for _, b := range m.TagKeys {
l = len(b)
n += 1 + l + sovStorage(uint64(l))
}
}
if len(m.PartitionKeyVals) > 0 {
for _, b := range m.PartitionKeyVals {
l = len(b)
n += 1 + l + sovStorage(uint64(l))
}
}
return n
}
func (m *ReadResponse_SeriesFrame) Size() (n int) {
var l int
_ = l
@ -1713,7 +1877,7 @@ func (m *ReadRequest) Unmarshal(dAtA []byte) error {
m.Descending = bool(v != 0)
case 4:
if wireType != 2 {
return fmt.Errorf("proto: wrong wireType = %d for field Grouping", wireType)
return fmt.Errorf("proto: wrong wireType = %d for field GroupKeys", wireType)
}
var stringLen uint64
for shift := uint(0); ; shift += 7 {
@ -1738,7 +1902,7 @@ func (m *ReadRequest) Unmarshal(dAtA []byte) error {
if postIndex > l {
return io.ErrUnexpectedEOF
}
m.Grouping = append(m.Grouping, string(dAtA[iNdEx:postIndex]))
m.GroupKeys = append(m.GroupKeys, string(dAtA[iNdEx:postIndex]))
iNdEx = postIndex
case 5:
if wireType != 2 {
@ -1787,7 +1951,7 @@ func (m *ReadRequest) Unmarshal(dAtA []byte) error {
}
b := dAtA[iNdEx]
iNdEx++
m.SeriesLimit |= (uint64(b) & 0x7F) << shift
m.SeriesLimit |= (int64(b) & 0x7F) << shift
if b < 0x80 {
break
}
@ -1806,7 +1970,7 @@ func (m *ReadRequest) Unmarshal(dAtA []byte) error {
}
b := dAtA[iNdEx]
iNdEx++
m.SeriesOffset |= (uint64(b) & 0x7F) << shift
m.SeriesOffset |= (int64(b) & 0x7F) << shift
if b < 0x80 {
break
}
@ -1825,7 +1989,7 @@ func (m *ReadRequest) Unmarshal(dAtA []byte) error {
}
b := dAtA[iNdEx]
iNdEx++
m.PointsLimit |= (uint64(b) & 0x7F) << shift
m.PointsLimit |= (int64(b) & 0x7F) << shift
if b < 0x80 {
break
}
@ -1983,9 +2147,9 @@ func (m *ReadRequest) Unmarshal(dAtA []byte) error {
iNdEx = postIndex
case 11:
if wireType != 0 {
return fmt.Errorf("proto: wrong wireType = %d for field RequestType", wireType)
return fmt.Errorf("proto: wrong wireType = %d for field Group", wireType)
}
m.RequestType = 0
m.Group = 0
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowStorage
@ -1995,40 +2159,24 @@ func (m *ReadRequest) Unmarshal(dAtA []byte) error {
}
b := dAtA[iNdEx]
iNdEx++
m.RequestType |= (ReadRequest_Type(b) & 0x7F) << shift
m.Group |= (ReadRequest_Group(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
case 12:
if wireType != 2 {
return fmt.Errorf("proto: wrong wireType = %d for field OrgID", wireType)
if wireType != 5 {
return fmt.Errorf("proto: wrong wireType = %d for field Hints", wireType)
}
var stringLen uint64
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowStorage
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
stringLen |= (uint64(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
intStringLen := int(stringLen)
if intStringLen < 0 {
return ErrInvalidLengthStorage
}
postIndex := iNdEx + intStringLen
if postIndex > l {
m.Hints = 0
if (iNdEx + 4) > l {
return io.ErrUnexpectedEOF
}
m.OrgID = string(dAtA[iNdEx:postIndex])
iNdEx = postIndex
iNdEx += 4
m.Hints = HintFlags(dAtA[iNdEx-4])
m.Hints |= HintFlags(dAtA[iNdEx-3]) << 8
m.Hints |= HintFlags(dAtA[iNdEx-2]) << 16
m.Hints |= HintFlags(dAtA[iNdEx-1]) << 24
default:
iNdEx = preIndex
skippy, err := skipStorage(dAtA[iNdEx:])
@ -2533,6 +2681,146 @@ func (m *ReadResponse_Frame) Unmarshal(dAtA []byte) error {
}
m.Data = &ReadResponse_Frame_StringPoints{v}
iNdEx = postIndex
case 7:
if wireType != 2 {
return fmt.Errorf("proto: wrong wireType = %d for field Group", wireType)
}
var msglen int
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowStorage
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
msglen |= (int(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
if msglen < 0 {
return ErrInvalidLengthStorage
}
postIndex := iNdEx + msglen
if postIndex > l {
return io.ErrUnexpectedEOF
}
v := &ReadResponse_GroupFrame{}
if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
return err
}
m.Data = &ReadResponse_Frame_Group{v}
iNdEx = postIndex
default:
iNdEx = preIndex
skippy, err := skipStorage(dAtA[iNdEx:])
if err != nil {
return err
}
if skippy < 0 {
return ErrInvalidLengthStorage
}
if (iNdEx + skippy) > l {
return io.ErrUnexpectedEOF
}
iNdEx += skippy
}
}
if iNdEx > l {
return io.ErrUnexpectedEOF
}
return nil
}
func (m *ReadResponse_GroupFrame) Unmarshal(dAtA []byte) error {
l := len(dAtA)
iNdEx := 0
for iNdEx < l {
preIndex := iNdEx
var wire uint64
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowStorage
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
wire |= (uint64(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
fieldNum := int32(wire >> 3)
wireType := int(wire & 0x7)
if wireType == 4 {
return fmt.Errorf("proto: GroupFrame: wiretype end group for non-group")
}
if fieldNum <= 0 {
return fmt.Errorf("proto: GroupFrame: illegal tag %d (wire type %d)", fieldNum, wire)
}
switch fieldNum {
case 1:
if wireType != 2 {
return fmt.Errorf("proto: wrong wireType = %d for field TagKeys", wireType)
}
var byteLen int
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowStorage
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
byteLen |= (int(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
if byteLen < 0 {
return ErrInvalidLengthStorage
}
postIndex := iNdEx + byteLen
if postIndex > l {
return io.ErrUnexpectedEOF
}
m.TagKeys = append(m.TagKeys, make([]byte, postIndex-iNdEx))
copy(m.TagKeys[len(m.TagKeys)-1], dAtA[iNdEx:postIndex])
iNdEx = postIndex
case 2:
if wireType != 2 {
return fmt.Errorf("proto: wrong wireType = %d for field PartitionKeyVals", wireType)
}
var byteLen int
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowStorage
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
byteLen |= (int(b) & 0x7F) << shift
if b < 0x80 {
break
}
}
if byteLen < 0 {
return ErrInvalidLengthStorage
}
postIndex := iNdEx + byteLen
if postIndex > l {
return io.ErrUnexpectedEOF
}
m.PartitionKeyVals = append(m.PartitionKeyVals, make([]byte, postIndex-iNdEx))
copy(m.PartitionKeyVals[len(m.PartitionKeyVals)-1], dAtA[iNdEx:postIndex])
iNdEx = postIndex
default:
iNdEx = preIndex
skippy, err := skipStorage(dAtA[iNdEx:])
@ -3883,88 +4171,103 @@ var (
func init() { proto.RegisterFile("storage.proto", fileDescriptorStorage) }
var fileDescriptorStorage = []byte{
// 1316 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x56, 0xbf, 0x6f, 0xdb, 0xc6,
0x17, 0x27, 0x2d, 0x4a, 0xb6, 0x9e, 0x24, 0x9b, 0xbe, 0x38, 0xfe, 0xea, 0xcb, 0x34, 0x12, 0xa3,
0x21, 0x55, 0x87, 0x28, 0x81, 0xda, 0xa2, 0x69, 0x83, 0x02, 0xb5, 0x12, 0xc5, 0x56, 0x63, 0x53,
0xc1, 0x49, 0x06, 0x3a, 0x14, 0x70, 0x4f, 0xd6, 0x89, 0x21, 0x2a, 0x91, 0x2c, 0x49, 0x15, 0xf1,
0xd6, 0xb1, 0x10, 0x3a, 0x74, 0xe8, 0xaa, 0x29, 0x7f, 0x43, 0xbb, 0x74, 0xeb, 0x94, 0xb1, 0x63,
0x27, 0xa3, 0x55, 0xff, 0x91, 0xe2, 0xee, 0x48, 0x8a, 0x94, 0xe5, 0x00, 0x5e, 0x04, 0xbe, 0xf7,
0x3e, 0xef, 0xf3, 0x7e, 0xdc, 0x7b, 0x77, 0x82, 0x92, 0x1f, 0x38, 0x1e, 0x31, 0x69, 0xc3, 0xf5,
0x9c, 0xc0, 0x41, 0x9b, 0xa1, 0xa8, 0x3d, 0x30, 0xad, 0xe0, 0xd5, 0x74, 0xd0, 0x38, 0x77, 0x26,
0x0f, 0x4d, 0xc7, 0x74, 0x1e, 0x72, 0xfb, 0x60, 0x3a, 0xe2, 0x12, 0x17, 0xf8, 0x97, 0xf0, 0xd3,
0xee, 0x98, 0x8e, 0x63, 0x8e, 0xe9, 0x12, 0x45, 0x27, 0x6e, 0x70, 0x11, 0x1a, 0x9b, 0x09, 0x2e,
0xcb, 0x1e, 0x8d, 0xa7, 0xaf, 0x87, 0x24, 0x20, 0x0f, 0x2f, 0x88, 0xe7, 0x9e, 0x8b, 0x5f, 0xc1,
0xc7, 0x3f, 0x43, 0x9f, 0x1d, 0xd7, 0xa3, 0x43, 0xeb, 0x9c, 0x04, 0x61, 0x66, 0xb5, 0x37, 0x39,
0x28, 0x60, 0x4a, 0x86, 0x98, 0x7e, 0x37, 0xa5, 0x7e, 0x80, 0x34, 0xd8, 0x62, 0x2c, 0x03, 0xe2,
0xd3, 0xb2, 0xac, 0xcb, 0xf5, 0x3c, 0x8e, 0x65, 0xf4, 0x15, 0xec, 0x04, 0xd6, 0x84, 0xfa, 0x01,
0x99, 0xb8, 0x67, 0x1e, 0xb1, 0x4d, 0x5a, 0xde, 0xd0, 0xe5, 0x7a, 0xa1, 0xf9, 0xbf, 0x46, 0x54,
0x6e, 0x3f, 0xb2, 0x63, 0x66, 0x6e, 0xed, 0xbf, 0xbd, 0xac, 0x4a, 0x8b, 0xcb, 0xea, 0x76, 0x5a,
0x8f, 0xb7, 0x83, 0x94, 0x8c, 0x2a, 0x00, 0x43, 0xea, 0x9f, 0x53, 0x7b, 0x68, 0xd9, 0x66, 0x39,
0xa3, 0xcb, 0xf5, 0x2d, 0x9c, 0xd0, 0xb0, 0xac, 0x4c, 0xcf, 0x99, 0xba, 0xcc, 0xaa, 0xe8, 0x19,
0x96, 0x55, 0x24, 0xa3, 0x47, 0x90, 0x8f, 0x8b, 0x2a, 0x67, 0x79, 0x3e, 0x28, 0xce, 0xe7, 0x65,
0x64, 0xc1, 0x4b, 0x10, 0x6a, 0x42, 0xd1, 0xa7, 0x9e, 0x45, 0xfd, 0xb3, 0xb1, 0x35, 0xb1, 0x82,
0x72, 0x4e, 0x97, 0xeb, 0x4a, 0x6b, 0x67, 0x71, 0x59, 0x2d, 0xf4, 0xb8, 0xfe, 0x98, 0xa9, 0x71,
0xc1, 0x5f, 0x0a, 0xe8, 0x63, 0x28, 0x85, 0x3e, 0xce, 0x68, 0xe4, 0xd3, 0xa0, 0xbc, 0xc9, 0x9d,
0xd4, 0xc5, 0x65, 0xb5, 0x28, 0x9c, 0xba, 0x5c, 0x8f, 0x43, 0x6a, 0x21, 0xb1, 0x50, 0xae, 0x63,
0xd9, 0x41, 0x14, 0x6a, 0x6b, 0x19, 0xea, 0x25, 0xd7, 0x87, 0xa1, 0xdc, 0xa5, 0xc0, 0x0a, 0x22,
0xa6, 0xe9, 0x51, 0x93, 0x15, 0x94, 0x5f, 0x29, 0xe8, 0x20, 0xb2, 0xe0, 0x25, 0x08, 0x7d, 0x01,
0xd9, 0xc0, 0x23, 0xe7, 0xb4, 0x0c, 0x7a, 0xa6, 0x5e, 0x68, 0x56, 0x63, 0x74, 0xe2, 0x64, 0x1b,
0x7d, 0x86, 0x68, 0xdb, 0x81, 0x77, 0xd1, 0xca, 0x2f, 0x2e, 0xab, 0x59, 0x2e, 0x63, 0xe1, 0x88,
0x4e, 0xa0, 0xe8, 0x09, 0xdc, 0x59, 0x70, 0xe1, 0xd2, 0x72, 0x41, 0x97, 0xeb, 0xdb, 0xcd, 0xff,
0xaf, 0x27, 0xba, 0x70, 0xa9, 0x28, 0x21, 0xd4, 0x30, 0x05, 0x2e, 0x78, 0x4b, 0x01, 0xe9, 0x90,
0x73, 0x3c, 0xf3, 0xcc, 0x1a, 0x96, 0x8b, 0x6c, 0x86, 0x44, 0xc0, 0xae, 0x67, 0x76, 0x9e, 0xe1,
0xac, 0xe3, 0x99, 0x9d, 0xa1, 0xf6, 0x18, 0x60, 0x99, 0x10, 0x52, 0x21, 0xf3, 0x2d, 0xbd, 0x08,
0x07, 0x8e, 0x7d, 0xa2, 0x3d, 0xc8, 0x7e, 0x4f, 0xc6, 0x53, 0x31, 0x61, 0x79, 0x2c, 0x84, 0xcf,
0x36, 0x1e, 0xcb, 0x35, 0x0f, 0x14, 0x1e, 0xa3, 0x09, 0xa5, 0x5e, 0xc7, 0x38, 0x3c, 0x6e, 0x9f,
0xf5, 0xdb, 0xc6, 0x81, 0xd1, 0x57, 0x25, 0xad, 0x3a, 0x9b, 0xeb, 0x77, 0x12, 0xa9, 0x32, 0x5c,
0xcf, 0xb2, 0xcd, 0x31, 0xed, 0x53, 0x9b, 0xd8, 0xac, 0xb5, 0xc5, 0x93, 0xd3, 0xe3, 0x7e, 0x27,
0x72, 0x91, 0xb5, 0xca, 0x6c, 0xae, 0x6b, 0x2b, 0x2e, 0x27, 0xd3, 0x71, 0x60, 0x09, 0x0f, 0x4d,
0xf9, 0xf1, 0x4d, 0x45, 0xaa, 0xfd, 0x2e, 0x43, 0x3e, 0xee, 0x3c, 0xfa, 0x08, 0x14, 0xde, 0x24,
0x99, 0x37, 0x49, 0xbf, 0x7a, 0x36, 0xcb, 0x2f, 0xde, 0x1a, 0x8e, 0xae, 0xbd, 0x86, 0x52, 0x4a,
0x8d, 0xaa, 0xa0, 0x18, 0x5d, 0xa3, 0xad, 0x4a, 0xda, 0xed, 0xd9, 0x5c, 0xdf, 0x4d, 0x19, 0x0d,
0xc7, 0xa6, 0xe8, 0x2e, 0x64, 0x7a, 0xa7, 0x27, 0xaa, 0xac, 0xed, 0xcd, 0xe6, 0xba, 0x9a, 0xb2,
0xf7, 0xa6, 0x13, 0x74, 0x0f, 0xb2, 0x4f, 0xbb, 0xa7, 0x46, 0x5f, 0xdd, 0xd0, 0xf6, 0x67, 0x73,
0x1d, 0xa5, 0x00, 0x4f, 0x9d, 0x69, 0x9c, 0xfd, 0x03, 0xc8, 0xf4, 0x89, 0x99, 0x6c, 0x72, 0x71,
0x4d, 0x93, 0x8b, 0x61, 0x93, 0x6b, 0xbf, 0x14, 0xa0, 0x28, 0x3a, 0xe2, 0xbb, 0x8e, 0xed, 0x53,
0xf4, 0x29, 0xe4, 0x46, 0x1e, 0x99, 0x50, 0xbf, 0x2c, 0xf3, 0xf9, 0xba, 0xb3, 0x32, 0x16, 0x02,
0xd6, 0x78, 0xce, 0x30, 0x2d, 0x85, 0xad, 0x3c, 0x0e, 0x1d, 0xb4, 0x3f, 0x14, 0xc8, 0x72, 0x3d,
0x7a, 0x02, 0x39, 0xb1, 0x19, 0x3c, 0x81, 0x42, 0xf3, 0xde, 0x7a, 0x12, 0xb1, 0x4b, 0xdc, 0xe5,
0x48, 0xc2, 0xa1, 0x0b, 0xfa, 0x1a, 0x8a, 0xa3, 0xb1, 0x43, 0x82, 0x33, 0xb1, 0x27, 0xe1, 0xb5,
0x73, 0xff, 0x9a, 0x3c, 0x18, 0x52, 0x6c, 0x97, 0x48, 0x89, 0xcf, 0x6a, 0x42, 0x7b, 0x24, 0xe1,
0xc2, 0x68, 0x29, 0xa2, 0x21, 0x6c, 0x5b, 0x76, 0x40, 0x4d, 0xea, 0x45, 0xfc, 0x19, 0xce, 0x5f,
0x5f, 0xcf, 0xdf, 0x11, 0xd8, 0x64, 0x84, 0xdd, 0xc5, 0x65, 0xb5, 0x94, 0xd2, 0x1f, 0x49, 0xb8,
0x64, 0x25, 0x15, 0xe8, 0x15, 0xec, 0x4c, 0x6d, 0xdf, 0x32, 0x6d, 0x3a, 0x8c, 0xc2, 0x28, 0x3c,
0xcc, 0x07, 0xeb, 0xc3, 0x9c, 0x86, 0xe0, 0x64, 0x1c, 0xc4, 0xee, 0xd2, 0xb4, 0xe1, 0x48, 0xc2,
0xdb, 0xd3, 0x94, 0x86, 0xd5, 0x33, 0x70, 0x9c, 0x31, 0x25, 0x76, 0x14, 0x28, 0xfb, 0xae, 0x7a,
0x5a, 0x02, 0x7b, 0xa5, 0x9e, 0x94, 0x9e, 0xd5, 0x33, 0x48, 0x2a, 0xd0, 0x37, 0xec, 0x91, 0xf3,
0x2c, 0xdb, 0x8c, 0x82, 0xe4, 0x78, 0x90, 0xf7, 0xaf, 0x39, 0x57, 0x0e, 0x4d, 0xc6, 0x10, 0x57,
0x67, 0x42, 0x7d, 0x24, 0xe1, 0xa2, 0x9f, 0x90, 0x5b, 0x39, 0x50, 0xd8, 0xdb, 0xa3, 0x79, 0x50,
0x48, 0x8c, 0x05, 0xba, 0x0f, 0x4a, 0x40, 0xcc, 0x68, 0x18, 0x8b, 0xcb, 0xb7, 0x87, 0x98, 0xe1,
0xf4, 0x71, 0x3b, 0x7a, 0x02, 0x79, 0xe6, 0x2e, 0x2e, 0xb4, 0x0d, 0xbe, 0xab, 0x95, 0xf5, 0xc9,
0x3d, 0x23, 0x01, 0xe1, 0x9b, 0xca, 0xdf, 0x3a, 0xf6, 0xa5, 0x7d, 0x09, 0xea, 0xea, 0x1c, 0xb1,
0x57, 0x2a, 0x7e, 0xb7, 0x44, 0x78, 0x15, 0x27, 0x34, 0x68, 0x1f, 0x72, 0x7c, 0x83, 0xd8, 0x7c,
0x66, 0xea, 0x32, 0x0e, 0x25, 0xed, 0x18, 0xd0, 0xd5, 0x99, 0xb9, 0x21, 0x5b, 0x26, 0x66, 0x3b,
0x81, 0x5b, 0x6b, 0x46, 0xe3, 0x86, 0x74, 0x4a, 0x32, 0xb9, 0xab, 0x03, 0x70, 0x43, 0xb6, 0xad,
0x98, 0xed, 0x05, 0xec, 0x5e, 0x39, 0xe9, 0x1b, 0x92, 0xe5, 0x23, 0xb2, 0x5a, 0x0f, 0xf2, 0x9c,
0x20, 0xbc, 0x2d, 0x73, 0xbd, 0x36, 0xee, 0xb4, 0x7b, 0xaa, 0xa4, 0xdd, 0x9a, 0xcd, 0xf5, 0x9d,
0xd8, 0x24, 0x66, 0x83, 0x01, 0x5e, 0x76, 0x3b, 0x46, 0xbf, 0xa7, 0xca, 0x2b, 0x00, 0x91, 0x4b,
0x78, 0x19, 0xfe, 0x26, 0xc3, 0x56, 0x74, 0xde, 0xe8, 0x3d, 0xc8, 0x3e, 0x3f, 0xee, 0x1e, 0xb0,
0xb7, 0x63, 0x77, 0x36, 0xd7, 0x4b, 0x91, 0x81, 0x1f, 0x3d, 0xd2, 0x61, 0xb3, 0x63, 0xf4, 0xdb,
0x87, 0x6d, 0x1c, 0x51, 0x46, 0xf6, 0xf0, 0x38, 0x51, 0x0d, 0xb6, 0x4e, 0x8d, 0x5e, 0xe7, 0xd0,
0x68, 0x3f, 0x53, 0x37, 0xc4, 0x35, 0x1d, 0x41, 0xa2, 0x33, 0x62, 0x2c, 0xad, 0x6e, 0xf7, 0xb8,
0x7d, 0x60, 0xa8, 0x99, 0x34, 0x4b, 0xd8, 0x77, 0x54, 0x81, 0x5c, 0xaf, 0x8f, 0x3b, 0xc6, 0xa1,
0xaa, 0x68, 0x68, 0x36, 0xd7, 0xb7, 0x23, 0x80, 0x68, 0x65, 0x98, 0xf8, 0x4f, 0x32, 0xec, 0x3d,
0x25, 0x2e, 0x19, 0x58, 0x63, 0x2b, 0xb0, 0xa8, 0x1f, 0x5f, 0xcf, 0x4f, 0x40, 0x39, 0x27, 0x6e,
0xb4, 0x0f, 0xcb, 0xfd, 0x5b, 0x07, 0x66, 0x4a, 0x9f, 0xbf, 0xb9, 0x98, 0x3b, 0x69, 0x9f, 0x40,
0x3e, 0x56, 0xdd, 0xe8, 0x19, 0xde, 0x81, 0xd2, 0x11, 0x6b, 0x6b, 0xc4, 0x5c, 0x7b, 0x0c, 0x2b,
0xff, 0xf2, 0x98, 0xb3, 0x1f, 0x10, 0x2f, 0xe0, 0x84, 0x19, 0x2c, 0x04, 0x16, 0x84, 0xda, 0x43,
0x4e, 0x98, 0xc1, 0xec, 0xb3, 0xf9, 0x97, 0x0c, 0x9b, 0x3d, 0x91, 0x34, 0x2b, 0x86, 0xad, 0x26,
0xda, 0x5b, 0xf7, 0xd7, 0x43, 0xbb, 0xbd, 0x76, 0x7f, 0x6b, 0xca, 0x0f, 0xbf, 0x96, 0xa5, 0x47,
0x32, 0x7a, 0x01, 0xc5, 0x64, 0xd1, 0x68, 0xbf, 0x21, 0xfe, 0x3f, 0x37, 0xa2, 0xff, 0xcf, 0x8d,
0x36, 0xfb, 0xff, 0xac, 0xdd, 0x7d, 0x67, 0x8f, 0x38, 0x9d, 0x8c, 0x3e, 0x87, 0x2c, 0x2f, 0xf0,
0x5a, 0x96, 0xfd, 0x98, 0x25, 0xdd, 0x08, 0xe6, 0xbe, 0xa1, 0xf1, 0x9c, 0x5a, 0x7b, 0x6f, 0xff,
0xa9, 0x48, 0x6f, 0x17, 0x15, 0xf9, 0xcf, 0x45, 0x45, 0xfe, 0x7b, 0x51, 0x91, 0x7f, 0xfe, 0xb7,
0x22, 0x0d, 0x72, 0x9c, 0xe9, 0xc3, 0xff, 0x02, 0x00, 0x00, 0xff, 0xff, 0xc3, 0xd6, 0xa7, 0xe4,
0x26, 0x0c, 0x00, 0x00,
// 1565 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x57, 0xcd, 0x8f, 0xe3, 0x4a,
0x11, 0x8f, 0xf3, 0xed, 0xca, 0x97, 0xa7, 0xdf, 0x30, 0x0a, 0x7e, 0x6c, 0xe2, 0x17, 0x21, 0x94,
0xc3, 0x7b, 0x19, 0x14, 0x40, 0xac, 0x56, 0x7c, 0x68, 0x32, 0x9b, 0x99, 0x84, 0x9d, 0x49, 0x86,
0x4e, 0x06, 0x2d, 0x08, 0x29, 0x74, 0x32, 0x3d, 0x1e, 0x6b, 0x13, 0xdb, 0xd8, 0xce, 0x6a, 0x72,
0xdb, 0x03, 0x12, 0xab, 0xc0, 0x61, 0xaf, 0x1c, 0x72, 0xe2, 0xc8, 0x09, 0x09, 0xf8, 0x03, 0x38,
0xed, 0x91, 0xbf, 0x20, 0x82, 0xec, 0x5f, 0x01, 0x27, 0xd4, 0xdd, 0x76, 0xe2, 0xec, 0xec, 0x3b,
0x64, 0xb3, 0x17, 0xab, 0xeb, 0xa3, 0x7f, 0x55, 0x5d, 0x5d, 0x55, 0x5d, 0x86, 0x9c, 0xeb, 0x59,
0x0e, 0xd1, 0x69, 0xcd, 0x76, 0x2c, 0xcf, 0x42, 0xc7, 0x23, 0x6b, 0x52, 0xd3, 0x0d, 0xef, 0x6e,
0x3a, 0xac, 0x19, 0xe6, 0xed, 0x78, 0x7a, 0x7f, 0x43, 0x3c, 0x12, 0x2c, 0x87, 0x35, 0x97, 0x3a,
0x2f, 0x8d, 0x11, 0x75, 0x6b, 0xfe, 0x36, 0xf5, 0x2b, 0x5f, 0x79, 0x64, 0x4d, 0x8e, 0x75, 0x4b,
0xb7, 0x8e, 0x39, 0xce, 0x70, 0x7a, 0xcb, 0x29, 0x4e, 0xf0, 0x95, 0xc0, 0x57, 0x3f, 0xd7, 0x2d,
0x4b, 0x1f, 0xd3, 0x8d, 0x16, 0x9d, 0xd8, 0xde, 0xcc, 0x17, 0xd6, 0x43, 0x58, 0x1b, 0xe3, 0xc7,
0x33, 0xe2, 0xd8, 0x23, 0xf1, 0x15, 0x78, 0x7c, 0xe9, 0xef, 0x29, 0xd8, 0x0e, 0xbd, 0x31, 0x46,
0xc4, 0xf3, 0x4f, 0x50, 0xf9, 0xab, 0x0c, 0x19, 0x4c, 0xc9, 0x0d, 0xa6, 0xbf, 0x9d, 0x52, 0xd7,
0x43, 0x2a, 0xa4, 0x19, 0xca, 0x90, 0xb8, 0xb4, 0x28, 0x69, 0x52, 0x55, 0xc6, 0x6b, 0x1a, 0xbd,
0x92, 0xa0, 0xe0, 0x19, 0x13, 0xea, 0x7a, 0x64, 0x62, 0x0f, 0x1c, 0x62, 0xea, 0xb4, 0x18, 0xd5,
0xa4, 0x6a, 0xa6, 0xfe, 0xd3, 0xda, 0x8e, 0x81, 0xa8, 0xf5, 0x03, 0x1c, 0xcc, 0x60, 0x1a, 0x47,
0x6f, 0x97, 0xe5, 0xc8, 0x6a, 0x59, 0xce, 0x6f, 0xf3, 0x71, 0xde, 0xdb, 0xa2, 0x51, 0x09, 0xe0,
0x86, 0xba, 0x23, 0x6a, 0xde, 0x18, 0xa6, 0x5e, 0x8c, 0x69, 0x52, 0x35, 0x8d, 0x43, 0x1c, 0xf4,
0x25, 0x80, 0xee, 0x58, 0x53, 0x7b, 0xf0, 0x82, 0xce, 0xdc, 0x62, 0x5c, 0x8b, 0x55, 0xe5, 0x46,
0x6e, 0xb5, 0x2c, 0xcb, 0xe7, 0x8c, 0xfb, 0x8c, 0xce, 0x5c, 0x2c, 0xeb, 0xc1, 0x12, 0x3d, 0x07,
0x79, 0x1d, 0x8f, 0x62, 0x82, 0x9f, 0xe4, 0xc9, 0xce, 0x27, 0xb9, 0x0a, 0x10, 0xf0, 0x06, 0x0c,
0xd5, 0x21, 0xeb, 0x52, 0xc7, 0xa0, 0xee, 0x60, 0x6c, 0x4c, 0x0c, 0xaf, 0x98, 0xd4, 0xa4, 0x6a,
0xac, 0x51, 0x58, 0x2d, 0xcb, 0x99, 0x1e, 0xe7, 0x5f, 0x30, 0x36, 0xce, 0xb8, 0x1b, 0x02, 0xfd,
0x00, 0x72, 0xfe, 0x1e, 0xeb, 0xf6, 0xd6, 0xa5, 0x5e, 0x31, 0xc5, 0x37, 0x29, 0xab, 0x65, 0x39,
0x2b, 0x36, 0x75, 0x39, 0x1f, 0xfb, 0xd0, 0x82, 0x62, 0xa6, 0x6c, 0xcb, 0x30, 0xbd, 0xc0, 0x54,
0x7a, 0x63, 0xea, 0x8a, 0xf3, 0x7d, 0x53, 0xf6, 0x86, 0x60, 0x07, 0x27, 0xba, 0xee, 0x50, 0x9d,
0x1d, 0x5c, 0xfe, 0xc8, 0x83, 0x9f, 0x04, 0x08, 0x78, 0x03, 0x86, 0xee, 0x20, 0xe1, 0x39, 0x64,
0x44, 0x8b, 0xa0, 0xc5, 0xaa, 0x99, 0xfa, 0xf9, 0xce, 0xa8, 0xa1, 0x64, 0xac, 0xf5, 0x19, 0x52,
0xd3, 0xf4, 0x9c, 0x59, 0x43, 0x5e, 0x2d, 0xcb, 0x09, 0x4e, 0x63, 0x61, 0x00, 0x3d, 0x87, 0x04,
0xbf, 0xc9, 0x62, 0x46, 0x93, 0xaa, 0xf9, 0x7a, 0x63, 0x2f, 0x4b, 0x3c, 0x3d, 0xb0, 0x00, 0x44,
0x5f, 0x42, 0xe2, 0x8e, 0xc5, 0xaa, 0x98, 0xd5, 0xa4, 0x6a, 0xaa, 0x71, 0xc4, 0x4c, 0xb7, 0x18,
0xe3, 0x7f, 0xcb, 0xb2, 0xcc, 0x16, 0x67, 0x63, 0xa2, 0xbb, 0x58, 0x28, 0xa9, 0x8f, 0x01, 0x36,
0x7e, 0x22, 0x05, 0x62, 0x2f, 0xe8, 0xcc, 0x2f, 0x1d, 0xb6, 0x44, 0x87, 0x90, 0x78, 0x49, 0xc6,
0x53, 0x51, 0x2a, 0x32, 0x16, 0xc4, 0x93, 0xe8, 0x63, 0xa9, 0xf2, 0x7b, 0x09, 0x12, 0xdc, 0x30,
0x7a, 0x04, 0x70, 0x8e, 0xbb, 0xd7, 0x57, 0x83, 0x4e, 0xb7, 0xd3, 0x54, 0x22, 0x6a, 0x6e, 0xbe,
0xd0, 0x44, 0xca, 0x76, 0x2c, 0x93, 0xa2, 0xcf, 0x41, 0x16, 0xe2, 0x93, 0x8b, 0x0b, 0x45, 0x52,
0xb3, 0xf3, 0x85, 0x96, 0xe6, 0xd2, 0x93, 0xf1, 0x18, 0x7d, 0x13, 0xd2, 0x42, 0xd8, 0xf8, 0xa5,
0x12, 0x55, 0x33, 0xf3, 0x85, 0x96, 0xe2, 0xb2, 0xc6, 0x0c, 0x7d, 0x01, 0x59, 0x21, 0x6a, 0x3e,
0x3f, 0x6d, 0x5e, 0xf5, 0x95, 0x98, 0x5a, 0x98, 0x2f, 0xb4, 0x0c, 0x17, 0x37, 0xef, 0x47, 0xd4,
0xf6, 0xd4, 0xf8, 0xeb, 0x3f, 0x97, 0x22, 0x95, 0xbf, 0x48, 0xb0, 0x39, 0x18, 0x33, 0xd7, 0x6a,
0x77, 0xfa, 0x81, 0x33, 0xdc, 0x1c, 0x93, 0x72, 0x5f, 0xbe, 0x0d, 0x79, 0x5f, 0x38, 0xb8, 0xea,
0xb6, 0x3b, 0xfd, 0x9e, 0x22, 0xa9, 0xca, 0x7c, 0xa1, 0x65, 0x85, 0x86, 0x48, 0xb9, 0xb0, 0x56,
0xaf, 0x89, 0xdb, 0xcd, 0x9e, 0x12, 0x0d, 0x6b, 0x89, 0x74, 0x46, 0xc7, 0x70, 0xc8, 0xb5, 0x7a,
0xa7, 0xad, 0xe6, 0xe5, 0x09, 0x3b, 0xdd, 0xa0, 0xdf, 0xbe, 0x6c, 0x2a, 0x71, 0xf5, 0x1b, 0xf3,
0x85, 0x76, 0xc0, 0x74, 0x7b, 0xa3, 0x3b, 0x3a, 0x21, 0x27, 0xe3, 0x31, 0x6b, 0x0c, 0xbe, 0xb7,
0xef, 0x24, 0x90, 0xd7, 0xc9, 0x87, 0x7e, 0x0d, 0x71, 0x6f, 0x66, 0x8b, 0x6e, 0x95, 0xaf, 0xb7,
0x3e, 0x3e, 0x8d, 0x37, 0xab, 0xfe, 0xcc, 0xa6, 0x98, 0xa3, 0x56, 0xee, 0x21, 0xb7, 0xc5, 0x46,
0x65, 0x88, 0xfb, 0x71, 0xe1, 0x3e, 0x6e, 0x09, 0x79, 0x80, 0x1e, 0x41, 0xac, 0x77, 0x7d, 0xa9,
0x48, 0xea, 0xe1, 0x7c, 0xa1, 0x29, 0x5b, 0xf2, 0xde, 0x74, 0x82, 0xbe, 0x80, 0xc4, 0x69, 0xf7,
0xba, 0xd3, 0x57, 0xa2, 0xea, 0xd1, 0x7c, 0xa1, 0xa1, 0x2d, 0x85, 0x53, 0x6b, 0x6a, 0x06, 0x77,
0xf2, 0x15, 0xc4, 0xfa, 0x44, 0x0f, 0x27, 0x54, 0xf6, 0x03, 0x09, 0x95, 0xf5, 0x13, 0xaa, 0xb2,
0x2c, 0x40, 0x56, 0x64, 0xb4, 0x6b, 0x5b, 0xa6, 0x4b, 0x11, 0x81, 0xe4, 0xad, 0x43, 0x26, 0xd4,
0x2d, 0x4a, 0xbc, 0x14, 0x4f, 0x3f, 0xb2, 0x40, 0x04, 0x5c, 0xed, 0x8c, 0x61, 0x35, 0xe2, 0xac,
0x4f, 0x63, 0x1f, 0x58, 0xfd, 0x53, 0x0a, 0x12, 0x9c, 0x8f, 0x46, 0x90, 0x14, 0x4d, 0x89, 0x3b,
0x9a, 0xa9, 0xb7, 0xf7, 0x33, 0x26, 0xf2, 0x83, 0x43, 0xb7, 0x22, 0xd8, 0x87, 0x46, 0xbf, 0x93,
0x20, 0x7b, 0x3b, 0xb6, 0x88, 0x37, 0x10, 0xbd, 0xcc, 0x7f, 0x7c, 0x3a, 0x7b, 0x1e, 0x8c, 0x21,
0x8a, 0xb4, 0x15, 0x67, 0xe4, 0xad, 0x33, 0xc4, 0x6d, 0x45, 0x70, 0xe6, 0x76, 0x43, 0xa2, 0x3f,
0x4a, 0x90, 0x37, 0x4c, 0x8f, 0xea, 0xd4, 0x09, 0x1c, 0x89, 0x71, 0x47, 0xae, 0xf6, 0x73, 0xa4,
0x2d, 0x30, 0xc3, 0xae, 0x1c, 0xac, 0x96, 0xe5, 0xdc, 0x16, 0xbf, 0x15, 0xc1, 0x39, 0x23, 0xcc,
0x40, 0x6f, 0x24, 0x28, 0x4c, 0x4d, 0xd7, 0xd0, 0x4d, 0x7a, 0x13, 0xf8, 0x13, 0xe7, 0xfe, 0xfc,
0x7c, 0x3f, 0x7f, 0xae, 0x7d, 0xd0, 0xb0, 0x43, 0x88, 0xbd, 0xd1, 0xdb, 0x82, 0x56, 0x04, 0xe7,
0xa7, 0x5b, 0x1c, 0x1e, 0xa1, 0xa1, 0x65, 0x8d, 0x29, 0x31, 0x03, 0x8f, 0x12, 0x9f, 0x22, 0x42,
0x0d, 0x81, 0xf9, 0x20, 0x42, 0x5b, 0x7c, 0x16, 0xa1, 0x61, 0x98, 0x81, 0x5e, 0x4b, 0x6c, 0x6e,
0x73, 0x0c, 0x53, 0x0f, 0xbc, 0x49, 0x72, 0x6f, 0xba, 0x7b, 0x26, 0x29, 0x87, 0x0c, 0x3b, 0x23,
0x9e, 0xea, 0x10, 0xbb, 0x15, 0xc1, 0x59, 0x37, 0x44, 0xa3, 0xdf, 0x04, 0x8f, 0x56, 0x8a, 0x7b,
0xd0, 0xda, 0xcf, 0x03, 0xde, 0xc8, 0x83, 0x2a, 0x11, 0xc0, 0x8d, 0x24, 0xc4, 0x19, 0x84, 0x7a,
0x0f, 0xb0, 0x11, 0xa3, 0xef, 0x40, 0xda, 0x23, 0xba, 0x98, 0x8a, 0x58, 0x3b, 0xc8, 0x36, 0x32,
0xab, 0x65, 0x39, 0xd5, 0x27, 0x3a, 0x9f, 0x89, 0x52, 0x9e, 0x58, 0xa0, 0x06, 0x20, 0x9b, 0x38,
0x9e, 0xe1, 0x19, 0x96, 0xc9, 0xb4, 0x07, 0x2f, 0xc9, 0x98, 0xd5, 0x19, 0xdb, 0x71, 0xb8, 0x5a,
0x96, 0x95, 0xab, 0x40, 0xfa, 0x8c, 0xce, 0x7e, 0x41, 0xc6, 0x2e, 0x56, 0xec, 0xf7, 0x38, 0xea,
0x3f, 0x25, 0xc8, 0x84, 0x0a, 0x18, 0x75, 0x20, 0xee, 0x11, 0x3d, 0x68, 0x43, 0xdf, 0xdf, 0x7d,
0x54, 0x24, 0xba, 0xdf, 0x77, 0x38, 0x0e, 0x1a, 0x81, 0xcc, 0x76, 0x0c, 0x78, 0xd7, 0x8f, 0xf2,
0xae, 0x7f, 0xb6, 0x5f, 0x1c, 0x9f, 0x12, 0x8f, 0xf0, 0x9e, 0xcf, 0x67, 0x5d, 0xb6, 0x52, 0x7f,
0x06, 0xca, 0xfb, 0x8d, 0x81, 0x0d, 0x9f, 0xeb, 0x71, 0x54, 0x1c, 0x47, 0xc1, 0x21, 0x0e, 0x3a,
0x82, 0x24, 0xef, 0xc5, 0x22, 0x60, 0x12, 0xf6, 0x29, 0xf5, 0x02, 0xd0, 0xc3, 0xda, 0xde, 0x11,
0x2d, 0xb6, 0x46, 0xbb, 0x84, 0xcf, 0x3e, 0x50, 0x99, 0x3b, 0xc2, 0xc5, 0xc3, 0xce, 0x3d, 0x2c,
0xab, 0x1d, 0xd1, 0xd2, 0x6b, 0xb4, 0x67, 0x70, 0xf0, 0xa0, 0x2c, 0x76, 0x04, 0x93, 0x03, 0xb0,
0x4a, 0x0f, 0x64, 0x0e, 0xe0, 0xbf, 0xbb, 0x49, 0x7f, 0x92, 0x88, 0xa8, 0x9f, 0xcd, 0x17, 0x5a,
0x61, 0x2d, 0xf2, 0x87, 0x89, 0x32, 0x24, 0xd7, 0x03, 0xc9, 0xb6, 0x82, 0xf0, 0xc5, 0x7f, 0x56,
0xff, 0x21, 0x41, 0x3a, 0xb8, 0x6f, 0xf4, 0x2d, 0x48, 0x9c, 0x5d, 0x74, 0x4f, 0xfa, 0x4a, 0x44,
0x3d, 0x98, 0x2f, 0xb4, 0x5c, 0x20, 0xe0, 0x57, 0x8f, 0x34, 0x48, 0xb5, 0x3b, 0xfd, 0xe6, 0x79,
0x13, 0x07, 0x90, 0x81, 0xdc, 0xbf, 0x4e, 0x54, 0x81, 0xf4, 0x75, 0xa7, 0xd7, 0x3e, 0xef, 0x34,
0x9f, 0x2a, 0x51, 0xf1, 0xe0, 0x07, 0x2a, 0xc1, 0x1d, 0x31, 0x94, 0x46, 0xb7, 0x7b, 0xd1, 0x3c,
0xe9, 0x28, 0xb1, 0x6d, 0x14, 0x3f, 0xee, 0xa8, 0x04, 0xc9, 0x5e, 0x1f, 0xb7, 0x3b, 0xe7, 0x4a,
0x5c, 0x45, 0xf3, 0x85, 0x96, 0x0f, 0x14, 0x44, 0x28, 0x7d, 0xc7, 0xff, 0x26, 0xc1, 0xe1, 0x29,
0xb1, 0xc9, 0xd0, 0x18, 0x1b, 0x9e, 0x41, 0xdd, 0xf5, 0x43, 0x3f, 0x82, 0xf8, 0x88, 0xd8, 0x41,
0x7d, 0xed, 0xde, 0xd4, 0x3e, 0x04, 0xca, 0x98, 0x2e, 0x9f, 0x68, 0x31, 0x07, 0x57, 0x7f, 0x08,
0xf2, 0x9a, 0xb5, 0xd3, 0x90, 0x5b, 0x80, 0x1c, 0x9f, 0x9d, 0x03, 0xe4, 0xca, 0x63, 0x78, 0xef,
0x27, 0x8f, 0x6d, 0x76, 0x3d, 0xe2, 0x78, 0x1c, 0x30, 0x86, 0x05, 0xc1, 0x8c, 0x50, 0xf3, 0x86,
0x03, 0xc6, 0x30, 0x5b, 0xd6, 0xff, 0x1b, 0x85, 0x54, 0x4f, 0x38, 0x8d, 0xfe, 0x20, 0x41, 0x9c,
0xd5, 0x30, 0xfa, 0xd1, 0x3e, 0x73, 0xbf, 0xfa, 0xe3, 0xbd, 0x1a, 0x47, 0x25, 0xfe, 0xea, 0xef,
0xc5, 0xc8, 0x77, 0x25, 0xe4, 0x42, 0x36, 0x1c, 0x45, 0x74, 0x54, 0x13, 0x3f, 0xee, 0xb5, 0xe0,
0xc7, 0xbd, 0xd6, 0x64, 0x3f, 0xee, 0x6a, 0xf3, 0x93, 0x5c, 0x0e, 0x37, 0x2b, 0x21, 0x0a, 0xe2,
0xaf, 0xe4, 0x6b, 0xad, 0xfd, 0x64, 0x67, 0x6b, 0xdb, 0x37, 0xc5, 0xcc, 0x44, 0x55, 0x7e, 0xc6,
0xc6, 0xa3, 0xb7, 0xff, 0x29, 0x45, 0xde, 0xae, 0x4a, 0xd2, 0xbf, 0x56, 0x25, 0xe9, 0xdf, 0xab,
0x92, 0xf4, 0xe6, 0x5d, 0x29, 0xf2, 0xab, 0x94, 0xbf, 0x71, 0x98, 0xe4, 0xa6, 0xbf, 0xf7, 0xff,
0x00, 0x00, 0x00, 0xff, 0xff, 0x78, 0x1c, 0xf2, 0x5d, 0x20, 0x11, 0x00, 0x00,
}

View File

@ -1,5 +1,6 @@
syntax = "proto3";
package storage;
package com.github.influxdata.influxdb.services.storage;
option go_package = "storage";
import "github.com/gogo/protobuf/gogoproto/gogo.proto";
import "google/protobuf/empty.proto";
@ -34,19 +35,33 @@ service Storage {
// Request message for Storage.Read.
message ReadRequest {
enum Type {
enum Group {
option (gogoproto.goproto_enum_prefix) = false;
SINGLE_TENANT = 0 [(gogoproto.enumvalue_customname) = "ReadRequestTypeSingleTenant"];
MULTI_TENANT = 1 [(gogoproto.enumvalue_customname) = "ReadRequestTypeMultiTenant"];
// GroupNone returns all series as a single group.
// The single GroupFrame.TagKeys will be the union of all tag keys.
GROUP_NONE = 0 [(gogoproto.enumvalue_customname) = "GroupNone"];
// GroupAll returns a unique group for each series.
// As an optimization, no GroupFrames will be generated.
GROUP_ALL = 1 [(gogoproto.enumvalue_customname) = "GroupAll"];
// GroupBy returns a group for each unique value of the specified GroupKeys.
GROUP_BY = 2 [(gogoproto.enumvalue_customname) = "GroupBy"];
// GroupExcept in not implemented.
GROUP_EXCEPT = 3 [(gogoproto.enumvalue_customname) = "GroupExcept"];
}
// RequestType specifies the request type as either single or multi tenant.
Type request_type = 11 [(gogoproto.customname) = "RequestType"];
enum HintFlags {
option (gogoproto.goproto_enum_prefix) = false;
// OrgID specifies the organization identifier for this request.
string org_id = 12 [(gogoproto.customname) = "OrgID"];
HINT_NONE = 0x00 [(gogoproto.enumvalue_customname) = "HintNone"];
HINT_NO_POINTS = 0x01 [(gogoproto.enumvalue_customname) = "HintNoPoints"];
HINT_NO_SERIES = 0x02 [(gogoproto.enumvalue_customname) = "HintNoSeries"];
// HintSchemaAllTime performs schema queries without using time ranges
HINT_SCHEMA_ALL_TIME = 0x04 [(gogoproto.enumvalue_customname) = "HintSchemaAllTime"];
}
// Database specifies the database name (single tenant) or bucket identifier (multi tenant).
string database = 1;
@ -56,8 +71,12 @@ message ReadRequest {
// Descending indicates whether points should be returned in descending order.
bool descending = 3;
// Grouping specifies a list of tags used to order the data
repeated string grouping = 4;
// 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
@ -66,17 +85,21 @@ message ReadRequest {
Predicate predicate = 5;
// SeriesLimit determines the maximum number of series to be returned for the request. Specify 0 for no limit.
uint64 series_limit = 6 [(gogoproto.customname) = "SeriesLimit"];
int64 series_limit = 6 [(gogoproto.customname) = "SeriesLimit"];
// SeriesOffset determines how many series to skip before processing the request.
uint64 series_offset = 7 [(gogoproto.customname) = "SeriesOffset"];
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.
uint64 points_limit = 8 [(gogoproto.customname) = "PointsLimit"];
// 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"];
}
message Aggregate {
@ -119,6 +142,7 @@ message ReadResponse {
message Frame {
oneof data {
GroupFrame group = 7;
SeriesFrame series = 1;
FloatPointsFrame float_points = 2 [(gogoproto.customname) = "FloatPoints"];
IntegerPointsFrame integer_points = 3 [(gogoproto.customname) = "IntegerPoints"];
@ -128,6 +152,13 @@ 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
repeated bytes partition_key_vals = 2 [(gogoproto.customname) = "PartitionKeyVals"];
}
message SeriesFrame {
repeated Tag tags = 1 [(gogoproto.nullable) = false];
DataType data_type = 2;

View File

@ -23,6 +23,7 @@ package storage
import (
context "context"
yarpc "github.com/influxdata/yarpc"
)
@ -163,7 +164,7 @@ func _Storage_Hints_Handler(srv interface{}, ctx context.Context, dec func(inter
}
var _Storage_serviceDesc = yarpc.ServiceDesc{
ServiceName: "storage.Storage",
ServiceName: "com.github.influxdata.influxdb.services.storage.Storage",
Index: 0,
HandlerType: (*StorageServer)(nil),
Methods: []yarpc.MethodDesc{

View File

@ -28,41 +28,7 @@ func (s *Store) WithLogger(log *zap.Logger) {
s.Logger = log.With(zap.String("service", "store"))
}
func (s *Store) Read(ctx context.Context, req *ReadRequest) (*ResultSet, error) {
database, rp := req.Database, ""
if req.RequestType == ReadRequestTypeMultiTenant {
// TODO(sgc): this should be moved to configuration
database, rp = "db", "rp"
} else {
if p := strings.IndexByte(database, '/'); p > -1 {
database, rp = database[:p], database[p+1:]
}
}
di := s.MetaClient.Database(database)
if di == nil {
return nil, errors.New("no database")
}
if rp == "" {
rp = di.DefaultRetentionPolicy
}
rpi := di.RetentionPolicy(rp)
if rpi == nil {
return nil, errors.New("invalid retention policy")
}
var start, end = models.MinNanoTime, models.MaxNanoTime
if req.TimestampRange.Start > 0 {
start = req.TimestampRange.Start
}
if req.TimestampRange.End > 0 {
end = req.TimestampRange.End
}
func (s *Store) findShardIDs(database, rp string, desc bool, start, end int64) ([]uint64, error) {
groups, err := s.MetaClient.ShardGroupsByTimeRange(database, rp, time.Unix(0, start), time.Unix(0, end))
if err != nil {
return nil, err
@ -72,7 +38,7 @@ func (s *Store) Read(ctx context.Context, req *ReadRequest) (*ResultSet, error)
return nil, nil
}
if req.Descending {
if desc {
sort.Sort(sort.Reverse(meta.ShardGroupInfos(groups)))
} else {
sort.Sort(meta.ShardGroupInfos(groups))
@ -84,9 +50,58 @@ func (s *Store) Read(ctx context.Context, req *ReadRequest) (*ResultSet, error)
shardIDs = append(shardIDs, si.ID)
}
}
return shardIDs, nil
}
func (s *Store) validateArgs(database string, start, end int64) (string, string, int64, int64, error) {
rp := ""
if p := strings.IndexByte(database, '/'); p > -1 {
database, rp = database[:p], database[p+1:]
}
di := s.MetaClient.Database(database)
if di == nil {
return "", "", 0, 0, errors.New("no database")
}
if rp == "" {
rp = di.DefaultRetentionPolicy
}
rpi := di.RetentionPolicy(rp)
if rpi == nil {
return "", "", 0, 0, errors.New("invalid retention policy")
}
if start <= 0 {
start = models.MinNanoTime
}
if end <= 0 {
end = models.MaxNanoTime
}
return database, rp, start, end, nil
}
func (s *Store) Read(ctx context.Context, req *ReadRequest) (*ResultSet, error) {
if len(req.GroupKeys) > 0 {
panic("Read: len(Grouping) > 0")
}
database, rp, start, end, err := s.validateArgs(req.Database, req.TimestampRange.Start, req.TimestampRange.End)
if err != nil {
return nil, err
}
shardIDs, err := s.findShardIDs(database, rp, req.Descending, start, end)
if err != nil {
return nil, err
}
if len(shardIDs) == 0 {
return nil, nil
}
var cur seriesCursor
if ic, err := newIndexSeriesCursor(ctx, req, s.TSDBStore.Shards(shardIDs)); err != nil {
if ic, err := newIndexSeriesCursor(ctx, req.Predicate, s.TSDBStore.Shards(shardIDs)); err != nil {
return nil, err
} else if ic == nil {
return nil, nil
@ -94,23 +109,52 @@ func (s *Store) Read(ctx context.Context, req *ReadRequest) (*ResultSet, error)
cur = ic
}
if len(req.Grouping) > 0 {
cur = newGroupSeriesCursor(ctx, cur, req.Grouping)
}
if req.SeriesLimit > 0 || req.SeriesOffset > 0 {
cur = newLimitSeriesCursor(ctx, cur, req.SeriesLimit, req.SeriesOffset)
}
rr := readRequest{
ctx: ctx,
start: start,
end: end,
asc: !req.Descending,
limit: req.PointsLimit,
aggregate: req.Aggregate,
}
return &ResultSet{
req: readRequest{
ctx: ctx,
start: start,
end: end,
asc: !req.Descending,
limit: req.PointsLimit,
aggregate: req.Aggregate,
},
req: rr,
cur: cur,
mb: newMultiShardBatchCursors(ctx, &rr),
}, nil
}
func (s *Store) GroupRead(ctx context.Context, req *ReadRequest) (*groupResultSet, error) {
if req.SeriesLimit > 0 || req.SeriesOffset > 0 {
return nil, errors.New("GroupRead: SeriesLimit and SeriesOffset not supported when Grouping")
}
database, rp, start, end, err := s.validateArgs(req.Database, req.TimestampRange.Start, req.TimestampRange.End)
if err != nil {
return nil, err
}
shardIDs, err := s.findShardIDs(database, rp, req.Descending, start, end)
if err != nil {
return nil, err
}
if len(shardIDs) == 0 {
return nil, nil
}
shards := s.TSDBStore.Shards(shardIDs)
req.TimestampRange.Start = start
req.TimestampRange.End = end
newCursor := func() (seriesCursor, error) {
return newIndexSeriesCursor(ctx, req.Predicate, shards)
}
return newGroupResultSet(ctx, req, newCursor), nil
}

View File

@ -0,0 +1,30 @@
package storage
import (
"github.com/influxdata/influxdb/models"
)
type tagsBuffer struct {
sz int
i int
buf models.Tags
}
func (tb *tagsBuffer) copyTags(src models.Tags) models.Tags {
var buf models.Tags
if len(src) > tb.sz {
buf = make(models.Tags, len(src))
} else {
if tb.i+len(src) > len(tb.buf) {
tb.buf = make(models.Tags, tb.sz)
tb.i = 0
}
buf = tb.buf[tb.i : tb.i+len(src)]
tb.i += len(src)
}
copy(buf, src)
return buf
}