Upgrade flux to the latest version and remove the platform dependency

This integrates the influxdb 1.x series to the latest version of Flux
and updates the code to use it. It also removes the dependency on
platform and copies the necessary code from storage into the 1.x series
so the dependency is unneeded.

The flux functions specific to 1.x have been moved to the same structure
that flux changed to with having a `stdlib` directory instead of a
`functions` directory. It also adds a `databases()` function that
returns the databases from the meta client.
pull/13153/head
Jonathan A. Sternberg 2019-04-04 10:55:09 -05:00
parent 932521b6ff
commit 31501c9dcf
No known key found for this signature in database
GPG Key ID: 4A0C1200CB8B9D2E
67 changed files with 19300 additions and 314 deletions

147
Gopkg.lock generated
View File

@ -29,14 +29,6 @@
revision = "c7af12943936e8c39859482e61f0574c2fd7fc75"
version = "v1.4.2"
[[projects]]
digest = "1:72fff13e105db55c6d4b1acb69216b88d9d3009433f43d7df3488f842b0b6929"
name = "github.com/RoaringBitmap/roaring"
packages = ["."]
pruneopts = "UT"
revision = "3d677d3262197ee558b85029301eb69b8239f91a"
version = "v0.4.16"
[[projects]]
digest = "1:c06d9e11d955af78ac3bbb26bd02e01d2f61f689e1a3bce2ef6fb683ef8a7f2d"
name = "github.com/alecthomas/kingpin"
@ -253,38 +245,14 @@
version = "v1.4.0"
[[projects]]
digest = "1:84fa2f03428ad6597620424756169cdb2a47ef1ef6c1fc94100f2de809ea0b72"
digest = "1:3f7c586d2fd571c4f2e3d313c0638b33b999055de507d5f11fafe27e5adc30db"
name = "github.com/gogo/protobuf"
packages = [
"gogoproto",
"plugin/compare",
"plugin/defaultcheck",
"plugin/description",
"plugin/embedcheck",
"plugin/enumstringer",
"plugin/equal",
"plugin/face",
"plugin/gostring",
"plugin/marshalto",
"plugin/oneofcheck",
"plugin/populate",
"plugin/size",
"plugin/stringer",
"plugin/testgen",
"plugin/union",
"plugin/unmarshal",
"proto",
"protoc-gen-gogo",
"protoc-gen-gogo/descriptor",
"protoc-gen-gogo/generator",
"protoc-gen-gogo/generator/internal/remap",
"protoc-gen-gogo/grpc",
"protoc-gen-gogo/plugin",
"protoc-gen-gogofaster",
"sortkeys",
"types",
"vanity",
"vanity/command",
]
pruneopts = "UT"
revision = "636bf0302bc95575d69441b25a2603156ffdddf1"
@ -412,7 +380,7 @@
version = "v0.3.6"
[[projects]]
digest = "1:9608dc2c0716b655b535fc0f20f42472e43b6cead49f4c9ad20eded341c5df3b"
digest = "1:0e29c493c2538a653b6c8787312ba93884f654415d782c85e51d07772baec1de"
name = "github.com/influxdata/flux"
packages = [
".",
@ -422,10 +390,6 @@
"control",
"csv",
"execute",
"functions",
"functions/inputs",
"functions/outputs",
"functions/transformations",
"influxql",
"internal/parser",
"internal/pkg/syncutil",
@ -434,17 +398,32 @@
"interpreter",
"iocounter",
"lang",
"line",
"memory",
"options",
"parser",
"plan",
"repl",
"semantic",
"stdlib",
"stdlib/csv",
"stdlib/generate",
"stdlib/http",
"stdlib/influxdata/influxdb",
"stdlib/influxdata/influxdb/v1",
"stdlib/kafka",
"stdlib/math",
"stdlib/socket",
"stdlib/sql",
"stdlib/strings",
"stdlib/system",
"stdlib/testing",
"stdlib/testing/testdata",
"stdlib/universe",
"values",
]
pruneopts = "UT"
revision = "8c9d0ad49204d3bbb171e96d872cf663ee7f1b4d"
version = "v0.12.0"
revision = "bcfc535fb7443776501797df4ba596483c50c7fe"
version = "v0.24.0"
[[projects]]
branch = "master"
@ -465,44 +444,6 @@
pruneopts = "UT"
revision = "32c6aa80de5eb09d190ad284a8214a531c6bce57"
[[projects]]
branch = "master"
digest = "1:211733975798a6f796a57d0f1bd3e3be2f8ada48c6d3289b98fefa6345a76fbf"
name = "github.com/influxdata/platform"
packages = [
".",
"logger",
"models",
"pkg/binaryutil",
"pkg/bloom",
"pkg/bytesutil",
"pkg/data/gen",
"pkg/encoding/simple8b",
"pkg/escape",
"pkg/file",
"pkg/limiter",
"pkg/metrics",
"pkg/mmap",
"pkg/pool",
"pkg/rhh",
"pkg/slices",
"pkg/snowflake",
"query",
"query/functions/inputs/storage",
"storage/reads",
"storage/reads/datatypes",
"telegraf/plugins",
"telegraf/plugins/inputs",
"telegraf/plugins/outputs",
"toml",
"tsdb",
"tsdb/cursors",
"tsdb/tsi1",
"tsdb/tsm1",
]
pruneopts = "UT"
revision = "0f79e4ea3248354c789cba274542e0a8e55971db"
[[projects]]
branch = "master"
digest = "1:ca1ab73a1e54a6e820ce3d55c7f7ae02a73815d270c42a5b272727b01af0aa36"
@ -550,17 +491,6 @@
pruneopts = "UT"
revision = "b4e1701a28efcc637d9afcca7d38e495fe909a09"
[[projects]]
digest = "1:f5ecd306f6eb9f9902757feaef040afe172292d7ee69e46e81442b81f8b6ea57"
name = "github.com/kevinburke/go-bindata"
packages = [
".",
"go-bindata",
]
pruneopts = "UT"
revision = "06af60a4461b70d84a2b173d92f9f425d78baf55"
version = "v3.11.0"
[[projects]]
digest = "1:15ec2166e33ef6c60b344a04d050eec79193517e7f5082b6233b2d09ef0d10b8"
name = "github.com/kisielk/gotool"
@ -674,18 +604,6 @@
pruneopts = "UT"
revision = "ae18d6b8b3205b561c79e8e5f69bff09736185f4"
[[projects]]
digest = "1:d4d60a22dc6c84bf1f8d2991cddcb98fb25f433f8a856fb6888fc21885998a13"
name = "github.com/mna/pigeon"
packages = [
".",
"ast",
"builder",
]
pruneopts = "UT"
revision = "9df264905d4734c0133161d8c67828ff522b154a"
version = "v1.0.0"
[[projects]]
branch = "master"
digest = "1:c22fdadfe2de0e1df75987cac34f1c721157dc24f180c76b4ef66a3c9637a799"
@ -1054,24 +972,21 @@
revision = "5420a8b6744d3b0345ab293f6fcba19c978f1183"
[[projects]]
digest = "1:ec626c5a8d1abbb2e052d195da2d089ff6a3afb60fa7cd45638d648da96d41dd"
digest = "1:7af71138a81cc2a505e0cd39be2a2d16fd6685b8ce49efe4f7f7cd0181797bdb"
name = "honnef.co/go/tools"
packages = [
"callgraph",
"callgraph/static",
"cmd/megacheck",
"cmd/staticcheck",
"deprecated",
"functions",
"internal/sharedcheck",
"lint",
"lint/lintutil",
"simple",
"ssa",
"ssa/ssautil",
"staticcheck",
"staticcheck/vrp",
"unused",
"version",
]
pruneopts = "UT"
@ -1085,6 +1000,7 @@
"collectd.org/api",
"collectd.org/network",
"github.com/BurntSushi/toml",
"github.com/apache/arrow/go/arrow/array",
"github.com/bmizerany/pat",
"github.com/boltdb/bolt",
"github.com/cespare/xxhash",
@ -1098,25 +1014,22 @@
"github.com/google/go-cmp/cmp",
"github.com/google/go-cmp/cmp/cmpopts",
"github.com/influxdata/flux",
"github.com/influxdata/flux/arrow",
"github.com/influxdata/flux/ast",
"github.com/influxdata/flux/control",
"github.com/influxdata/flux/csv",
"github.com/influxdata/flux/execute",
"github.com/influxdata/flux/functions/inputs",
"github.com/influxdata/flux/functions/outputs",
"github.com/influxdata/flux/functions/transformations",
"github.com/influxdata/flux/lang",
"github.com/influxdata/flux/memory",
"github.com/influxdata/flux/options",
"github.com/influxdata/flux/plan",
"github.com/influxdata/flux/repl",
"github.com/influxdata/flux/semantic",
"github.com/influxdata/flux/stdlib",
"github.com/influxdata/flux/stdlib/influxdata/influxdb",
"github.com/influxdata/flux/stdlib/influxdata/influxdb/v1",
"github.com/influxdata/flux/stdlib/universe",
"github.com/influxdata/flux/values",
"github.com/influxdata/influxql",
"github.com/influxdata/platform/models",
"github.com/influxdata/platform/pkg/data/gen",
"github.com/influxdata/platform/query/functions/inputs/storage",
"github.com/influxdata/platform/storage/reads",
"github.com/influxdata/platform/storage/reads/datatypes",
"github.com/influxdata/platform/tsdb/cursors",
"github.com/influxdata/roaring",
"github.com/influxdata/usage-client/v1",
"github.com/jsternberg/zap-logfmt",
@ -1138,12 +1051,14 @@
"go.uber.org/zap/zaptest/observer",
"golang.org/x/crypto/bcrypt",
"golang.org/x/crypto/ssh/terminal",
"golang.org/x/net/context",
"golang.org/x/sync/errgroup",
"golang.org/x/sys/unix",
"golang.org/x/text/encoding/unicode",
"golang.org/x/text/transform",
"golang.org/x/time/rate",
"google.golang.org/grpc",
"google.golang.org/grpc/metadata",
]
solver-name = "gps-cdcl"
solver-version = 1

View File

@ -1,5 +1,3 @@
required = ["github.com/influxdata/platform/pkg/data/gen"]
[[constraint]]
name = "collectd.org"
version = "0.3.0"
@ -74,8 +72,8 @@ required = ["github.com/influxdata/platform/pkg/data/gen"]
[[constraint]]
name = "github.com/influxdata/flux"
version = "0.12.0"
version = "= 0.24.0"
[[constraint]]
name = "github.com/spf13/cast"
version = "1.3.x"
version = "1.3.x"

View File

@ -31,9 +31,9 @@ import (
"github.com/influxdata/influxdb/services/storage"
"github.com/influxdata/influxdb/services/subscriber"
"github.com/influxdata/influxdb/services/udp"
"github.com/influxdata/influxdb/storage/reads"
"github.com/influxdata/influxdb/tcp"
"github.com/influxdata/influxdb/tsdb"
"github.com/influxdata/platform/storage/reads"
client "github.com/influxdata/usage-client/v1"
"go.uber.org/zap"

View File

@ -16,8 +16,8 @@ import (
"github.com/gogo/protobuf/types"
"github.com/influxdata/influxdb/models"
"github.com/influxdata/influxdb/services/storage"
"github.com/influxdata/influxdb/storage/reads/datatypes"
"github.com/influxdata/influxql"
"github.com/influxdata/platform/storage/reads/datatypes"
"go.uber.org/zap"
"google.golang.org/grpc"
)

View File

@ -5,12 +5,8 @@ package builtin
import (
"github.com/influxdata/flux"
_ "github.com/influxdata/flux/functions/inputs" // Import the built-in input functions
_ "github.com/influxdata/flux/functions/outputs" // Import the built-in output functions
_ "github.com/influxdata/flux/functions/transformations" // Import the built-in transformations
_ "github.com/influxdata/flux/options" // Import the built-in options
_ "github.com/influxdata/influxdb/flux/functions/inputs" // Import the built-in functions
_ "github.com/influxdata/flux/stdlib"
_ "github.com/influxdata/influxdb/flux/stdlib"
)
func init() {

View File

@ -3,16 +3,17 @@ package control
import (
"github.com/influxdata/flux/control"
"github.com/influxdata/flux/execute"
"github.com/influxdata/influxdb/coordinator"
_ "github.com/influxdata/influxdb/flux/builtin"
"github.com/influxdata/influxdb/flux/functions/inputs"
fstorage "github.com/influxdata/platform/query/functions/inputs/storage"
"github.com/influxdata/influxdb/flux/stdlib/influxdata/influxdb"
v1 "github.com/influxdata/influxdb/flux/stdlib/influxdata/influxdb/v1"
"go.uber.org/zap"
)
type MetaClient = inputs.MetaClient
type Authorizer = inputs.Authorizer
type MetaClient = coordinator.MetaClient
type Authorizer = influxdb.Authorizer
func NewController(mc MetaClient, reader fstorage.Reader, auth Authorizer, authEnabled bool, logger *zap.Logger) *control.Controller {
func NewController(mc MetaClient, reader influxdb.Reader, auth Authorizer, authEnabled bool, logger *zap.Logger) *control.Controller {
// flux
var (
concurrencyQuota = 10
@ -26,13 +27,28 @@ func NewController(mc MetaClient, reader fstorage.Reader, auth Authorizer, authE
Logger: logger,
}
err := inputs.InjectFromDependencies(cc.ExecutorDependencies, inputs.Dependencies{Reader: reader, MetaClient: mc, Authorizer: auth, AuthEnabled: authEnabled})
if err != nil {
if err := influxdb.InjectFromDependencies(cc.ExecutorDependencies, influxdb.Dependencies{
Reader: reader,
MetaClient: mc,
Authorizer: auth,
AuthEnabled: authEnabled,
}); err != nil {
panic(err)
}
err = inputs.InjectBucketDependencies(cc.ExecutorDependencies, inputs.BucketDependencies{MetaClient: mc, Authorizer: auth, AuthEnabled: authEnabled})
if err != nil {
if err := v1.InjectDatabaseDependencies(cc.ExecutorDependencies, v1.DatabaseDependencies{
MetaClient: mc,
Authorizer: auth,
AuthEnabled: authEnabled,
}); err != nil {
panic(err)
}
if err := influxdb.InjectBucketDependencies(cc.ExecutorDependencies, influxdb.BucketDependencies{
MetaClient: mc,
Authorizer: auth,
AuthEnabled: authEnabled,
}); err != nil {
panic(err)
}

View File

@ -1,127 +0,0 @@
package inputs
import (
"strings"
"github.com/influxdata/flux/execute"
"github.com/influxdata/flux/functions/inputs"
"github.com/influxdata/flux/plan"
"github.com/influxdata/influxdb/services/meta"
"github.com/influxdata/influxql"
"github.com/influxdata/platform/query/functions/inputs/storage"
"github.com/pkg/errors"
)
func init() {
execute.RegisterSource(inputs.FromKind, createFromSource)
}
func createFromSource(prSpec plan.ProcedureSpec, dsid execute.DatasetID, a execute.Administration) (execute.Source, error) {
spec := prSpec.(*inputs.FromProcedureSpec)
var w execute.Window
bounds := a.StreamContext().Bounds()
if bounds == nil {
return nil, errors.New("nil bounds passed to from")
}
if spec.WindowSet {
w = execute.Window{
Every: execute.Duration(spec.Window.Every),
Period: execute.Duration(spec.Window.Period),
Round: execute.Duration(spec.Window.Round),
Start: bounds.Start,
}
} else {
duration := execute.Duration(bounds.Stop) - execute.Duration(bounds.Start)
w = execute.Window{
Every: duration,
Period: duration,
Start: bounds.Start,
}
}
currentTime := w.Start + execute.Time(w.Period)
deps := a.Dependencies()[inputs.FromKind].(Dependencies)
var db, rp string
if i := strings.IndexByte(spec.Bucket, '/'); i == -1 {
db = spec.Bucket
} else {
rp = spec.Bucket[i+1:]
db = spec.Bucket[:i]
}
// validate and resolve db/rp
di := deps.MetaClient.Database(db)
if di == nil {
return nil, errors.New("no database")
}
if deps.AuthEnabled {
user := meta.UserFromContext(a.Context())
if user == nil {
return nil, errors.New("createFromSource: no user")
}
if err := deps.Authorizer.AuthorizeDatabase(user, influxql.ReadPrivilege, db); err != nil {
return nil, err
}
}
if rp == "" {
rp = di.DefaultRetentionPolicy
}
if rpi := di.RetentionPolicy(rp); rpi == nil {
return nil, errors.New("invalid retention policy")
}
return storage.NewSource(
dsid,
deps.Reader,
storage.ReadSpec{
Database: db,
RetentionPolicy: rp,
Predicate: spec.Filter,
PointsLimit: spec.PointsLimit,
SeriesLimit: spec.SeriesLimit,
SeriesOffset: spec.SeriesOffset,
Descending: spec.Descending,
OrderByTime: spec.OrderByTime,
GroupMode: storage.ToGroupMode(spec.GroupMode),
GroupKeys: spec.GroupKeys,
AggregateMethod: spec.AggregateMethod,
},
*bounds,
w,
currentTime,
), nil
}
type Authorizer interface {
AuthorizeDatabase(u meta.User, priv influxql.Privilege, database string) error
}
type Dependencies struct {
Reader storage.Reader
MetaClient MetaClient
Authorizer Authorizer
AuthEnabled bool
}
func (d Dependencies) Validate() error {
if d.Reader == nil {
return errors.New("missing reader dependency")
}
if d.AuthEnabled && d.Authorizer == nil {
return errors.New("validate Dependencies: missing Authorizer")
}
return nil
}
func InjectFromDependencies(depsMap execute.Dependencies, deps Dependencies) error {
if err := deps.Validate(); err != nil {
return err
}
depsMap[inputs.FromKind] = deps
return nil
}

View File

@ -1,21 +1,21 @@
package inputs
package influxdb
import (
"errors"
"fmt"
"github.com/influxdata/flux"
"github.com/influxdata/flux/execute"
"github.com/influxdata/flux/functions/inputs"
"github.com/influxdata/flux/memory"
"github.com/influxdata/flux/plan"
"github.com/influxdata/flux/stdlib/influxdata/influxdb"
"github.com/influxdata/flux/values"
"github.com/influxdata/influxdb/services/meta"
"github.com/influxdata/influxql"
"github.com/pkg/errors"
)
func init() {
execute.RegisterSource(inputs.BucketsKind, createBucketsSource)
execute.RegisterSource(influxdb.BucketsKind, createBucketsSource)
}
type BucketsDecoder struct {
@ -34,7 +34,7 @@ func (bd *BucketsDecoder) Fetch() (bool, error) {
func (bd *BucketsDecoder) Decode() (flux.Table, error) {
kb := execute.NewGroupKeyBuilder(nil)
kb.AddKeyValue("organizationID", values.NewString("influxdb"))
kb.AddKeyValue("organizationID", values.NewString(""))
gk, err := kb.Build()
if err != nil {
return nil, err
@ -42,27 +42,27 @@ func (bd *BucketsDecoder) Decode() (flux.Table, error) {
b := execute.NewColListTableBuilder(gk, bd.alloc)
b.AddCol(flux.ColMeta{
_, _ = b.AddCol(flux.ColMeta{
Label: "name",
Type: flux.TString,
})
b.AddCol(flux.ColMeta{
_, _ = b.AddCol(flux.ColMeta{
Label: "id",
Type: flux.TString,
})
b.AddCol(flux.ColMeta{
_, _ = b.AddCol(flux.ColMeta{
Label: "organization",
Type: flux.TString,
})
b.AddCol(flux.ColMeta{
_, _ = b.AddCol(flux.ColMeta{
Label: "organizationID",
Type: flux.TString,
})
b.AddCol(flux.ColMeta{
_, _ = b.AddCol(flux.ColMeta{
Label: "retentionPolicy",
Type: flux.TString,
})
b.AddCol(flux.ColMeta{
_, _ = b.AddCol(flux.ColMeta{
Label: "retentionPeriod",
Type: flux.TInt,
})
@ -79,30 +79,35 @@ func (bd *BucketsDecoder) Decode() (flux.Table, error) {
}
}
for _, bucket := range bd.deps.MetaClient.Databases() {
if hasAccess(bucket.Name) {
rp := bucket.RetentionPolicy(bucket.DefaultRetentionPolicy)
b.AppendString(0, bucket.Name)
b.AppendString(1, "")
b.AppendString(2, "influxdb")
b.AppendString(3, "")
b.AppendString(4, rp.Name)
b.AppendInt(5, rp.Duration.Nanoseconds())
for _, db := range bd.deps.MetaClient.Databases() {
if hasAccess(db.Name) {
for _, rp := range db.RetentionPolicies {
_ = b.AppendString(0, db.Name+"/"+rp.Name)
_ = b.AppendString(1, "")
_ = b.AppendString(2, "influxdb")
_ = b.AppendString(3, "")
_ = b.AppendString(4, rp.Name)
_ = b.AppendInt(5, rp.Duration.Nanoseconds())
}
}
}
return b.Table()
}
func (bd *BucketsDecoder) Close() error {
return nil
}
func createBucketsSource(prSpec plan.ProcedureSpec, dsid execute.DatasetID, a execute.Administration) (execute.Source, error) {
_, ok := prSpec.(*inputs.BucketsProcedureSpec)
_, ok := prSpec.(*influxdb.BucketsProcedureSpec)
if !ok {
return nil, fmt.Errorf("invalid spec type %T", prSpec)
}
// the dependencies used for FromKind are adequate for what we need here
// so there's no need to inject custom dependencies for buckets()
deps := a.Dependencies()[inputs.BucketsKind].(BucketDependencies)
deps := a.Dependencies()[influxdb.BucketsKind].(BucketDependencies)
var user meta.User
if deps.AuthEnabled {
@ -114,7 +119,7 @@ func createBucketsSource(prSpec plan.ProcedureSpec, dsid execute.DatasetID, a ex
bd := &BucketsDecoder{deps: deps, alloc: a.Allocator(), user: user}
return inputs.CreateSourceFromDecoder(bd, dsid, a)
return execute.CreateSourceFromDecoder(bd, dsid, a)
}
@ -143,6 +148,6 @@ func InjectBucketDependencies(depsMap execute.Dependencies, deps BucketDependenc
if err := deps.Validate(); err != nil {
return err
}
depsMap[inputs.BucketsKind] = deps
depsMap[influxdb.BucketsKind] = deps
return nil
}

View File

@ -0,0 +1,744 @@
package influxdb
import (
"fmt"
"strings"
"github.com/influxdata/flux"
"github.com/influxdata/flux/ast"
"github.com/influxdata/flux/execute"
"github.com/influxdata/flux/plan"
"github.com/influxdata/flux/semantic"
"github.com/influxdata/flux/stdlib/influxdata/influxdb"
"github.com/influxdata/flux/stdlib/universe"
"github.com/influxdata/flux/values"
"github.com/influxdata/influxdb/services/meta"
"github.com/influxdata/influxql"
"github.com/pkg/errors"
)
const FromKind = "influxDBFrom"
type FromOpSpec struct {
Bucket string `json:"bucket,omitempty"`
BucketID string `json:"bucketID,omitempty"`
}
func init() {
fromSignature := semantic.FunctionPolySignature{
Parameters: map[string]semantic.PolyType{
"bucket": semantic.String,
"bucketID": semantic.String,
},
Required: nil,
Return: flux.TableObjectType,
}
flux.ReplacePackageValue("influxdata/influxdb", influxdb.FromKind, flux.FunctionValue(FromKind, createFromOpSpec, fromSignature))
flux.RegisterOpSpec(FromKind, newFromOp)
plan.RegisterProcedureSpec(FromKind, newFromProcedure, FromKind)
plan.RegisterPhysicalRules(
FromConversionRule{},
MergeFromRangeRule{},
MergeFromFilterRule{},
FromDistinctRule{},
MergeFromGroupRule{},
FromKeysRule{},
)
execute.RegisterSource(PhysicalFromKind, createFromSource)
}
func createFromOpSpec(args flux.Arguments, a *flux.Administration) (flux.OperationSpec, error) {
spec := new(FromOpSpec)
if bucket, ok, err := args.GetString("bucket"); err != nil {
return nil, err
} else if ok {
spec.Bucket = bucket
}
if bucketID, ok, err := args.GetString("bucketID"); err != nil {
return nil, err
} else if ok {
spec.BucketID = bucketID
}
if spec.Bucket == "" && spec.BucketID == "" {
return nil, errors.New("must specify one of bucket or bucketID")
}
if spec.Bucket != "" && spec.BucketID != "" {
return nil, errors.New("must specify only one of bucket or bucketID")
}
return spec, nil
}
func newFromOp() flux.OperationSpec {
return new(FromOpSpec)
}
func (s *FromOpSpec) Kind() flux.OperationKind {
return FromKind
}
type FromProcedureSpec struct {
Bucket string
BucketID string
}
func newFromProcedure(qs flux.OperationSpec, pa plan.Administration) (plan.ProcedureSpec, error) {
spec, ok := qs.(*FromOpSpec)
if !ok {
return nil, fmt.Errorf("invalid spec type %T", qs)
}
return &FromProcedureSpec{
Bucket: spec.Bucket,
BucketID: spec.BucketID,
}, nil
}
func (s *FromProcedureSpec) Kind() plan.ProcedureKind {
return FromKind
}
func (s *FromProcedureSpec) Copy() plan.ProcedureSpec {
ns := new(FromProcedureSpec)
ns.Bucket = s.Bucket
ns.BucketID = s.BucketID
return ns
}
func (s FromProcedureSpec) PostPhysicalValidate(id plan.NodeID) error {
// FromProcedureSpec has no bounds, so must be invalid.
var bucket string
if len(s.Bucket) > 0 {
bucket = s.Bucket
} else {
bucket = s.BucketID
}
return fmt.Errorf(`%s: results from "%s" must be bounded`, id, bucket)
}
const PhysicalFromKind = "physFrom"
type PhysicalFromProcedureSpec struct {
FromProcedureSpec
plan.DefaultCost
BoundsSet bool
Bounds flux.Bounds
FilterSet bool
Filter *semantic.FunctionExpression
DescendingSet bool
Descending bool
LimitSet bool
PointsLimit int64
SeriesLimit int64
SeriesOffset int64
WindowSet bool
Window plan.WindowSpec
GroupingSet bool
OrderByTime bool
GroupMode flux.GroupMode
GroupKeys []string
AggregateSet bool
AggregateMethod string
}
func (PhysicalFromProcedureSpec) Kind() plan.ProcedureKind {
return PhysicalFromKind
}
func (s *PhysicalFromProcedureSpec) Copy() plan.ProcedureSpec {
ns := new(PhysicalFromProcedureSpec)
ns.Bucket = s.Bucket
ns.BucketID = s.BucketID
ns.BoundsSet = s.BoundsSet
ns.Bounds = s.Bounds
ns.FilterSet = s.FilterSet
ns.Filter = s.Filter.Copy().(*semantic.FunctionExpression)
ns.DescendingSet = s.DescendingSet
ns.Descending = s.Descending
ns.LimitSet = s.LimitSet
ns.PointsLimit = s.PointsLimit
ns.SeriesLimit = s.SeriesLimit
ns.SeriesOffset = s.SeriesOffset
ns.WindowSet = s.WindowSet
ns.Window = s.Window
ns.GroupingSet = s.GroupingSet
ns.OrderByTime = s.OrderByTime
ns.GroupMode = s.GroupMode
ns.GroupKeys = s.GroupKeys
ns.AggregateSet = s.AggregateSet
ns.AggregateMethod = s.AggregateMethod
return ns
}
// TimeBounds implements plan.BoundsAwareProcedureSpec.
func (s *PhysicalFromProcedureSpec) TimeBounds(predecessorBounds *plan.Bounds) *plan.Bounds {
if s.BoundsSet {
bounds := &plan.Bounds{
Start: values.ConvertTime(s.Bounds.Start.Time(s.Bounds.Now)),
Stop: values.ConvertTime(s.Bounds.Stop.Time(s.Bounds.Now)),
}
return bounds
}
return nil
}
func (s PhysicalFromProcedureSpec) PostPhysicalValidate(id plan.NodeID) error {
if !s.BoundsSet || (s.Bounds.Start.IsZero() && s.Bounds.Stop.IsZero()) {
var bucket string
if len(s.Bucket) > 0 {
bucket = s.Bucket
} else {
bucket = s.BucketID
}
return fmt.Errorf(`%s: results from "%s" must be bounded`, id, bucket)
}
return nil
}
// FromConversionRule converts a logical `from` node into a physical `from` node.
// TODO(cwolff): this rule can go away when we require a `range`
// to be pushed into a logical `from` to create a physical `from.`
type FromConversionRule struct {
}
func (FromConversionRule) Name() string {
return "FromConversionRule"
}
func (FromConversionRule) Pattern() plan.Pattern {
return plan.Pat(FromKind)
}
func (FromConversionRule) Rewrite(pn plan.Node) (plan.Node, bool, error) {
logicalFromSpec := pn.ProcedureSpec().(*FromProcedureSpec)
newNode := plan.CreatePhysicalNode(pn.ID(), &PhysicalFromProcedureSpec{
FromProcedureSpec: *logicalFromSpec,
})
plan.ReplaceNode(pn, newNode)
return newNode, true, nil
}
// MergeFromRangeRule pushes a `range` into a `from`.
type MergeFromRangeRule struct{}
// Name returns the name of the rule.
func (rule MergeFromRangeRule) Name() string {
return "MergeFromRangeRule"
}
// Pattern returns the pattern that matches `from -> range`.
func (rule MergeFromRangeRule) Pattern() plan.Pattern {
return plan.Pat(universe.RangeKind, plan.Pat(PhysicalFromKind))
}
// Rewrite attempts to rewrite a `from -> range` into a `FromRange`.
func (rule MergeFromRangeRule) Rewrite(node plan.Node) (plan.Node, bool, error) {
from := node.Predecessors()[0]
fromSpec := from.ProcedureSpec().(*PhysicalFromProcedureSpec)
rangeSpec := node.ProcedureSpec().(*universe.RangeProcedureSpec)
fromRange := fromSpec.Copy().(*PhysicalFromProcedureSpec)
// Set new bounds to `range` bounds initially
fromRange.Bounds = rangeSpec.Bounds
var (
now = rangeSpec.Bounds.Now
start = rangeSpec.Bounds.Start
stop = rangeSpec.Bounds.Stop
)
bounds := &plan.Bounds{
Start: values.ConvertTime(start.Time(now)),
Stop: values.ConvertTime(stop.Time(now)),
}
// Intersect bounds if `from` already bounded
if fromSpec.BoundsSet {
now = fromSpec.Bounds.Now
start = fromSpec.Bounds.Start
stop = fromSpec.Bounds.Stop
fromBounds := &plan.Bounds{
Start: values.ConvertTime(start.Time(now)),
Stop: values.ConvertTime(stop.Time(now)),
}
bounds = bounds.Intersect(fromBounds)
fromRange.Bounds = flux.Bounds{
Start: flux.Time{Absolute: bounds.Start.Time()},
Stop: flux.Time{Absolute: bounds.Stop.Time()},
}
}
fromRange.BoundsSet = true
// Finally merge nodes into single operation
merged, err := plan.MergeToPhysicalNode(node, from, fromRange)
if err != nil {
return nil, false, err
}
return merged, true, nil
}
// MergeFromFilterRule is a rule that pushes filters into from procedures to be evaluated in the storage layer.
// This rule is likely to be replaced by a more generic rule when we have a better
// framework for pushing filters, etc into sources.
type MergeFromFilterRule struct{}
func (MergeFromFilterRule) Name() string {
return "MergeFromFilterRule"
}
func (MergeFromFilterRule) Pattern() plan.Pattern {
return plan.Pat(universe.FilterKind, plan.Pat(PhysicalFromKind))
}
func (MergeFromFilterRule) Rewrite(filterNode plan.Node) (plan.Node, bool, error) {
filterSpec := filterNode.ProcedureSpec().(*universe.FilterProcedureSpec)
fromNode := filterNode.Predecessors()[0]
fromSpec := fromNode.ProcedureSpec().(*PhysicalFromProcedureSpec)
if fromSpec.AggregateSet || fromSpec.GroupingSet {
return filterNode, false, nil
}
bodyExpr, ok := filterSpec.Fn.Block.Body.(semantic.Expression)
if !ok {
return filterNode, false, nil
}
if len(filterSpec.Fn.Block.Parameters.List) != 1 {
// I would expect that type checking would catch this, but just to be safe...
return filterNode, false, nil
}
paramName := filterSpec.Fn.Block.Parameters.List[0].Key.Name
pushable, notPushable, err := semantic.PartitionPredicates(bodyExpr, func(e semantic.Expression) (bool, error) {
return isPushableExpr(paramName, e)
})
if err != nil {
return nil, false, err
}
if pushable == nil {
// Nothing could be pushed down, no rewrite can happen
return filterNode, false, nil
}
newFromSpec := fromSpec.Copy().(*PhysicalFromProcedureSpec)
if newFromSpec.FilterSet {
newBody := semantic.ExprsToConjunction(newFromSpec.Filter.Block.Body.(semantic.Expression), pushable)
newFromSpec.Filter.Block.Body = newBody
} else {
newFromSpec.FilterSet = true
newFromSpec.Filter = filterSpec.Fn.Copy().(*semantic.FunctionExpression)
newFromSpec.Filter.Block.Body = pushable
}
if notPushable == nil {
// All predicates could be pushed down, so eliminate the filter
mergedNode, err := plan.MergeToPhysicalNode(filterNode, fromNode, newFromSpec)
if err != nil {
return nil, false, err
}
return mergedNode, true, nil
}
err = fromNode.ReplaceSpec(newFromSpec)
if err != nil {
return nil, false, err
}
newFilterSpec := filterSpec.Copy().(*universe.FilterProcedureSpec)
newFilterSpec.Fn.Block.Body = notPushable
err = filterNode.ReplaceSpec(newFilterSpec)
if err != nil {
return nil, false, err
}
return filterNode, true, nil
}
// isPushableExpr determines if a predicate expression can be pushed down into the storage layer.
func isPushableExpr(paramName string, expr semantic.Expression) (bool, error) {
switch e := expr.(type) {
case *semantic.LogicalExpression:
b, err := isPushableExpr(paramName, e.Left)
if err != nil {
return false, err
}
if !b {
return false, nil
}
return isPushableExpr(paramName, e.Right)
case *semantic.BinaryExpression:
if isPushablePredicate(paramName, e) {
return true, nil
}
}
return false, nil
}
func isPushablePredicate(paramName string, be *semantic.BinaryExpression) bool {
// Manual testing seems to indicate that (at least right now) we can
// only handle predicates of the form <fn param>.<property> <op> <literal>
// and the literal must be on the RHS.
if !isLiteral(be.Right) {
return false
}
if isField(paramName, be.Left) && isPushableFieldOperator(be.Operator) {
return true
}
if isTag(paramName, be.Left) && isPushableTagOperator(be.Operator) {
return true
}
return false
}
func isLiteral(e semantic.Expression) bool {
switch e.(type) {
case *semantic.StringLiteral:
return true
case *semantic.IntegerLiteral:
return true
case *semantic.BooleanLiteral:
return true
case *semantic.FloatLiteral:
return true
case *semantic.RegexpLiteral:
return true
}
return false
}
const fieldValueProperty = "_value"
func isTag(paramName string, e semantic.Expression) bool {
memberExpr := validateMemberExpr(paramName, e)
return memberExpr != nil && memberExpr.Property != fieldValueProperty
}
func isField(paramName string, e semantic.Expression) bool {
memberExpr := validateMemberExpr(paramName, e)
return memberExpr != nil && memberExpr.Property == fieldValueProperty
}
func validateMemberExpr(paramName string, e semantic.Expression) *semantic.MemberExpression {
memberExpr, ok := e.(*semantic.MemberExpression)
if !ok {
return nil
}
idExpr, ok := memberExpr.Object.(*semantic.IdentifierExpression)
if !ok {
return nil
}
if idExpr.Name != paramName {
return nil
}
return memberExpr
}
func isPushableTagOperator(kind ast.OperatorKind) bool {
pushableOperators := []ast.OperatorKind{
ast.EqualOperator,
ast.NotEqualOperator,
ast.RegexpMatchOperator,
ast.NotRegexpMatchOperator,
}
for _, op := range pushableOperators {
if op == kind {
return true
}
}
return false
}
func isPushableFieldOperator(kind ast.OperatorKind) bool {
if isPushableTagOperator(kind) {
return true
}
// Fields can be filtered by anything that tags can be filtered by,
// plus range operators.
moreOperators := []ast.OperatorKind{
ast.LessThanEqualOperator,
ast.LessThanOperator,
ast.GreaterThanEqualOperator,
ast.GreaterThanOperator,
}
for _, op := range moreOperators {
if op == kind {
return true
}
}
return false
}
type FromDistinctRule struct {
}
func (FromDistinctRule) Name() string {
return "FromDistinctRule"
}
func (FromDistinctRule) Pattern() plan.Pattern {
return plan.Pat(universe.DistinctKind, plan.Pat(PhysicalFromKind))
}
func (FromDistinctRule) Rewrite(distinctNode plan.Node) (plan.Node, bool, error) {
fromNode := distinctNode.Predecessors()[0]
distinctSpec := distinctNode.ProcedureSpec().(*universe.DistinctProcedureSpec)
fromSpec := fromNode.ProcedureSpec().(*PhysicalFromProcedureSpec)
if fromSpec.LimitSet && fromSpec.PointsLimit == -1 {
return distinctNode, false, nil
}
groupStar := !fromSpec.GroupingSet && distinctSpec.Column != execute.DefaultValueColLabel && distinctSpec.Column != execute.DefaultTimeColLabel
groupByColumn := fromSpec.GroupingSet && len(fromSpec.GroupKeys) > 0 &&
((fromSpec.GroupMode == flux.GroupModeBy && execute.ContainsStr(fromSpec.GroupKeys, distinctSpec.Column)) ||
(fromSpec.GroupMode == flux.GroupModeExcept && !execute.ContainsStr(fromSpec.GroupKeys, distinctSpec.Column)))
if groupStar || groupByColumn {
newFromSpec := fromSpec.Copy().(*PhysicalFromProcedureSpec)
newFromSpec.LimitSet = true
newFromSpec.PointsLimit = -1
if err := fromNode.ReplaceSpec(newFromSpec); err != nil {
return nil, false, err
}
return distinctNode, true, nil
}
return distinctNode, false, nil
}
type MergeFromGroupRule struct {
}
func (MergeFromGroupRule) Name() string {
return "MergeFromGroupRule"
}
func (MergeFromGroupRule) Pattern() plan.Pattern {
return plan.Pat(universe.GroupKind, plan.Pat(PhysicalFromKind))
}
func (MergeFromGroupRule) Rewrite(groupNode plan.Node) (plan.Node, bool, error) {
fromNode := groupNode.Predecessors()[0]
groupSpec := groupNode.ProcedureSpec().(*universe.GroupProcedureSpec)
fromSpec := fromNode.ProcedureSpec().(*PhysicalFromProcedureSpec)
if fromSpec.GroupingSet ||
fromSpec.LimitSet ||
groupSpec.GroupMode != flux.GroupModeBy {
return groupNode, false, nil
}
for _, c := range groupSpec.GroupKeys {
// Storage can only do grouping over tag keys.
// Note: _start and _stop are okay, since storage is always implicitly grouping by them anyway.
if c == execute.DefaultTimeColLabel || c == execute.DefaultValueColLabel {
return groupNode, false, nil
}
}
newFromSpec := fromSpec.Copy().(*PhysicalFromProcedureSpec)
newFromSpec.GroupingSet = true
newFromSpec.GroupMode = groupSpec.GroupMode
newFromSpec.GroupKeys = groupSpec.GroupKeys
merged, err := plan.MergeToPhysicalNode(groupNode, fromNode, newFromSpec)
if err != nil {
return nil, false, err
}
return merged, true, nil
}
type FromKeysRule struct {
}
func (FromKeysRule) Name() string {
return "FromKeysRule"
}
func (FromKeysRule) Pattern() plan.Pattern {
return plan.Pat(universe.KeysKind, plan.Pat(PhysicalFromKind))
}
func (FromKeysRule) Rewrite(keysNode plan.Node) (plan.Node, bool, error) {
fromNode := keysNode.Predecessors()[0]
fromSpec := fromNode.ProcedureSpec().(*PhysicalFromProcedureSpec)
if fromSpec.LimitSet && fromSpec.PointsLimit == -1 {
return keysNode, false, nil
}
newFromSpec := fromSpec.Copy().(*PhysicalFromProcedureSpec)
newFromSpec.LimitSet = true
newFromSpec.PointsLimit = -1
if err := fromNode.ReplaceSpec(newFromSpec); err != nil {
return nil, false, err
}
return keysNode, true, nil
}
func createFromSource(prSpec plan.ProcedureSpec, dsid execute.DatasetID, a execute.Administration) (execute.Source, error) {
spec := prSpec.(*PhysicalFromProcedureSpec)
var w execute.Window
bounds := a.StreamContext().Bounds()
if bounds == nil {
return nil, errors.New("nil bounds passed to from")
}
// Note: currently no planner rules will push a window() into from()
// so the following is dead code.
if spec.WindowSet {
w = execute.Window{
Every: execute.Duration(spec.Window.Every),
Period: execute.Duration(spec.Window.Period),
Offset: execute.Duration(spec.Window.Offset),
}
} else {
duration := execute.Duration(bounds.Stop) - execute.Duration(bounds.Start)
w = execute.Window{
Every: duration,
Period: duration,
Offset: bounds.Start.Remainder(duration),
}
}
currentTime := bounds.Start + execute.Time(w.Period)
deps := a.Dependencies()[FromKind].(Dependencies)
if len(spec.BucketID) != 0 {
return nil, errors.New("cannot refer to buckets by their id in 1.x")
}
var db, rp string
if i := strings.IndexByte(spec.Bucket, '/'); i == -1 {
db = spec.Bucket
} else {
rp = spec.Bucket[i+1:]
db = spec.Bucket[:i]
}
// validate and resolve db/rp
di := deps.MetaClient.Database(db)
if di == nil {
return nil, errors.New("no database")
}
if deps.AuthEnabled {
user := meta.UserFromContext(a.Context())
if user == nil {
return nil, errors.New("createFromSource: no user")
}
if err := deps.Authorizer.AuthorizeDatabase(user, influxql.ReadPrivilege, db); err != nil {
return nil, err
}
}
if rp == "" {
rp = di.DefaultRetentionPolicy
}
if rpi := di.RetentionPolicy(rp); rpi == nil {
return nil, errors.New("invalid retention policy")
}
return NewSource(
dsid,
deps.Reader,
ReadSpec{
Database: db,
RetentionPolicy: rp,
Predicate: spec.Filter,
PointsLimit: spec.PointsLimit,
SeriesLimit: spec.SeriesLimit,
SeriesOffset: spec.SeriesOffset,
Descending: spec.Descending,
OrderByTime: spec.OrderByTime,
GroupMode: ToGroupMode(spec.GroupMode),
GroupKeys: spec.GroupKeys,
AggregateMethod: spec.AggregateMethod,
},
*bounds,
w,
currentTime,
a.Allocator(),
), nil
}
type Authorizer interface {
AuthorizeDatabase(u meta.User, priv influxql.Privilege, database string) error
}
type Dependencies struct {
Reader Reader
MetaClient MetaClient
Authorizer Authorizer
AuthEnabled bool
}
func (d Dependencies) Validate() error {
if d.Reader == nil {
return errors.New("missing reader dependency")
}
if d.MetaClient == nil {
return errors.New("missing meta client dependency")
}
if d.AuthEnabled && d.Authorizer == nil {
return errors.New("validate Dependencies: missing Authorizer")
}
return nil
}
func InjectFromDependencies(depsMap execute.Dependencies, deps Dependencies) error {
if err := deps.Validate(); err != nil {
return err
}
depsMap[FromKind] = deps
return nil
}

View File

@ -0,0 +1,176 @@
package influxdb
import (
"context"
"fmt"
"log"
"math"
"github.com/influxdata/flux"
"github.com/influxdata/flux/execute"
"github.com/influxdata/flux/memory"
"github.com/influxdata/flux/semantic"
)
// source performs storage reads
type source struct {
id execute.DatasetID
reader Reader
readSpec ReadSpec
window execute.Window
bounds execute.Bounds
alloc *memory.Allocator
ts []execute.Transformation
currentTime execute.Time
overflow bool
}
func NewSource(id execute.DatasetID, r Reader, readSpec ReadSpec, bounds execute.Bounds, w execute.Window, currentTime execute.Time, alloc *memory.Allocator) execute.Source {
return &source{
id: id,
reader: r,
readSpec: readSpec,
bounds: bounds,
window: w,
currentTime: currentTime,
alloc: alloc,
}
}
func (s *source) AddTransformation(t execute.Transformation) {
s.ts = append(s.ts, t)
}
func (s *source) Run(ctx context.Context) {
err := s.run(ctx)
for _, t := range s.ts {
t.Finish(s.id, err)
}
}
func (s *source) run(ctx context.Context) error {
//TODO(nathanielc): Pass through context to actual network I/O.
for tables, mark, ok := s.next(ctx); ok; tables, mark, ok = s.next(ctx) {
err := tables.Do(func(tbl flux.Table) error {
for _, t := range s.ts {
if err := t.Process(s.id, tbl); err != nil {
return err
}
//TODO(nathanielc): Also add mechanism to send UpdateProcessingTime calls, when no data is arriving.
// This is probably not needed for this source, but other sources should do so.
if err := t.UpdateProcessingTime(s.id, execute.Now()); err != nil {
return err
}
}
return nil
})
if err != nil {
return err
}
for _, t := range s.ts {
if err := t.UpdateWatermark(s.id, mark); err != nil {
return err
}
}
}
return nil
}
func (s *source) next(ctx context.Context) (flux.TableIterator, execute.Time, bool) {
if s.overflow {
return nil, 0, false
}
start := s.currentTime - execute.Time(s.window.Period)
stop := s.currentTime
if stop > s.bounds.Stop {
return nil, 0, false
}
// Check if we will overflow, if so we are done after this pass
every := execute.Time(s.window.Every)
if every > 0 {
s.overflow = s.currentTime > math.MaxInt64-every
} else {
s.overflow = s.currentTime < math.MinInt64-every
}
s.currentTime = s.currentTime + every
bi, err := s.reader.Read(
ctx,
s.readSpec,
start,
stop,
s.alloc,
)
if err != nil {
log.Println("E!", err)
return nil, 0, false
}
return bi, stop, true
}
type GroupMode int
const (
// GroupModeDefault specifies the default grouping mode, which is GroupModeAll.
GroupModeDefault GroupMode = 0
// GroupModeNone merges all series into a single group.
GroupModeNone GroupMode = 1 << iota
// GroupModeAll produces a separate table for each series.
GroupModeAll
// GroupModeBy produces a table for each unique value of the specified GroupKeys.
GroupModeBy
// GroupModeExcept produces a table for the unique values of all keys, except those specified by GroupKeys.
GroupModeExcept
)
// ToGroupMode accepts the group mode from Flux and produces the appropriate storage group mode.
func ToGroupMode(fluxMode flux.GroupMode) GroupMode {
switch fluxMode {
case flux.GroupModeNone:
return GroupModeDefault
case flux.GroupModeBy:
return GroupModeBy
case flux.GroupModeExcept:
return GroupModeExcept
default:
panic(fmt.Sprint("unknown group mode: ", fluxMode))
}
}
type ReadSpec struct {
Database string
RetentionPolicy string
RAMLimit uint64
Hosts []string
Predicate *semantic.FunctionExpression
PointsLimit int64
SeriesLimit int64
SeriesOffset int64
Descending bool
AggregateMethod string
// OrderByTime indicates that series reads should produce all
// series for a time before producing any series for a larger time.
// By default this is false meaning all values of time are produced for a given series,
// before any values are produced from the next series.
OrderByTime bool
// GroupMode instructs
GroupMode GroupMode
// GroupKeys is the list of dimensions along which to group.
//
// When GroupMode is GroupModeBy, the results will be grouped by the specified keys.
// When GroupMode is GroupModeExcept, the results will be grouped by all keys, except those specified.
GroupKeys []string
}
type Reader interface {
Read(ctx context.Context, rs ReadSpec, start, stop execute.Time, alloc *memory.Allocator) (flux.TableIterator, error)
Close()
}

View File

@ -0,0 +1,199 @@
package v1
import (
"context"
"errors"
"fmt"
"github.com/influxdata/flux"
"github.com/influxdata/flux/execute"
"github.com/influxdata/flux/memory"
"github.com/influxdata/flux/plan"
v1 "github.com/influxdata/flux/stdlib/influxdata/influxdb/v1"
"github.com/influxdata/flux/values"
"github.com/influxdata/influxdb/coordinator"
"github.com/influxdata/influxdb/flux/stdlib/influxdata/influxdb"
"github.com/influxdata/influxdb/services/meta"
"github.com/influxdata/influxql"
)
const DatabasesKind = v1.DatabasesKind
type DatabasesOpSpec struct {
}
func init() {
flux.ReplacePackageValue("influxdata/influxdb/v1", DatabasesKind, flux.FunctionValue(DatabasesKind, createDatabasesOpSpec, v1.DatabasesSignature))
flux.RegisterOpSpec(DatabasesKind, newDatabasesOp)
plan.RegisterProcedureSpec(DatabasesKind, newDatabasesProcedure, DatabasesKind)
}
func createDatabasesOpSpec(args flux.Arguments, a *flux.Administration) (flux.OperationSpec, error) {
spec := new(DatabasesOpSpec)
return spec, nil
}
func newDatabasesOp() flux.OperationSpec {
return new(DatabasesOpSpec)
}
func (s *DatabasesOpSpec) Kind() flux.OperationKind {
return DatabasesKind
}
type DatabasesProcedureSpec struct {
plan.DefaultCost
}
func newDatabasesProcedure(qs flux.OperationSpec, pa plan.Administration) (plan.ProcedureSpec, error) {
_, ok := qs.(*DatabasesOpSpec)
if !ok {
return nil, fmt.Errorf("invalid spec type %T", qs)
}
return &DatabasesProcedureSpec{}, nil
}
func (s *DatabasesProcedureSpec) Kind() plan.ProcedureKind {
return DatabasesKind
}
func (s *DatabasesProcedureSpec) Copy() plan.ProcedureSpec {
ns := new(DatabasesProcedureSpec)
return ns
}
func init() {
execute.RegisterSource(DatabasesKind, createDatabasesSource)
}
type DatabasesDecoder struct {
deps *DatabaseDependencies
databases []meta.DatabaseInfo
user meta.User
alloc *memory.Allocator
ctx context.Context
}
func (bd *DatabasesDecoder) Connect() error {
return nil
}
func (bd *DatabasesDecoder) Fetch() (bool, error) {
bd.databases = bd.deps.MetaClient.Databases()
return false, nil
}
func (bd *DatabasesDecoder) Decode() (flux.Table, error) {
kb := execute.NewGroupKeyBuilder(nil)
kb.AddKeyValue("organizationID", values.NewString(""))
gk, err := kb.Build()
if err != nil {
return nil, err
}
b := execute.NewColListTableBuilder(gk, bd.alloc)
if _, err := b.AddCol(flux.ColMeta{
Label: "organizationID",
Type: flux.TString,
}); err != nil {
return nil, err
}
if _, err := b.AddCol(flux.ColMeta{
Label: "databaseName",
Type: flux.TString,
}); err != nil {
return nil, err
}
if _, err := b.AddCol(flux.ColMeta{
Label: "retentionPolicy",
Type: flux.TString,
}); err != nil {
return nil, err
}
if _, err := b.AddCol(flux.ColMeta{
Label: "retentionPeriod",
Type: flux.TInt,
}); err != nil {
return nil, err
}
if _, err := b.AddCol(flux.ColMeta{
Label: "default",
Type: flux.TBool,
}); err != nil {
return nil, err
}
if _, err := b.AddCol(flux.ColMeta{
Label: "bucketId",
Type: flux.TString,
}); err != nil {
return nil, err
}
var hasAccess func(db string) bool
if bd.user == nil {
hasAccess = func(db string) bool {
return true
}
} else {
hasAccess = func(db string) bool {
return bd.deps.Authorizer.AuthorizeDatabase(bd.user, influxql.ReadPrivilege, db) == nil ||
bd.deps.Authorizer.AuthorizeDatabase(bd.user, influxql.WritePrivilege, db) == nil
}
}
for _, db := range bd.databases {
if hasAccess(db.Name) {
for _, rp := range db.RetentionPolicies {
_ = b.AppendString(0, "")
_ = b.AppendString(1, db.Name)
_ = b.AppendString(2, rp.Name)
_ = b.AppendInt(3, rp.Duration.Nanoseconds())
_ = b.AppendBool(4, db.DefaultRetentionPolicy == rp.Name)
_ = b.AppendString(5, "")
}
}
}
return b.Table()
}
func (bd *DatabasesDecoder) Close() error {
return nil
}
func createDatabasesSource(prSpec plan.ProcedureSpec, dsid execute.DatasetID, a execute.Administration) (execute.Source, error) {
_, ok := prSpec.(*DatabasesProcedureSpec)
if !ok {
return nil, fmt.Errorf("invalid spec type %T", prSpec)
}
deps := a.Dependencies()[DatabasesKind].(DatabaseDependencies)
var user meta.User
if deps.AuthEnabled {
user = meta.UserFromContext(a.Context())
if user == nil {
return nil, errors.New("createDatabasesSource: no user")
}
}
bd := &DatabasesDecoder{deps: &deps, alloc: a.Allocator(), ctx: a.Context(), user: user}
return execute.CreateSourceFromDecoder(bd, dsid, a)
}
type DatabaseDependencies struct {
MetaClient coordinator.MetaClient
Authorizer influxdb.Authorizer
AuthEnabled bool
}
func InjectDatabaseDependencies(depsMap execute.Dependencies, deps DatabaseDependencies) error {
if deps.MetaClient == nil {
return errors.New("missing meta client dependency")
}
if deps.AuthEnabled && deps.Authorizer == nil {
return errors.New("missing authorizer with auth enabled")
}
depsMap[DatabasesKind] = deps
return nil
}

7
flux/stdlib/packages.go Normal file
View File

@ -0,0 +1,7 @@
package stdlib
// Import all stdlib packages
import (
_ "github.com/influxdata/influxdb/flux/stdlib/influxdata/influxdb"
_ "github.com/influxdata/influxdb/flux/stdlib/influxdata/influxdb/v1"
)

View File

@ -4,9 +4,9 @@ import (
"context"
"github.com/influxdata/influxdb/models"
"github.com/influxdata/influxdb/storage/reads"
"github.com/influxdata/influxdb/storage/reads/datatypes"
"github.com/influxdata/influxdb/tsdb"
"github.com/influxdata/platform/storage/reads"
"github.com/influxdata/platform/storage/reads/datatypes"
"go.uber.org/zap"
)

187
mock/storage_reads.go Normal file
View File

@ -0,0 +1,187 @@
package mock
import (
"github.com/influxdata/influxdb/models"
"github.com/influxdata/influxdb/storage/reads"
"github.com/influxdata/influxdb/storage/reads/datatypes"
"github.com/influxdata/influxdb/tsdb/cursors"
"google.golang.org/grpc/metadata"
)
type ResponseStream struct {
SendFunc func(*datatypes.ReadResponse) error
SetTrailerFunc func(metadata.MD)
}
func NewResponseStream() *ResponseStream {
return &ResponseStream{
SendFunc: func(*datatypes.ReadResponse) error { return nil },
SetTrailerFunc: func(mds metadata.MD) {},
}
}
func (s *ResponseStream) Send(r *datatypes.ReadResponse) error {
return s.SendFunc(r)
}
func (s *ResponseStream) SetTrailer(m metadata.MD) {
s.SetTrailerFunc(m)
}
type ResultSet struct {
NextFunc func() bool
CursorFunc func() cursors.Cursor
TagsFunc func() models.Tags
CloseFunc func()
ErrFunc func() error
StatsFunc func() cursors.CursorStats
}
func NewResultSet() *ResultSet {
return &ResultSet{
NextFunc: func() bool { return false },
CursorFunc: func() cursors.Cursor { return nil },
TagsFunc: func() models.Tags { return nil },
CloseFunc: func() {},
ErrFunc: func() error { return nil },
StatsFunc: func() cursors.CursorStats { return cursors.CursorStats{} },
}
}
func (rs *ResultSet) Next() bool {
return rs.NextFunc()
}
func (rs *ResultSet) Cursor() cursors.Cursor {
return rs.CursorFunc()
}
func (rs *ResultSet) Tags() models.Tags {
return rs.TagsFunc()
}
func (rs *ResultSet) Close() {
rs.CloseFunc()
}
func (rs *ResultSet) Err() error {
return rs.ErrFunc()
}
func (rs *ResultSet) Stats() cursors.CursorStats {
return rs.StatsFunc()
}
type GroupResultSet struct {
NextFunc func() reads.GroupCursor
CloseFunc func()
ErrFunc func() error
}
func NewGroupResultSet() *GroupResultSet {
return &GroupResultSet{
NextFunc: func() reads.GroupCursor { return nil },
CloseFunc: func() {},
ErrFunc: func() error { return nil },
}
}
func (rs *GroupResultSet) Next() reads.GroupCursor {
return rs.NextFunc()
}
func (rs *GroupResultSet) Close() {
rs.CloseFunc()
}
func (rs *GroupResultSet) Err() error {
return rs.ErrFunc()
}
type IntegerArrayCursor struct {
CloseFunc func()
Errfunc func() error
StatsFunc func() cursors.CursorStats
NextFunc func() *cursors.IntegerArray
}
func NewIntegerArrayCursor() *IntegerArrayCursor {
return &IntegerArrayCursor{
CloseFunc: func() {},
Errfunc: func() error { return nil },
StatsFunc: func() cursors.CursorStats { return cursors.CursorStats{} },
NextFunc: func() *cursors.IntegerArray { return &cursors.IntegerArray{} },
}
}
func (c *IntegerArrayCursor) Close() {
c.CloseFunc()
}
func (c *IntegerArrayCursor) Err() error {
return c.Errfunc()
}
func (c *IntegerArrayCursor) Stats() cursors.CursorStats {
return c.StatsFunc()
}
func (c *IntegerArrayCursor) Next() *cursors.IntegerArray {
return c.NextFunc()
}
type GroupCursor struct {
NextFunc func() bool
CursorFunc func() cursors.Cursor
TagsFunc func() models.Tags
KeysFunc func() [][]byte
PartitionKeyValsFunc func() [][]byte
CloseFunc func()
ErrFunc func() error
StatsFunc func() cursors.CursorStats
}
func NewGroupCursor() *GroupCursor {
return &GroupCursor{
NextFunc: func() bool { return false },
CursorFunc: func() cursors.Cursor { return nil },
TagsFunc: func() models.Tags { return nil },
KeysFunc: func() [][]byte { return nil },
PartitionKeyValsFunc: func() [][]byte { return nil },
CloseFunc: func() {},
ErrFunc: func() error { return nil },
StatsFunc: func() cursors.CursorStats { return cursors.CursorStats{} },
}
}
func (c *GroupCursor) Next() bool {
return c.NextFunc()
}
func (c *GroupCursor) Cursor() cursors.Cursor {
return c.CursorFunc()
}
func (c *GroupCursor) Tags() models.Tags {
return c.TagsFunc()
}
func (c *GroupCursor) Keys() [][]byte {
return c.KeysFunc()
}
func (c *GroupCursor) PartitionKeyVals() [][]byte {
return c.PartitionKeyValsFunc()
}
func (c *GroupCursor) Close() {
c.CloseFunc()
}
func (c *GroupCursor) Err() error {
return c.ErrFunc()
}
func (c *GroupCursor) Stats() cursors.CursorStats {
return c.StatsFunc()
}

View File

@ -16,7 +16,19 @@ import (
"unicode/utf8"
"github.com/influxdata/influxdb/pkg/escape"
"github.com/influxdata/platform/models"
)
// Values used to store the field key and measurement name as special internal
// tags.
const (
FieldKeyTagKey = "\xff"
MeasurementTagKey = "\x00"
)
// Predefined byte representations of special tag keys.
var (
FieldKeyTagKeyBytes = []byte(FieldKeyTagKey)
MeasurementTagKeyBytes = []byte(MeasurementTagKey)
)
type escapeSet struct {
@ -1882,7 +1894,10 @@ func (p *point) Split(size int) []Point {
}
// Tag represents a single key/value tag pair.
type Tag = models.Tag
type Tag struct {
Key []byte
Value []byte
}
// NewTag returns a new Tag.
func NewTag(key, value []byte) Tag {
@ -1892,8 +1907,38 @@ func NewTag(key, value []byte) Tag {
}
}
// Size returns the size of the key and value.
func (t Tag) Size() int { return len(t.Key) + len(t.Value) }
// Clone returns a shallow copy of Tag.
//
// Tags associated with a Point created by ParsePointsWithPrecision will hold references to the byte slice that was parsed.
// Use Clone to create a Tag with new byte slices that do not refer to the argument to ParsePointsWithPrecision.
func (t Tag) Clone() Tag {
other := Tag{
Key: make([]byte, len(t.Key)),
Value: make([]byte, len(t.Value)),
}
copy(other.Key, t.Key)
copy(other.Value, t.Value)
return other
}
// String returns the string reprsentation of the tag.
func (t *Tag) String() string {
var buf bytes.Buffer
buf.WriteByte('{')
buf.WriteString(string(t.Key))
buf.WriteByte(' ')
buf.WriteString(string(t.Value))
buf.WriteByte('}')
return buf.String()
}
// Tags represents a sorted list of tags.
type Tags = models.Tags
type Tags []Tag
// NewTags returns a new Tags from a map.
func NewTags(m map[string]string) Tags {
@ -1908,6 +1953,89 @@ func NewTags(m map[string]string) Tags {
return a
}
// Keys returns the list of keys for a tag set.
func (a Tags) Keys() []string {
if len(a) == 0 {
return nil
}
keys := make([]string, len(a))
for i, tag := range a {
keys[i] = string(tag.Key)
}
return keys
}
// Values returns the list of values for a tag set.
func (a Tags) Values() []string {
if len(a) == 0 {
return nil
}
values := make([]string, len(a))
for i, tag := range a {
values[i] = string(tag.Value)
}
return values
}
// String returns the string representation of the tags.
func (a Tags) String() string {
var buf bytes.Buffer
buf.WriteByte('[')
for i := range a {
buf.WriteString(a[i].String())
if i < len(a)-1 {
buf.WriteByte(' ')
}
}
buf.WriteByte(']')
return buf.String()
}
// Size returns the number of bytes needed to store all tags. Note, this is
// the number of bytes needed to store all keys and values and does not account
// for data structures or delimiters for example.
func (a Tags) Size() int {
var total int
for i := range a {
total += a[i].Size()
}
return total
}
// Clone returns a copy of the slice where the elements are a result of calling `Clone` on the original elements
//
// Tags associated with a Point created by ParsePointsWithPrecision will hold references to the byte slice that was parsed.
// Use Clone to create Tags with new byte slices that do not refer to the argument to ParsePointsWithPrecision.
func (a Tags) Clone() Tags {
if len(a) == 0 {
return nil
}
others := make(Tags, len(a))
for i := range a {
others[i] = a[i].Clone()
}
return others
}
func (a Tags) Len() int { return len(a) }
func (a Tags) Less(i, j int) bool { return bytes.Compare(a[i].Key, a[j].Key) == -1 }
func (a Tags) Swap(i, j int) { a[i], a[j] = a[j], a[i] }
// Equal returns true if a equals other.
func (a Tags) Equal(other Tags) bool {
if len(a) != len(other) {
return false
}
for i := range a {
if !bytes.Equal(a[i].Key, other[i].Key) || !bytes.Equal(a[i].Value, other[i].Value) {
return false
}
}
return true
}
// CompareTags returns -1 if a < b, 1 if a > b, and 0 if a == b.
func CompareTags(a, b Tags) int {
// Compare each key & value until a mismatch.
@ -1931,6 +2059,151 @@ func CompareTags(a, b Tags) int {
return 0
}
// Get returns the value for a key.
func (a Tags) Get(key []byte) []byte {
// OPTIMIZE: Use sort.Search if tagset is large.
for _, t := range a {
if bytes.Equal(t.Key, key) {
return t.Value
}
}
return nil
}
// GetString returns the string value for a string key.
func (a Tags) GetString(key string) string {
return string(a.Get([]byte(key)))
}
// Set sets the value for a key.
func (a *Tags) Set(key, value []byte) {
for i, t := range *a {
if bytes.Equal(t.Key, key) {
(*a)[i].Value = value
return
}
}
*a = append(*a, Tag{Key: key, Value: value})
sort.Sort(*a)
}
// SetString sets the string value for a string key.
func (a *Tags) SetString(key, value string) {
a.Set([]byte(key), []byte(value))
}
// Delete removes a tag by key.
func (a *Tags) Delete(key []byte) {
for i, t := range *a {
if bytes.Equal(t.Key, key) {
copy((*a)[i:], (*a)[i+1:])
(*a)[len(*a)-1] = Tag{}
*a = (*a)[:len(*a)-1]
return
}
}
}
// Map returns a map representation of the tags.
func (a Tags) Map() map[string]string {
m := make(map[string]string, len(a))
for _, t := range a {
m[string(t.Key)] = string(t.Value)
}
return m
}
// Merge merges the tags combining the two. If both define a tag with the
// same key, the merged value overwrites the old value.
// A new map is returned.
func (a Tags) Merge(other map[string]string) Tags {
merged := make(map[string]string, len(a)+len(other))
for _, t := range a {
merged[string(t.Key)] = string(t.Value)
}
for k, v := range other {
merged[k] = v
}
return NewTags(merged)
}
// HashKey hashes all of a tag's keys.
func (a Tags) HashKey() []byte {
return a.AppendHashKey(nil)
}
func (a Tags) needsEscape() bool {
for i := range a {
t := &a[i]
for j := range tagEscapeCodes {
c := &tagEscapeCodes[j]
if bytes.IndexByte(t.Key, c.k[0]) != -1 || bytes.IndexByte(t.Value, c.k[0]) != -1 {
return true
}
}
}
return false
}
// AppendHashKey appends the result of hashing all of a tag's keys and values to dst and returns the extended buffer.
func (a Tags) AppendHashKey(dst []byte) []byte {
// Empty maps marshal to empty bytes.
if len(a) == 0 {
return dst
}
// Type invariant: Tags are sorted
sz := 0
var escaped Tags
if a.needsEscape() {
var tmp [20]Tag
if len(a) < len(tmp) {
escaped = tmp[:len(a)]
} else {
escaped = make(Tags, len(a))
}
for i := range a {
t := &a[i]
nt := &escaped[i]
nt.Key = escapeTag(t.Key)
nt.Value = escapeTag(t.Value)
sz += len(nt.Key) + len(nt.Value)
}
} else {
sz = a.Size()
escaped = a
}
sz += len(escaped) + (len(escaped) * 2) // separators
// Generate marshaled bytes.
if cap(dst)-len(dst) < sz {
nd := make([]byte, len(dst), len(dst)+sz)
copy(nd, dst)
dst = nd
}
buf := dst[len(dst) : len(dst)+sz]
idx := 0
for i := range escaped {
k := &escaped[i]
if len(k.Value) == 0 {
continue
}
buf[idx] = ','
idx++
copy(buf[idx:], k.Key)
idx += len(k.Key)
buf[idx] = '='
idx++
copy(buf[idx:], k.Value)
idx += len(k.Value)
}
return dst[:len(dst)+idx]
}
// CopyTags returns a shallow copy of tags.
func CopyTags(a Tags) Tags {
other := make(Tags, len(a))

View File

@ -1,5 +1,7 @@
package slices
import "bytes"
// BytesToStrings converts a slice of []byte into a slice of strings.
func BytesToStrings(a [][]byte) []string {
s := make([]string, 0, len(a))
@ -35,3 +37,28 @@ func CopyChunkedByteSlices(src [][]byte, chunkSize int) [][]byte {
return dst
}
// CompareSlice returns an integer comparing two slices of byte slices
// lexicographically.
// The result will be 0 if a==b, -1 if a < b, and +1 if a > b.
func CompareSlice(a, b [][]byte) int {
i := 0
for i < len(a) && i < len(b) {
if v := bytes.Compare(a[i], b[i]); v == 0 {
i++
continue
} else {
return v
}
}
if i < len(b) {
// b is longer, so assume a is less
return -1
} else if i < len(a) {
// a is longer, so assume b is less
return 1
} else {
return 0
}
}

View File

@ -1,10 +1,15 @@
package slices
import (
"bytes"
"fmt"
"math"
"reflect"
"testing"
"unsafe"
"github.com/google/go-cmp/cmp"
"github.com/influxdata/influxdb/pkg/bytesutil"
)
func TestCopyChunkedByteSlices_oneChunk(t *testing.T) {
@ -76,3 +81,124 @@ func TestCopyChunkedByteSlices_multipleChunks(t *testing.T) {
t.Error("destination should not match source")
}
}
const NIL = "<nil>"
// ss returns a sorted slice of byte slices.
func ss(s ...string) [][]byte {
r := make([][]byte, len(s))
for i := range s {
if s[i] != NIL {
r[i] = []byte(s[i])
}
}
bytesutil.Sort(r)
return r
}
func TestCompareSlice(t *testing.T) {
name := func(a, b [][]byte, exp int) string {
var as string
if a != nil {
as = string(bytes.Join(a, nil))
} else {
as = NIL
}
var bs string
if b != nil {
bs = string(bytes.Join(b, nil))
} else {
bs = NIL
}
return fmt.Sprintf("%s <=> %s is %d", as, bs, exp)
}
tests := []struct {
a, b [][]byte
exp int
}{
{
a: ss("aaa", "bbb", "ccc"),
b: ss("aaa", "bbb", "ccc"),
exp: 0,
},
{
a: ss("aaa", "bbb", "ccc", "ddd"),
b: ss("aaa", "bbb", "ccc"),
exp: 1,
},
{
a: ss("aaa", "bbb"),
b: ss("aaa", "bbb", "ccc"),
exp: -1,
},
{
a: ss("aaa", "bbbb"),
b: ss("aaa", "bbb", "ccc"),
exp: 1,
},
{
a: ss("aaa", "ccc"),
b: ss("aaa", "bbb", "ccc"),
exp: 1,
},
{
a: ss("aaa", "bbb", NIL),
b: ss("aaa", "bbb", "ccc"),
exp: -1,
},
{
a: ss("aaa", NIL, "ccc"),
b: ss("aaa", NIL, "ccc"),
exp: 0,
},
{
a: ss(NIL, "bbb", "ccc"),
b: ss("aaa", "bbb", "ccc"),
exp: -1,
},
{
a: ss("aaa", "aaa"),
b: ss("aaa", "bbb", "ccc"),
exp: -1,
},
{
a: nil,
b: ss("aaa", "bbb", "ccc"),
exp: -1,
},
{
a: ss("aaa", "bbb"),
b: nil,
exp: 1,
},
{
a: nil,
b: nil,
exp: 0,
},
{
a: [][]byte{},
b: nil,
exp: 0,
},
}
for _, test := range tests {
t.Run(name(test.a, test.b, test.exp), func(t *testing.T) {
if got := CompareSlice(test.a, test.b); got != test.exp {
t.Errorf("unexpected result, -got/+exp\n%s", cmp.Diff(got, test.exp))
}
})
}
}

View File

@ -10,7 +10,7 @@ import (
"github.com/influxdata/influxdb/models"
"github.com/influxdata/influxdb/prometheus/remote"
"github.com/influxdata/influxdb/services/storage"
"github.com/influxdata/platform/storage/reads/datatypes"
"github.com/influxdata/influxdb/storage/reads/datatypes"
)
const (

View File

@ -36,11 +36,11 @@ import (
"github.com/influxdata/influxdb/query"
"github.com/influxdata/influxdb/services/meta"
"github.com/influxdata/influxdb/services/storage"
"github.com/influxdata/influxdb/storage/reads"
"github.com/influxdata/influxdb/storage/reads/datatypes"
"github.com/influxdata/influxdb/tsdb"
"github.com/influxdata/influxdb/uuid"
"github.com/influxdata/influxql"
"github.com/influxdata/platform/storage/reads"
"github.com/influxdata/platform/storage/reads/datatypes"
prom "github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/client_golang/prometheus/promhttp"
"go.uber.org/zap"
@ -1221,11 +1221,9 @@ func (h *Handler) serveFluxQuery(w http.ResponseWriter, r *http.Request, user me
encoder := pr.Dialect.Encoder()
results := flux.NewResultIteratorFromQuery(q)
if h.Config.FluxLogEnabled {
if s, ok := results.(flux.Statisticser); ok {
defer func() {
stats = s.Statistics()
}()
}
defer func() {
stats = results.Statistics()
}()
}
defer results.Release()

View File

@ -20,7 +20,7 @@ import (
"testing"
"time"
"github.com/dgrijalva/jwt-go"
jwt "github.com/dgrijalva/jwt-go"
"github.com/gogo/protobuf/proto"
"github.com/golang/snappy"
"github.com/google/go-cmp/cmp"
@ -34,10 +34,10 @@ import (
"github.com/influxdata/influxdb/query"
"github.com/influxdata/influxdb/services/httpd"
"github.com/influxdata/influxdb/services/meta"
"github.com/influxdata/influxdb/storage/reads"
"github.com/influxdata/influxdb/storage/reads/datatypes"
"github.com/influxdata/influxdb/tsdb"
"github.com/influxdata/influxql"
"github.com/influxdata/platform/storage/reads"
"github.com/influxdata/platform/storage/reads/datatypes"
)
// Ensure the handler returns results from a query (including nil results).

View File

@ -5,9 +5,9 @@ import (
"github.com/influxdata/influxdb/pkg/testing/assert"
"github.com/influxdata/influxdb/services/storage"
"github.com/influxdata/influxdb/storage/reads"
"github.com/influxdata/influxdb/storage/reads/datatypes"
"github.com/influxdata/influxql"
"github.com/influxdata/platform/storage/reads"
"github.com/influxdata/platform/storage/reads/datatypes"
)
func TestHasSingleMeasurementNoOR(t *testing.T) {

View File

@ -7,10 +7,10 @@ import (
"github.com/influxdata/influxdb/models"
"github.com/influxdata/influxdb/query"
"github.com/influxdata/influxdb/storage/reads"
"github.com/influxdata/influxdb/storage/reads/datatypes"
"github.com/influxdata/influxdb/tsdb"
"github.com/influxdata/influxql"
"github.com/influxdata/platform/storage/reads"
"github.com/influxdata/platform/storage/reads/datatypes"
opentracing "github.com/opentracing/opentracing-go"
)

View File

@ -4,8 +4,8 @@ import (
"testing"
"github.com/google/go-cmp/cmp"
"github.com/influxdata/influxdb/models"
"github.com/influxdata/influxql"
"github.com/influxdata/platform/models"
)
func exprEqual(x, y influxql.Expr) bool {

View File

@ -9,12 +9,12 @@ import (
"github.com/gogo/protobuf/proto"
"github.com/gogo/protobuf/types"
"github.com/influxdata/influxdb/flux/stdlib/influxdata/influxdb"
"github.com/influxdata/influxdb/models"
"github.com/influxdata/influxdb/services/meta"
"github.com/influxdata/influxdb/storage/reads"
"github.com/influxdata/influxdb/storage/reads/datatypes"
"github.com/influxdata/influxdb/tsdb"
"github.com/influxdata/platform/query/functions/inputs/storage"
"github.com/influxdata/platform/storage/reads"
"github.com/influxdata/platform/storage/reads/datatypes"
"go.uber.org/zap"
)
@ -211,6 +211,6 @@ func (s *Store) GroupRead(ctx context.Context, req *datatypes.ReadRequest) (read
return rs, nil
}
func (s *Store) GetSource(rs storage.ReadSpec) (proto.Message, error) {
func (s *Store) GetSource(rs influxdb.ReadSpec) (proto.Message, error) {
return &ReadSource{Database: rs.Database, RetentionPolicy: rs.RetentionPolicy}, nil
}

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,163 @@
package reads
import (
"context"
"fmt"
"github.com/influxdata/influxdb/storage/reads/datatypes"
"github.com/influxdata/influxdb/tsdb/cursors"
)
type singleValue struct {
v interface{}
}
func (v *singleValue) Value(key string) (interface{}, bool) {
return v.v, true
}
func newAggregateArrayCursor(ctx context.Context, agg *datatypes.Aggregate, cursor cursors.Cursor) cursors.Cursor {
if cursor == nil {
return nil
}
switch agg.Type {
case datatypes.AggregateTypeSum:
return newSumArrayCursor(cursor)
case datatypes.AggregateTypeCount:
return newCountArrayCursor(cursor)
default:
// TODO(sgc): should be validated higher up
panic("invalid aggregate")
}
}
func newSumArrayCursor(cur cursors.Cursor) cursors.Cursor {
switch cur := cur.(type) {
case cursors.FloatArrayCursor:
return newFloatArraySumCursor(cur)
case cursors.IntegerArrayCursor:
return newIntegerArraySumCursor(cur)
case cursors.UnsignedArrayCursor:
return newUnsignedArraySumCursor(cur)
default:
// TODO(sgc): propagate an error instead?
return nil
}
}
func newCountArrayCursor(cur cursors.Cursor) cursors.Cursor {
switch cur := cur.(type) {
case cursors.FloatArrayCursor:
return &integerFloatCountArrayCursor{FloatArrayCursor: cur}
case cursors.IntegerArrayCursor:
return &integerIntegerCountArrayCursor{IntegerArrayCursor: cur}
case cursors.UnsignedArrayCursor:
return &integerUnsignedCountArrayCursor{UnsignedArrayCursor: cur}
case cursors.StringArrayCursor:
return &integerStringCountArrayCursor{StringArrayCursor: cur}
case cursors.BooleanArrayCursor:
return &integerBooleanCountArrayCursor{BooleanArrayCursor: cur}
default:
panic(fmt.Sprintf("unreachable: %T", cur))
}
}
type cursorContext struct {
ctx context.Context
req *cursors.CursorRequest
itrs cursors.CursorIterators
limit int64
count int64
err error
}
type multiShardArrayCursors struct {
ctx context.Context
limit int64
req cursors.CursorRequest
cursors struct {
i integerMultiShardArrayCursor
f floatMultiShardArrayCursor
u unsignedMultiShardArrayCursor
b booleanMultiShardArrayCursor
s stringMultiShardArrayCursor
}
}
func newMultiShardArrayCursors(ctx context.Context, start, end int64, asc bool, limit int64) *multiShardArrayCursors {
if limit < 0 {
limit = 1
}
m := &multiShardArrayCursors{
ctx: ctx,
limit: limit,
req: cursors.CursorRequest{
Ascending: asc,
StartTime: start,
EndTime: end,
},
}
cc := cursorContext{
ctx: ctx,
limit: limit,
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 *multiShardArrayCursors) createCursor(row SeriesRow) cursors.Cursor {
m.req.Name = row.Name
m.req.Tags = row.SeriesTags
m.req.Field = row.Field
var cond expression
if row.ValueCond != nil {
cond = &astExpr{row.ValueCond}
}
var shard cursors.CursorIterator
var cur cursors.Cursor
for cur == nil && len(row.Query) > 0 {
shard, row.Query = row.Query[0], row.Query[1:]
cur, _ = shard.Next(m.ctx, &m.req)
}
if cur == nil {
return nil
}
switch c := cur.(type) {
case cursors.IntegerArrayCursor:
m.cursors.i.reset(c, row.Query, cond)
return &m.cursors.i
case cursors.FloatArrayCursor:
m.cursors.f.reset(c, row.Query, cond)
return &m.cursors.f
case cursors.UnsignedArrayCursor:
m.cursors.u.reset(c, row.Query, cond)
return &m.cursors.u
case cursors.StringArrayCursor:
m.cursors.s.reset(c, row.Query, cond)
return &m.cursors.s
case cursors.BooleanArrayCursor:
m.cursors.b.reset(c, row.Query, cond)
return &m.cursors.b
default:
panic(fmt.Sprintf("unreachable: %T", cur))
}
}
func (m *multiShardArrayCursors) newAggregateCursor(ctx context.Context, agg *datatypes.Aggregate, cursor cursors.Cursor) cursors.Cursor {
return newAggregateArrayCursor(ctx, agg, cursor)
}

View File

@ -0,0 +1,30 @@
# List any generated files here
TARGETS = predicate.pb.go \
storage_common.pb.go
# List any source files used to generate the targets here
SOURCES = gen.go \
predicate.proto \
storage_common.proto
# List any directories that have their own Makefile here
SUBDIRS =
# Default target
all: $(SUBDIRS) $(TARGETS)
# Recurse into subdirs for same make goal
$(SUBDIRS):
$(MAKE) -C $@ $(MAKECMDGOALS)
# Clean all targets recursively
clean: $(SUBDIRS)
rm -f $(TARGETS)
# Define go generate if not already defined
GO_GENERATE := go generate
$(TARGETS): $(SOURCES)
$(GO_GENERATE) -x
.PHONY: all clean $(SUBDIRS)

View File

@ -0,0 +1,55 @@
package datatypes
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("influxdata.platform.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, ",")
}

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,60 @@
syntax = "proto3";
package influxdata.platform.storage;
option go_package = "datatypes";
import "gogoproto/gogo.proto";
message Node {
enum Type {
option (gogoproto.goproto_enum_prefix) = false;
LOGICAL_EXPRESSION = 0 [(gogoproto.enumvalue_customname) = "NodeTypeLogicalExpression"];
COMPARISON_EXPRESSION = 1 [(gogoproto.enumvalue_customname) = "NodeTypeComparisonExpression"];
PAREN_EXPRESSION = 2 [(gogoproto.enumvalue_customname) = "NodeTypeParenExpression"];
TAG_REF = 3 [(gogoproto.enumvalue_customname) = "NodeTypeTagRef"];
LITERAL = 4 [(gogoproto.enumvalue_customname) = "NodeTypeLiteral"];
FIELD_REF = 5 [(gogoproto.enumvalue_customname) = "NodeTypeFieldRef"];
}
enum Comparison {
option (gogoproto.goproto_enum_prefix) = false;
EQUAL = 0 [(gogoproto.enumvalue_customname) = "ComparisonEqual"];
NOT_EQUAL = 1 [(gogoproto.enumvalue_customname) = "ComparisonNotEqual"];
STARTS_WITH = 2 [(gogoproto.enumvalue_customname) = "ComparisonStartsWith"];
REGEX = 3 [(gogoproto.enumvalue_customname) = "ComparisonRegex"];
NOT_REGEX = 4 [(gogoproto.enumvalue_customname) = "ComparisonNotRegex"];
LT = 5 [(gogoproto.enumvalue_customname) = "ComparisonLess"];
LTE = 6 [(gogoproto.enumvalue_customname) = "ComparisonLessEqual"];
GT = 7 [(gogoproto.enumvalue_customname) = "ComparisonGreater"];
GTE = 8 [(gogoproto.enumvalue_customname) = "ComparisonGreaterEqual"];
}
// Logical operators apply to boolean values and combine to produce a single boolean result.
enum Logical {
option (gogoproto.goproto_enum_prefix) = false;
AND = 0 [(gogoproto.enumvalue_customname) = "LogicalAnd"];
OR = 1 [(gogoproto.enumvalue_customname) = "LogicalOr"];
}
Type node_type = 1 [(gogoproto.customname) = "NodeType", (gogoproto.jsontag) = "nodeType"];
repeated Node children = 2;
oneof value {
string string_value = 3 [(gogoproto.customname) = "StringValue"];
bool bool_value = 4 [(gogoproto.customname) = "BooleanValue"];
int64 int_value = 5 [(gogoproto.customname) = "IntegerValue"];
uint64 uint_value = 6 [(gogoproto.customname) = "UnsignedValue"];
double float_value = 7 [(gogoproto.customname) = "FloatValue"];
string regex_value = 8 [(gogoproto.customname) = "RegexValue"];
string tag_ref_value = 9 [(gogoproto.customname) = "TagRefValue"];
string field_ref_value = 10 [(gogoproto.customname) = "FieldRefValue"];
Logical logical = 11;
Comparison comparison = 12;
}
}
message Predicate {
Node root = 1;
}

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,206 @@
syntax = "proto3";
package influxdata.platform.storage;
option go_package = "datatypes";
import "gogoproto/gogo.proto";
import "google/protobuf/empty.proto";
import "google/protobuf/any.proto";
import "storage/reads/datatypes/predicate.proto";
option (gogoproto.marshaler_all) = true;
option (gogoproto.sizer_all) = true;
option (gogoproto.unmarshaler_all) = true;
option (gogoproto.goproto_getters_all) = false;
service Storage {
// Read performs a read operation using the given ReadRequest
rpc Read (ReadRequest) returns (stream ReadResponse);
// Capabilities returns a map of keys and values identifying the capabilities supported by the storage engine
rpc Capabilities (google.protobuf.Empty) returns (CapabilitiesResponse);
rpc Hints (google.protobuf.Empty) returns (HintsResponse);
// Explain describes the costs associated with executing a given Read request
// rpc Explain(google.protobuf.Empty) returns (ExplainResponse){}
}
// Request message for Storage.Read.
message ReadRequest {
enum Group {
option (gogoproto.goproto_enum_prefix) = false;
// 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"];
}
enum HintFlags {
option (gogoproto.goproto_enum_prefix) = false;
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"];
}
google.protobuf.Any read_source = 13 [(gogoproto.customname) = "ReadSource"];
TimestampRange timestamp_range = 2 [(gogoproto.customname) = "TimestampRange", (gogoproto.nullable) = false];
// Descending indicates whether points should be returned in descending order.
bool descending = 3;
// GroupKeys specifies a list of tag keys used to order the data. It is dependent on the Group property to determine
// its behavior.
repeated string group_keys = 4 [(gogoproto.customname) = "GroupKeys"];
//
Group group = 11;
// Aggregate specifies an optional aggregate to apply to the data.
// TODO(sgc): switch to slice for multiple aggregates in a single request
Aggregate aggregate = 9;
Predicate predicate = 5;
// SeriesLimit determines the maximum number of series to be returned for the request. Specify 0 for no limit.
int64 series_limit = 6 [(gogoproto.customname) = "SeriesLimit"];
// SeriesOffset determines how many series to skip before processing the request.
int64 series_offset = 7 [(gogoproto.customname) = "SeriesOffset"];
// PointsLimit determines the maximum number of values per series to be returned for the request.
// Specify 0 for no limit. -1 to return series frames only.
int64 points_limit = 8 [(gogoproto.customname) = "PointsLimit"];
// Trace contains opaque data if a trace is active.
map<string, string> trace = 10 [(gogoproto.customname) = "Trace"];
// Hints is a bitwise OR of HintFlags to control the behavior
// of the read request.
fixed32 hints = 12 [(gogoproto.customname) = "Hints", (gogoproto.casttype) = "HintFlags"];
}
message Aggregate {
enum AggregateType {
option (gogoproto.goproto_enum_prefix) = false;
NONE = 0 [(gogoproto.enumvalue_customname) = "AggregateTypeNone"];
SUM = 1 [(gogoproto.enumvalue_customname) = "AggregateTypeSum"];
COUNT = 2 [(gogoproto.enumvalue_customname) = "AggregateTypeCount"];
}
AggregateType type = 1;
// additional arguments?
}
message Tag {
bytes key = 1;
bytes value = 2;
}
// Response message for Storage.Read.
message ReadResponse {
enum FrameType {
option (gogoproto.goproto_enum_prefix) = false;
SERIES = 0 [(gogoproto.enumvalue_customname) = "FrameTypeSeries"];
POINTS = 1 [(gogoproto.enumvalue_customname) = "FrameTypePoints"];
}
enum DataType {
option (gogoproto.goproto_enum_prefix) = false;
FLOAT = 0 [(gogoproto.enumvalue_customname) = "DataTypeFloat"];
INTEGER = 1 [(gogoproto.enumvalue_customname) = "DataTypeInteger"];
UNSIGNED = 2 [(gogoproto.enumvalue_customname) = "DataTypeUnsigned"];
BOOLEAN = 3 [(gogoproto.enumvalue_customname) = "DataTypeBoolean"];
STRING = 4 [(gogoproto.enumvalue_customname) = "DataTypeString"];
}
message Frame {
oneof data {
GroupFrame group = 7;
SeriesFrame series = 1;
FloatPointsFrame float_points = 2 [(gogoproto.customname) = "FloatPoints"];
IntegerPointsFrame integer_points = 3 [(gogoproto.customname) = "IntegerPoints"];
UnsignedPointsFrame unsigned_points = 4 [(gogoproto.customname) = "UnsignedPoints"];
BooleanPointsFrame boolean_points = 5 [(gogoproto.customname) = "BooleanPoints"];
StringPointsFrame string_points = 6 [(gogoproto.customname) = "StringPoints"];
}
}
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;
}
message FloatPointsFrame {
repeated sfixed64 timestamps = 1;
repeated double values = 2;
}
message IntegerPointsFrame {
repeated sfixed64 timestamps = 1;
repeated int64 values = 2;
}
message UnsignedPointsFrame {
repeated sfixed64 timestamps = 1;
repeated uint64 values = 2;
}
message BooleanPointsFrame {
repeated sfixed64 timestamps = 1;
repeated bool values = 2;
}
message StringPointsFrame {
repeated sfixed64 timestamps = 1;
repeated string values = 2;
}
repeated Frame frames = 1 [(gogoproto.nullable) = false];
}
message CapabilitiesResponse {
map<string, string> caps = 1;
}
message HintsResponse {
}
// Specifies a continuous range of nanosecond timestamps.
message TimestampRange {
// Start defines the inclusive lower bound.
int64 start = 1;
// End defines the inclusive upper bound.
int64 end = 2;
}
//message ExplainRequest {
// ReadRequest read_request = 1 [(gogoproto.customname) = "ReadRequest"];
//}
//
//message ExplainResponse {}

284
storage/reads/eval.go Normal file
View File

@ -0,0 +1,284 @@
package reads
import (
"math"
"regexp"
"github.com/influxdata/influxql"
)
// evalExpr evaluates expr against a map.
func evalExpr(expr influxql.Expr, m Valuer) interface{} {
if expr == nil {
return nil
}
switch expr := expr.(type) {
case *influxql.BinaryExpr:
return evalBinaryExpr(expr, m)
case *influxql.BooleanLiteral:
return expr.Val
case *influxql.IntegerLiteral:
return expr.Val
case *influxql.UnsignedLiteral:
return expr.Val
case *influxql.NumberLiteral:
return expr.Val
case *influxql.ParenExpr:
return evalExpr(expr.Expr, m)
case *influxql.RegexLiteral:
return expr.Val
case *influxql.StringLiteral:
return expr.Val
case *influxql.VarRef:
v, _ := m.Value(expr.Val)
return v
default:
return nil
}
}
func evalBinaryExpr(expr *influxql.BinaryExpr, m Valuer) interface{} {
lhs := evalExpr(expr.LHS, m)
rhs := evalExpr(expr.RHS, m)
if lhs == nil && rhs != nil {
// When the LHS is nil and the RHS is a boolean, implicitly cast the
// nil to false.
if _, ok := rhs.(bool); ok {
lhs = false
}
} else if lhs != nil && rhs == nil {
// Implicit cast of the RHS nil to false when the LHS is a boolean.
if _, ok := lhs.(bool); ok {
rhs = false
}
}
// Evaluate if both sides are simple types.
switch lhs := lhs.(type) {
case bool:
rhs, ok := rhs.(bool)
switch expr.Op {
case influxql.AND:
return ok && (lhs && rhs)
case influxql.OR:
return ok && (lhs || rhs)
case influxql.BITWISE_AND:
return ok && (lhs && rhs)
case influxql.BITWISE_OR:
return ok && (lhs || rhs)
case influxql.BITWISE_XOR:
return ok && (lhs != rhs)
case influxql.EQ:
return ok && (lhs == rhs)
case influxql.NEQ:
return ok && (lhs != rhs)
}
case float64:
// Try the rhs as a float64 or int64
rhsf, ok := rhs.(float64)
if !ok {
var rhsi int64
if rhsi, ok = rhs.(int64); ok {
rhsf = float64(rhsi)
}
}
rhs := rhsf
switch expr.Op {
case influxql.EQ:
return ok && (lhs == rhs)
case influxql.NEQ:
return ok && (lhs != rhs)
case influxql.LT:
return ok && (lhs < rhs)
case influxql.LTE:
return ok && (lhs <= rhs)
case influxql.GT:
return ok && (lhs > rhs)
case influxql.GTE:
return ok && (lhs >= rhs)
case influxql.ADD:
if !ok {
return nil
}
return lhs + rhs
case influxql.SUB:
if !ok {
return nil
}
return lhs - rhs
case influxql.MUL:
if !ok {
return nil
}
return lhs * rhs
case influxql.DIV:
if !ok {
return nil
} else if rhs == 0 {
return float64(0)
}
return lhs / rhs
case influxql.MOD:
if !ok {
return nil
}
return math.Mod(lhs, rhs)
}
case int64:
// Try as a float64 to see if a float cast is required.
rhsf, ok := rhs.(float64)
if ok {
lhs := float64(lhs)
rhs := rhsf
switch expr.Op {
case influxql.EQ:
return lhs == rhs
case influxql.NEQ:
return lhs != rhs
case influxql.LT:
return lhs < rhs
case influxql.LTE:
return lhs <= rhs
case influxql.GT:
return lhs > rhs
case influxql.GTE:
return lhs >= rhs
case influxql.ADD:
return lhs + rhs
case influxql.SUB:
return lhs - rhs
case influxql.MUL:
return lhs * rhs
case influxql.DIV:
if rhs == 0 {
return float64(0)
}
return lhs / rhs
case influxql.MOD:
return math.Mod(lhs, rhs)
}
} else {
rhs, ok := rhs.(int64)
switch expr.Op {
case influxql.EQ:
return ok && (lhs == rhs)
case influxql.NEQ:
return ok && (lhs != rhs)
case influxql.LT:
return ok && (lhs < rhs)
case influxql.LTE:
return ok && (lhs <= rhs)
case influxql.GT:
return ok && (lhs > rhs)
case influxql.GTE:
return ok && (lhs >= rhs)
case influxql.ADD:
if !ok {
return nil
}
return lhs + rhs
case influxql.SUB:
if !ok {
return nil
}
return lhs - rhs
case influxql.MUL:
if !ok {
return nil
}
return lhs * rhs
case influxql.DIV:
if !ok {
return nil
} else if rhs == 0 {
return float64(0)
}
return lhs / rhs
case influxql.MOD:
if !ok {
return nil
} else if rhs == 0 {
return int64(0)
}
return lhs % rhs
case influxql.BITWISE_AND:
if !ok {
return nil
}
return lhs & rhs
case influxql.BITWISE_OR:
if !ok {
return nil
}
return lhs | rhs
case influxql.BITWISE_XOR:
if !ok {
return nil
}
return lhs ^ rhs
}
}
case string:
switch expr.Op {
case influxql.EQ:
rhs, ok := rhs.(string)
if !ok {
return nil
}
return lhs == rhs
case influxql.NEQ:
rhs, ok := rhs.(string)
if !ok {
return nil
}
return lhs != rhs
case influxql.EQREGEX:
rhs, ok := rhs.(*regexp.Regexp)
if !ok {
return nil
}
return rhs.MatchString(lhs)
case influxql.NEQREGEX:
rhs, ok := rhs.(*regexp.Regexp)
if !ok {
return nil
}
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
}
func EvalExprBool(expr influxql.Expr, m Valuer) bool {
v, _ := evalExpr(expr, m).(bool)
return v
}

25
storage/reads/expr.go Normal file
View File

@ -0,0 +1,25 @@
package reads
import (
"github.com/influxdata/influxql"
)
// TODO(sgc): build expression evaluator that does not use influxql AST
type expression interface {
EvalBool(v Valuer) bool
}
type astExpr struct {
expr influxql.Expr
}
func (e *astExpr) EvalBool(v Valuer) bool {
return EvalExprBool(e.expr, v)
}
// Valuer is the interface that wraps the Value() method.
type Valuer interface {
// Value returns the value and existence flag for a given key.
Value(key string) (interface{}, bool)
}

View File

@ -0,0 +1,342 @@
package reads
import (
"bytes"
"context"
"fmt"
"sort"
"github.com/influxdata/influxdb/models"
"github.com/influxdata/influxdb/storage/reads/datatypes"
"github.com/influxdata/influxdb/tsdb/cursors"
)
type groupResultSet struct {
ctx context.Context
req *datatypes.ReadRequest
agg *datatypes.Aggregate
mb multiShardCursors
i int
rows []*SeriesRow
keys [][]byte
nilSort []byte
rgc groupByCursor
km keyMerger
newCursorFn func() (SeriesCursor, error)
nextGroupFn func(c *groupResultSet) GroupCursor
sortFn func(c *groupResultSet) (int, error)
eof bool
}
type GroupOption func(g *groupResultSet)
// GroupOptionNilSortLo configures nil values to be sorted lower than any
// other value
func GroupOptionNilSortLo() GroupOption {
return func(g *groupResultSet) {
g.nilSort = nilSortLo
}
}
func NewGroupResultSet(ctx context.Context, req *datatypes.ReadRequest, newCursorFn func() (SeriesCursor, error), opts ...GroupOption) GroupResultSet {
g := &groupResultSet{
ctx: ctx,
req: req,
agg: req.Aggregate,
keys: make([][]byte, len(req.GroupKeys)),
nilSort: nilSortHi,
newCursorFn: newCursorFn,
}
for _, o := range opts {
o(g)
}
g.mb = newMultiShardArrayCursors(ctx, req.TimestampRange.Start, req.TimestampRange.End, !req.Descending, req.PointsLimit)
for i, k := range req.GroupKeys {
g.keys[i] = []byte(k)
}
switch req.Group {
case datatypes.GroupBy:
g.sortFn = groupBySort
g.nextGroupFn = groupByNextGroup
g.rgc = groupByCursor{
ctx: ctx,
mb: g.mb,
agg: req.Aggregate,
vals: make([][]byte, len(req.GroupKeys)),
}
case datatypes.GroupNone:
g.sortFn = groupNoneSort
g.nextGroupFn = groupNoneNextGroup
default:
panic("not implemented")
}
n, err := g.sort()
if n == 0 || err != nil {
return nil
}
return g
}
// nilSort values determine the lexicographical order of nil values in the
// partition key
var (
// nil sorts lowest
nilSortLo = []byte{0x00}
// nil sorts highest
nilSortHi = []byte{0xff} // sort nil values
)
func (g *groupResultSet) Err() error { return nil }
func (g *groupResultSet) Close() {}
func (g *groupResultSet) Next() GroupCursor {
if g.eof {
return nil
}
return g.nextGroupFn(g)
}
func (g *groupResultSet) sort() (int, error) {
return g.sortFn(g)
}
// 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 cursors.IntegerArrayCursor:
a := c.Next()
ts = a.Timestamps
case cursors.FloatArrayCursor:
a := c.Next()
ts = a.Timestamps
case cursors.UnsignedArrayCursor:
a := c.Next()
ts = a.Timestamps
case cursors.BooleanArrayCursor:
a := c.Next()
ts = a.Timestamps
case cursors.StringArrayCursor:
a := c.Next()
ts = a.Timestamps
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{
ctx: g.ctx,
mb: g.mb,
agg: g.agg,
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
}
allTime := g.req.Hints.HintSchemaAllTime()
g.km.clear()
n := 0
row := cur.Next()
for row != nil {
if allTime || g.seriesHasPoints(row) {
n++
g.km.mergeTagKeys(row.Tags)
}
row = cur.Next()
}
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])
}
g.km.clear()
rowKey := row.SortKey
j := g.i
for j < len(g.rows) && bytes.Equal(rowKey, g.rows[j].SortKey) {
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}
allTime := g.req.Hints.HintSchemaAllTime()
row := cur.Next()
for row != nil {
if allTime || g.seriesHasPoints(row) {
nr := *row
nr.SeriesTags = tagsBuf.copyTags(nr.SeriesTags)
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] = g.nilSort
}
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 {
ctx context.Context
mb multiShardCursors
agg *datatypes.Aggregate
cur SeriesCursor
row SeriesRow
keys [][]byte
}
func (c *groupNoneCursor) Err() error { return nil }
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) Stats() cursors.CursorStats { return c.row.Query.Stats() }
func (c *groupNoneCursor) Next() bool {
row := c.cur.Next()
if row == nil {
return false
}
c.row = *row
return true
}
func (c *groupNoneCursor) Cursor() cursors.Cursor {
cur := c.mb.createCursor(c.row)
if c.agg != nil {
cur = c.mb.newAggregateCursor(c.ctx, c.agg, cur)
}
return cur
}
type groupByCursor struct {
ctx context.Context
mb multiShardCursors
agg *datatypes.Aggregate
i int
rows []*SeriesRow
keys [][]byte
vals [][]byte
}
func (c *groupByCursor) reset(rows []*SeriesRow) {
c.i = 0
c.rows = rows
}
func (c *groupByCursor) Err() error { return nil }
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() cursors.Cursor {
cur := c.mb.createCursor(*c.rows[c.i-1])
if c.agg != nil {
cur = c.mb.newAggregateCursor(c.ctx, c.agg, cur)
}
return cur
}
func (c *groupByCursor) Stats() cursors.CursorStats {
var stats cursors.CursorStats
for _, row := range c.rows {
stats.Add(row.Query.Stats())
}
return stats
}

View File

@ -0,0 +1,372 @@
package reads_test
import (
"context"
"strings"
"testing"
"github.com/google/go-cmp/cmp"
"github.com/influxdata/influxdb/models"
"github.com/influxdata/influxdb/pkg/data/gen"
"github.com/influxdata/influxdb/storage/reads"
"github.com/influxdata/influxdb/storage/reads/datatypes"
)
func TestGroupGroupResultSetSorting(t *testing.T) {
tests := []struct {
name string
cur reads.SeriesCursor
group datatypes.ReadRequest_Group
keys []string
exp string
}{
{
name: "group by tag1 in all series",
cur: &sliceSeriesCursor{
rows: newSeriesRows(
"cpu,tag0=val00,tag1=val10",
"cpu,tag0=val00,tag1=val11",
"cpu,tag0=val00,tag1=val12",
"cpu,tag0=val01,tag1=val10",
"cpu,tag0=val01,tag1=val11",
"cpu,tag0=val01,tag1=val12",
)},
group: datatypes.GroupBy,
keys: []string{"tag1"},
exp: `group:
tag key : _m,tag0,tag1
partition key: val10
series: _m=cpu,tag0=val00,tag1=val10
series: _m=cpu,tag0=val01,tag1=val10
group:
tag key : _m,tag0,tag1
partition key: val11
series: _m=cpu,tag0=val00,tag1=val11
series: _m=cpu,tag0=val01,tag1=val11
group:
tag key : _m,tag0,tag1
partition key: val12
series: _m=cpu,tag0=val00,tag1=val12
series: _m=cpu,tag0=val01,tag1=val12
`,
},
{
name: "group by tag1 in partial series",
cur: &sliceSeriesCursor{
rows: newSeriesRows(
"aaa,tag0=val00",
"aaa,tag0=val01",
"cpu,tag0=val00,tag1=val10",
"cpu,tag0=val00,tag1=val11",
"cpu,tag0=val00,tag1=val12",
"cpu,tag0=val01,tag1=val10",
"cpu,tag0=val01,tag1=val11",
"cpu,tag0=val01,tag1=val12",
)},
group: datatypes.GroupBy,
keys: []string{"tag1"},
exp: `group:
tag key : _m,tag0,tag1
partition key: val10
series: _m=cpu,tag0=val00,tag1=val10
series: _m=cpu,tag0=val01,tag1=val10
group:
tag key : _m,tag0,tag1
partition key: val11
series: _m=cpu,tag0=val01,tag1=val11
series: _m=cpu,tag0=val00,tag1=val11
group:
tag key : _m,tag0,tag1
partition key: val12
series: _m=cpu,tag0=val01,tag1=val12
series: _m=cpu,tag0=val00,tag1=val12
group:
tag key : _m,tag0
partition key: <nil>
series: _m=aaa,tag0=val00
series: _m=aaa,tag0=val01
`,
},
{
name: "group by tag2,tag1 with partial series",
cur: &sliceSeriesCursor{
rows: newSeriesRows(
"aaa,tag0=val00",
"aaa,tag0=val01",
"cpu,tag0=val00,tag1=val10",
"cpu,tag0=val00,tag1=val11",
"cpu,tag0=val00,tag1=val12",
"mem,tag1=val10,tag2=val20",
"mem,tag1=val11,tag2=val20",
"mem,tag1=val11,tag2=val21",
)},
group: datatypes.GroupBy,
keys: []string{"tag2", "tag1"},
exp: `group:
tag key : _m,tag1,tag2
partition key: val20,val10
series: _m=mem,tag1=val10,tag2=val20
group:
tag key : _m,tag1,tag2
partition key: val20,val11
series: _m=mem,tag1=val11,tag2=val20
group:
tag key : _m,tag1,tag2
partition key: val21,val11
series: _m=mem,tag1=val11,tag2=val21
group:
tag key : _m,tag0,tag1
partition key: <nil>,val10
series: _m=cpu,tag0=val00,tag1=val10
group:
tag key : _m,tag0,tag1
partition key: <nil>,val11
series: _m=cpu,tag0=val00,tag1=val11
group:
tag key : _m,tag0,tag1
partition key: <nil>,val12
series: _m=cpu,tag0=val00,tag1=val12
group:
tag key : _m,tag0
partition key: <nil>,<nil>
series: _m=aaa,tag0=val00
series: _m=aaa,tag0=val01
`,
},
{
name: "group by tag0,tag2 with partial series",
cur: &sliceSeriesCursor{
rows: newSeriesRows(
"aaa,tag0=val00",
"aaa,tag0=val01",
"cpu,tag0=val00,tag1=val10",
"cpu,tag0=val00,tag1=val11",
"cpu,tag0=val00,tag1=val12",
"mem,tag1=val10,tag2=val20",
"mem,tag1=val11,tag2=val20",
"mem,tag1=val11,tag2=val21",
)},
group: datatypes.GroupBy,
keys: []string{"tag0", "tag2"},
exp: `group:
tag key : _m,tag0,tag1
partition key: val00,<nil>
series: _m=aaa,tag0=val00
series: _m=cpu,tag0=val00,tag1=val10
series: _m=cpu,tag0=val00,tag1=val11
series: _m=cpu,tag0=val00,tag1=val12
group:
tag key : _m,tag0
partition key: val01,<nil>
series: _m=aaa,tag0=val01
group:
tag key : _m,tag1,tag2
partition key: <nil>,val20
series: _m=mem,tag1=val10,tag2=val20
series: _m=mem,tag1=val11,tag2=val20
group:
tag key : _m,tag1,tag2
partition key: <nil>,val21
series: _m=mem,tag1=val11,tag2=val21
`,
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
newCursor := func() (reads.SeriesCursor, error) {
return tt.cur, nil
}
var hints datatypes.HintFlags
hints.SetHintSchemaAllTime()
rs := reads.NewGroupResultSet(context.Background(), &datatypes.ReadRequest{Group: tt.group, GroupKeys: tt.keys, Hints: hints}, newCursor)
sb := new(strings.Builder)
GroupResultSetToString(sb, rs, SkipNilCursor())
if got := sb.String(); !cmp.Equal(got, tt.exp) {
t.Errorf("unexpected value; -got/+exp\n%s", cmp.Diff(strings.Split(got, "\n"), strings.Split(tt.exp, "\n")))
}
})
}
}
func TestNewGroupResultSet_GroupNone_NoDataReturnsNil(t *testing.T) {
newCursor := func() (reads.SeriesCursor, error) {
return &sliceSeriesCursor{
rows: newSeriesRows(
"aaa,tag0=val00",
"aaa,tag0=val01",
)}, nil
}
rs := reads.NewGroupResultSet(context.Background(), &datatypes.ReadRequest{Group: datatypes.GroupNone}, newCursor)
if rs != nil {
t.Errorf("expected nil cursor")
}
}
func TestNewGroupResultSet_GroupBy_NoDataReturnsNil(t *testing.T) {
newCursor := func() (reads.SeriesCursor, error) {
return &sliceSeriesCursor{
rows: newSeriesRows(
"aaa,tag0=val00",
"aaa,tag0=val01",
)}, nil
}
rs := reads.NewGroupResultSet(context.Background(), &datatypes.ReadRequest{Group: datatypes.GroupBy, GroupKeys: []string{"tag0"}}, newCursor)
if rs != nil {
t.Errorf("expected nil cursor")
}
}
func TestNewGroupResultSet_Sorting(t *testing.T) {
tests := []struct {
name string
keys []string
opts []reads.GroupOption
exp string
}{
{
name: "nil hi",
keys: []string{"tag0", "tag2"},
exp: `group:
tag key : _m,tag0,tag1
partition key: val00,<nil>
series: _m=aaa,tag0=val00
series: _m=cpu,tag0=val00,tag1=val10
series: _m=cpu,tag0=val00,tag1=val11
series: _m=cpu,tag0=val00,tag1=val12
group:
tag key : _m,tag0
partition key: val01,<nil>
series: _m=aaa,tag0=val01
group:
tag key : _m,tag1,tag2
partition key: <nil>,val20
series: _m=mem,tag1=val10,tag2=val20
series: _m=mem,tag1=val11,tag2=val20
group:
tag key : _m,tag1,tag2
partition key: <nil>,val21
series: _m=mem,tag1=val11,tag2=val21
`,
},
{
name: "nil lo",
keys: []string{"tag0", "tag2"},
opts: []reads.GroupOption{reads.GroupOptionNilSortLo()},
exp: `group:
tag key : _m,tag1,tag2
partition key: <nil>,val20
series: _m=mem,tag1=val11,tag2=val20
series: _m=mem,tag1=val10,tag2=val20
group:
tag key : _m,tag1,tag2
partition key: <nil>,val21
series: _m=mem,tag1=val11,tag2=val21
group:
tag key : _m,tag0,tag1
partition key: val00,<nil>
series: _m=cpu,tag0=val00,tag1=val10
series: _m=cpu,tag0=val00,tag1=val11
series: _m=cpu,tag0=val00,tag1=val12
series: _m=aaa,tag0=val00
group:
tag key : _m,tag0
partition key: val01,<nil>
series: _m=aaa,tag0=val01
`,
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
newCursor := func() (reads.SeriesCursor, error) {
return &sliceSeriesCursor{
rows: newSeriesRows(
"aaa,tag0=val00",
"aaa,tag0=val01",
"cpu,tag0=val00,tag1=val10",
"cpu,tag0=val00,tag1=val11",
"cpu,tag0=val00,tag1=val12",
"mem,tag1=val10,tag2=val20",
"mem,tag1=val11,tag2=val20",
"mem,tag1=val11,tag2=val21",
)}, nil
}
var hints datatypes.HintFlags
hints.SetHintSchemaAllTime()
rs := reads.NewGroupResultSet(context.Background(), &datatypes.ReadRequest{Group: datatypes.GroupBy, GroupKeys: tt.keys, Hints: hints}, newCursor, tt.opts...)
sb := new(strings.Builder)
GroupResultSetToString(sb, rs, SkipNilCursor())
if got := sb.String(); !cmp.Equal(got, tt.exp) {
t.Errorf("unexpected value; -got/+exp\n%s", cmp.Diff(strings.Split(got, "\n"), strings.Split(tt.exp, "\n")))
}
})
}
}
type sliceSeriesCursor struct {
rows []reads.SeriesRow
i int
}
func newSeriesRows(keys ...string) []reads.SeriesRow {
rows := make([]reads.SeriesRow, len(keys))
for i := range keys {
rows[i].Name, rows[i].SeriesTags = models.ParseKeyBytes([]byte(keys[i]))
rows[i].Tags = rows[i].SeriesTags.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() *reads.SeriesRow {
if s.i < len(s.rows) {
s.i++
return &s.rows[s.i-1]
}
return nil
}
func BenchmarkNewGroupResultSet_GroupBy(b *testing.B) {
card := []int{10, 10, 10}
vals := make([]gen.CountableSequence, len(card))
for i := range card {
vals[i] = gen.NewCounterByteSequenceCount(card[i])
}
tags := gen.NewTagsValuesSequenceValues("tag", vals)
rows := make([]reads.SeriesRow, tags.Count())
for i := range rows {
tags.Next()
t := tags.Value().Clone()
rows[i].SeriesTags = t
rows[i].Tags = t
rows[i].Name = []byte("m0")
}
cur := &sliceSeriesCursor{rows: rows}
newCursor := func() (reads.SeriesCursor, error) {
cur.i = 0
return cur, nil
}
b.ResetTimer()
b.ReportAllocs()
for i := 0; i < b.N; i++ {
var hints datatypes.HintFlags
hints.SetHintSchemaAllTime()
rs := reads.NewGroupResultSet(context.Background(), &datatypes.ReadRequest{Group: datatypes.GroupBy, GroupKeys: []string{"tag2"}, Hints: hints}, newCursor)
rs.Close()
}
}

109
storage/reads/keymerger.go Normal file
View File

@ -0,0 +1,109 @@
package reads
import (
"bytes"
"strings"
"github.com/influxdata/influxdb/models"
)
// tagsKeyMerger is responsible for determining a merged set of tag keys
type keyMerger struct {
i int
tmp [][]byte
keys [2][][]byte
}
func (km *keyMerger) clear() {
km.i = 0
km.keys[0] = km.keys[0][:0]
if km.tmp != nil {
tmp := km.tmp[:cap(km.tmp)]
for i := range tmp {
tmp[i] = nil
}
}
}
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) {
if cap(km.tmp) < len(tags) {
km.tmp = make([][]byte, len(tags))
} else {
km.tmp = km.tmp[:len(tags)]
}
for i := range tags {
km.tmp[i] = tags[i].Key
}
km.mergeKeys(km.tmp)
}
func (km *keyMerger) mergeKeys(in [][]byte) {
keys := km.keys[km.i&1]
i, j := 0, 0
for i < len(keys) && j < len(in) && bytes.Equal(keys[i], in[j]) {
i++
j++
}
if j == len(in) {
// no new tags
return
}
km.i = (km.i + 1) & 1
l := len(keys) + len(in)
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(in) {
cmp := bytes.Compare(keys[i], in[j])
if cmp < 0 {
keya[k] = keys[i]
i++
} else if cmp > 0 {
keya[k] = in[j]
j++
} else {
keya[k] = keys[i]
i++
j++
}
k++
}
if i < len(keys) {
k += copy(keya[k:], keys[i:])
}
if j < len(in) {
k += copy(keya[k:], in[j:])
}
km.keys[km.i] = keya[:k]
}

View File

@ -0,0 +1,204 @@
package reads
import (
"bytes"
"math/rand"
"strconv"
"testing"
"github.com/google/go-cmp/cmp"
"github.com/influxdata/influxdb/models"
)
func TestKeyMerger_MergeTagKeys(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",
},
{
name: "new tags,verify clear",
tags: []models.Tags{
models.ParseTags([]byte("foo,tag9=v0")),
models.ParseTags([]byte("foo,tag8=v0")),
},
exp: "tag8,tag9",
},
}
var km keyMerger
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
km.clear()
for _, tags := range tt.tags {
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))
}
})
}
}
var commaB = []byte(",")
func TestKeyMerger_MergeKeys(t *testing.T) {
tests := []struct {
name string
keys [][][]byte
exp string
}{
{
name: "mixed",
keys: [][][]byte{
bytes.Split([]byte("tag0,tag1,tag2"), commaB),
bytes.Split([]byte("tag0,tag1,tag2"), commaB),
bytes.Split([]byte("tag0"), commaB),
bytes.Split([]byte("tag0,tag3"), commaB),
},
exp: "tag0,tag1,tag2,tag3",
},
{
name: "mixed 2",
keys: [][][]byte{
bytes.Split([]byte("tag0"), commaB),
bytes.Split([]byte("tag0,tag3"), commaB),
bytes.Split([]byte("tag0,tag1,tag2"), commaB),
bytes.Split([]byte("tag0,tag1,tag2"), commaB),
},
exp: "tag0,tag1,tag2,tag3",
},
{
name: "all different",
keys: [][][]byte{
bytes.Split([]byte("tag0"), commaB),
bytes.Split([]byte("tag3"), commaB),
bytes.Split([]byte("tag1"), commaB),
bytes.Split([]byte("tag2"), commaB),
},
exp: "tag0,tag1,tag2,tag3",
},
{
name: "new tags,verify clear",
keys: [][][]byte{
bytes.Split([]byte("tag9"), commaB),
bytes.Split([]byte("tag8"), commaB),
},
exp: "tag8,tag9",
},
}
var km keyMerger
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
km.clear()
for _, keys := range tt.keys {
km.mergeKeys(keys)
}
if got := km.String(); !cmp.Equal(got, tt.exp) {
t.Errorf("unexpected keys -got/+exp\n%s", cmp.Diff(got, tt.exp))
}
})
}
}
func BenchmarkKeyMerger_MergeKeys(b *testing.B) {
keys := [][][]byte{
bytes.Split([]byte("tag00,tag01,tag02"), commaB),
bytes.Split([]byte("tag00,tag01,tag02"), commaB),
bytes.Split([]byte("tag00,tag01,tag05,tag06,tag10,tag11,tag12,tag13,tag14,tag15"), commaB),
bytes.Split([]byte("tag00"), commaB),
bytes.Split([]byte("tag00,tag03"), commaB),
bytes.Split([]byte("tag01,tag03,tag13,tag14,tag15"), commaB),
bytes.Split([]byte("tag04,tag05"), commaB),
}
rand.Seed(20040409)
tests := []int{
10,
1000,
1000000,
}
for _, n := range tests {
b.Run(strconv.Itoa(n), func(b *testing.B) {
b.ResetTimer()
var km keyMerger
for i := 0; i < b.N; i++ {
for j := 0; j < n; j++ {
km.mergeKeys(keys[rand.Int()%len(keys)])
}
km.clear()
}
})
}
}
func BenchmarkKeyMerger_MergeTagKeys(b *testing.B) {
tags := []models.Tags{
models.ParseTags([]byte("foo,tag00=v0,tag01=v0,tag02=v0")),
models.ParseTags([]byte("foo,tag00=v0,tag01=v0,tag02=v0")),
models.ParseTags([]byte("foo,tag00=v0,tag01=v0,tag05=v0,tag06=v0,tag10=v0,tag11=v0,tag12=v0,tag13=v0,tag14=v0,tag15=v0")),
models.ParseTags([]byte("foo,tag00=v0")),
models.ParseTags([]byte("foo,tag00=v0,tag03=v0")),
models.ParseTags([]byte("foo,tag01=v0,tag03=v0,tag13=v0,tag14=v0,tag15=v0")),
models.ParseTags([]byte("foo,tag04=v0,tag05=v0")),
}
rand.Seed(20040409)
tests := []int{
10,
1000,
1000000,
}
for _, n := range tests {
b.Run(strconv.Itoa(n), func(b *testing.B) {
b.ResetTimer()
var km keyMerger
for i := 0; i < b.N; i++ {
for j := 0; j < n; j++ {
km.mergeTagKeys(tags[rand.Int()%len(tags)])
}
km.clear()
}
})
}
}

123
storage/reads/merge.go Normal file
View File

@ -0,0 +1,123 @@
package reads
import (
"container/heap"
"github.com/influxdata/influxdb/models"
"github.com/influxdata/influxdb/tsdb/cursors"
)
type mergedResultSet struct {
heap resultSetHeap
err error
first bool
stats cursors.CursorStats
}
func NewMergedResultSet(results []ResultSet) ResultSet {
if len(results) == 0 {
return nil
} else if len(results) == 1 {
return results[0]
}
mrs := &mergedResultSet{first: true}
mrs.heap.init(results)
return mrs
}
func (r *mergedResultSet) Err() error { return r.err }
func (r *mergedResultSet) Close() {
for _, rs := range r.heap.items {
rs.Close()
}
r.heap.items = nil
}
func (r *mergedResultSet) Next() bool {
if len(r.heap.items) == 0 {
return false
}
if !r.first {
top := r.heap.items[0]
if top.Next() {
heap.Fix(&r.heap, 0)
return true
}
err := top.Err()
stats := top.Stats()
top.Close()
heap.Pop(&r.heap)
if err != nil {
r.err = err
r.Close()
}
r.stats.Add(stats)
return len(r.heap.items) > 0
}
r.first = false
return true
}
func (r *mergedResultSet) Cursor() cursors.Cursor {
return r.heap.items[0].Cursor()
}
func (r *mergedResultSet) Tags() models.Tags {
return r.heap.items[0].Tags()
}
func (r *mergedResultSet) Stats() cursors.CursorStats {
return r.stats
}
type resultSetHeap struct {
items []ResultSet
}
func (h *resultSetHeap) init(results []ResultSet) {
if cap(h.items) < len(results) {
h.items = make([]ResultSet, 0, len(results))
} else {
h.items = h.items[:0]
}
for _, rs := range results {
if rs.Next() {
h.items = append(h.items, rs)
} else {
rs.Close()
}
}
heap.Init(h)
}
func (h *resultSetHeap) Less(i, j int) bool {
return models.CompareTags(h.items[i].Tags(), h.items[j].Tags()) == -1
}
func (h *resultSetHeap) Len() int {
return len(h.items)
}
func (h *resultSetHeap) Swap(i, j int) {
h.items[i], h.items[j] = h.items[j], h.items[i]
}
func (h *resultSetHeap) Push(x interface{}) {
panic("not implemented")
}
func (h *resultSetHeap) Pop() interface{} {
old := h.items
n := len(old)
item := old[n-1]
old[n-1] = nil
h.items = old[0 : n-1]
return item
}

105
storage/reads/merge_test.go Normal file
View File

@ -0,0 +1,105 @@
package reads_test
import (
"strings"
"testing"
"github.com/google/go-cmp/cmp"
"github.com/influxdata/influxdb/storage/reads"
"github.com/influxdata/influxdb/storage/reads/datatypes"
)
func newStreamSeries(v ...string) *sliceStreamReader {
var frames []datatypes.ReadResponse_Frame
for _, s := range v {
frames = append(frames, seriesF(Float, s))
}
return newStreamReader(response(frames...))
}
func TestNewMergedResultSet(t *testing.T) {
exp := `series: _m=m0,tag0=val00
cursor:Float
series: _m=m0,tag0=val01
cursor:Float
series: _m=m0,tag0=val02
cursor:Float
series: _m=m0,tag0=val03
cursor:Float
`
tests := []struct {
name string
streams []*sliceStreamReader
exp string
}{
{
name: "outer inner",
streams: []*sliceStreamReader{
newStreamSeries("m0,tag0=val01", "m0,tag0=val02"),
newStreamSeries("m0,tag0=val00", "m0,tag0=val03"),
},
exp: exp,
},
{
name: "sequential",
streams: []*sliceStreamReader{
newStreamSeries("m0,tag0=val00", "m0,tag0=val01"),
newStreamSeries("m0,tag0=val02", "m0,tag0=val03"),
},
exp: exp,
},
{
name: "interleaved",
streams: []*sliceStreamReader{
newStreamSeries("m0,tag0=val01", "m0,tag0=val03"),
newStreamSeries("m0,tag0=val00", "m0,tag0=val02"),
},
exp: exp,
},
{
name: "single resultset",
streams: []*sliceStreamReader{
newStreamSeries("m0,tag0=val00", "m0,tag0=val01", "m0,tag0=val02", "m0,tag0=val03"),
},
exp: exp,
},
{
name: "single series ordered",
streams: []*sliceStreamReader{
newStreamSeries("m0,tag0=val00"),
newStreamSeries("m0,tag0=val01"),
newStreamSeries("m0,tag0=val02"),
newStreamSeries("m0,tag0=val03"),
},
exp: exp,
},
{
name: "single series random order",
streams: []*sliceStreamReader{
newStreamSeries("m0,tag0=val02"),
newStreamSeries("m0,tag0=val03"),
newStreamSeries("m0,tag0=val00"),
newStreamSeries("m0,tag0=val01"),
},
exp: exp,
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
rss := make([]reads.ResultSet, len(tt.streams))
for i := range tt.streams {
rss[i] = reads.NewResultSetStreamReader(tt.streams[i])
}
rs := reads.NewMergedResultSet(rss)
sb := new(strings.Builder)
ResultSetToString(sb, rs)
if got := sb.String(); !cmp.Equal(got, tt.exp) {
t.Errorf("unexpected value; -got/+exp\n%s", cmp.Diff(got, tt.exp))
}
})
}
}

View File

@ -0,0 +1,251 @@
package reads
import (
"bytes"
"sort"
"github.com/influxdata/influxdb/pkg/slices"
)
// groupNoneMergedGroupResultSet produces a single GroupCursor, merging all
// GroupResultSet#Keys
type groupNoneMergedGroupResultSet struct {
g []GroupResultSet
gc groupNoneMergedGroupCursor
done bool
}
// Returns a GroupResultSet that merges results using the datatypes.GroupNone
// strategy. Each source GroupResultSet in g must be configured using the
// GroupNone strategy or the results are undefined.
//
// The GroupNone strategy must merge the partition key and tag keys
// from each source GroupResultSet when producing its
func NewGroupNoneMergedGroupResultSet(g []GroupResultSet) GroupResultSet {
if len(g) == 0 {
return nil
} else if len(g) == 1 {
return g[0]
}
grs := &groupNoneMergedGroupResultSet{
g: g,
gc: groupNoneMergedGroupCursor{
mergedResultSet: mergedResultSet{first: true},
},
}
var km keyMerger
results := make([]ResultSet, 0, len(g))
for _, rs := range g {
if gc := rs.Next(); gc != nil {
results = append(results, gc)
km.mergeKeys(gc.Keys())
} else if rs.Err() != nil {
grs.done = true
grs.gc.err = rs.Err()
results = nil
break
}
}
if len(results) > 0 {
grs.gc.keys = km.get()
grs.gc.heap.init(results)
}
return grs
}
func (r *groupNoneMergedGroupResultSet) Next() GroupCursor {
if !r.done {
r.done = true
return &r.gc
}
return nil
}
func (r *groupNoneMergedGroupResultSet) Err() error { return r.gc.err }
func (r *groupNoneMergedGroupResultSet) Close() {
r.gc.Close()
for _, grs := range r.g {
grs.Close()
}
r.g = nil
}
type groupNoneMergedGroupCursor struct {
mergedResultSet
keys [][]byte
}
func (r *groupNoneMergedGroupCursor) Keys() [][]byte {
return r.keys
}
func (r *groupNoneMergedGroupCursor) PartitionKeyVals() [][]byte {
return nil
}
// groupByMergedGroupResultSet implements the GroupBy strategy.
type groupByMergedGroupResultSet struct {
items []*groupCursorItem
alt []*groupCursorItem
groupCursors []GroupCursor
resultSets []ResultSet
nilVal []byte
err error
gc groupByMergedGroupCursor
}
// Returns a GroupResultSet that merges results using the datatypes.GroupBy
// strategy. Each source GroupResultSet in g must be configured using the
// GroupBy strategy with the same GroupKeys or the results are undefined.
func NewGroupByMergedGroupResultSet(g []GroupResultSet) GroupResultSet {
if len(g) == 0 {
return nil
} else if len(g) == 1 {
return g[0]
}
grs := &groupByMergedGroupResultSet{}
grs.nilVal = nilSortHi
grs.groupCursors = make([]GroupCursor, 0, len(g))
grs.resultSets = make([]ResultSet, 0, len(g))
grs.items = make([]*groupCursorItem, 0, len(g))
grs.alt = make([]*groupCursorItem, 0, len(g))
for _, rs := range g {
grs.items = append(grs.items, &groupCursorItem{grs: rs})
}
return grs
}
// next determines the cursors for the next partition key.
func (r *groupByMergedGroupResultSet) next() {
r.alt = r.alt[:0]
for i, item := range r.items {
if item.gc == nil {
item.gc = item.grs.Next()
if item.gc != nil {
r.alt = append(r.alt, item)
} else {
r.err = item.grs.Err()
item.grs.Close()
}
} else {
// append remaining non-nil cursors
r.alt = append(r.alt, r.items[i:]...)
break
}
}
r.items, r.alt = r.alt, r.items
if len(r.items) == 0 {
r.groupCursors = r.groupCursors[:0]
r.resultSets = r.resultSets[:0]
return
}
if r.err != nil {
r.Close()
return
}
sort.Slice(r.items, func(i, j int) bool {
return comparePartitionKey(r.items[i].gc.PartitionKeyVals(), r.items[j].gc.PartitionKeyVals(), r.nilVal) == -1
})
r.groupCursors = r.groupCursors[:1]
r.resultSets = r.resultSets[:1]
first := r.items[0].gc
r.groupCursors[0] = first
r.resultSets[0] = first
r.items[0].gc = nil
for i := 1; i < len(r.items); i++ {
if slices.CompareSlice(first.PartitionKeyVals(), r.items[i].gc.PartitionKeyVals()) == 0 {
r.groupCursors = append(r.groupCursors, r.items[i].gc)
r.resultSets = append(r.resultSets, r.items[i].gc)
r.items[i].gc = nil
}
}
}
func (r *groupByMergedGroupResultSet) Next() GroupCursor {
r.next()
if len(r.groupCursors) == 0 {
return nil
}
r.gc.first = true
r.gc.heap.init(r.resultSets)
r.gc.keys = r.groupCursors[0].Keys()
r.gc.vals = r.groupCursors[0].PartitionKeyVals()
return &r.gc
}
func (r *groupByMergedGroupResultSet) Err() error { return r.err }
func (r *groupByMergedGroupResultSet) Close() {
r.gc.Close()
for _, grs := range r.items {
if grs.gc != nil {
grs.gc.Close()
}
grs.grs.Close()
}
r.items = nil
r.alt = nil
}
type groupByMergedGroupCursor struct {
mergedResultSet
keys [][]byte
vals [][]byte
}
func (r *groupByMergedGroupCursor) Keys() [][]byte {
return r.keys
}
func (r *groupByMergedGroupCursor) PartitionKeyVals() [][]byte {
return r.vals
}
type groupCursorItem struct {
grs GroupResultSet
gc GroupCursor
}
func comparePartitionKey(a, b [][]byte, nilVal []byte) int {
i := 0
for i < len(a) && i < len(b) {
av, bv := a[i], b[i]
if len(av) == 0 {
av = nilVal
}
if len(bv) == 0 {
bv = nilVal
}
if v := bytes.Compare(av, bv); v == 0 {
i++
continue
} else {
return v
}
}
if i < len(b) {
// b is longer, so assume a is less
return -1
} else if i < len(a) {
// a is longer, so assume b is less
return 1
} else {
return 0
}
}

View File

@ -0,0 +1,248 @@
package reads_test
import (
"errors"
"strings"
"testing"
"github.com/google/go-cmp/cmp"
"github.com/influxdata/influxdb/storage/reads"
"github.com/influxdata/influxdb/storage/reads/datatypes"
)
func newGroupNoneStreamSeries(tagKeys string, v ...string) *sliceStreamReader {
var frames []datatypes.ReadResponse_Frame
frames = append(frames, groupF(tagKeys, ""))
for _, s := range v {
frames = append(frames, seriesF(Float, s))
}
return newStreamReader(response(frames...))
}
func TestNewGroupNoneMergedGroupResultSet(t *testing.T) {
exp := `group:
tag key : m0,tag0,tag1,tag2
partition key:
series: _m=m0,tag0=val00
cursor:Float
series: _m=m0,tag0=val01
cursor:Float
series: _m=m0,tag1=val10
cursor:Float
series: _m=m0,tag2=val20
cursor:Float
`
tests := []struct {
name string
streams []*sliceStreamReader
exp string
}{
{
name: "merge tagKey schemas series total order",
streams: []*sliceStreamReader{
newGroupNoneStreamSeries("m0,tag0", "m0,tag0=val00", "m0,tag0=val01"),
newGroupNoneStreamSeries("m0,tag1,tag2", "m0,tag1=val10", "m0,tag2=val20"),
},
exp: exp,
},
{
name: "merge tagKey schemas series mixed",
streams: []*sliceStreamReader{
newGroupNoneStreamSeries("m0,tag0,tag2", "m0,tag0=val01", "m0,tag2=val20"),
newGroupNoneStreamSeries("m0,tag0,tag1", "m0,tag0=val00", "m0,tag1=val10"),
},
exp: exp,
},
{
name: "merge single group schemas ordered",
streams: []*sliceStreamReader{
newGroupNoneStreamSeries("m0,tag0", "m0,tag0=val00"),
newGroupNoneStreamSeries("m0,tag0", "m0,tag0=val01"),
newGroupNoneStreamSeries("m0,tag1", "m0,tag1=val10"),
newGroupNoneStreamSeries("m0,tag2", "m0,tag2=val20"),
},
exp: exp,
},
{
name: "merge single group schemas unordered",
streams: []*sliceStreamReader{
newGroupNoneStreamSeries("m0,tag2", "m0,tag2=val20"),
newGroupNoneStreamSeries("m0,tag0", "m0,tag0=val00"),
newGroupNoneStreamSeries("m0,tag1", "m0,tag1=val10"),
newGroupNoneStreamSeries("m0,tag0", "m0,tag0=val01"),
},
exp: exp,
},
{
name: "merge single group",
streams: []*sliceStreamReader{
newGroupNoneStreamSeries("m0,tag0,tag1,tag2", "m0,tag0=val00", "m0,tag0=val01", "m0,tag1=val10", "m0,tag2=val20"),
},
exp: exp,
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
grss := make([]reads.GroupResultSet, len(tt.streams))
for i := range tt.streams {
grss[i] = reads.NewGroupResultSetStreamReader(tt.streams[i])
}
grs := reads.NewGroupNoneMergedGroupResultSet(grss)
sb := new(strings.Builder)
GroupResultSetToString(sb, grs)
if got := sb.String(); !cmp.Equal(got, tt.exp) {
t.Errorf("unexpected value; -got/+exp\n%s", cmp.Diff(strings.Split(got, "\n"), strings.Split(tt.exp, "\n")))
}
grs.Close()
})
}
}
func TestGroupNoneMergedGroupResultSet_ErrNoData(t *testing.T) {
exp := "no data"
streams := []reads.StreamReader{
newGroupNoneStreamSeries("m0,tag2", "m0,tag2=val20"),
errStreamReader(exp),
}
grss := make([]reads.GroupResultSet, len(streams))
for i := range streams {
grss[i] = reads.NewGroupResultSetStreamReader(streams[i])
}
grs := reads.NewGroupNoneMergedGroupResultSet(grss)
if got := grs.Next(); got != nil {
t.Errorf("expected nil")
}
if got, expErr := grs.Err(), errors.New(exp); !cmp.Equal(got, expErr, cmp.Comparer(errCmp)) {
t.Errorf("unexpected error; -got/+exp\n%s", cmp.Diff(got, expErr, cmp.Transformer("err", errTr)))
}
}
func TestGroupNoneMergedGroupResultSet_ErrStreamNoData(t *testing.T) {
streams := []reads.StreamReader{
newGroupNoneStreamSeries("m0,tag2", "m0,tag2=val20"),
&emptyStreamReader{},
}
grss := make([]reads.GroupResultSet, len(streams))
for i := range streams {
grss[i] = reads.NewGroupResultSetStreamReader(streams[i])
}
grs := reads.NewGroupNoneMergedGroupResultSet(grss)
if got := grs.Next(); got != nil {
t.Errorf("expected nil")
}
if got, expErr := grs.Err(), reads.ErrStreamNoData; !cmp.Equal(got, expErr, cmp.Comparer(errCmp)) {
t.Errorf("unexpected error; -got/+exp\n%s", cmp.Diff(got, expErr, cmp.Transformer("err", errTr)))
}
}
func groupByF(tagKeys, parKeys string, v ...string) datatypes.ReadResponse {
var frames []datatypes.ReadResponse_Frame
frames = append(frames, groupF(tagKeys, parKeys))
for _, s := range v {
frames = append(frames, seriesF(Float, s))
}
return response(frames...)
}
func TestNewGroupByMergedGroupResultSet(t *testing.T) {
exp := `group:
tag key : _m,tag0,tag1
partition key: val00,<nil>
series: _m=aaa,tag0=val00
cursor:Float
series: _m=cpu,tag0=val00,tag1=val10
cursor:Float
series: _m=cpu,tag0=val00,tag1=val11
cursor:Float
series: _m=cpu,tag0=val00,tag1=val12
cursor:Float
group:
tag key : _m,tag0
partition key: val01,<nil>
series: _m=aaa,tag0=val01
cursor:Float
group:
tag key : _m,tag1,tag2
partition key: <nil>,val20
series: _m=mem,tag1=val10,tag2=val20
cursor:Float
series: _m=mem,tag1=val11,tag2=val20
cursor:Float
group:
tag key : _m,tag1,tag2
partition key: <nil>,val21
series: _m=mem,tag1=val11,tag2=val21
cursor:Float
`
tests := []struct {
name string
streams []*sliceStreamReader
exp string
}{
{
streams: []*sliceStreamReader{
newStreamReader(
groupByF("_m,tag0,tag1", "val00,<nil>", "aaa,tag0=val00", "cpu,tag0=val00,tag1=val11"),
groupByF("_m,tag1,tag2", "<nil>,val20", "mem,tag1=val10,tag2=val20"),
groupByF("_m,tag1,tag2", "<nil>,val21", "mem,tag1=val11,tag2=val21"),
),
newStreamReader(
groupByF("_m,tag0,tag1", "val00,<nil>", "cpu,tag0=val00,tag1=val10", "cpu,tag0=val00,tag1=val12"),
groupByF("_m,tag0", "val01,<nil>", "aaa,tag0=val01"),
),
newStreamReader(
groupByF("_m,tag1,tag2", "<nil>,val20", "mem,tag1=val11,tag2=val20"),
),
},
exp: exp,
},
{
streams: []*sliceStreamReader{
newStreamReader(
groupByF("_m,tag1,tag2", "<nil>,val20", "mem,tag1=val10,tag2=val20"),
groupByF("_m,tag1,tag2", "<nil>,val21", "mem,tag1=val11,tag2=val21"),
),
newStreamReader(
groupByF("_m,tag1,tag2", "<nil>,val20", "mem,tag1=val11,tag2=val20"),
),
newStreamReader(
groupByF("_m,tag0,tag1", "val00,<nil>", "cpu,tag0=val00,tag1=val10", "cpu,tag0=val00,tag1=val12"),
groupByF("_m,tag0", "val01,<nil>", "aaa,tag0=val01"),
),
newStreamReader(
groupByF("_m,tag0,tag1", "val00,<nil>", "aaa,tag0=val00", "cpu,tag0=val00,tag1=val11"),
),
},
exp: exp,
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
grss := make([]reads.GroupResultSet, len(tt.streams))
for i := range tt.streams {
grss[i] = reads.NewGroupResultSetStreamReader(tt.streams[i])
}
grs := reads.NewGroupByMergedGroupResultSet(grss)
sb := new(strings.Builder)
GroupResultSetToString(sb, grs, SkipNilCursor())
if got := sb.String(); !cmp.Equal(got, tt.exp) {
t.Errorf("unexpected value; -got/+exp\n%s", cmp.Diff(strings.Split(got, "\n"), strings.Split(tt.exp, "\n")))
}
grs.Close()
})
}
}

582
storage/reads/predicate.go Normal file
View File

@ -0,0 +1,582 @@
package reads
import (
"bytes"
"fmt"
"regexp"
"strconv"
"github.com/influxdata/flux/ast"
"github.com/influxdata/flux/semantic"
"github.com/influxdata/influxdb/models"
"github.com/influxdata/influxdb/storage/reads/datatypes"
"github.com/influxdata/influxql"
"github.com/pkg/errors"
)
const (
fieldKey = "_field"
measurementKey = "_measurement"
valueKey = "_value"
)
// NodeVisitor can be called by Walk to traverse the Node hierarchy.
// The Visit() function is called once per node.
type NodeVisitor interface {
Visit(*datatypes.Node) NodeVisitor
}
func WalkChildren(v NodeVisitor, node *datatypes.Node) {
for _, n := range node.Children {
WalkNode(v, n)
}
}
func WalkNode(v NodeVisitor, node *datatypes.Node) {
if v = v.Visit(node); v == nil {
return
}
WalkChildren(v, node)
}
func PredicateToExprString(p *datatypes.Predicate) string {
if p == nil {
return "[none]"
}
var v predicateExpressionPrinter
WalkNode(&v, p.Root)
return v.Buffer.String()
}
type predicateExpressionPrinter struct {
bytes.Buffer
}
func (v *predicateExpressionPrinter) Visit(n *datatypes.Node) NodeVisitor {
switch n.NodeType {
case datatypes.NodeTypeLogicalExpression:
if len(n.Children) > 0 {
var op string
if n.GetLogical() == datatypes.LogicalAnd {
op = " AND "
} else {
op = " OR "
}
WalkNode(v, n.Children[0])
for _, e := range n.Children[1:] {
v.Buffer.WriteString(op)
WalkNode(v, e)
}
}
return nil
case datatypes.NodeTypeParenExpression:
if len(n.Children) == 1 {
v.Buffer.WriteString("( ")
WalkNode(v, n.Children[0])
v.Buffer.WriteString(" )")
}
return nil
case datatypes.NodeTypeComparisonExpression:
WalkNode(v, n.Children[0])
v.Buffer.WriteByte(' ')
switch n.GetComparison() {
case datatypes.ComparisonEqual:
v.Buffer.WriteByte('=')
case datatypes.ComparisonNotEqual:
v.Buffer.WriteString("!=")
case datatypes.ComparisonStartsWith:
v.Buffer.WriteString("startsWith")
case datatypes.ComparisonRegex:
v.Buffer.WriteString("=~")
case datatypes.ComparisonNotRegex:
v.Buffer.WriteString("!~")
case datatypes.ComparisonLess:
v.Buffer.WriteByte('<')
case datatypes.ComparisonLessEqual:
v.Buffer.WriteString("<=")
case datatypes.ComparisonGreater:
v.Buffer.WriteByte('>')
case datatypes.ComparisonGreaterEqual:
v.Buffer.WriteString(">=")
}
v.Buffer.WriteByte(' ')
WalkNode(v, n.Children[1])
return nil
case datatypes.NodeTypeTagRef:
v.Buffer.WriteByte('\'')
v.Buffer.WriteString(n.GetTagRefValue())
v.Buffer.WriteByte('\'')
return nil
case datatypes.NodeTypeFieldRef:
v.Buffer.WriteByte('$')
return nil
case datatypes.NodeTypeLiteral:
switch val := n.Value.(type) {
case *datatypes.Node_StringValue:
v.Buffer.WriteString(strconv.Quote(val.StringValue))
case *datatypes.Node_RegexValue:
v.Buffer.WriteByte('/')
v.Buffer.WriteString(val.RegexValue)
v.Buffer.WriteByte('/')
case *datatypes.Node_IntegerValue:
v.Buffer.WriteString(strconv.FormatInt(val.IntegerValue, 10))
case *datatypes.Node_UnsignedValue:
v.Buffer.WriteString(strconv.FormatUint(val.UnsignedValue, 10))
case *datatypes.Node_FloatValue:
v.Buffer.WriteString(strconv.FormatFloat(val.FloatValue, 'f', 10, 64))
case *datatypes.Node_BooleanValue:
if val.BooleanValue {
v.Buffer.WriteString("true")
} else {
v.Buffer.WriteString("false")
}
}
return nil
default:
return v
}
}
func toStoragePredicate(f *semantic.FunctionExpression) (*datatypes.Predicate, error) {
if f.Block.Parameters == nil || len(f.Block.Parameters.List) != 1 {
return nil, errors.New("storage predicate functions must have exactly one parameter")
}
root, err := toStoragePredicateHelper(f.Block.Body.(semantic.Expression), f.Block.Parameters.List[0].Key.Name)
if err != nil {
return nil, err
}
return &datatypes.Predicate{
Root: root,
}, nil
}
func toStoragePredicateHelper(n semantic.Expression, objectName string) (*datatypes.Node, error) {
switch n := n.(type) {
case *semantic.LogicalExpression:
left, err := toStoragePredicateHelper(n.Left, objectName)
if err != nil {
return nil, errors.Wrap(err, "left hand side")
}
right, err := toStoragePredicateHelper(n.Right, objectName)
if err != nil {
return nil, errors.Wrap(err, "right hand side")
}
children := []*datatypes.Node{left, right}
switch n.Operator {
case ast.AndOperator:
return &datatypes.Node{
NodeType: datatypes.NodeTypeLogicalExpression,
Value: &datatypes.Node_Logical_{Logical: datatypes.LogicalAnd},
Children: children,
}, nil
case ast.OrOperator:
return &datatypes.Node{
NodeType: datatypes.NodeTypeLogicalExpression,
Value: &datatypes.Node_Logical_{Logical: datatypes.LogicalOr},
Children: children,
}, nil
default:
return nil, fmt.Errorf("unknown logical operator %v", n.Operator)
}
case *semantic.BinaryExpression:
left, err := toStoragePredicateHelper(n.Left, objectName)
if err != nil {
return nil, errors.Wrap(err, "left hand side")
}
right, err := toStoragePredicateHelper(n.Right, objectName)
if err != nil {
return nil, errors.Wrap(err, "right hand side")
}
children := []*datatypes.Node{left, right}
op, err := toComparisonOperator(n.Operator)
if err != nil {
return nil, err
}
return &datatypes.Node{
NodeType: datatypes.NodeTypeComparisonExpression,
Value: &datatypes.Node_Comparison_{Comparison: op},
Children: children,
}, nil
case *semantic.StringLiteral:
return &datatypes.Node{
NodeType: datatypes.NodeTypeLiteral,
Value: &datatypes.Node_StringValue{
StringValue: n.Value,
},
}, nil
case *semantic.IntegerLiteral:
return &datatypes.Node{
NodeType: datatypes.NodeTypeLiteral,
Value: &datatypes.Node_IntegerValue{
IntegerValue: n.Value,
},
}, nil
case *semantic.BooleanLiteral:
return &datatypes.Node{
NodeType: datatypes.NodeTypeLiteral,
Value: &datatypes.Node_BooleanValue{
BooleanValue: n.Value,
},
}, nil
case *semantic.FloatLiteral:
return &datatypes.Node{
NodeType: datatypes.NodeTypeLiteral,
Value: &datatypes.Node_FloatValue{
FloatValue: n.Value,
},
}, nil
case *semantic.RegexpLiteral:
return &datatypes.Node{
NodeType: datatypes.NodeTypeLiteral,
Value: &datatypes.Node_RegexValue{
RegexValue: n.Value.String(),
},
}, nil
case *semantic.MemberExpression:
// Sanity check that the object is the objectName identifier
if ident, ok := n.Object.(*semantic.IdentifierExpression); !ok || ident.Name != objectName {
return nil, fmt.Errorf("unknown object %q", n.Object)
}
switch n.Property {
case fieldKey:
return &datatypes.Node{
NodeType: datatypes.NodeTypeTagRef,
Value: &datatypes.Node_TagRefValue{
TagRefValue: models.FieldKeyTagKey,
},
}, nil
case measurementKey:
return &datatypes.Node{
NodeType: datatypes.NodeTypeTagRef,
Value: &datatypes.Node_TagRefValue{
TagRefValue: models.MeasurementTagKey,
},
}, nil
case valueKey:
return &datatypes.Node{
NodeType: datatypes.NodeTypeFieldRef,
Value: &datatypes.Node_FieldRefValue{
FieldRefValue: valueKey,
},
}, nil
}
return &datatypes.Node{
NodeType: datatypes.NodeTypeTagRef,
Value: &datatypes.Node_TagRefValue{
TagRefValue: n.Property,
},
}, nil
case *semantic.DurationLiteral:
return nil, errors.New("duration literals not supported in storage predicates")
case *semantic.DateTimeLiteral:
return nil, errors.New("time literals not supported in storage predicates")
default:
return nil, fmt.Errorf("unsupported semantic expression type %T", n)
}
}
func toComparisonOperator(o ast.OperatorKind) (datatypes.Node_Comparison, error) {
switch o {
case ast.EqualOperator:
return datatypes.ComparisonEqual, nil
case ast.NotEqualOperator:
return datatypes.ComparisonNotEqual, nil
case ast.RegexpMatchOperator:
return datatypes.ComparisonRegex, nil
case ast.NotRegexpMatchOperator:
return datatypes.ComparisonNotRegex, nil
case ast.StartsWithOperator:
return datatypes.ComparisonStartsWith, nil
case ast.LessThanOperator:
return datatypes.ComparisonLess, nil
case ast.LessThanEqualOperator:
return datatypes.ComparisonLessEqual, nil
case ast.GreaterThanOperator:
return datatypes.ComparisonGreater, nil
case ast.GreaterThanEqualOperator:
return datatypes.ComparisonGreaterEqual, nil
default:
return 0, fmt.Errorf("unknown operator %v", o)
}
}
// NodeToExpr transforms a predicate node to an influxql.Expr.
func NodeToExpr(node *datatypes.Node, remap map[string]string) (influxql.Expr, error) {
v := &nodeToExprVisitor{remap: remap}
WalkNode(v, node)
if err := v.Err(); err != nil {
return nil, err
}
if len(v.exprs) > 1 {
return nil, errors.New("invalid expression")
}
if len(v.exprs) == 0 {
return nil, nil
}
// TODO(edd): It would be preferable if RewriteRegexConditions was a
// package level function in influxql.
stmt := &influxql.SelectStatement{
Condition: v.exprs[0],
}
stmt.RewriteRegexConditions()
return stmt.Condition, nil
}
type nodeToExprVisitor struct {
remap map[string]string
exprs []influxql.Expr
err error
}
func (v *nodeToExprVisitor) Visit(n *datatypes.Node) NodeVisitor {
if v.err != nil {
return nil
}
switch n.NodeType {
case datatypes.NodeTypeLogicalExpression:
if len(n.Children) > 1 {
op := influxql.AND
if n.GetLogical() == datatypes.LogicalOr {
op = influxql.OR
}
WalkNode(v, n.Children[0])
if v.err != nil {
return nil
}
for i := 1; i < len(n.Children); i++ {
WalkNode(v, n.Children[i])
if v.err != nil {
return nil
}
if len(v.exprs) >= 2 {
lhs, rhs := v.pop2()
v.exprs = append(v.exprs, &influxql.BinaryExpr{LHS: lhs, Op: op, RHS: rhs})
}
}
return nil
}
case datatypes.NodeTypeParenExpression:
if len(n.Children) != 1 {
v.err = errors.New("parenExpression expects one child")
return nil
}
WalkNode(v, n.Children[0])
if v.err != nil {
return nil
}
if len(v.exprs) > 0 {
v.exprs = append(v.exprs, &influxql.ParenExpr{Expr: v.pop()})
}
return nil
case datatypes.NodeTypeComparisonExpression:
WalkChildren(v, n)
if len(v.exprs) < 2 {
v.err = errors.New("comparisonExpression expects two children")
return nil
}
lhs, rhs := v.pop2()
be := &influxql.BinaryExpr{LHS: lhs, RHS: rhs}
switch n.GetComparison() {
case datatypes.ComparisonEqual:
be.Op = influxql.EQ
case datatypes.ComparisonNotEqual:
be.Op = influxql.NEQ
case datatypes.ComparisonStartsWith:
// TODO(sgc): rewrite to anchored RE, as index does not support startsWith yet
v.err = errors.New("startsWith not implemented")
return nil
case datatypes.ComparisonRegex:
be.Op = influxql.EQREGEX
case datatypes.ComparisonNotRegex:
be.Op = influxql.NEQREGEX
case datatypes.ComparisonLess:
be.Op = influxql.LT
case datatypes.ComparisonLessEqual:
be.Op = influxql.LTE
case datatypes.ComparisonGreater:
be.Op = influxql.GT
case datatypes.ComparisonGreaterEqual:
be.Op = influxql.GTE
default:
v.err = errors.New("invalid comparison operator")
return nil
}
v.exprs = append(v.exprs, be)
return nil
case datatypes.NodeTypeTagRef:
ref := n.GetTagRefValue()
if v.remap != nil {
if nk, ok := v.remap[ref]; ok {
ref = nk
}
}
v.exprs = append(v.exprs, &influxql.VarRef{Val: ref, Type: influxql.Tag})
return nil
case datatypes.NodeTypeFieldRef:
v.exprs = append(v.exprs, &influxql.VarRef{Val: "$"})
return nil
case datatypes.NodeTypeLiteral:
switch val := n.Value.(type) {
case *datatypes.Node_StringValue:
v.exprs = append(v.exprs, &influxql.StringLiteral{Val: val.StringValue})
case *datatypes.Node_RegexValue:
// TODO(sgc): consider hashing the RegexValue and cache compiled version
re, err := regexp.Compile(val.RegexValue)
if err != nil {
v.err = err
}
v.exprs = append(v.exprs, &influxql.RegexLiteral{Val: re})
return nil
case *datatypes.Node_IntegerValue:
v.exprs = append(v.exprs, &influxql.IntegerLiteral{Val: val.IntegerValue})
case *datatypes.Node_UnsignedValue:
v.exprs = append(v.exprs, &influxql.UnsignedLiteral{Val: val.UnsignedValue})
case *datatypes.Node_FloatValue:
v.exprs = append(v.exprs, &influxql.NumberLiteral{Val: val.FloatValue})
case *datatypes.Node_BooleanValue:
v.exprs = append(v.exprs, &influxql.BooleanLiteral{Val: val.BooleanValue})
default:
v.err = errors.New("unexpected literal type")
return nil
}
return nil
default:
return v
}
return nil
}
func (v *nodeToExprVisitor) Err() error {
return v.err
}
func (v *nodeToExprVisitor) pop() influxql.Expr {
if len(v.exprs) == 0 {
panic("stack empty")
}
var top influxql.Expr
top, v.exprs = v.exprs[len(v.exprs)-1], v.exprs[:len(v.exprs)-1]
return top
}
func (v *nodeToExprVisitor) pop2() (influxql.Expr, influxql.Expr) {
if len(v.exprs) < 2 {
panic("stack empty")
}
rhs := v.exprs[len(v.exprs)-1]
lhs := v.exprs[len(v.exprs)-2]
v.exprs = v.exprs[:len(v.exprs)-2]
return lhs, rhs
}
func IsTrueBooleanLiteral(expr influxql.Expr) bool {
b, ok := expr.(*influxql.BooleanLiteral)
if ok {
return b.Val
}
return false
}
func RewriteExprRemoveFieldValue(expr influxql.Expr) influxql.Expr {
return influxql.RewriteExpr(expr, func(expr influxql.Expr) influxql.Expr {
if be, ok := expr.(*influxql.BinaryExpr); ok {
if ref, ok := be.LHS.(*influxql.VarRef); ok {
if ref.Val == "$" {
return &influxql.BooleanLiteral{Val: true}
}
}
}
return expr
})
}
type hasRefs struct {
refs []string
found []bool
}
func (v *hasRefs) allFound() bool {
for _, val := range v.found {
if !val {
return false
}
}
return true
}
func (v *hasRefs) Visit(node influxql.Node) influxql.Visitor {
if v.allFound() {
return nil
}
if n, ok := node.(*influxql.VarRef); ok {
for i, r := range v.refs {
if !v.found[i] && r == n.Val {
v.found[i] = true
if v.allFound() {
return nil
}
}
}
}
return v
}
func HasFieldValueKey(expr influxql.Expr) bool {
refs := hasRefs{refs: []string{valueKey}, found: make([]bool, 1)}
influxql.Walk(&refs, expr)
return refs.found[0]
}

View File

@ -0,0 +1,58 @@
package reads_test
import (
"testing"
"github.com/influxdata/influxdb/storage/reads"
"github.com/influxdata/influxdb/storage/reads/datatypes"
)
func TestPredicateToExprString(t *testing.T) {
cases := []struct {
n string
r *datatypes.Predicate
e string
}{
{
n: "returns [none] for nil",
r: nil,
e: "[none]",
},
{
n: "logical AND",
r: &datatypes.Predicate{
Root: &datatypes.Node{
NodeType: datatypes.NodeTypeLogicalExpression,
Value: &datatypes.Node_Logical_{Logical: datatypes.LogicalAnd},
Children: []*datatypes.Node{
{
NodeType: datatypes.NodeTypeComparisonExpression,
Value: &datatypes.Node_Comparison_{Comparison: datatypes.ComparisonEqual},
Children: []*datatypes.Node{
{NodeType: datatypes.NodeTypeTagRef, Value: &datatypes.Node_TagRefValue{TagRefValue: "host"}},
{NodeType: datatypes.NodeTypeLiteral, Value: &datatypes.Node_StringValue{StringValue: "host1"}},
},
},
{
NodeType: datatypes.NodeTypeComparisonExpression,
Value: &datatypes.Node_Comparison_{Comparison: datatypes.ComparisonRegex},
Children: []*datatypes.Node{
{NodeType: datatypes.NodeTypeTagRef, Value: &datatypes.Node_TagRefValue{TagRefValue: "region"}},
{NodeType: datatypes.NodeTypeLiteral, Value: &datatypes.Node_RegexValue{RegexValue: "^us-west"}},
},
},
},
},
},
e: `'host' = "host1" AND 'region' =~ /^us-west/`,
},
}
for _, tc := range cases {
t.Run(tc.n, func(t *testing.T) {
if got, wanted := reads.PredicateToExprString(tc.r), tc.e; got != wanted {
t.Fatal("got:", got, "wanted:", wanted)
}
})
}
}

542
storage/reads/reader.go Normal file
View File

@ -0,0 +1,542 @@
package reads
import (
"bytes"
"context"
"fmt"
"strings"
"github.com/gogo/protobuf/types"
"github.com/influxdata/flux"
"github.com/influxdata/flux/execute"
"github.com/influxdata/flux/memory"
"github.com/influxdata/flux/values"
"github.com/influxdata/influxdb/flux/stdlib/influxdata/influxdb"
"github.com/influxdata/influxdb/models"
"github.com/influxdata/influxdb/storage/reads/datatypes"
"github.com/influxdata/influxdb/tsdb/cursors"
)
type storageTable interface {
flux.Table
Close()
Cancel()
Statistics() cursors.CursorStats
}
type storeReader struct {
s Store
}
func NewReader(s Store) influxdb.Reader {
return &storeReader{s: s}
}
func (r *storeReader) Read(ctx context.Context, rs influxdb.ReadSpec, start, stop execute.Time, alloc *memory.Allocator) (flux.TableIterator, error) {
var predicate *datatypes.Predicate
if rs.Predicate != nil {
p, err := toStoragePredicate(rs.Predicate)
if err != nil {
return nil, err
}
predicate = p
}
return &tableIterator{
ctx: ctx,
bounds: execute.Bounds{Start: start, Stop: stop},
s: r.s,
readSpec: rs,
predicate: predicate,
alloc: alloc,
}, nil
}
func (r *storeReader) Close() {}
type tableIterator struct {
ctx context.Context
bounds execute.Bounds
s Store
readSpec influxdb.ReadSpec
predicate *datatypes.Predicate
stats cursors.CursorStats
alloc *memory.Allocator
}
func (bi *tableIterator) Statistics() cursors.CursorStats { return bi.stats }
func (bi *tableIterator) Do(f func(flux.Table) error) error {
src, err := bi.s.GetSource(bi.readSpec)
if err != nil {
return err
}
// Setup read request
var req datatypes.ReadRequest
if any, err := types.MarshalAny(src); err != nil {
return err
} else {
req.ReadSource = any
}
req.Predicate = bi.predicate
req.Descending = bi.readSpec.Descending
req.TimestampRange.Start = int64(bi.bounds.Start)
req.TimestampRange.End = int64(bi.bounds.Stop)
req.Group = convertGroupMode(bi.readSpec.GroupMode)
req.GroupKeys = bi.readSpec.GroupKeys
req.SeriesLimit = bi.readSpec.SeriesLimit
req.PointsLimit = bi.readSpec.PointsLimit
req.SeriesOffset = bi.readSpec.SeriesOffset
if req.PointsLimit == -1 {
req.Hints.SetNoPoints()
}
if agg, err := determineAggregateMethod(bi.readSpec.AggregateMethod); err != nil {
return err
} else if agg != datatypes.AggregateTypeNone {
req.Aggregate = &datatypes.Aggregate{Type: agg}
}
switch {
case req.Group != datatypes.GroupAll:
rs, err := bi.s.GroupRead(bi.ctx, &req)
if err != nil {
return err
}
if rs == nil {
return nil
}
if req.Hints.NoPoints() {
return bi.handleGroupReadNoPoints(f, rs)
}
return bi.handleGroupRead(f, rs)
default:
rs, err := bi.s.Read(bi.ctx, &req)
if err != nil {
return err
}
if rs == nil {
return nil
}
if req.Hints.NoPoints() {
return bi.handleReadNoPoints(f, rs)
}
return bi.handleRead(f, rs)
}
}
func (bi *tableIterator) handleRead(f func(flux.Table) error, rs ResultSet) error {
// these resources must be closed if not nil on return
var (
cur cursors.Cursor
table storageTable
)
defer func() {
if table != nil {
table.Close()
}
if cur != nil {
cur.Close()
}
rs.Close()
}()
READ:
for rs.Next() {
cur = rs.Cursor()
if cur == nil {
// no data for series key + field combination
continue
}
key := groupKeyForSeries(rs.Tags(), &bi.readSpec, bi.bounds)
done := make(chan struct{})
switch typedCur := cur.(type) {
case cursors.IntegerArrayCursor:
cols, defs := determineTableColsForSeries(rs.Tags(), flux.TInt)
table = newIntegerTable(done, typedCur, bi.bounds, key, cols, rs.Tags(), defs, bi.alloc)
case cursors.FloatArrayCursor:
cols, defs := determineTableColsForSeries(rs.Tags(), flux.TFloat)
table = newFloatTable(done, typedCur, bi.bounds, key, cols, rs.Tags(), defs, bi.alloc)
case cursors.UnsignedArrayCursor:
cols, defs := determineTableColsForSeries(rs.Tags(), flux.TUInt)
table = newUnsignedTable(done, typedCur, bi.bounds, key, cols, rs.Tags(), defs, bi.alloc)
case cursors.BooleanArrayCursor:
cols, defs := determineTableColsForSeries(rs.Tags(), flux.TBool)
table = newBooleanTable(done, typedCur, bi.bounds, key, cols, rs.Tags(), defs, bi.alloc)
case cursors.StringArrayCursor:
cols, defs := determineTableColsForSeries(rs.Tags(), flux.TString)
table = newStringTable(done, typedCur, bi.bounds, key, cols, rs.Tags(), defs, bi.alloc)
default:
panic(fmt.Sprintf("unreachable: %T", typedCur))
}
cur = nil
if !table.Empty() {
if err := f(table); err != nil {
table.Close()
table = nil
return err
}
select {
case <-done:
case <-bi.ctx.Done():
table.Cancel()
break READ
}
}
stats := table.Statistics()
bi.stats.ScannedValues += stats.ScannedValues
bi.stats.ScannedBytes += stats.ScannedBytes
table.Close()
table = nil
}
return rs.Err()
}
func (bi *tableIterator) handleReadNoPoints(f func(flux.Table) error, rs ResultSet) error {
// these resources must be closed if not nil on return
var table storageTable
defer func() {
if table != nil {
table.Close()
}
rs.Close()
}()
READ:
for rs.Next() {
cur := rs.Cursor()
if !hasPoints(cur) {
// no data for series key + field combination
continue
}
key := groupKeyForSeries(rs.Tags(), &bi.readSpec, bi.bounds)
done := make(chan struct{})
cols, defs := determineTableColsForSeries(rs.Tags(), flux.TString)
table = newTableNoPoints(done, bi.bounds, key, cols, rs.Tags(), defs, bi.alloc)
if err := f(table); err != nil {
table.Close()
table = nil
return err
}
select {
case <-done:
case <-bi.ctx.Done():
table.Cancel()
break READ
}
table.Close()
table = nil
}
return rs.Err()
}
func (bi *tableIterator) handleGroupRead(f func(flux.Table) error, rs GroupResultSet) error {
// these resources must be closed if not nil on return
var (
gc GroupCursor
cur cursors.Cursor
table storageTable
)
defer func() {
if table != nil {
table.Close()
}
if cur != nil {
cur.Close()
}
if gc != nil {
gc.Close()
}
rs.Close()
}()
gc = rs.Next()
READ:
for gc != nil {
for gc.Next() {
cur = gc.Cursor()
if cur != nil {
break
}
}
if cur == nil {
gc.Close()
gc = rs.Next()
continue
}
key := groupKeyForGroup(gc.PartitionKeyVals(), &bi.readSpec, bi.bounds)
done := make(chan struct{})
switch typedCur := cur.(type) {
case cursors.IntegerArrayCursor:
cols, defs := determineTableColsForGroup(gc.Keys(), flux.TInt)
table = newIntegerGroupTable(done, gc, typedCur, bi.bounds, key, cols, gc.Tags(), defs, bi.alloc)
case cursors.FloatArrayCursor:
cols, defs := determineTableColsForGroup(gc.Keys(), flux.TFloat)
table = newFloatGroupTable(done, gc, typedCur, bi.bounds, key, cols, gc.Tags(), defs, bi.alloc)
case cursors.UnsignedArrayCursor:
cols, defs := determineTableColsForGroup(gc.Keys(), flux.TUInt)
table = newUnsignedGroupTable(done, gc, typedCur, bi.bounds, key, cols, gc.Tags(), defs, bi.alloc)
case cursors.BooleanArrayCursor:
cols, defs := determineTableColsForGroup(gc.Keys(), flux.TBool)
table = newBooleanGroupTable(done, gc, typedCur, bi.bounds, key, cols, gc.Tags(), defs, bi.alloc)
case cursors.StringArrayCursor:
cols, defs := determineTableColsForGroup(gc.Keys(), flux.TString)
table = newStringGroupTable(done, gc, typedCur, bi.bounds, key, cols, gc.Tags(), defs, bi.alloc)
default:
panic(fmt.Sprintf("unreachable: %T", typedCur))
}
// table owns these resources and is responsible for closing them
cur = nil
gc = nil
if err := f(table); err != nil {
table.Close()
table = nil
return err
}
select {
case <-done:
case <-bi.ctx.Done():
table.Cancel()
break READ
}
stats := table.Statistics()
bi.stats.ScannedValues += stats.ScannedValues
bi.stats.ScannedBytes += stats.ScannedBytes
table.Close()
table = nil
gc = rs.Next()
}
return rs.Err()
}
func (bi *tableIterator) handleGroupReadNoPoints(f func(flux.Table) error, rs GroupResultSet) error {
// these resources must be closed if not nil on return
var (
gc GroupCursor
table storageTable
)
defer func() {
if table != nil {
table.Close()
}
if gc != nil {
gc.Close()
}
rs.Close()
}()
gc = rs.Next()
READ:
for gc != nil {
key := groupKeyForGroup(gc.PartitionKeyVals(), &bi.readSpec, bi.bounds)
done := make(chan struct{})
cols, defs := determineTableColsForGroup(gc.Keys(), flux.TString)
table = newGroupTableNoPoints(done, bi.bounds, key, cols, defs, bi.alloc)
gc.Close()
gc = nil
if err := f(table); err != nil {
table.Close()
table = nil
return err
}
select {
case <-done:
case <-bi.ctx.Done():
table.Cancel()
break READ
}
table.Close()
table = nil
gc = rs.Next()
}
return rs.Err()
}
func determineAggregateMethod(agg string) (datatypes.Aggregate_AggregateType, error) {
if agg == "" {
return datatypes.AggregateTypeNone, nil
}
if t, ok := datatypes.Aggregate_AggregateType_value[strings.ToUpper(agg)]; ok {
return datatypes.Aggregate_AggregateType(t), nil
}
return 0, fmt.Errorf("unknown aggregate type %q", agg)
}
func convertGroupMode(m influxdb.GroupMode) datatypes.ReadRequest_Group {
switch m {
case influxdb.GroupModeNone:
return datatypes.GroupNone
case influxdb.GroupModeBy:
return datatypes.GroupBy
case influxdb.GroupModeExcept:
return datatypes.GroupExcept
case influxdb.GroupModeDefault, influxdb.GroupModeAll:
fallthrough
default:
return datatypes.GroupAll
}
}
const (
startColIdx = 0
stopColIdx = 1
timeColIdx = 2
valueColIdx = 3
)
func determineTableColsForSeries(tags models.Tags, typ flux.ColType) ([]flux.ColMeta, [][]byte) {
cols := make([]flux.ColMeta, 4+len(tags))
defs := make([][]byte, 4+len(tags))
cols[startColIdx] = flux.ColMeta{
Label: execute.DefaultStartColLabel,
Type: flux.TTime,
}
cols[stopColIdx] = flux.ColMeta{
Label: execute.DefaultStopColLabel,
Type: flux.TTime,
}
cols[timeColIdx] = flux.ColMeta{
Label: execute.DefaultTimeColLabel,
Type: flux.TTime,
}
cols[valueColIdx] = flux.ColMeta{
Label: execute.DefaultValueColLabel,
Type: typ,
}
for j, tag := range tags {
cols[4+j] = flux.ColMeta{
Label: string(tag.Key),
Type: flux.TString,
}
defs[4+j] = []byte("")
}
return cols, defs
}
func groupKeyForSeries(tags models.Tags, readSpec *influxdb.ReadSpec, bnds execute.Bounds) flux.GroupKey {
cols := make([]flux.ColMeta, 2, len(tags))
vs := make([]values.Value, 2, len(tags))
cols[0] = flux.ColMeta{
Label: execute.DefaultStartColLabel,
Type: flux.TTime,
}
vs[0] = values.NewTime(bnds.Start)
cols[1] = flux.ColMeta{
Label: execute.DefaultStopColLabel,
Type: flux.TTime,
}
vs[1] = values.NewTime(bnds.Stop)
switch readSpec.GroupMode {
case influxdb.GroupModeBy:
// group key in GroupKeys order, including tags in the GroupKeys slice
for _, k := range readSpec.GroupKeys {
bk := []byte(k)
for _, t := range tags {
if bytes.Equal(t.Key, bk) && len(t.Value) > 0 {
cols = append(cols, flux.ColMeta{
Label: k,
Type: flux.TString,
})
vs = append(vs, values.NewString(string(t.Value)))
}
}
}
case influxdb.GroupModeExcept:
// group key in GroupKeys order, skipping tags in the GroupKeys slice
panic("not implemented")
case influxdb.GroupModeDefault, influxdb.GroupModeAll:
for i := range tags {
cols = append(cols, flux.ColMeta{
Label: string(tags[i].Key),
Type: flux.TString,
})
vs = append(vs, values.NewString(string(tags[i].Value)))
}
}
return execute.NewGroupKey(cols, vs)
}
func determineTableColsForGroup(tagKeys [][]byte, typ flux.ColType) ([]flux.ColMeta, [][]byte) {
cols := make([]flux.ColMeta, 4+len(tagKeys))
defs := make([][]byte, 4+len(tagKeys))
cols[startColIdx] = flux.ColMeta{
Label: execute.DefaultStartColLabel,
Type: flux.TTime,
}
cols[stopColIdx] = flux.ColMeta{
Label: execute.DefaultStopColLabel,
Type: flux.TTime,
}
cols[timeColIdx] = flux.ColMeta{
Label: execute.DefaultTimeColLabel,
Type: flux.TTime,
}
cols[valueColIdx] = flux.ColMeta{
Label: execute.DefaultValueColLabel,
Type: typ,
}
for j, tag := range tagKeys {
cols[4+j] = flux.ColMeta{
Label: string(tag),
Type: flux.TString,
}
defs[4+j] = []byte("")
}
return cols, defs
}
func groupKeyForGroup(kv [][]byte, readSpec *influxdb.ReadSpec, bnds execute.Bounds) flux.GroupKey {
cols := make([]flux.ColMeta, 2, len(readSpec.GroupKeys)+2)
vs := make([]values.Value, 2, len(readSpec.GroupKeys)+2)
cols[0] = flux.ColMeta{
Label: execute.DefaultStartColLabel,
Type: flux.TTime,
}
vs[0] = values.NewTime(bnds.Start)
cols[1] = flux.ColMeta{
Label: execute.DefaultStopColLabel,
Type: flux.TTime,
}
vs[1] = values.NewTime(bnds.Stop)
for i := range readSpec.GroupKeys {
if readSpec.GroupKeys[i] == execute.DefaultStartColLabel || readSpec.GroupKeys[i] == execute.DefaultStopColLabel {
continue
}
cols = append(cols, flux.ColMeta{
Label: readSpec.GroupKeys[i],
Type: flux.TString,
})
vs = append(vs, values.NewString(string(kv[i])))
}
return execute.NewGroupKey(cols, vs)
}

View File

@ -0,0 +1,16 @@
// Code generated by "stringer -type=readState -trimprefix=state"; DO NOT EDIT.
package reads
import "strconv"
const _readState_name = "ReadGroupReadSeriesReadPointsReadFloatPointsReadIntegerPointsReadUnsignedPointsReadBooleanPointsReadStringPointsReadErrDone"
var _readState_index = [...]uint8{0, 9, 19, 29, 44, 61, 79, 96, 112, 119, 123}
func (i readState) String() string {
if i >= readState(len(_readState_index)-1) {
return "readState(" + strconv.FormatInt(int64(i), 10) + ")"
}
return _readState_name[_readState_index[i]:_readState_index[i+1]]
}

View File

@ -0,0 +1,457 @@
// Generated by tmpl
// https://github.com/benbjohnson/tmpl
//
// DO NOT EDIT!
// Source: response_writer.gen.go.tmpl
package reads
import (
"github.com/influxdata/influxdb/storage/reads/datatypes"
"github.com/influxdata/influxdb/tsdb/cursors"
)
func (w *ResponseWriter) getFloatPointsFrame() *datatypes.ReadResponse_Frame_FloatPoints {
var res *datatypes.ReadResponse_Frame_FloatPoints
if len(w.buffer.Float) > 0 {
i := len(w.buffer.Float) - 1
res = w.buffer.Float[i]
w.buffer.Float[i] = nil
w.buffer.Float = w.buffer.Float[:i]
} else {
res = &datatypes.ReadResponse_Frame_FloatPoints{
FloatPoints: &datatypes.ReadResponse_FloatPointsFrame{
Timestamps: make([]int64, 0, batchSize),
Values: make([]float64, 0, batchSize),
},
}
}
return res
}
func (w *ResponseWriter) putFloatPointsFrame(f *datatypes.ReadResponse_Frame_FloatPoints) {
f.FloatPoints.Timestamps = f.FloatPoints.Timestamps[:0]
f.FloatPoints.Values = f.FloatPoints.Values[:0]
w.buffer.Float = append(w.buffer.Float, f)
}
func (w *ResponseWriter) streamFloatArraySeries(cur cursors.FloatArrayCursor) {
w.sf.DataType = datatypes.DataTypeFloat
ss := len(w.res.Frames) - 1
a := cur.Next()
if len(a.Timestamps) == 0 {
w.sz -= w.sf.Size()
w.putSeriesFrame(w.res.Frames[ss].Data.(*datatypes.ReadResponse_Frame_Series))
w.res.Frames = w.res.Frames[:ss]
} else if w.sz > writeSize {
w.Flush()
}
}
func (w *ResponseWriter) streamFloatArrayPoints(cur cursors.FloatArrayCursor) {
w.sf.DataType = datatypes.DataTypeFloat
ss := len(w.res.Frames) - 1
p := w.getFloatPointsFrame()
frame := p.FloatPoints
w.res.Frames = append(w.res.Frames, datatypes.ReadResponse_Frame{Data: p})
var (
seriesValueCount = 0
b = 0
)
for {
a := cur.Next()
if len(a.Timestamps) == 0 {
break
}
frame.Timestamps = append(frame.Timestamps, a.Timestamps...)
frame.Values = append(frame.Values, a.Values...)
b = len(frame.Timestamps)
if b >= batchSize {
seriesValueCount += b
b = 0
w.sz += frame.Size()
if w.sz >= writeSize {
w.Flush()
if w.err != nil {
break
}
}
p = w.getFloatPointsFrame()
frame = p.FloatPoints
w.res.Frames = append(w.res.Frames, datatypes.ReadResponse_Frame{Data: p})
}
}
seriesValueCount += b
w.vc += seriesValueCount
if seriesValueCount == 0 {
w.sz -= w.sf.Size()
w.putSeriesFrame(w.res.Frames[ss].Data.(*datatypes.ReadResponse_Frame_Series))
w.res.Frames = w.res.Frames[:ss]
} else if w.sz > writeSize {
w.Flush()
}
}
func (w *ResponseWriter) getIntegerPointsFrame() *datatypes.ReadResponse_Frame_IntegerPoints {
var res *datatypes.ReadResponse_Frame_IntegerPoints
if len(w.buffer.Integer) > 0 {
i := len(w.buffer.Integer) - 1
res = w.buffer.Integer[i]
w.buffer.Integer[i] = nil
w.buffer.Integer = w.buffer.Integer[:i]
} else {
res = &datatypes.ReadResponse_Frame_IntegerPoints{
IntegerPoints: &datatypes.ReadResponse_IntegerPointsFrame{
Timestamps: make([]int64, 0, batchSize),
Values: make([]int64, 0, batchSize),
},
}
}
return res
}
func (w *ResponseWriter) putIntegerPointsFrame(f *datatypes.ReadResponse_Frame_IntegerPoints) {
f.IntegerPoints.Timestamps = f.IntegerPoints.Timestamps[:0]
f.IntegerPoints.Values = f.IntegerPoints.Values[:0]
w.buffer.Integer = append(w.buffer.Integer, f)
}
func (w *ResponseWriter) streamIntegerArraySeries(cur cursors.IntegerArrayCursor) {
w.sf.DataType = datatypes.DataTypeInteger
ss := len(w.res.Frames) - 1
a := cur.Next()
if len(a.Timestamps) == 0 {
w.sz -= w.sf.Size()
w.putSeriesFrame(w.res.Frames[ss].Data.(*datatypes.ReadResponse_Frame_Series))
w.res.Frames = w.res.Frames[:ss]
} else if w.sz > writeSize {
w.Flush()
}
}
func (w *ResponseWriter) streamIntegerArrayPoints(cur cursors.IntegerArrayCursor) {
w.sf.DataType = datatypes.DataTypeInteger
ss := len(w.res.Frames) - 1
p := w.getIntegerPointsFrame()
frame := p.IntegerPoints
w.res.Frames = append(w.res.Frames, datatypes.ReadResponse_Frame{Data: p})
var (
seriesValueCount = 0
b = 0
)
for {
a := cur.Next()
if len(a.Timestamps) == 0 {
break
}
frame.Timestamps = append(frame.Timestamps, a.Timestamps...)
frame.Values = append(frame.Values, a.Values...)
b = len(frame.Timestamps)
if b >= batchSize {
seriesValueCount += b
b = 0
w.sz += frame.Size()
if w.sz >= writeSize {
w.Flush()
if w.err != nil {
break
}
}
p = w.getIntegerPointsFrame()
frame = p.IntegerPoints
w.res.Frames = append(w.res.Frames, datatypes.ReadResponse_Frame{Data: p})
}
}
seriesValueCount += b
w.vc += seriesValueCount
if seriesValueCount == 0 {
w.sz -= w.sf.Size()
w.putSeriesFrame(w.res.Frames[ss].Data.(*datatypes.ReadResponse_Frame_Series))
w.res.Frames = w.res.Frames[:ss]
} else if w.sz > writeSize {
w.Flush()
}
}
func (w *ResponseWriter) getUnsignedPointsFrame() *datatypes.ReadResponse_Frame_UnsignedPoints {
var res *datatypes.ReadResponse_Frame_UnsignedPoints
if len(w.buffer.Unsigned) > 0 {
i := len(w.buffer.Unsigned) - 1
res = w.buffer.Unsigned[i]
w.buffer.Unsigned[i] = nil
w.buffer.Unsigned = w.buffer.Unsigned[:i]
} else {
res = &datatypes.ReadResponse_Frame_UnsignedPoints{
UnsignedPoints: &datatypes.ReadResponse_UnsignedPointsFrame{
Timestamps: make([]int64, 0, batchSize),
Values: make([]uint64, 0, batchSize),
},
}
}
return res
}
func (w *ResponseWriter) putUnsignedPointsFrame(f *datatypes.ReadResponse_Frame_UnsignedPoints) {
f.UnsignedPoints.Timestamps = f.UnsignedPoints.Timestamps[:0]
f.UnsignedPoints.Values = f.UnsignedPoints.Values[:0]
w.buffer.Unsigned = append(w.buffer.Unsigned, f)
}
func (w *ResponseWriter) streamUnsignedArraySeries(cur cursors.UnsignedArrayCursor) {
w.sf.DataType = datatypes.DataTypeUnsigned
ss := len(w.res.Frames) - 1
a := cur.Next()
if len(a.Timestamps) == 0 {
w.sz -= w.sf.Size()
w.putSeriesFrame(w.res.Frames[ss].Data.(*datatypes.ReadResponse_Frame_Series))
w.res.Frames = w.res.Frames[:ss]
} else if w.sz > writeSize {
w.Flush()
}
}
func (w *ResponseWriter) streamUnsignedArrayPoints(cur cursors.UnsignedArrayCursor) {
w.sf.DataType = datatypes.DataTypeUnsigned
ss := len(w.res.Frames) - 1
p := w.getUnsignedPointsFrame()
frame := p.UnsignedPoints
w.res.Frames = append(w.res.Frames, datatypes.ReadResponse_Frame{Data: p})
var (
seriesValueCount = 0
b = 0
)
for {
a := cur.Next()
if len(a.Timestamps) == 0 {
break
}
frame.Timestamps = append(frame.Timestamps, a.Timestamps...)
frame.Values = append(frame.Values, a.Values...)
b = len(frame.Timestamps)
if b >= batchSize {
seriesValueCount += b
b = 0
w.sz += frame.Size()
if w.sz >= writeSize {
w.Flush()
if w.err != nil {
break
}
}
p = w.getUnsignedPointsFrame()
frame = p.UnsignedPoints
w.res.Frames = append(w.res.Frames, datatypes.ReadResponse_Frame{Data: p})
}
}
seriesValueCount += b
w.vc += seriesValueCount
if seriesValueCount == 0 {
w.sz -= w.sf.Size()
w.putSeriesFrame(w.res.Frames[ss].Data.(*datatypes.ReadResponse_Frame_Series))
w.res.Frames = w.res.Frames[:ss]
} else if w.sz > writeSize {
w.Flush()
}
}
func (w *ResponseWriter) getStringPointsFrame() *datatypes.ReadResponse_Frame_StringPoints {
var res *datatypes.ReadResponse_Frame_StringPoints
if len(w.buffer.String) > 0 {
i := len(w.buffer.String) - 1
res = w.buffer.String[i]
w.buffer.String[i] = nil
w.buffer.String = w.buffer.String[:i]
} else {
res = &datatypes.ReadResponse_Frame_StringPoints{
StringPoints: &datatypes.ReadResponse_StringPointsFrame{
Timestamps: make([]int64, 0, batchSize),
Values: make([]string, 0, batchSize),
},
}
}
return res
}
func (w *ResponseWriter) putStringPointsFrame(f *datatypes.ReadResponse_Frame_StringPoints) {
f.StringPoints.Timestamps = f.StringPoints.Timestamps[:0]
f.StringPoints.Values = f.StringPoints.Values[:0]
w.buffer.String = append(w.buffer.String, f)
}
func (w *ResponseWriter) streamStringArraySeries(cur cursors.StringArrayCursor) {
w.sf.DataType = datatypes.DataTypeString
ss := len(w.res.Frames) - 1
a := cur.Next()
if len(a.Timestamps) == 0 {
w.sz -= w.sf.Size()
w.putSeriesFrame(w.res.Frames[ss].Data.(*datatypes.ReadResponse_Frame_Series))
w.res.Frames = w.res.Frames[:ss]
} else if w.sz > writeSize {
w.Flush()
}
}
func (w *ResponseWriter) streamStringArrayPoints(cur cursors.StringArrayCursor) {
w.sf.DataType = datatypes.DataTypeString
ss := len(w.res.Frames) - 1
p := w.getStringPointsFrame()
frame := p.StringPoints
w.res.Frames = append(w.res.Frames, datatypes.ReadResponse_Frame{Data: p})
var (
seriesValueCount = 0
b = 0
)
for {
a := cur.Next()
if len(a.Timestamps) == 0 {
break
}
frame.Timestamps = append(frame.Timestamps, a.Timestamps...)
frame.Values = append(frame.Values, a.Values...)
b = len(frame.Timestamps)
if b >= batchSize {
seriesValueCount += b
b = 0
w.sz += frame.Size()
if w.sz >= writeSize {
w.Flush()
if w.err != nil {
break
}
}
p = w.getStringPointsFrame()
frame = p.StringPoints
w.res.Frames = append(w.res.Frames, datatypes.ReadResponse_Frame{Data: p})
}
}
seriesValueCount += b
w.vc += seriesValueCount
if seriesValueCount == 0 {
w.sz -= w.sf.Size()
w.putSeriesFrame(w.res.Frames[ss].Data.(*datatypes.ReadResponse_Frame_Series))
w.res.Frames = w.res.Frames[:ss]
} else if w.sz > writeSize {
w.Flush()
}
}
func (w *ResponseWriter) getBooleanPointsFrame() *datatypes.ReadResponse_Frame_BooleanPoints {
var res *datatypes.ReadResponse_Frame_BooleanPoints
if len(w.buffer.Boolean) > 0 {
i := len(w.buffer.Boolean) - 1
res = w.buffer.Boolean[i]
w.buffer.Boolean[i] = nil
w.buffer.Boolean = w.buffer.Boolean[:i]
} else {
res = &datatypes.ReadResponse_Frame_BooleanPoints{
BooleanPoints: &datatypes.ReadResponse_BooleanPointsFrame{
Timestamps: make([]int64, 0, batchSize),
Values: make([]bool, 0, batchSize),
},
}
}
return res
}
func (w *ResponseWriter) putBooleanPointsFrame(f *datatypes.ReadResponse_Frame_BooleanPoints) {
f.BooleanPoints.Timestamps = f.BooleanPoints.Timestamps[:0]
f.BooleanPoints.Values = f.BooleanPoints.Values[:0]
w.buffer.Boolean = append(w.buffer.Boolean, f)
}
func (w *ResponseWriter) streamBooleanArraySeries(cur cursors.BooleanArrayCursor) {
w.sf.DataType = datatypes.DataTypeBoolean
ss := len(w.res.Frames) - 1
a := cur.Next()
if len(a.Timestamps) == 0 {
w.sz -= w.sf.Size()
w.putSeriesFrame(w.res.Frames[ss].Data.(*datatypes.ReadResponse_Frame_Series))
w.res.Frames = w.res.Frames[:ss]
} else if w.sz > writeSize {
w.Flush()
}
}
func (w *ResponseWriter) streamBooleanArrayPoints(cur cursors.BooleanArrayCursor) {
w.sf.DataType = datatypes.DataTypeBoolean
ss := len(w.res.Frames) - 1
p := w.getBooleanPointsFrame()
frame := p.BooleanPoints
w.res.Frames = append(w.res.Frames, datatypes.ReadResponse_Frame{Data: p})
var (
seriesValueCount = 0
b = 0
)
for {
a := cur.Next()
if len(a.Timestamps) == 0 {
break
}
frame.Timestamps = append(frame.Timestamps, a.Timestamps...)
frame.Values = append(frame.Values, a.Values...)
b = len(frame.Timestamps)
if b >= batchSize {
seriesValueCount += b
b = 0
w.sz += frame.Size()
if w.sz >= writeSize {
w.Flush()
if w.err != nil {
break
}
}
p = w.getBooleanPointsFrame()
frame = p.BooleanPoints
w.res.Frames = append(w.res.Frames, datatypes.ReadResponse_Frame{Data: p})
}
}
seriesValueCount += b
w.vc += seriesValueCount
if seriesValueCount == 0 {
w.sz -= w.sf.Size()
w.putSeriesFrame(w.res.Frames[ss].Data.(*datatypes.ReadResponse_Frame_Series))
w.res.Frames = w.res.Frames[:ss]
} else if w.sz > writeSize {
w.Flush()
}
}

View File

@ -0,0 +1,285 @@
package reads
import (
"fmt"
"google.golang.org/grpc/metadata"
"github.com/influxdata/influxdb/models"
"github.com/influxdata/influxdb/storage/reads/datatypes"
"github.com/influxdata/influxdb/tsdb/cursors"
)
type ResponseStream interface {
Send(*datatypes.ReadResponse) error
// SetTrailer sets the trailer metadata which will be sent with the RPC status.
// When called more than once, all the provided metadata will be merged.
SetTrailer(metadata.MD)
}
const (
batchSize = 1000
frameCount = 50
writeSize = 64 << 10 // 64k
)
type ResponseWriter struct {
stream ResponseStream
res *datatypes.ReadResponse
err error
// current series
sf *datatypes.ReadResponse_SeriesFrame
ss int // pointer to current series frame; used to skip writing if no points
sz int // estimated size in bytes for pending write
vc int // total value count
buffer struct {
Float []*datatypes.ReadResponse_Frame_FloatPoints
Integer []*datatypes.ReadResponse_Frame_IntegerPoints
Unsigned []*datatypes.ReadResponse_Frame_UnsignedPoints
Boolean []*datatypes.ReadResponse_Frame_BooleanPoints
String []*datatypes.ReadResponse_Frame_StringPoints
Series []*datatypes.ReadResponse_Frame_Series
Group []*datatypes.ReadResponse_Frame_Group
}
hints datatypes.HintFlags
}
func NewResponseWriter(stream ResponseStream, hints datatypes.HintFlags) *ResponseWriter {
rw := &ResponseWriter{stream: stream,
res: &datatypes.ReadResponse{
Frames: make([]datatypes.ReadResponse_Frame, 0, frameCount),
},
hints: hints,
}
return rw
}
// WrittenN returns the number of values written to the response stream.
func (w *ResponseWriter) WrittenN() int { return w.vc }
func (w *ResponseWriter) WriteResultSet(rs ResultSet) error {
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
}
}
stats := rs.Stats()
w.stream.SetTrailer(metadata.Pairs(
"scanned-bytes", fmt.Sprint(stats.ScannedBytes),
"scanned-values", fmt.Sprint(stats.ScannedValues)))
return nil
}
func (w *ResponseWriter) WriteGroupResultSet(rs GroupResultSet) error {
stats := cursors.CursorStats{}
gc := rs.Next()
for gc != nil {
w.startGroup(gc.Keys(), gc.PartitionKeyVals())
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
}
stats.Add(gc.Stats())
}
gc.Close()
gc = rs.Next()
}
w.stream.SetTrailer(metadata.Pairs(
"scanned-bytes", fmt.Sprint(stats.ScannedBytes),
"scanned-values", fmt.Sprint(stats.ScannedValues)))
return nil
}
func (w *ResponseWriter) Err() error { return w.err }
func (w *ResponseWriter) getGroupFrame(keys, partitionKey [][]byte) *datatypes.ReadResponse_Frame_Group {
var res *datatypes.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 = &datatypes.ReadResponse_Frame_Group{Group: &datatypes.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 *datatypes.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) *datatypes.ReadResponse_Frame_Series {
var res *datatypes.ReadResponse_Frame_Series
if len(w.buffer.Series) > 0 {
i := len(w.buffer.Series) - 1
res = w.buffer.Series[i]
w.buffer.Series[i] = nil
w.buffer.Series = w.buffer.Series[:i]
} else {
res = &datatypes.ReadResponse_Frame_Series{Series: &datatypes.ReadResponse_SeriesFrame{}}
}
if cap(res.Series.Tags) < len(next) {
res.Series.Tags = make([]datatypes.Tag, len(next))
} else if len(res.Series.Tags) != len(next) {
res.Series.Tags = res.Series.Tags[:len(next)]
}
return res
}
func (w *ResponseWriter) putSeriesFrame(f *datatypes.ReadResponse_Frame_Series) {
tags := f.Series.Tags
for i := range tags {
tags[i].Key = nil
tags[i].Value = nil
}
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, datatypes.ReadResponse_Frame{Data: 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)
w.sf = f.Series
for i, t := range next {
w.sf.Tags[i] = datatypes.Tag{
Key: t.Key,
Value: t.Value,
}
}
w.res.Frames = append(w.res.Frames, datatypes.ReadResponse_Frame{Data: f})
w.sz += w.sf.Size()
}
func (w *ResponseWriter) streamCursor(cur cursors.Cursor) {
switch {
case w.hints.NoSeries():
// skip
case w.hints.NoPoints():
switch cur := cur.(type) {
case cursors.IntegerArrayCursor:
w.streamIntegerArraySeries(cur)
case cursors.FloatArrayCursor:
w.streamFloatArraySeries(cur)
case cursors.UnsignedArrayCursor:
w.streamUnsignedArraySeries(cur)
case cursors.BooleanArrayCursor:
w.streamBooleanArraySeries(cur)
case cursors.StringArrayCursor:
w.streamStringArraySeries(cur)
default:
panic(fmt.Sprintf("unreachable: %T", cur))
}
default:
switch cur := cur.(type) {
case cursors.IntegerArrayCursor:
w.streamIntegerArrayPoints(cur)
case cursors.FloatArrayCursor:
w.streamFloatArrayPoints(cur)
case cursors.UnsignedArrayCursor:
w.streamUnsignedArrayPoints(cur)
case cursors.BooleanArrayCursor:
w.streamBooleanArrayPoints(cur)
case cursors.StringArrayCursor:
w.streamStringArrayPoints(cur)
default:
panic(fmt.Sprintf("unreachable: %T", cur))
}
}
cur.Close()
}
func (w *ResponseWriter) Flush() {
if w.err != nil || w.sz == 0 {
return
}
w.sz = 0
if w.err = w.stream.Send(w.res); w.err != nil {
return
}
for i := range w.res.Frames {
d := w.res.Frames[i].Data
w.res.Frames[i].Data = nil
switch p := d.(type) {
case *datatypes.ReadResponse_Frame_FloatPoints:
w.putFloatPointsFrame(p)
case *datatypes.ReadResponse_Frame_IntegerPoints:
w.putIntegerPointsFrame(p)
case *datatypes.ReadResponse_Frame_UnsignedPoints:
w.putUnsignedPointsFrame(p)
case *datatypes.ReadResponse_Frame_BooleanPoints:
w.putBooleanPointsFrame(p)
case *datatypes.ReadResponse_Frame_StringPoints:
w.putStringPointsFrame(p)
case *datatypes.ReadResponse_Frame_Series:
w.putSeriesFrame(p)
case *datatypes.ReadResponse_Frame_Group:
w.putGroupFrame(p)
}
}
w.res.Frames = w.res.Frames[:0]
}

View File

@ -0,0 +1,125 @@
package reads_test
import (
"fmt"
"reflect"
"testing"
"github.com/influxdata/influxdb/mock"
"github.com/influxdata/influxdb/storage/reads"
"github.com/influxdata/influxdb/tsdb/cursors"
"google.golang.org/grpc/metadata"
)
func TestResponseWriter_WriteResultSet_Stats(t *testing.T) {
scannedValues := 37
scannedBytes := 41
var gotTrailer metadata.MD = nil
stream := mock.NewResponseStream()
stream.SetTrailerFunc = func(trailer metadata.MD) {
if gotTrailer != nil {
t.Error("trailer expected to be set once, but SetTrailer was called more than once")
} else {
gotTrailer = trailer
}
}
rs := mock.NewResultSet()
rs.StatsFunc = func() cursors.CursorStats {
return cursors.CursorStats{
ScannedValues: scannedValues,
ScannedBytes: scannedBytes,
}
}
nextHasBeenCalledOnce := false
rs.NextFunc = func() bool { // Returns true exactly once
if !nextHasBeenCalledOnce {
nextHasBeenCalledOnce = true
return true
}
return false
}
cursorHasBeenCalledOnce := false
rs.CursorFunc = func() cursors.Cursor {
if !cursorHasBeenCalledOnce {
cursorHasBeenCalledOnce = true
return mock.NewIntegerArrayCursor()
}
return nil
}
// This is what we're testing.
rw := reads.NewResponseWriter(stream, 0)
err := rw.WriteResultSet(rs)
if err != nil {
t.Fatal(err)
}
if !reflect.DeepEqual(gotTrailer.Get("scanned-values"), []string{fmt.Sprint(scannedValues)}) {
t.Errorf("expected scanned-values '%v' but got '%v'", []string{fmt.Sprint(scannedValues)}, gotTrailer.Get("scanned-values"))
}
if !reflect.DeepEqual(gotTrailer.Get("scanned-bytes"), []string{fmt.Sprint(scannedBytes)}) {
t.Errorf("expected scanned-bytes '%v' but got '%v'", []string{fmt.Sprint(scannedBytes)}, gotTrailer.Get("scanned-bytes"))
}
}
func TestResponseWriter_WriteGroupResultSet_Stats(t *testing.T) {
scannedValues := 37
scannedBytes := 41
var gotTrailer metadata.MD = nil
stream := mock.NewResponseStream()
stream.SetTrailerFunc = func(trailer metadata.MD) {
if gotTrailer != nil {
t.Error("trailer expected to be set once, but SetTrailer was called more than once")
} else {
gotTrailer = trailer
}
}
gc := mock.NewGroupCursor()
gc.StatsFunc = func() cursors.CursorStats {
return cursors.CursorStats{
ScannedValues: scannedValues,
ScannedBytes: scannedBytes,
}
}
cNextHasBeenCalledOnce := false
gc.NextFunc = func() bool {
if !cNextHasBeenCalledOnce {
cNextHasBeenCalledOnce = true
return true
}
return false
}
gc.CursorFunc = func() cursors.Cursor {
return mock.NewIntegerArrayCursor()
}
rs := mock.NewGroupResultSet()
rsNextHasBeenCalledOnce := false
rs.NextFunc = func() reads.GroupCursor {
if !rsNextHasBeenCalledOnce {
rsNextHasBeenCalledOnce = true
return gc
}
return nil
}
// This is what we're testing.
rw := reads.NewResponseWriter(stream, 0)
err := rw.WriteGroupResultSet(rs)
if err != nil {
t.Fatal(err)
}
if !reflect.DeepEqual(gotTrailer.Get("scanned-values"), []string{fmt.Sprint(scannedValues)}) {
t.Errorf("expected scanned-values '%v' but got '%v'", []string{fmt.Sprint(scannedValues)}, gotTrailer.Get("scanned-values"))
}
if !reflect.DeepEqual(gotTrailer.Get("scanned-bytes"), []string{fmt.Sprint(scannedBytes)}) {
t.Errorf("expected scanned-bytes '%v' but got '%v'", []string{fmt.Sprint(scannedBytes)}, gotTrailer.Get("scanned-bytes"))
}
}

View File

@ -0,0 +1,74 @@
package reads
import (
"context"
"github.com/influxdata/influxdb/models"
"github.com/influxdata/influxdb/storage/reads/datatypes"
"github.com/influxdata/influxdb/tsdb/cursors"
)
type multiShardCursors interface {
createCursor(row SeriesRow) cursors.Cursor
newAggregateCursor(ctx context.Context, agg *datatypes.Aggregate, cursor cursors.Cursor) cursors.Cursor
}
type resultSet struct {
ctx context.Context
agg *datatypes.Aggregate
cur SeriesCursor
row SeriesRow
mb multiShardCursors
}
func NewResultSet(ctx context.Context, req *datatypes.ReadRequest, cur SeriesCursor) ResultSet {
return &resultSet{
ctx: ctx,
agg: req.Aggregate,
cur: cur,
mb: newMultiShardArrayCursors(ctx, req.TimestampRange.Start, req.TimestampRange.End, !req.Descending, req.PointsLimit),
}
}
func (r *resultSet) Err() error { return nil }
// Close closes the result set. Close is idempotent.
func (r *resultSet) Close() {
if r == nil {
return // Nothing to do.
}
r.row.Query = nil
r.cur.Close()
}
// Next returns true if there are more results available.
func (r *resultSet) Next() bool {
if r == nil {
return false
}
row := r.cur.Next()
if row == nil {
return false
}
r.row = *row
return true
}
func (r *resultSet) Cursor() cursors.Cursor {
cur := r.mb.createCursor(r.row)
if r.agg != nil {
cur = r.mb.newAggregateCursor(r.ctx, r.agg, cur)
}
return cur
}
func (r *resultSet) Tags() models.Tags {
return r.row.Tags
}
// Stats returns the stats for the underlying cursors.
// Available after resultset has been scanned.
func (r *resultSet) Stats() cursors.CursorStats { return r.row.Query.Stats() }

View File

@ -0,0 +1,51 @@
package reads
import (
"context"
"github.com/influxdata/influxdb/models"
"github.com/influxdata/influxdb/tsdb/cursors"
"github.com/influxdata/influxql"
)
type SeriesCursor interface {
Close()
Next() *SeriesRow
Err() error
}
type SeriesRow struct {
SortKey []byte
Name []byte // measurement name
SeriesTags models.Tags // unmodified series tags
Tags models.Tags
Field string
Query cursors.CursorIterators
ValueCond influxql.Expr
}
type limitSeriesCursor struct {
SeriesCursor
n, o, c int64
}
func NewLimitSeriesCursor(ctx context.Context, cur SeriesCursor, n, o int64) SeriesCursor {
return &limitSeriesCursor{SeriesCursor: cur, o: o, n: n}
}
func (c *limitSeriesCursor) Next() *SeriesRow {
if c.o > 0 {
for i := int64(0); i < c.o; i++ {
if c.SeriesCursor.Next() == nil {
break
}
}
c.o = 0
}
if c.c >= c.n {
return nil
}
c.c++
return c.SeriesCursor.Next()
}

81
storage/reads/store.go Normal file
View File

@ -0,0 +1,81 @@
package reads
import (
"context"
"github.com/gogo/protobuf/proto"
"github.com/influxdata/influxdb/flux/stdlib/influxdata/influxdb"
"github.com/influxdata/influxdb/models"
"github.com/influxdata/influxdb/storage/reads/datatypes"
"github.com/influxdata/influxdb/tsdb/cursors"
)
type ResultSet interface {
// Next advances the ResultSet to the next cursor. It returns false
// when there are no more cursors.
Next() bool
// Cursor returns the most recent cursor after a call to Next.
Cursor() cursors.Cursor
// Tags returns the tags for the most recent cursor after a call to Next.
Tags() models.Tags
// Close releases any resources allocated by the ResultSet.
Close()
// Err returns the first error encountered by the ResultSet.
Err() error
Stats() cursors.CursorStats
}
type GroupResultSet interface {
// Next advances the GroupResultSet and returns the next GroupCursor. It
// returns nil if there are no more groups.
Next() GroupCursor
// Close releases any resources allocated by the GroupResultSet.
Close()
// Err returns the first error encountered by the GroupResultSet.
Err() error
}
type GroupCursor interface {
// Next advances to the next cursor. Next will return false when there are no
// more cursors in the current group.
Next() bool
// Cursor returns the most recent cursor after a call to Next.
Cursor() cursors.Cursor
// Tags returns the tags for the most recent cursor after a call to Next.
Tags() models.Tags
// Keys returns the union of all tag key names for all series produced by
// this GroupCursor.
Keys() [][]byte
// PartitionKeyVals returns the values of all tags identified by the
// keys specified in ReadRequest#GroupKeys. The tag values values will
// appear in the same order as the GroupKeys.
//
// When the datatypes.GroupNone strategy is specified, PartitionKeyVals will
// be nil.
PartitionKeyVals() [][]byte
// Close releases any resources allocated by the GroupCursor.
Close()
// Err returns the first error encountered by the GroupCursor.
Err() error
Stats() cursors.CursorStats
}
type Store interface {
Read(ctx context.Context, req *datatypes.ReadRequest) (ResultSet, error)
GroupRead(ctx context.Context, req *datatypes.ReadRequest) (GroupResultSet, error)
GetSource(rs influxdb.ReadSpec) (proto.Message, error)
}

223
storage/reads/store_test.go Normal file
View File

@ -0,0 +1,223 @@
package reads_test
import (
"bytes"
"fmt"
"io"
"strings"
"github.com/influxdata/influxdb/models"
"github.com/influxdata/influxdb/storage/reads"
"github.com/influxdata/influxdb/tsdb/cursors"
)
func CursorToString(wr io.Writer, cur cursors.Cursor, opts ...optionFn) {
switch ccur := cur.(type) {
case cursors.IntegerArrayCursor:
fmt.Fprintln(wr, "Integer")
for {
a := ccur.Next()
if a.Len() > 0 {
for i := range a.Timestamps {
fmt.Fprintf(wr, "%20d | %20d\n", a.Timestamps[i], a.Values[i])
}
} else {
break
}
}
case cursors.FloatArrayCursor:
fmt.Fprintln(wr, "Float")
for {
a := ccur.Next()
if a.Len() > 0 {
for i := range a.Timestamps {
fmt.Fprintf(wr, "%20d | %18.2f\n", a.Timestamps[i], a.Values[i])
}
} else {
break
}
}
case cursors.UnsignedArrayCursor:
fmt.Fprintln(wr, "Unsigned")
for {
a := ccur.Next()
if a.Len() > 0 {
for i := range a.Timestamps {
fmt.Fprintf(wr, "%20d | %20d\n", a.Timestamps[i], a.Values[i])
}
} else {
break
}
}
case cursors.BooleanArrayCursor:
fmt.Fprintln(wr, "Boolean")
for {
a := ccur.Next()
if a.Len() > 0 {
for i := range a.Timestamps {
fmt.Fprintf(wr, "%20d | %t\n", a.Timestamps[i], a.Values[i])
}
} else {
break
}
}
case cursors.StringArrayCursor:
fmt.Fprintln(wr, "String")
for {
a := ccur.Next()
if a.Len() > 0 {
for i := range a.Timestamps {
fmt.Fprintf(wr, "%20d | %20s\n", a.Timestamps[i], a.Values[i])
}
} else {
break
}
}
default:
fmt.Fprintln(wr, "Invalid")
fmt.Fprintf(wr, "unreachable: %T\n", cur)
}
if err := cur.Err(); err != nil && err != io.EOF {
fmt.Fprintf(wr, "cursor err: %s\n", cur.Err().Error())
}
cur.Close()
}
const nilVal = "<nil>"
var (
nilValBytes = []byte(nilVal)
)
func joinString(b [][]byte) string {
s := make([]string, len(b))
for i := range b {
v := b[i]
if len(v) == 0 {
s[i] = nilVal
} else {
s[i] = string(v)
}
}
return strings.Join(s, ",")
}
func TagsToString(wr io.Writer, tags models.Tags, opts ...optionFn) {
if k := tags.HashKey(); len(k) > 0 {
fmt.Fprintf(wr, "%s", string(k[1:]))
}
fmt.Fprintln(wr)
}
func ResultSetToString(wr io.Writer, rs reads.ResultSet, opts ...optionFn) {
var po PrintOptions
for _, o := range opts {
o(&po)
}
iw := ensureIndentWriter(wr)
wr = iw
for rs.Next() {
fmt.Fprint(wr, "series: ")
TagsToString(wr, rs.Tags())
cur := rs.Cursor()
if po.SkipNilCursor && cur == nil {
continue
}
iw.Indent(2)
fmt.Fprint(wr, "cursor:")
if cur == nil {
fmt.Fprintln(wr, nilVal)
goto LOOP
}
CursorToString(wr, cur)
LOOP:
iw.Indent(-2)
}
}
func GroupResultSetToString(wr io.Writer, rs reads.GroupResultSet, opts ...optionFn) {
iw := ensureIndentWriter(wr)
wr = iw
gc := rs.Next()
for gc != nil {
fmt.Fprintln(wr, "group:")
iw.Indent(2)
fmt.Fprintf(wr, "tag key : %s\n", joinString(gc.Keys()))
fmt.Fprintf(wr, "partition key: %s\n", joinString(gc.PartitionKeyVals()))
iw.Indent(2)
ResultSetToString(wr, gc, opts...)
iw.Indent(-4)
gc = rs.Next()
}
}
type PrintOptions struct {
SkipNilCursor bool
}
type optionFn func(o *PrintOptions)
func SkipNilCursor() optionFn {
return func(o *PrintOptions) {
o.SkipNilCursor = true
}
}
type indentWriter struct {
l int
p []byte
wr io.Writer
bol bool
}
func ensureIndentWriter(wr io.Writer) *indentWriter {
if iw, ok := wr.(*indentWriter); ok {
return iw
} else {
return newIndentWriter(wr)
}
}
func newIndentWriter(wr io.Writer) *indentWriter {
return &indentWriter{
wr: wr,
bol: true,
}
}
func (w *indentWriter) Indent(n int) {
w.l += n
if w.l < 0 {
panic("negative indent")
}
w.p = bytes.Repeat([]byte(" "), w.l)
}
func (w *indentWriter) Write(p []byte) (n int, err error) {
for _, c := range p {
if w.bol {
_, err = w.wr.Write(w.p)
if err != nil {
break
}
w.bol = false
}
_, err = w.wr.Write([]byte{c})
if err != nil {
break
}
n++
w.bol = c == '\n'
}
return n, err
}

View File

@ -0,0 +1,263 @@
// Generated by tmpl
// https://github.com/benbjohnson/tmpl
//
// DO NOT EDIT!
// Source: stream_reader.gen.go.tmpl
package reads
import (
"fmt"
"github.com/influxdata/influxdb/storage/reads/datatypes"
"github.com/influxdata/influxdb/tsdb/cursors"
)
type streamCursor interface {
streamCursor()
}
type floatCursorStreamReader struct {
fr *frameReader
a cursors.FloatArray
}
func (c *floatCursorStreamReader) streamCursor() {}
func (c *floatCursorStreamReader) Close() {
for c.fr.state == stateReadFloatPoints {
c.readFrame()
}
}
func (c *floatCursorStreamReader) Err() error { return c.fr.err }
func (c *floatCursorStreamReader) Next() *cursors.FloatArray {
if c.fr.state == stateReadFloatPoints {
c.readFrame()
}
return &c.a
}
func (c *floatCursorStreamReader) readFrame() {
c.a.Timestamps = nil
c.a.Values = nil
if f := c.fr.peekFrame(); f != nil {
switch ff := f.Data.(type) {
case *datatypes.ReadResponse_Frame_FloatPoints:
c.a.Timestamps = ff.FloatPoints.Timestamps
c.a.Values = ff.FloatPoints.Values
c.fr.nextFrame()
case *datatypes.ReadResponse_Frame_Series:
c.fr.state = stateReadSeries
case *datatypes.ReadResponse_Frame_Group:
c.fr.state = stateReadGroup
default:
c.fr.setErr(fmt.Errorf("floatCursorStreamReader: unexpected frame type %T", f.Data))
}
}
}
func (c *floatCursorStreamReader) Stats() cursors.CursorStats {
return c.fr.stats.Stats()
}
type integerCursorStreamReader struct {
fr *frameReader
a cursors.IntegerArray
}
func (c *integerCursorStreamReader) streamCursor() {}
func (c *integerCursorStreamReader) Close() {
for c.fr.state == stateReadIntegerPoints {
c.readFrame()
}
}
func (c *integerCursorStreamReader) Err() error { return c.fr.err }
func (c *integerCursorStreamReader) Next() *cursors.IntegerArray {
if c.fr.state == stateReadIntegerPoints {
c.readFrame()
}
return &c.a
}
func (c *integerCursorStreamReader) readFrame() {
c.a.Timestamps = nil
c.a.Values = nil
if f := c.fr.peekFrame(); f != nil {
switch ff := f.Data.(type) {
case *datatypes.ReadResponse_Frame_IntegerPoints:
c.a.Timestamps = ff.IntegerPoints.Timestamps
c.a.Values = ff.IntegerPoints.Values
c.fr.nextFrame()
case *datatypes.ReadResponse_Frame_Series:
c.fr.state = stateReadSeries
case *datatypes.ReadResponse_Frame_Group:
c.fr.state = stateReadGroup
default:
c.fr.setErr(fmt.Errorf("integerCursorStreamReader: unexpected frame type %T", f.Data))
}
}
}
func (c *integerCursorStreamReader) Stats() cursors.CursorStats {
return c.fr.stats.Stats()
}
type unsignedCursorStreamReader struct {
fr *frameReader
a cursors.UnsignedArray
}
func (c *unsignedCursorStreamReader) streamCursor() {}
func (c *unsignedCursorStreamReader) Close() {
for c.fr.state == stateReadUnsignedPoints {
c.readFrame()
}
}
func (c *unsignedCursorStreamReader) Err() error { return c.fr.err }
func (c *unsignedCursorStreamReader) Next() *cursors.UnsignedArray {
if c.fr.state == stateReadUnsignedPoints {
c.readFrame()
}
return &c.a
}
func (c *unsignedCursorStreamReader) readFrame() {
c.a.Timestamps = nil
c.a.Values = nil
if f := c.fr.peekFrame(); f != nil {
switch ff := f.Data.(type) {
case *datatypes.ReadResponse_Frame_UnsignedPoints:
c.a.Timestamps = ff.UnsignedPoints.Timestamps
c.a.Values = ff.UnsignedPoints.Values
c.fr.nextFrame()
case *datatypes.ReadResponse_Frame_Series:
c.fr.state = stateReadSeries
case *datatypes.ReadResponse_Frame_Group:
c.fr.state = stateReadGroup
default:
c.fr.setErr(fmt.Errorf("unsignedCursorStreamReader: unexpected frame type %T", f.Data))
}
}
}
func (c *unsignedCursorStreamReader) Stats() cursors.CursorStats {
return c.fr.stats.Stats()
}
type stringCursorStreamReader struct {
fr *frameReader
a cursors.StringArray
}
func (c *stringCursorStreamReader) streamCursor() {}
func (c *stringCursorStreamReader) Close() {
for c.fr.state == stateReadStringPoints {
c.readFrame()
}
}
func (c *stringCursorStreamReader) Err() error { return c.fr.err }
func (c *stringCursorStreamReader) Next() *cursors.StringArray {
if c.fr.state == stateReadStringPoints {
c.readFrame()
}
return &c.a
}
func (c *stringCursorStreamReader) readFrame() {
c.a.Timestamps = nil
c.a.Values = nil
if f := c.fr.peekFrame(); f != nil {
switch ff := f.Data.(type) {
case *datatypes.ReadResponse_Frame_StringPoints:
c.a.Timestamps = ff.StringPoints.Timestamps
c.a.Values = ff.StringPoints.Values
c.fr.nextFrame()
case *datatypes.ReadResponse_Frame_Series:
c.fr.state = stateReadSeries
case *datatypes.ReadResponse_Frame_Group:
c.fr.state = stateReadGroup
default:
c.fr.setErr(fmt.Errorf("stringCursorStreamReader: unexpected frame type %T", f.Data))
}
}
}
func (c *stringCursorStreamReader) Stats() cursors.CursorStats {
return c.fr.stats.Stats()
}
type booleanCursorStreamReader struct {
fr *frameReader
a cursors.BooleanArray
}
func (c *booleanCursorStreamReader) streamCursor() {}
func (c *booleanCursorStreamReader) Close() {
for c.fr.state == stateReadBooleanPoints {
c.readFrame()
}
}
func (c *booleanCursorStreamReader) Err() error { return c.fr.err }
func (c *booleanCursorStreamReader) Next() *cursors.BooleanArray {
if c.fr.state == stateReadBooleanPoints {
c.readFrame()
}
return &c.a
}
func (c *booleanCursorStreamReader) readFrame() {
c.a.Timestamps = nil
c.a.Values = nil
if f := c.fr.peekFrame(); f != nil {
switch ff := f.Data.(type) {
case *datatypes.ReadResponse_Frame_BooleanPoints:
c.a.Timestamps = ff.BooleanPoints.Timestamps
c.a.Values = ff.BooleanPoints.Values
c.fr.nextFrame()
case *datatypes.ReadResponse_Frame_Series:
c.fr.state = stateReadSeries
case *datatypes.ReadResponse_Frame_Group:
c.fr.state = stateReadGroup
default:
c.fr.setErr(fmt.Errorf("booleanCursorStreamReader: unexpected frame type %T", f.Data))
}
}
}
func (c *booleanCursorStreamReader) Stats() cursors.CursorStats {
return c.fr.stats.Stats()
}

View File

@ -0,0 +1,443 @@
package reads
import (
"errors"
"fmt"
"io"
"strconv"
"github.com/influxdata/influxdb/models"
"github.com/influxdata/influxdb/storage/reads/datatypes"
"github.com/influxdata/influxdb/tsdb/cursors"
"google.golang.org/grpc/metadata"
)
var (
// ErrSeriesKeyOrder means the series keys for a ResultSetStreamReader were
// incorrectly ordered.
ErrSeriesKeyOrder = errors.New("invalid series key order")
// ErrPartitionKeyOrder means the partition keys for a
// GroupResultSetStreamReader were incorrectly ordered.
ErrPartitionKeyOrder = errors.New("invalid partition key order")
// ErrStreamNoData means the StreamReader repeatedly returned no data
// when calling Recv
ErrStreamNoData = errors.New("peekFrame: no data")
)
// peekFrameRetries specifies the number of times peekFrame will
// retry before returning ErrStreamNoData when StreamReader.Recv
// returns an empty result.
const peekFrameRetries = 2
type StreamReader interface {
Recv() (*datatypes.ReadResponse, error)
}
// statistics is the interface which wraps the Stats method.
type statistics interface {
Stats() cursors.CursorStats
}
var zeroStatistics statistics = &emptyStatistics{}
type emptyStatistics struct{}
func (*emptyStatistics) Stats() cursors.CursorStats {
return cursors.CursorStats{}
}
// StorageReadClient adapts a Storage_ReadClient to implement cursors.Statistics
// and read the statistics from the gRPC trailer.
type StorageReadClient struct {
c datatypes.Storage_ReadClient
trailer metadata.MD
}
// NewStorageReadClient returns a new StorageReadClient which implements
// StreamReader and reads the gRPC trailer to return CursorStats.
func NewStorageReadClient(c datatypes.Storage_ReadClient) *StorageReadClient {
return &StorageReadClient{c: c}
}
func (rc *StorageReadClient) Recv() (res *datatypes.ReadResponse, err error) {
res, err = rc.c.Recv()
if err != nil {
rc.trailer = rc.c.Trailer()
}
return res, err
}
func (rc *StorageReadClient) Stats() (stats cursors.CursorStats) {
for _, s := range rc.trailer.Get("scanned-bytes") {
v, err := strconv.Atoi(s)
if err != nil {
continue
}
stats.ScannedBytes += v
}
for _, s := range rc.trailer.Get("scanned-values") {
v, err := strconv.Atoi(s)
if err != nil {
continue
}
stats.ScannedValues += v
}
return stats
}
type ResultSetStreamReader struct {
fr frameReader
cur cursorReaders
tags models.Tags
prev models.Tags
}
func NewResultSetStreamReader(stream StreamReader) *ResultSetStreamReader {
r := &ResultSetStreamReader{fr: frameReader{s: stream, state: stateReadSeries}}
r.fr.init()
r.cur.setFrameReader(&r.fr)
return r
}
func (r *ResultSetStreamReader) Err() error { return r.fr.err }
func (r *ResultSetStreamReader) Close() { r.fr.state = stateDone }
func (r *ResultSetStreamReader) Cursor() cursors.Cursor { return r.cur.cursor() }
func (r *ResultSetStreamReader) Stats() cursors.CursorStats {
return r.fr.stats.Stats()
}
func (r *ResultSetStreamReader) Next() bool {
if r.fr.state == stateReadSeries {
return r.readSeriesFrame()
}
if r.fr.state == stateDone || r.fr.state == stateReadErr {
return false
}
r.fr.setErr(fmt.Errorf("expected reader in state %v, was in state %v", stateReadSeries, r.fr.state))
return false
}
func (r *ResultSetStreamReader) readSeriesFrame() bool {
f := r.fr.peekFrame()
if f == nil {
return false
}
r.fr.nextFrame()
if sf, ok := f.Data.(*datatypes.ReadResponse_Frame_Series); ok {
r.fr.state = stateReadPoints
r.prev, r.tags = r.tags, r.prev
if cap(r.tags) < len(sf.Series.Tags) {
r.tags = make(models.Tags, len(sf.Series.Tags))
} else {
r.tags = r.tags[:len(sf.Series.Tags)]
}
for i := range sf.Series.Tags {
r.tags[i].Key = sf.Series.Tags[i].Key
r.tags[i].Value = sf.Series.Tags[i].Value
}
if models.CompareTags(r.tags, r.prev) == 1 || r.prev == nil {
r.cur.nextType = sf.Series.DataType
return true
}
r.fr.setErr(ErrSeriesKeyOrder)
} else {
r.fr.setErr(fmt.Errorf("expected series frame, got %T", f.Data))
}
return false
}
func (r *ResultSetStreamReader) Tags() models.Tags {
return r.tags
}
type GroupResultSetStreamReader struct {
fr frameReader
gc groupCursorStreamReader
}
func NewGroupResultSetStreamReader(stream StreamReader) *GroupResultSetStreamReader {
r := &GroupResultSetStreamReader{fr: frameReader{s: stream, state: stateReadGroup}}
r.fr.init()
r.gc.fr = &r.fr
r.gc.cur.setFrameReader(&r.fr)
return r
}
func (r *GroupResultSetStreamReader) Err() error { return r.fr.err }
func (r *GroupResultSetStreamReader) Next() GroupCursor {
if r.fr.state == stateReadGroup {
return r.readGroupFrame()
}
if r.fr.state == stateDone || r.fr.state == stateReadErr {
return nil
}
r.fr.setErr(fmt.Errorf("expected reader in state %v, was in state %v", stateReadGroup, r.fr.state))
return nil
}
func (r *GroupResultSetStreamReader) readGroupFrame() GroupCursor {
f := r.fr.peekFrame()
if f == nil {
return nil
}
r.fr.nextFrame()
if sf, ok := f.Data.(*datatypes.ReadResponse_Frame_Group); ok {
r.fr.state = stateReadSeries
if cap(r.gc.tagKeys) < len(sf.Group.TagKeys) {
r.gc.tagKeys = make([][]byte, len(sf.Group.TagKeys))
} else {
r.gc.tagKeys = r.gc.tagKeys[:len(sf.Group.TagKeys)]
}
copy(r.gc.tagKeys, sf.Group.TagKeys)
r.gc.partitionKeyVals, r.gc.prevKey = r.gc.prevKey, r.gc.partitionKeyVals
if cap(r.gc.partitionKeyVals) < len(sf.Group.PartitionKeyVals) {
r.gc.partitionKeyVals = make([][]byte, len(sf.Group.PartitionKeyVals))
} else {
r.gc.partitionKeyVals = r.gc.partitionKeyVals[:len(sf.Group.PartitionKeyVals)]
}
copy(r.gc.partitionKeyVals, sf.Group.PartitionKeyVals)
if comparePartitionKey(r.gc.partitionKeyVals, r.gc.prevKey, nilSortHi) == 1 || r.gc.prevKey == nil {
return &r.gc
}
r.fr.setErr(ErrPartitionKeyOrder)
} else {
r.fr.setErr(fmt.Errorf("expected group frame, got %T", f.Data))
}
return nil
}
func (r *GroupResultSetStreamReader) Close() {
r.fr.state = stateDone
}
type groupCursorStreamReader struct {
fr *frameReader
cur cursorReaders
tagKeys [][]byte
partitionKeyVals [][]byte
prevKey [][]byte
tags models.Tags
}
func (gc *groupCursorStreamReader) Err() error { return gc.fr.err }
func (gc *groupCursorStreamReader) Tags() models.Tags { return gc.tags }
func (gc *groupCursorStreamReader) Keys() [][]byte { return gc.tagKeys }
func (gc *groupCursorStreamReader) PartitionKeyVals() [][]byte { return gc.partitionKeyVals }
func (gc *groupCursorStreamReader) Cursor() cursors.Cursor { return gc.cur.cursor() }
func (gc *groupCursorStreamReader) Stats() cursors.CursorStats {
return gc.fr.stats.Stats()
}
func (gc *groupCursorStreamReader) Next() bool {
if gc.fr.state == stateReadSeries {
return gc.readSeriesFrame()
}
if gc.fr.state == stateDone || gc.fr.state == stateReadErr || gc.fr.state == stateReadGroup {
return false
}
gc.fr.setErr(fmt.Errorf("expected reader in state %v, was in state %v", stateReadSeries, gc.fr.state))
return false
}
func (gc *groupCursorStreamReader) readSeriesFrame() bool {
f := gc.fr.peekFrame()
if f == nil {
return false
}
if sf, ok := f.Data.(*datatypes.ReadResponse_Frame_Series); ok {
gc.fr.nextFrame()
gc.fr.state = stateReadPoints
if cap(gc.tags) < len(sf.Series.Tags) {
gc.tags = make(models.Tags, len(sf.Series.Tags))
} else {
gc.tags = gc.tags[:len(sf.Series.Tags)]
}
for i := range sf.Series.Tags {
gc.tags[i].Key = sf.Series.Tags[i].Key
gc.tags[i].Value = sf.Series.Tags[i].Value
}
gc.cur.nextType = sf.Series.DataType
return true
} else if _, ok := f.Data.(*datatypes.ReadResponse_Frame_Group); ok {
gc.fr.state = stateReadGroup
return false
}
gc.fr.setErr(fmt.Errorf("expected series frame, got %T", f.Data))
return false
}
func (gc *groupCursorStreamReader) Close() {
RETRY:
if gc.fr.state == stateReadPoints {
cur := gc.Cursor()
if cur != nil {
cur.Close()
}
}
if gc.fr.state == stateReadSeries {
gc.readSeriesFrame()
goto RETRY
}
}
type readState byte
const (
stateReadGroup readState = iota
stateReadSeries
stateReadPoints
stateReadFloatPoints
stateReadIntegerPoints
stateReadUnsignedPoints
stateReadBooleanPoints
stateReadStringPoints
stateReadErr
stateDone
)
type frameReader struct {
s StreamReader
stats statistics
state readState
buf []datatypes.ReadResponse_Frame
p int
err error
}
func (r *frameReader) init() {
if stats, ok := r.s.(statistics); ok {
r.stats = stats
} else {
r.stats = zeroStatistics
}
}
func (r *frameReader) peekFrame() *datatypes.ReadResponse_Frame {
retries := peekFrameRetries
RETRY:
if r.p < len(r.buf) {
f := &r.buf[r.p]
return f
}
r.p = 0
r.buf = nil
res, err := r.s.Recv()
if err == nil {
if res != nil {
r.buf = res.Frames
}
if retries > 0 {
retries--
goto RETRY
}
r.setErr(ErrStreamNoData)
} else if err == io.EOF {
r.state = stateDone
} else {
r.setErr(err)
}
return nil
}
func (r *frameReader) nextFrame() { r.p++ }
func (r *frameReader) setErr(err error) {
r.err = err
r.state = stateReadErr
}
type cursorReaders struct {
fr *frameReader
nextType datatypes.ReadResponse_DataType
cc cursors.Cursor
f floatCursorStreamReader
i integerCursorStreamReader
u unsignedCursorStreamReader
b booleanCursorStreamReader
s stringCursorStreamReader
}
func (cur *cursorReaders) setFrameReader(fr *frameReader) {
cur.fr = fr
cur.f.fr = fr
cur.i.fr = fr
cur.u.fr = fr
cur.b.fr = fr
cur.s.fr = fr
}
func (cur *cursorReaders) cursor() cursors.Cursor {
cur.cc = nil
if cur.fr.state != stateReadPoints {
cur.fr.setErr(fmt.Errorf("expected reader in state %v, was in state %v", stateReadPoints, cur.fr.state))
return cur.cc
}
switch cur.nextType {
case datatypes.DataTypeFloat:
cur.fr.state = stateReadFloatPoints
cur.cc = &cur.f
case datatypes.DataTypeInteger:
cur.fr.state = stateReadIntegerPoints
cur.cc = &cur.i
case datatypes.DataTypeUnsigned:
cur.fr.state = stateReadUnsignedPoints
cur.cc = &cur.u
case datatypes.DataTypeBoolean:
cur.fr.state = stateReadBooleanPoints
cur.cc = &cur.b
case datatypes.DataTypeString:
cur.fr.state = stateReadStringPoints
cur.cc = &cur.s
default:
cur.fr.setErr(fmt.Errorf("unexpected data type, %d", cur.nextType))
}
return cur.cc
}

View File

@ -0,0 +1,163 @@
// Generated by tmpl
// https://github.com/benbjohnson/tmpl
//
// DO NOT EDIT!
// Source: stream_reader_gen_test.go.tmpl
package reads_test
import (
"sort"
"github.com/influxdata/influxdb/storage/reads/datatypes"
)
type FloatPoints datatypes.ReadResponse_FloatPointsFrame
func (a FloatPoints) Len() int { return len(a.Timestamps) }
func (a FloatPoints) Less(i, j int) bool { return a.Timestamps[i] < a.Timestamps[j] }
func (a FloatPoints) Swap(i, j int) {
a.Timestamps[i], a.Timestamps[j] = a.Timestamps[j], a.Timestamps[i]
a.Values[i], a.Values[j] = a.Values[j], a.Values[i]
}
type floatS map[int64]float64
func floatF(points floatS) datatypes.ReadResponse_Frame {
var block FloatPoints
for t, v := range points {
block.Timestamps = append(block.Timestamps, t)
block.Values = append(block.Values, v)
}
sort.Sort(block)
pointsFrame := datatypes.ReadResponse_FloatPointsFrame(block)
return datatypes.ReadResponse_Frame{
Data: &datatypes.ReadResponse_Frame_FloatPoints{
FloatPoints: &pointsFrame,
},
}
}
type IntegerPoints datatypes.ReadResponse_IntegerPointsFrame
func (a IntegerPoints) Len() int { return len(a.Timestamps) }
func (a IntegerPoints) Less(i, j int) bool { return a.Timestamps[i] < a.Timestamps[j] }
func (a IntegerPoints) Swap(i, j int) {
a.Timestamps[i], a.Timestamps[j] = a.Timestamps[j], a.Timestamps[i]
a.Values[i], a.Values[j] = a.Values[j], a.Values[i]
}
type integerS map[int64]int64
func integerF(points integerS) datatypes.ReadResponse_Frame {
var block IntegerPoints
for t, v := range points {
block.Timestamps = append(block.Timestamps, t)
block.Values = append(block.Values, v)
}
sort.Sort(block)
pointsFrame := datatypes.ReadResponse_IntegerPointsFrame(block)
return datatypes.ReadResponse_Frame{
Data: &datatypes.ReadResponse_Frame_IntegerPoints{
IntegerPoints: &pointsFrame,
},
}
}
type UnsignedPoints datatypes.ReadResponse_UnsignedPointsFrame
func (a UnsignedPoints) Len() int { return len(a.Timestamps) }
func (a UnsignedPoints) Less(i, j int) bool { return a.Timestamps[i] < a.Timestamps[j] }
func (a UnsignedPoints) Swap(i, j int) {
a.Timestamps[i], a.Timestamps[j] = a.Timestamps[j], a.Timestamps[i]
a.Values[i], a.Values[j] = a.Values[j], a.Values[i]
}
type unsignedS map[int64]uint64
func unsignedF(points unsignedS) datatypes.ReadResponse_Frame {
var block UnsignedPoints
for t, v := range points {
block.Timestamps = append(block.Timestamps, t)
block.Values = append(block.Values, v)
}
sort.Sort(block)
pointsFrame := datatypes.ReadResponse_UnsignedPointsFrame(block)
return datatypes.ReadResponse_Frame{
Data: &datatypes.ReadResponse_Frame_UnsignedPoints{
UnsignedPoints: &pointsFrame,
},
}
}
type StringPoints datatypes.ReadResponse_StringPointsFrame
func (a StringPoints) Len() int { return len(a.Timestamps) }
func (a StringPoints) Less(i, j int) bool { return a.Timestamps[i] < a.Timestamps[j] }
func (a StringPoints) Swap(i, j int) {
a.Timestamps[i], a.Timestamps[j] = a.Timestamps[j], a.Timestamps[i]
a.Values[i], a.Values[j] = a.Values[j], a.Values[i]
}
type stringS map[int64]string
func stringF(points stringS) datatypes.ReadResponse_Frame {
var block StringPoints
for t, v := range points {
block.Timestamps = append(block.Timestamps, t)
block.Values = append(block.Values, v)
}
sort.Sort(block)
pointsFrame := datatypes.ReadResponse_StringPointsFrame(block)
return datatypes.ReadResponse_Frame{
Data: &datatypes.ReadResponse_Frame_StringPoints{
StringPoints: &pointsFrame,
},
}
}
type BooleanPoints datatypes.ReadResponse_BooleanPointsFrame
func (a BooleanPoints) Len() int { return len(a.Timestamps) }
func (a BooleanPoints) Less(i, j int) bool { return a.Timestamps[i] < a.Timestamps[j] }
func (a BooleanPoints) Swap(i, j int) {
a.Timestamps[i], a.Timestamps[j] = a.Timestamps[j], a.Timestamps[i]
a.Values[i], a.Values[j] = a.Values[j], a.Values[i]
}
type booleanS map[int64]bool
func booleanF(points booleanS) datatypes.ReadResponse_Frame {
var block BooleanPoints
for t, v := range points {
block.Timestamps = append(block.Timestamps, t)
block.Values = append(block.Values, v)
}
sort.Sort(block)
pointsFrame := datatypes.ReadResponse_BooleanPointsFrame(block)
return datatypes.ReadResponse_Frame{
Data: &datatypes.ReadResponse_Frame_BooleanPoints{
BooleanPoints: &pointsFrame,
},
}
}

View File

@ -0,0 +1,825 @@
package reads_test
import (
"bytes"
"errors"
"io"
"strings"
"testing"
"github.com/google/go-cmp/cmp"
"github.com/influxdata/influxdb/models"
"github.com/influxdata/influxdb/storage/reads"
"github.com/influxdata/influxdb/storage/reads/datatypes"
)
func errCmp(x, y error) bool {
if x == nil {
return y == nil
}
if y == nil {
return false
}
return x.Error() == y.Error()
}
func errTr(x error) string {
if x == nil {
return ""
}
return x.Error()
}
func TestNewResultSetStreamReader(t *testing.T) {
tests := []struct {
name string
stream *sliceStreamReader
exp string
expErr error
}{
{
name: "float series",
stream: newStreamReader(
response(
seriesF(Float, "cpu,tag0=val0"),
floatF(floatS{
0: 1.0,
1: 2.0,
2: 3.0,
}),
seriesF(Float, "cpu,tag0=val1"),
floatF(floatS{
10: 11.0,
11: 12.0,
12: 13.0,
}),
),
),
exp: `series: _m=cpu,tag0=val0
cursor:Float
0 | 1.00
1 | 2.00
2 | 3.00
series: _m=cpu,tag0=val1
cursor:Float
10 | 11.00
11 | 12.00
12 | 13.00
`,
},
{
name: "some empty series",
stream: newStreamReader(
response(
seriesF(Float, "cpu,tag0=val0"),
seriesF(Float, "cpu,tag0=val1"),
floatF(floatS{
10: 11.0,
11: 12.0,
12: 13.0,
}),
),
),
exp: `series: _m=cpu,tag0=val0
cursor:Float
series: _m=cpu,tag0=val1
cursor:Float
10 | 11.00
11 | 12.00
12 | 13.00
`,
},
{
name: "all data types",
stream: newStreamReader(
response(
seriesF(Boolean, "cpu,tag0=booleans"),
booleanF(booleanS{
3: false,
4: true,
5: true,
}),
seriesF(Float, "cpu,tag0=floats"),
floatF(floatS{
0: 1.0,
1: 2.0,
2: 3.0,
}),
seriesF(Integer, "cpu,tag0=integers"),
integerF(integerS{
1: 1,
2: 2,
3: 3,
}),
seriesF(String, "cpu,tag0=strings"),
stringF(stringS{
33: "thing 1",
34: "thing 2",
35: "things",
}),
seriesF(Unsigned, "cpu,tag0=unsigned"),
unsignedF(unsignedS{
2: 55,
3: 56,
4: 57,
}),
),
),
exp: `series: _m=cpu,tag0=booleans
cursor:Boolean
3 | false
4 | true
5 | true
series: _m=cpu,tag0=floats
cursor:Float
0 | 1.00
1 | 2.00
2 | 3.00
series: _m=cpu,tag0=integers
cursor:Integer
1 | 1
2 | 2
3 | 3
series: _m=cpu,tag0=strings
cursor:String
33 | thing 1
34 | thing 2
35 | things
series: _m=cpu,tag0=unsigned
cursor:Unsigned
2 | 55
3 | 56
4 | 57
`,
},
{
name: "invalid_points_no_series",
stream: newStreamReader(
response(
floatF(floatS{0: 1.0}),
),
),
expErr: errors.New("expected series frame, got *datatypes.ReadResponse_Frame_FloatPoints"),
},
{
name: "no points frames",
stream: newStreamReader(
response(
seriesF(Boolean, "cpu,tag0=booleans"),
seriesF(Float, "cpu,tag0=floats"),
seriesF(Integer, "cpu,tag0=integers"),
seriesF(String, "cpu,tag0=strings"),
seriesF(Unsigned, "cpu,tag0=unsigned"),
),
),
exp: `series: _m=cpu,tag0=booleans
cursor:Boolean
series: _m=cpu,tag0=floats
cursor:Float
series: _m=cpu,tag0=integers
cursor:Integer
series: _m=cpu,tag0=strings
cursor:String
series: _m=cpu,tag0=unsigned
cursor:Unsigned
`,
},
{
name: "invalid_group_frame",
stream: newStreamReader(
response(
groupF("tag0", "val0"),
floatF(floatS{0: 1.0}),
),
),
expErr: errors.New("expected series frame, got *datatypes.ReadResponse_Frame_Group"),
},
{
name: "invalid_multiple_data_types",
stream: newStreamReader(
response(
seriesF(Float, "cpu,tag0=val0"),
floatF(floatS{0: 1.0}),
integerF(integerS{0: 1}),
),
),
exp: `series: _m=cpu,tag0=val0
cursor:Float
0 | 1.00
cursor err: floatCursorStreamReader: unexpected frame type *datatypes.ReadResponse_Frame_IntegerPoints
`,
expErr: errors.New("floatCursorStreamReader: unexpected frame type *datatypes.ReadResponse_Frame_IntegerPoints"),
},
{
name: "invalid series key order",
stream: newStreamReader(
response(
seriesF(Float, "cpu,tag0=val1"),
seriesF(Float, "cpu,tag0=val0"),
),
),
exp: `series: _m=cpu,tag0=val1
cursor:Float
`,
expErr: reads.ErrSeriesKeyOrder,
},
{
name: "some empty frames",
stream: newStreamReader(
response(
seriesF(Float, "cpu,tag0=val0"),
),
response(
floatF(floatS{
0: 1.0,
1: 2.0,
2: 3.0,
}),
),
response(),
response(
seriesF(Float, "cpu,tag0=val1"),
),
response(),
response(
floatF(floatS{
10: 11.0,
11: 12.0,
12: 13.0,
}),
),
response(),
),
exp: `series: _m=cpu,tag0=val0
cursor:Float
0 | 1.00
1 | 2.00
2 | 3.00
series: _m=cpu,tag0=val1
cursor:Float
10 | 11.00
11 | 12.00
12 | 13.00
`,
},
{
name: "last frame empty",
stream: newStreamReader(
response(
seriesF(Float, "cpu,tag0=val0"),
floatF(floatS{
0: 1.0,
1: 2.0,
2: 3.0,
}),
),
response(),
),
exp: `series: _m=cpu,tag0=val0
cursor:Float
0 | 1.00
1 | 2.00
2 | 3.00
`,
},
{
name: "ErrUnexpectedEOF",
stream: newStreamReader(
response(
seriesF(Float, "cpu,tag0=val0"),
),
response(
floatF(floatS{
0: 1.0,
1: 2.0,
2: 3.0,
}),
),
response(),
response(),
response(),
),
exp: `series: _m=cpu,tag0=val0
cursor:Float
0 | 1.00
1 | 2.00
2 | 3.00
cursor err: peekFrame: no data
`,
expErr: reads.ErrStreamNoData,
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
rs := reads.NewResultSetStreamReader(tt.stream)
sb := new(strings.Builder)
ResultSetToString(sb, rs)
if got := sb.String(); !cmp.Equal(got, tt.exp) {
t.Errorf("unexpected value; -got/+exp\n%s", cmp.Diff(got, tt.exp))
}
if got := rs.Err(); !cmp.Equal(got, tt.expErr, cmp.Comparer(errCmp)) {
t.Errorf("unexpected error; -got/+exp\n%s", cmp.Diff(got, tt.expErr, cmp.Transformer("err", errTr)))
}
})
}
}
func TestNewResultSetStreamReader_SkipSeriesCursors(t *testing.T) {
stream := newStreamReader(
response(
seriesF(Float, "cpu,tag0=floats"),
floatF(floatS{0: 1.0}),
seriesF(Integer, "cpu,tag0=integers"),
integerF(integerS{1: 1}),
seriesF(Unsigned, "cpu,tag0=unsigned"),
unsignedF(unsignedS{2: 55}),
),
)
expSeries := []string{"_m=cpu,tag0=floats", "_m=cpu,tag0=integers", "_m=cpu,tag0=unsigned"}
rs := reads.NewResultSetStreamReader(stream)
for i := 0; i < 3; i++ {
if got := rs.Next(); !cmp.Equal(got, true) {
t.Errorf("expected true")
}
sb := new(strings.Builder)
TagsToString(sb, rs.Tags())
if got := strings.TrimSpace(sb.String()); !cmp.Equal(got, expSeries[i]) {
t.Errorf("unexpected tags; -got/+exp\n%s", cmp.Diff(got, expSeries[i]))
}
cur := rs.Cursor()
if cur == nil {
t.Errorf("expected cursor")
}
cur.Close()
}
if got := rs.Next(); !cmp.Equal(got, false) {
t.Errorf("expected false")
}
rs.Close()
}
func TestNewGroupResultSetStreamReader(t *testing.T) {
tests := []struct {
name string
stream *sliceStreamReader
exp string
expErr error
}{
{
name: "groups none no series no points",
stream: newStreamReader(
response(
groupF("tag0,tag1", ""),
),
),
exp: `group:
tag key : tag0,tag1
partition key:
`,
},
{
name: "groups none series no points",
stream: newStreamReader(
response(
groupF("_m,tag0", ""),
seriesF(Float, "cpu,tag0=floats"),
seriesF(Integer, "cpu,tag0=integers"),
seriesF(Unsigned, "cpu,tag0=unsigned"),
),
),
exp: `group:
tag key : _m,tag0
partition key:
series: _m=cpu,tag0=floats
cursor:Float
series: _m=cpu,tag0=integers
cursor:Integer
series: _m=cpu,tag0=unsigned
cursor:Unsigned
`,
},
{
name: "groups none series points",
stream: newStreamReader(
response(
groupF("_m,tag0", ""),
seriesF(Float, "cpu,tag0=floats"),
floatF(floatS{0: 0.0, 1: 1.0, 2: 2.0}),
seriesF(Integer, "cpu,tag0=integers"),
integerF(integerS{10: 10, 20: 20, 30: 30}),
seriesF(Unsigned, "cpu,tag0=unsigned"),
unsignedF(unsignedS{100: 100, 200: 200, 300: 300}),
),
),
exp: `group:
tag key : _m,tag0
partition key:
series: _m=cpu,tag0=floats
cursor:Float
0 | 0.00
1 | 1.00
2 | 2.00
series: _m=cpu,tag0=integers
cursor:Integer
10 | 10
20 | 20
30 | 30
series: _m=cpu,tag0=unsigned
cursor:Unsigned
100 | 100
200 | 200
300 | 300
`,
},
{
name: "groups by no series no points",
stream: newStreamReader(
response(
groupF("tag00,tag10", "val00,val10"),
groupF("tag00,tag10", "val00,val11"),
groupF("tag00,tag10", "val01,val10"),
groupF("tag00,tag10", "val01,val11"),
),
),
exp: `group:
tag key : tag00,tag10
partition key: val00,val10
group:
tag key : tag00,tag10
partition key: val00,val11
group:
tag key : tag00,tag10
partition key: val01,val10
group:
tag key : tag00,tag10
partition key: val01,val11
`,
},
{
name: "groups by series no points",
stream: newStreamReader(
response(
groupF("_m,tag0", "cpu,val0"),
seriesF(Float, "cpu,tag0=val0"),
seriesF(Float, "cpu,tag0=val0,tag1=val0"),
groupF("_m,tag0", "cpu,val1"),
seriesF(Float, "cpu,tag0=val1"),
seriesF(Float, "cpu,tag0=val1,tag1=val0"),
),
),
exp: `group:
tag key : _m,tag0
partition key: cpu,val0
series: _m=cpu,tag0=val0
cursor:Float
series: _m=cpu,tag0=val0,tag1=val0
cursor:Float
group:
tag key : _m,tag0
partition key: cpu,val1
series: _m=cpu,tag0=val1
cursor:Float
series: _m=cpu,tag0=val1,tag1=val0
cursor:Float
`,
},
{
name: "missing group frame",
stream: newStreamReader(
response(
seriesF(Float, "cpu,tag0=val0"),
),
),
expErr: errors.New("expected group frame, got *datatypes.ReadResponse_Frame_Series"),
},
{
name: "incorrect points frame data type",
stream: newStreamReader(
response(
groupF("_m,tag0", "cpu,val0"),
seriesF(Float, "cpu,tag0=val0"),
integerF(integerS{0: 1}),
),
),
exp: `group:
tag key : _m,tag0
partition key: cpu,val0
series: _m=cpu,tag0=val0
cursor:Float
cursor err: floatCursorStreamReader: unexpected frame type *datatypes.ReadResponse_Frame_IntegerPoints
`,
expErr: errors.New("floatCursorStreamReader: unexpected frame type *datatypes.ReadResponse_Frame_IntegerPoints"),
},
{
name: "partition key order",
stream: newStreamReader(
response(
groupF("_m,tag0", "cpu,val1"),
groupF("_m,tag0", "cpu,val0"),
),
),
exp: `group:
tag key : _m,tag0
partition key: cpu,val1
`,
expErr: reads.ErrPartitionKeyOrder,
},
{
name: "partition key order",
stream: newStreamReader(
response(
groupF("_m", "cpu,"),
groupF("_m,tag0", "cpu,val0"),
),
),
exp: `group:
tag key : _m
partition key: cpu,<nil>
`,
expErr: reads.ErrPartitionKeyOrder,
},
{
name: "partition key order",
stream: newStreamReader(
response(
groupF("_m,tag0", ",val0"),
groupF("_m,tag0", "cpu,val0"),
),
),
exp: `group:
tag key : _m,tag0
partition key: <nil>,val0
`,
expErr: reads.ErrPartitionKeyOrder,
},
{
name: "partition key order",
stream: newStreamReader(
response(
groupF("_m,tag0", "cpu,val0"),
groupF("_m,tag0", "cpu,val1"),
groupF("_m,tag0", ","),
groupF("_m,tag0", ",val0"),
),
),
exp: `group:
tag key : _m,tag0
partition key: cpu,val0
group:
tag key : _m,tag0
partition key: cpu,val1
group:
tag key : _m,tag0
partition key: <nil>,<nil>
`,
expErr: reads.ErrPartitionKeyOrder,
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
rs := reads.NewGroupResultSetStreamReader(tt.stream)
sb := new(strings.Builder)
GroupResultSetToString(sb, rs)
if got := sb.String(); !cmp.Equal(got, tt.exp) {
t.Errorf("unexpected value; -got/+exp\n%s", cmp.Diff(got, tt.exp))
}
if got := rs.Err(); !cmp.Equal(got, tt.expErr, cmp.Comparer(errCmp)) {
t.Errorf("unexpected error; -got/+exp\n%s", cmp.Diff(got, tt.expErr, cmp.Transformer("err", errTr)))
}
})
}
}
func joinB(b [][]byte) string {
return string(bytes.Join(b, []byte(",")))
}
func TestNewGroupResultSetStreamReader_SkipGroupCursors(t *testing.T) {
stream := newStreamReader(
response(
groupF("_m,tag0", "cpu,val0"),
seriesF(Float, "cpu,tag0=val0"),
floatF(floatS{0: 1.0}),
groupF("_m,tag0", "cpu,val1"),
seriesF(Integer, "cpu,tag0=val1,tag1=val0"),
integerF(integerS{1: 1}),
seriesF(Integer, "cpu,tag0=val1,tag1=val1"),
unsignedF(unsignedS{2: 55}),
),
)
type expGroup struct {
tagKeys string
parKeys string
series []string
}
t.Run("skip series cursors", func(t *testing.T) {
exp := []expGroup{
{
tagKeys: "_m,tag0",
parKeys: "cpu,val0",
series: []string{"_m=cpu,tag0=val0"},
},
{
tagKeys: "_m,tag0",
parKeys: "cpu,val1",
series: []string{"_m=cpu,tag0=val1,tag1=val0", "_m=cpu,tag0=val1,tag1=val1"},
},
}
stream.reset()
grs := reads.NewGroupResultSetStreamReader(stream)
for i := range exp {
rs := grs.Next()
if rs == nil {
t.Errorf("expected group cursor")
}
if got := joinB(rs.Keys()); !cmp.Equal(got, exp[i].tagKeys) {
t.Errorf("unexpected group keys; -got/+exp\n%s", cmp.Diff(got, exp[i].tagKeys))
}
if got := joinB(rs.PartitionKeyVals()); !cmp.Equal(got, exp[i].parKeys) {
t.Errorf("unexpected group keys; -got/+exp\n%s", cmp.Diff(got, exp[i].parKeys))
}
for j := range exp[i].series {
if got := rs.Next(); !cmp.Equal(got, true) {
t.Errorf("expected true")
}
sb := new(strings.Builder)
TagsToString(sb, rs.Tags())
if got := strings.TrimSpace(sb.String()); !cmp.Equal(got, exp[i].series[j]) {
t.Errorf("unexpected tags; -got/+exp\n%s", cmp.Diff(got, exp[i].series[j]))
}
cur := rs.Cursor()
if cur == nil {
t.Errorf("expected cursor")
}
cur.Close()
}
if got := rs.Next(); !cmp.Equal(got, false) {
t.Errorf("expected false")
}
rs.Close()
}
if rs := grs.Next(); rs != nil {
t.Errorf("unexpected group cursor")
}
grs.Close()
})
t.Run("skip series", func(t *testing.T) {
exp := []expGroup{
{
tagKeys: "_m,tag0",
parKeys: "cpu,val0",
},
{
tagKeys: "_m,tag0",
parKeys: "cpu,val1",
},
}
stream.reset()
grs := reads.NewGroupResultSetStreamReader(stream)
for i := range exp {
rs := grs.Next()
if rs == nil {
t.Errorf("expected group cursor")
}
if got := joinB(rs.Keys()); !cmp.Equal(got, exp[i].tagKeys) {
t.Errorf("unexpected group keys; -got/+exp\n%s", cmp.Diff(got, exp[i].tagKeys))
}
if got := joinB(rs.PartitionKeyVals()); !cmp.Equal(got, exp[i].parKeys) {
t.Errorf("unexpected group keys; -got/+exp\n%s", cmp.Diff(got, exp[i].parKeys))
}
rs.Close()
}
if rs := grs.Next(); rs != nil {
t.Errorf("unexpected group cursor")
}
grs.Close()
})
}
func response(f ...datatypes.ReadResponse_Frame) datatypes.ReadResponse {
return datatypes.ReadResponse{Frames: f}
}
type sliceStreamReader struct {
res []datatypes.ReadResponse
p int
}
func newStreamReader(res ...datatypes.ReadResponse) *sliceStreamReader {
return &sliceStreamReader{res: res}
}
func (r *sliceStreamReader) reset() { r.p = 0 }
func (r *sliceStreamReader) Recv() (*datatypes.ReadResponse, error) {
if r.p < len(r.res) {
res := &r.res[r.p]
r.p++
return res, nil
}
return nil, io.EOF
}
func (r *sliceStreamReader) String() string {
return ""
}
// errStreamReader is a reads.StreamReader that always returns an error.
type errStreamReader string
func (e errStreamReader) Recv() (*datatypes.ReadResponse, error) {
return nil, errors.New(string(e))
}
// emptyStreamReader is a reads.StreamReader that returns no data.
type emptyStreamReader struct{}
func (s *emptyStreamReader) Recv() (*datatypes.ReadResponse, error) {
return nil, nil
}
func groupF(tagKeys string, partitionKeyVals string) datatypes.ReadResponse_Frame {
var pk [][]byte
if partitionKeyVals != "" {
pk = bytes.Split([]byte(partitionKeyVals), []byte(","))
for i := range pk {
if bytes.Equal(pk[i], nilValBytes) {
pk[i] = nil
}
}
}
return datatypes.ReadResponse_Frame{
Data: &datatypes.ReadResponse_Frame_Group{
Group: &datatypes.ReadResponse_GroupFrame{
TagKeys: bytes.Split([]byte(tagKeys), []byte(",")),
PartitionKeyVals: pk,
},
},
}
}
const (
Float = datatypes.DataTypeFloat
Integer = datatypes.DataTypeInteger
Unsigned = datatypes.DataTypeUnsigned
Boolean = datatypes.DataTypeBoolean
String = datatypes.DataTypeString
)
func seriesF(dt datatypes.ReadResponse_DataType, measurement string) datatypes.ReadResponse_Frame {
name, tags := models.ParseKeyBytes([]byte(measurement))
tags.Set([]byte("_m"), name)
t := make([]datatypes.Tag, len(tags))
for i, tag := range tags {
t[i].Key = tag.Key
t[i].Value = tag.Value
}
return datatypes.ReadResponse_Frame{
Data: &datatypes.ReadResponse_Frame_Series{
Series: &datatypes.ReadResponse_SeriesFrame{
DataType: dt,
Tags: t,
},
},
}
}

1194
storage/reads/table.gen.go Normal file

File diff suppressed because it is too large Load Diff

294
storage/reads/table.go Normal file
View File

@ -0,0 +1,294 @@
package reads
//go:generate env GO111MODULE=on go run github.com/benbjohnson/tmpl -data=@types.tmpldata table.gen.go.tmpl
import (
"fmt"
"sync/atomic"
"github.com/apache/arrow/go/arrow/array"
"github.com/influxdata/flux"
"github.com/influxdata/flux/arrow"
"github.com/influxdata/flux/execute"
"github.com/influxdata/flux/memory"
"github.com/influxdata/influxdb/models"
"github.com/influxdata/influxdb/tsdb/cursors"
)
type table struct {
bounds execute.Bounds
key flux.GroupKey
cols []flux.ColMeta
// cache of the tags on the current series.
// len(tags) == len(colMeta)
tags [][]byte
defs [][]byte
done chan struct{}
// The current number of records in memory
l int
colBufs []array.Interface
timeBuf []int64
err error
cancelled int32
alloc *memory.Allocator
}
func newTable(
done chan struct{},
bounds execute.Bounds,
key flux.GroupKey,
cols []flux.ColMeta,
defs [][]byte,
alloc *memory.Allocator,
) table {
return table{
done: done,
bounds: bounds,
key: key,
tags: make([][]byte, len(cols)),
defs: defs,
colBufs: make([]array.Interface, len(cols)),
cols: cols,
alloc: alloc,
}
}
func (t *table) Key() flux.GroupKey { return t.key }
func (t *table) Cols() []flux.ColMeta { return t.cols }
func (t *table) RefCount(n int) {}
func (t *table) Err() error { return t.err }
func (t *table) Empty() bool { return t.l == 0 }
func (t *table) Len() int { return t.l }
func (t *table) Cancel() {
atomic.StoreInt32(&t.cancelled, 1)
}
func (t *table) isCancelled() bool {
return atomic.LoadInt32(&t.cancelled) != 0
}
func (t *table) Bools(j int) *array.Boolean {
execute.CheckColType(t.cols[j], flux.TBool)
return t.colBufs[j].(*array.Boolean)
}
func (t *table) Ints(j int) *array.Int64 {
execute.CheckColType(t.cols[j], flux.TInt)
return t.colBufs[j].(*array.Int64)
}
func (t *table) UInts(j int) *array.Uint64 {
execute.CheckColType(t.cols[j], flux.TUInt)
return t.colBufs[j].(*array.Uint64)
}
func (t *table) Floats(j int) *array.Float64 {
execute.CheckColType(t.cols[j], flux.TFloat)
return t.colBufs[j].(*array.Float64)
}
func (t *table) Strings(j int) *array.Binary {
execute.CheckColType(t.cols[j], flux.TString)
return t.colBufs[j].(*array.Binary)
}
func (t *table) Times(j int) *array.Int64 {
execute.CheckColType(t.cols[j], flux.TTime)
return t.colBufs[j].(*array.Int64)
}
// readTags populates b.tags with the provided tags
func (t *table) readTags(tags models.Tags) {
for j := range t.tags {
t.tags[j] = t.defs[j]
}
if len(tags) == 0 {
return
}
for _, tag := range tags {
j := execute.ColIdx(string(tag.Key), t.cols)
t.tags[j] = tag.Value
}
}
// appendTags fills the colBufs for the tag columns with the tag value.
func (t *table) appendTags() {
for j := range t.cols {
v := t.tags[j]
if v != nil {
b := arrow.NewStringBuilder(t.alloc)
b.Reserve(t.l)
for i := 0; i < t.l; i++ {
b.Append(v)
}
t.colBufs[j] = b.NewArray()
b.Release()
}
}
}
// appendBounds fills the colBufs for the time bounds
func (t *table) appendBounds() {
bounds := []execute.Time{t.bounds.Start, t.bounds.Stop}
for j := range []int{startColIdx, stopColIdx} {
b := arrow.NewIntBuilder(t.alloc)
b.Reserve(t.l)
for i := 0; i < t.l; i++ {
b.UnsafeAppend(int64(bounds[j]))
}
t.colBufs[j] = b.NewArray()
b.Release()
}
}
func (t *table) closeDone() {
if t.done != nil {
close(t.done)
t.done = nil
}
}
// hasPoints returns true if the next block from cur has data. If cur is not
// nil, it will be closed.
func hasPoints(cur cursors.Cursor) bool {
if cur == nil {
return false
}
// TODO(sgc): this is a temporary fix to identify a remote cursor
// which will not stream points causing hasPoints to return false.
// This is the cause of https://github.com/influxdata/idpe/issues/2774
if _, ok := cur.(streamCursor); ok {
cur.Close()
return true
}
res := false
switch cur := cur.(type) {
case cursors.IntegerArrayCursor:
a := cur.Next()
res = a.Len() > 0
case cursors.FloatArrayCursor:
a := cur.Next()
res = a.Len() > 0
case cursors.UnsignedArrayCursor:
a := cur.Next()
res = a.Len() > 0
case cursors.BooleanArrayCursor:
a := cur.Next()
res = a.Len() > 0
case cursors.StringArrayCursor:
a := cur.Next()
res = a.Len() > 0
default:
panic(fmt.Sprintf("unreachable: %T", cur))
}
cur.Close()
return res
}
type tableNoPoints struct {
table
}
func newTableNoPoints(
done chan struct{},
bounds execute.Bounds,
key flux.GroupKey,
cols []flux.ColMeta,
tags models.Tags,
defs [][]byte,
alloc *memory.Allocator,
) *tableNoPoints {
t := &tableNoPoints{
table: newTable(done, bounds, key, cols, defs, alloc),
}
t.readTags(tags)
return t
}
func (t *tableNoPoints) Close() {}
func (t *tableNoPoints) Statistics() cursors.CursorStats { return cursors.CursorStats{} }
func (t *tableNoPoints) Do(f func(flux.ColReader) error) error {
if t.isCancelled() {
return nil
}
t.err = f(t)
t.closeDone()
return t.err
}
type groupTableNoPoints struct {
table
}
func newGroupTableNoPoints(
done chan struct{},
bounds execute.Bounds,
key flux.GroupKey,
cols []flux.ColMeta,
defs [][]byte,
alloc *memory.Allocator,
) *groupTableNoPoints {
t := &groupTableNoPoints{
table: newTable(done, bounds, key, cols, defs, alloc),
}
return t
}
func (t *groupTableNoPoints) Close() {}
func (t *groupTableNoPoints) Do(f func(flux.ColReader) error) error {
if t.isCancelled() {
return nil
}
t.err = f(t)
t.closeDone()
return t.err
}
func (t *groupTableNoPoints) Statistics() cursors.CursorStats { return cursors.CursorStats{} }
func (t *floatTable) toArrowBuffer(vs []float64) *array.Float64 {
return arrow.NewFloat(vs, t.alloc)
}
func (t *floatGroupTable) toArrowBuffer(vs []float64) *array.Float64 {
return arrow.NewFloat(vs, t.alloc)
}
func (t *integerTable) toArrowBuffer(vs []int64) *array.Int64 {
return arrow.NewInt(vs, t.alloc)
}
func (t *integerGroupTable) toArrowBuffer(vs []int64) *array.Int64 {
return arrow.NewInt(vs, t.alloc)
}
func (t *unsignedTable) toArrowBuffer(vs []uint64) *array.Uint64 {
return arrow.NewUint(vs, t.alloc)
}
func (t *unsignedGroupTable) toArrowBuffer(vs []uint64) *array.Uint64 {
return arrow.NewUint(vs, t.alloc)
}
func (t *stringTable) toArrowBuffer(vs []string) *array.Binary {
return arrow.NewString(vs, t.alloc)
}
func (t *stringGroupTable) toArrowBuffer(vs []string) *array.Binary {
return arrow.NewString(vs, t.alloc)
}
func (t *booleanTable) toArrowBuffer(vs []bool) *array.Boolean {
return arrow.NewBool(vs, t.alloc)
}
func (t *booleanGroupTable) toArrowBuffer(vs []bool) *array.Boolean {
return arrow.NewBool(vs, t.alloc)
}

View File

@ -0,0 +1,30 @@
package reads
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
}

View File

@ -4,7 +4,7 @@ import (
"context"
"github.com/influxdata/influxdb/query"
"github.com/influxdata/platform/tsdb/cursors"
"github.com/influxdata/influxdb/tsdb/cursors"
)
type (

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,208 @@
package cursors
{{range .}}
{{ $typename := print .Name "Array" }}
type {{ $typename }} struct {
Timestamps []int64
Values []{{.Type}}
}
func New{{$typename}}Len(sz int) *{{$typename}} {
return &{{$typename}}{
Timestamps: make([]int64, sz),
Values: make([]{{.Type}}, sz),
}
}
func (a *{{ $typename }}) MinTime() int64 {
return a.Timestamps[0]
}
func (a *{{ $typename }}) MaxTime() int64 {
return a.Timestamps[len(a.Timestamps)-1]
}
func (a *{{ $typename }}) Size() int {
panic("not implemented")
}
func (a *{{ $typename}}) Len() int {
return len(a.Timestamps)
}
// Exclude removes the subset of values in [min, max]. The values must
// be deduplicated and sorted before calling Exclude or the results are undefined.
func (a *{{ $typename }}) Exclude(min, max int64) {
rmin, rmax := a.FindRange(min, max)
if rmin == -1 && rmax == -1 {
return
}
// a.Timestamps[rmin] ≥ min
// a.Timestamps[rmax] ≥ max
if rmax < a.Len() {
if a.Timestamps[rmax] == max {
rmax++
}
rest := a.Len()-rmax
if rest > 0 {
ts := a.Timestamps[:rmin+rest]
copy(ts[rmin:], a.Timestamps[rmax:])
a.Timestamps = ts
vs := a.Values[:rmin+rest]
copy(vs[rmin:], a.Values[rmax:])
a.Values = vs
return
}
}
a.Timestamps = a.Timestamps[:rmin]
a.Values = a.Values[:rmin]
}
// Include returns the subset values between min and max inclusive. The values must
// be deduplicated and sorted before calling Include or the results are undefined.
func (a *{{ $typename }}) Include(min, max int64) {
rmin, rmax := a.FindRange(min, max)
if rmin == -1 && rmax == -1 {
a.Timestamps = a.Timestamps[:0]
a.Values = a.Values[:0]
return
}
// a.Timestamps[rmin] ≥ min
// a.Timestamps[rmax] ≥ max
if rmax < a.Len() && a.Timestamps[rmax] == max {
rmax++
}
if rmin > -1 {
ts := a.Timestamps[:rmax-rmin]
copy(ts, a.Timestamps[rmin:rmax])
a.Timestamps = ts
vs := a.Values[:rmax-rmin]
copy(vs, a.Values[rmin:rmax])
a.Values = vs
} else {
a.Timestamps = a.Timestamps[:rmax]
a.Values = a.Values[:rmax]
}
}
// search performs a binary search for UnixNano() v in a
// and returns the position, i, where v would be inserted.
// An additional check of a.Timestamps[i] == v is necessary
// to determine if the value v exists.
func (a *{{ $typename }}) search(v int64) int {
// Define: f(x) → a.Timestamps[x] < v
// Define: f(-1) == true, f(n) == false
// Invariant: f(lo-1) == true, f(hi) == false
lo := 0
hi := a.Len()
for lo < hi {
mid := int(uint(lo+hi) >> 1)
if a.Timestamps[mid] < v {
lo = mid + 1 // preserves f(lo-1) == true
} else {
hi = mid // preserves f(hi) == false
}
}
// lo == hi
return lo
}
// FindRange returns the positions where min and max would be
// inserted into the array. If a[0].UnixNano() > max or
// a[len-1].UnixNano() < min then FindRange returns (-1, -1)
// indicating the array is outside the [min, max]. The values must
// be deduplicated and sorted before calling Exclude or the results
// are undefined.
func (a *{{ $typename }}) FindRange(min, max int64) (int, int) {
if a.Len() == 0 || min > max {
return -1, -1
}
minVal := a.MinTime()
maxVal := a.MaxTime()
if maxVal < min || minVal > max {
return -1, -1
}
return a.search(min), a.search(max)
}
// Merge overlays b to top of a. If two values conflict with
// the same timestamp, b is used. Both a and b must be sorted
// in ascending order.
func (a *{{ $typename }}) Merge(b *{{ $typename }}) {
if a.Len() == 0 {
*a = *b
return
}
if b.Len() == 0 {
return
}
// Normally, both a and b should not contain duplicates. Due to a bug in older versions, it's
// possible stored blocks might contain duplicate values. Remove them if they exists before
// merging.
// a = a.Deduplicate()
// b = b.Deduplicate()
if a.MaxTime() < b.MinTime() {
a.Timestamps = append(a.Timestamps, b.Timestamps...)
a.Values = append(a.Values, b.Values...)
return
}
if b.MaxTime() < a.MinTime() {
var tmp {{$typename}}
tmp.Timestamps = append(b.Timestamps, a.Timestamps...)
tmp.Values = append(b.Values, a.Values...)
*a = tmp
return
}
out := New{{$typename}}Len(a.Len()+b.Len())
i, j, k := 0, 0, 0
for i < len(a.Timestamps) && j < len(b.Timestamps) {
if a.Timestamps[i] < b.Timestamps[j] {
out.Timestamps[k] = a.Timestamps[i]
out.Values[k] = a.Values[i]
i++
} else if a.Timestamps[i] == b.Timestamps[j] {
out.Timestamps[k] = b.Timestamps[j]
out.Values[k] = b.Values[j]
i++
j++
} else {
out.Timestamps[k] = b.Timestamps[j]
out.Values[k] = b.Values[j]
j++
}
k++
}
if i < len(a.Timestamps) {
n := copy(out.Timestamps[k:], a.Timestamps[i:])
copy(out.Values[k:], a.Values[i:])
k += n
} else if j < len(b.Timestamps) {
n := copy(out.Timestamps[k:], b.Timestamps[j:])
copy(out.Values[k:], b.Values[j:])
k += n
}
a.Timestamps = out.Timestamps[:k]
a.Values = out.Values[:k]
}
{{ end }}

View File

@ -0,0 +1,27 @@
[
{
"Name":"Float",
"name":"float",
"Type":"float64"
},
{
"Name":"Integer",
"name":"integer",
"Type":"int64"
},
{
"Name":"Unsigned",
"name":"unsigned",
"Type":"uint64"
},
{
"Name":"String",
"name":"string",
"Type":"string"
},
{
"Name":"Boolean",
"name":"boolean",
"Type":"bool"
}
]

View File

@ -0,0 +1,210 @@
package cursors
import (
"fmt"
"testing"
"github.com/google/go-cmp/cmp"
)
func makeIntegerArray(count int, min, max int64) *IntegerArray {
vals := NewIntegerArrayLen(count)
ts := min
inc := (max - min) / int64(count)
for i := 0; i < count; i++ {
vals.Timestamps[i] = ts
ts += inc
}
return vals
}
func makeIntegerArrayFromSlice(t []int64) *IntegerArray {
iv := NewIntegerArrayLen(len(t))
copy(iv.Timestamps, t)
return iv
}
func TestIntegerArray_FindRangeNoValues(t *testing.T) {
var vals IntegerArray
l, r := vals.FindRange(0, 100)
if exp := -1; l != exp {
t.Errorf("invalid l; exp=%d, got=%d", exp, l)
}
if exp := -1; r != exp {
t.Errorf("invalid r; exp=%d, got=%d", exp, r)
}
}
func TestIntegerArray_FindRange(t *testing.T) {
vals := makeIntegerArrayFromSlice([]int64{10, 11, 13, 15, 17, 20, 21})
cases := []struct {
min, max int64
l, r int
}{
{12, 20, 2, 5},
{22, 40, -1, -1},
{1, 9, -1, -1},
{1, 10, 0, 0},
{1, 11, 0, 1},
{15, 15, 3, 3},
}
for _, tc := range cases {
t.Run(fmt.Sprintf("%d→%d", tc.min, tc.max), func(t *testing.T) {
l, r := vals.FindRange(tc.min, tc.max)
if l != tc.l {
t.Errorf("left: got %d, exp %d", l, tc.l)
}
if r != tc.r {
t.Errorf("right: got %d, exp %d", r, tc.r)
}
})
}
}
func TestIntegerArray_Exclude(t *testing.T) {
cases := []struct {
n string
min, max int64
exp []int64
}{
{"excl bad range", 18, 11, []int64{10, 12, 14, 16, 18}},
{"excl none-lo", 0, 9, []int64{10, 12, 14, 16, 18}},
{"excl none-hi", 19, 30, []int64{10, 12, 14, 16, 18}},
{"excl first", 0, 10, []int64{12, 14, 16, 18}},
{"excl last", 18, 20, []int64{10, 12, 14, 16}},
{"excl all but first and last", 12, 16, []int64{10, 18}},
{"excl none in middle", 13, 13, []int64{10, 12, 14, 16, 18}},
{"excl middle", 14, 14, []int64{10, 12, 16, 18}},
{"excl suffix", 14, 18, []int64{10, 12}},
}
for _, tc := range cases {
t.Run(fmt.Sprintf("%s[%d,%d]", tc.n, tc.min, tc.max), func(t *testing.T) {
vals := makeIntegerArray(5, 10, 20)
vals.Exclude(tc.min, tc.max)
got := vals.Timestamps
if !cmp.Equal(got, tc.exp) {
t.Errorf("unexpected values -got/+exp\n%s", cmp.Diff(got, tc.exp))
}
})
}
}
func TestIntegerArray_Include(t *testing.T) {
cases := []struct {
n string
min, max int64
exp []int64
}{
{"incl none-lo", 0, 9, []int64{}},
{"incl none-hi", 19, 30, []int64{}},
{"incl first", 0, 10, []int64{10}},
{"incl last", 18, 20, []int64{18}},
{"incl all but first and last", 12, 16, []int64{12, 14, 16}},
{"incl none in middle", 13, 13, []int64{}},
{"incl middle", 14, 14, []int64{14}},
}
for _, tc := range cases {
t.Run(fmt.Sprintf("%s[%d,%d]", tc.n, tc.min, tc.max), func(t *testing.T) {
vals := makeIntegerArray(5, 10, 20)
vals.Include(tc.min, tc.max)
got := vals.Timestamps
if !cmp.Equal(got, tc.exp) {
t.Errorf("unexpected values -got/+exp\n%s", cmp.Diff(got, tc.exp))
}
})
}
}
func benchExclude(b *testing.B, vals *IntegerArray, min, max int64) {
b.ResetTimer()
for i := 0; i < b.N; i++ {
vals.Exclude(min, max)
}
}
func BenchmarkIntegerArray_ExcludeNone_1000(b *testing.B) {
benchExclude(b, makeIntegerArray(1000, 1000, 2000), 0, 500)
}
func BenchmarkIntegerArray_ExcludeMiddleHalf_1000(b *testing.B) {
benchExclude(b, makeIntegerArray(1000, 1000, 2000), 1250, 1750)
}
func BenchmarkIntegerArray_ExcludeFirst_1000(b *testing.B) {
benchExclude(b, makeIntegerArray(1000, 1000, 2000), 0, 1000)
}
func BenchmarkIntegerArray_ExcludeLast_1000(b *testing.B) {
benchExclude(b, makeIntegerArray(1000, 1000, 2000), 1999, 2000)
}
func BenchmarkIntegerArray_ExcludeNone_10000(b *testing.B) {
benchExclude(b, makeIntegerArray(10000, 10000, 20000), 00, 5000)
}
func BenchmarkIntegerArray_ExcludeMiddleHalf_10000(b *testing.B) {
benchExclude(b, makeIntegerArray(10000, 10000, 20000), 12500, 17500)
}
func BenchmarkIntegerArray_ExcludeFirst_10000(b *testing.B) {
benchExclude(b, makeIntegerArray(10000, 10000, 20000), 0, 10000)
}
func BenchmarkIntegerArray_ExcludeLast_10000(b *testing.B) {
benchExclude(b, makeIntegerArray(10000, 10000, 20000), 19999, 20000)
}
func benchInclude(b *testing.B, vals *IntegerArray, min, max int64) {
src := *vals
tmp := NewIntegerArrayLen(vals.Len())
copy(tmp.Timestamps, vals.Timestamps)
copy(tmp.Values, vals.Values)
b.ResetTimer()
for i := 0; i < b.N; i++ {
vals.Include(min, max)
*vals = src
copy(vals.Timestamps, tmp.Timestamps)
copy(vals.Values, tmp.Values)
}
}
func BenchmarkIntegerArray_IncludeNone_1000(b *testing.B) {
benchInclude(b, makeIntegerArray(1000, 1000, 2000), 0, 500)
}
func BenchmarkIntegerArray_IncludeMiddleHalf_1000(b *testing.B) {
benchInclude(b, makeIntegerArray(1000, 1000, 2000), 1250, 1750)
}
func BenchmarkIntegerArray_IncludeFirst_1000(b *testing.B) {
benchInclude(b, makeIntegerArray(1000, 1000, 2000), 0, 1000)
}
func BenchmarkIntegerArray_IncludeLast_1000(b *testing.B) {
benchInclude(b, makeIntegerArray(1000, 1000, 2000), 1999, 2000)
}
func BenchmarkIntegerArray_IncludeNone_10000(b *testing.B) {
benchInclude(b, makeIntegerArray(10000, 10000, 20000), 00, 5000)
}
func BenchmarkIntegerArray_IncludeMiddleHalf_10000(b *testing.B) {
benchInclude(b, makeIntegerArray(10000, 10000, 20000), 12500, 17500)
}
func BenchmarkIntegerArray_IncludeFirst_10000(b *testing.B) {
benchInclude(b, makeIntegerArray(10000, 10000, 20000), 0, 10000)
}
func BenchmarkIntegerArray_IncludeLast_10000(b *testing.B) {
benchInclude(b, makeIntegerArray(10000, 10000, 20000), 19999, 20000)
}

View File

@ -0,0 +1,459 @@
package cursors_test
import (
"strconv"
"testing"
"github.com/google/go-cmp/cmp"
"github.com/influxdata/influxdb/tsdb/cursors"
)
func makeBooleanArray(v ...interface{}) *cursors.BooleanArray {
if len(v)&1 == 1 {
panic("invalid array length")
}
a := cursors.NewBooleanArrayLen(len(v) / 2)
for i := 0; i < len(v); i += 2 {
a.Timestamps[i/2] = int64(v[i].(int))
a.Values[i/2] = v[i+1].(bool)
}
return a
}
func makeFloatArray(v ...interface{}) *cursors.FloatArray {
if len(v)&1 == 1 {
panic("invalid array length")
}
a := cursors.NewFloatArrayLen(len(v) / 2)
for i := 0; i < len(v); i += 2 {
a.Timestamps[i/2] = int64(v[i].(int))
a.Values[i/2] = v[i+1].(float64)
}
return a
}
func makeIntegerArray(v ...interface{}) *cursors.IntegerArray {
if len(v)&1 == 1 {
panic("invalid array length")
}
a := cursors.NewIntegerArrayLen(len(v) / 2)
for i := 0; i < len(v); i += 2 {
a.Timestamps[i/2] = int64(v[i].(int))
a.Values[i/2] = int64(v[i+1].(int))
}
return a
}
func makeUnsignedArray(v ...interface{}) *cursors.UnsignedArray {
if len(v)&1 == 1 {
panic("invalid array length")
}
a := cursors.NewUnsignedArrayLen(len(v) / 2)
for i := 0; i < len(v); i += 2 {
a.Timestamps[i/2] = int64(v[i].(int))
a.Values[i/2] = uint64(v[i+1].(int))
}
return a
}
func makeStringArray(v ...interface{}) *cursors.StringArray {
if len(v)&1 == 1 {
panic("invalid array length")
}
a := cursors.NewStringArrayLen(len(v) / 2)
for i := 0; i < len(v); i += 2 {
a.Timestamps[i/2] = int64(v[i].(int))
a.Values[i/2] = strconv.Itoa(v[i+1].(int))
}
return a
}
func TestBooleanArray_Merge(t *testing.T) {
tests := []struct {
name string
a, b, exp *cursors.BooleanArray
}{
{
name: "empty a",
a: makeBooleanArray(),
b: makeBooleanArray(1, true, 2, true),
exp: makeBooleanArray(1, true, 2, true),
},
{
name: "empty b",
a: makeBooleanArray(1, true, 2, true),
b: makeBooleanArray(),
exp: makeBooleanArray(1, true, 2, true),
},
{
name: "b replaces a",
a: makeBooleanArray(1, true),
b: makeBooleanArray(
0, false,
1, false, // overwrites a
2, false,
3, false,
4, false,
),
exp: makeBooleanArray(0, false, 1, false, 2, false, 3, false, 4, false),
},
{
name: "b replaces partial a",
a: makeBooleanArray(1, true, 2, true, 3, true, 4, true),
b: makeBooleanArray(
1, false, // overwrites a
2, false, // overwrites a
),
exp: makeBooleanArray(
1, false, // overwrites a
2, false, // overwrites a
3, true,
4, true,
),
},
{
name: "b replaces all a",
a: makeBooleanArray(1, true, 2, true, 3, true, 4, true),
b: makeBooleanArray(1, false, 2, false, 3, false, 4, false),
exp: makeBooleanArray(1, false, 2, false, 3, false, 4, false),
},
{
name: "b replaces a interleaved",
a: makeBooleanArray(0, true, 1, true, 2, true, 3, true, 4, true),
b: makeBooleanArray(0, false, 2, false, 4, false),
exp: makeBooleanArray(0, false, 1, true, 2, false, 3, true, 4, false),
},
{
name: "b merges a interleaved",
a: makeBooleanArray(0, true, 2, true, 4, true),
b: makeBooleanArray(1, false, 3, false, 5, false),
exp: makeBooleanArray(0, true, 1, false, 2, true, 3, false, 4, true, 5, false),
},
}
for _, test := range tests {
t.Run(test.name, func(t *testing.T) {
test.a.Merge(test.b)
if !cmp.Equal(test.a, test.exp) {
t.Fatalf("unexpected values -got/+exp\n%s", cmp.Diff(test.a, test.exp))
}
})
}
}
func TestFloatArray_Merge(t *testing.T) {
tests := []struct {
name string
a, b, exp *cursors.FloatArray
}{
{
name: "empty a",
a: makeFloatArray(),
b: makeFloatArray(1, 1.1, 2, 2.1),
exp: makeFloatArray(1, 1.1, 2, 2.1),
},
{
name: "empty b",
a: makeFloatArray(1, 1.0, 2, 2.0),
b: makeFloatArray(),
exp: makeFloatArray(1, 1.0, 2, 2.0),
},
{
name: "b replaces a",
a: makeFloatArray(1, 1.0),
b: makeFloatArray(
0, 0.1,
1, 1.1, // overwrites a
2, 2.1,
3, 3.1,
4, 4.1,
),
exp: makeFloatArray(0, 0.1, 1, 1.1, 2, 2.1, 3, 3.1, 4, 4.1),
},
{
name: "b replaces partial a",
a: makeFloatArray(1, 1.0, 2, 2.0, 3, 3.0, 4, 4.0),
b: makeFloatArray(
1, 1.1, // overwrites a
2, 2.1, // overwrites a
),
exp: makeFloatArray(
1, 1.1, // overwrites a
2, 2.1, // overwrites a
3, 3.0,
4, 4.0,
),
},
{
name: "b replaces all a",
a: makeFloatArray(1, 1.0, 2, 2.0, 3, 3.0, 4, 4.0),
b: makeFloatArray(1, 1.1, 2, 2.1, 3, 3.1, 4, 4.1),
exp: makeFloatArray(1, 1.1, 2, 2.1, 3, 3.1, 4, 4.1),
},
{
name: "b replaces a interleaved",
a: makeFloatArray(0, 0.0, 1, 1.0, 2, 2.0, 3, 3.0, 4, 4.0),
b: makeFloatArray(0, 0.1, 2, 2.1, 4, 4.1),
exp: makeFloatArray(0, 0.1, 1, 1.0, 2, 2.1, 3, 3.0, 4, 4.1),
},
{
name: "b merges a interleaved",
a: makeFloatArray(0, 0.0, 2, 2.0, 4, 4.0),
b: makeFloatArray(1, 1.1, 3, 3.1, 5, 5.1),
exp: makeFloatArray(0, 0.0, 1, 1.1, 2, 2.0, 3, 3.1, 4, 4.0, 5, 5.1),
},
}
for _, test := range tests {
t.Run(test.name, func(t *testing.T) {
test.a.Merge(test.b)
if !cmp.Equal(test.a, test.exp) {
t.Fatalf("unexpected values -got/+exp\n%s", cmp.Diff(test.a, test.exp))
}
})
}
}
func TestIntegerArray_Merge(t *testing.T) {
tests := []struct {
name string
a, b, exp *cursors.IntegerArray
}{
{
name: "empty a",
a: makeIntegerArray(),
b: makeIntegerArray(1, 11, 2, 21),
exp: makeIntegerArray(1, 11, 2, 21),
},
{
name: "empty b",
a: makeIntegerArray(1, 10, 2, 20),
b: makeIntegerArray(),
exp: makeIntegerArray(1, 10, 2, 20),
},
{
name: "b replaces a",
a: makeIntegerArray(1, 10),
b: makeIntegerArray(
0, 1,
1, 11, // overwrites a
2, 21,
3, 31,
4, 41,
),
exp: makeIntegerArray(0, 1, 1, 11, 2, 21, 3, 31, 4, 41),
},
{
name: "b replaces partial a",
a: makeIntegerArray(1, 10, 2, 20, 3, 30, 4, 40),
b: makeIntegerArray(
1, 11, // overwrites a
2, 21, // overwrites a
),
exp: makeIntegerArray(
1, 11, // overwrites a
2, 21, // overwrites a
3, 30,
4, 40,
),
},
{
name: "b replaces all a",
a: makeIntegerArray(1, 10, 2, 20, 3, 30, 4, 40),
b: makeIntegerArray(1, 11, 2, 21, 3, 31, 4, 41),
exp: makeIntegerArray(1, 11, 2, 21, 3, 31, 4, 41),
},
{
name: "b replaces a interleaved",
a: makeIntegerArray(0, 0, 1, 10, 2, 20, 3, 30, 4, 40),
b: makeIntegerArray(0, 1, 2, 21, 4, 41),
exp: makeIntegerArray(0, 1, 1, 10, 2, 21, 3, 30, 4, 41),
},
{
name: "b merges a interleaved",
a: makeIntegerArray(0, 00, 2, 20, 4, 40),
b: makeIntegerArray(1, 11, 3, 31, 5, 51),
exp: makeIntegerArray(0, 00, 1, 11, 2, 20, 3, 31, 4, 40, 5, 51),
},
}
for _, test := range tests {
t.Run(test.name, func(t *testing.T) {
test.a.Merge(test.b)
if !cmp.Equal(test.a, test.exp) {
t.Fatalf("unexpected values -got/+exp\n%s", cmp.Diff(test.a, test.exp))
}
})
}
}
func TestUnsignedArray_Merge(t *testing.T) {
tests := []struct {
name string
a, b, exp *cursors.UnsignedArray
}{
{
name: "empty a",
a: makeUnsignedArray(),
b: makeUnsignedArray(1, 11, 2, 21),
exp: makeUnsignedArray(1, 11, 2, 21),
},
{
name: "empty b",
a: makeUnsignedArray(1, 10, 2, 20),
b: makeUnsignedArray(),
exp: makeUnsignedArray(1, 10, 2, 20),
},
{
name: "b replaces a",
a: makeUnsignedArray(1, 10),
b: makeUnsignedArray(
0, 1,
1, 11, // overwrites a
2, 21,
3, 31,
4, 41,
),
exp: makeUnsignedArray(0, 1, 1, 11, 2, 21, 3, 31, 4, 41),
},
{
name: "b replaces partial a",
a: makeUnsignedArray(1, 10, 2, 20, 3, 30, 4, 40),
b: makeUnsignedArray(
1, 11, // overwrites a
2, 21, // overwrites a
),
exp: makeUnsignedArray(
1, 11, // overwrites a
2, 21, // overwrites a
3, 30,
4, 40,
),
},
{
name: "b replaces all a",
a: makeUnsignedArray(1, 10, 2, 20, 3, 30, 4, 40),
b: makeUnsignedArray(1, 11, 2, 21, 3, 31, 4, 41),
exp: makeUnsignedArray(1, 11, 2, 21, 3, 31, 4, 41),
},
{
name: "b replaces a interleaved",
a: makeUnsignedArray(0, 0, 1, 10, 2, 20, 3, 30, 4, 40),
b: makeUnsignedArray(0, 1, 2, 21, 4, 41),
exp: makeUnsignedArray(0, 1, 1, 10, 2, 21, 3, 30, 4, 41),
},
{
name: "b merges a interleaved",
a: makeUnsignedArray(0, 00, 2, 20, 4, 40),
b: makeUnsignedArray(1, 11, 3, 31, 5, 51),
exp: makeUnsignedArray(0, 00, 1, 11, 2, 20, 3, 31, 4, 40, 5, 51),
},
}
for _, test := range tests {
t.Run(test.name, func(t *testing.T) {
test.a.Merge(test.b)
if !cmp.Equal(test.a, test.exp) {
t.Fatalf("unexpected values -got/+exp\n%s", cmp.Diff(test.a, test.exp))
}
})
}
}
func TestStringArray_Merge(t *testing.T) {
tests := []struct {
name string
a, b, exp *cursors.StringArray
}{
{
name: "empty a",
a: makeStringArray(),
b: makeStringArray(1, 11, 2, 21),
exp: makeStringArray(1, 11, 2, 21),
},
{
name: "empty b",
a: makeStringArray(1, 10, 2, 20),
b: makeStringArray(),
exp: makeStringArray(1, 10, 2, 20),
},
{
name: "b replaces a",
a: makeStringArray(1, 10),
b: makeStringArray(
0, 1,
1, 11, // overwrites a
2, 21,
3, 31,
4, 41,
),
exp: makeStringArray(0, 1, 1, 11, 2, 21, 3, 31, 4, 41),
},
{
name: "b replaces partial a",
a: makeStringArray(1, 10, 2, 20, 3, 30, 4, 40),
b: makeStringArray(
1, 11, // overwrites a
2, 21, // overwrites a
),
exp: makeStringArray(
1, 11, // overwrites a
2, 21, // overwrites a
3, 30,
4, 40,
),
},
{
name: "b replaces all a",
a: makeStringArray(1, 10, 2, 20, 3, 30, 4, 40),
b: makeStringArray(1, 11, 2, 21, 3, 31, 4, 41),
exp: makeStringArray(1, 11, 2, 21, 3, 31, 4, 41),
},
{
name: "b replaces a interleaved",
a: makeStringArray(0, 0, 1, 10, 2, 20, 3, 30, 4, 40),
b: makeStringArray(0, 1, 2, 21, 4, 41),
exp: makeStringArray(0, 1, 1, 10, 2, 21, 3, 30, 4, 41),
},
{
name: "b merges a interleaved",
a: makeStringArray(0, 00, 2, 20, 4, 40),
b: makeStringArray(1, 11, 3, 31, 5, 51),
exp: makeStringArray(0, 00, 1, 11, 2, 20, 3, 31, 4, 40, 5, 51),
},
}
for _, test := range tests {
t.Run(test.name, func(t *testing.T) {
test.a.Merge(test.b)
if !cmp.Equal(test.a, test.exp) {
t.Fatalf("unexpected values -got/+exp\n%s", cmp.Diff(test.a, test.exp))
}
})
}
}

77
tsdb/cursors/cursor.go Normal file
View File

@ -0,0 +1,77 @@
package cursors
import (
"context"
"github.com/influxdata/influxdb/models"
)
const DefaultMaxPointsPerBlock = 1000
type Cursor interface {
Close()
Err() error
Stats() CursorStats
}
type IntegerArrayCursor interface {
Cursor
Next() *IntegerArray
}
type FloatArrayCursor interface {
Cursor
Next() *FloatArray
}
type UnsignedArrayCursor interface {
Cursor
Next() *UnsignedArray
}
type StringArrayCursor interface {
Cursor
Next() *StringArray
}
type BooleanArrayCursor interface {
Cursor
Next() *BooleanArray
}
type CursorRequest struct {
Name []byte
Tags models.Tags
Field string
Ascending bool
StartTime int64
EndTime int64
}
type CursorIterator interface {
Next(ctx context.Context, r *CursorRequest) (Cursor, error)
Stats() CursorStats
}
type CursorIterators []CursorIterator
// Stats returns the aggregate stats of all cursor iterators.
func (a CursorIterators) Stats() CursorStats {
var stats CursorStats
for _, itr := range a {
stats.Add(itr.Stats())
}
return stats
}
// CursorStats represents stats collected by a cursor.
type CursorStats struct {
ScannedValues int // number of values scanned
ScannedBytes int // number of uncompressed bytes scanned
}
// Add adds other to s and updates s.
func (s *CursorStats) Add(other CursorStats) {
s.ScannedValues += other.ScannedValues
s.ScannedBytes += other.ScannedBytes
}