2017-08-15 19:24:22 +00:00
|
|
|
package query
|
2015-11-04 21:06:06 +00:00
|
|
|
|
|
|
|
import (
|
2017-10-11 14:08:31 +00:00
|
|
|
"context"
|
2015-11-04 21:06:06 +00:00
|
|
|
"errors"
|
|
|
|
"fmt"
|
2017-08-22 18:18:41 +00:00
|
|
|
"io"
|
2016-12-27 23:12:11 +00:00
|
|
|
"math"
|
2015-11-04 21:06:06 +00:00
|
|
|
"sort"
|
2017-08-15 19:24:22 +00:00
|
|
|
|
2017-10-11 14:08:31 +00:00
|
|
|
"github.com/influxdata/influxdb/pkg/tracing"
|
2017-10-30 21:40:26 +00:00
|
|
|
"github.com/influxdata/influxql"
|
2015-11-04 21:06:06 +00:00
|
|
|
)
|
|
|
|
|
2016-02-11 20:11:04 +00:00
|
|
|
// SelectOptions are options that customize the select call.
|
|
|
|
type SelectOptions struct {
|
2017-05-05 17:20:00 +00:00
|
|
|
// Authorizer is used to limit access to data
|
|
|
|
Authorizer Authorizer
|
|
|
|
|
2016-06-10 15:14:21 +00:00
|
|
|
// Node to exclusively read from.
|
|
|
|
// If zero, all nodes are used.
|
|
|
|
NodeID uint64
|
|
|
|
|
2016-03-09 14:36:18 +00:00
|
|
|
// An optional channel that, if closed, signals that the select should be
|
|
|
|
// interrupted.
|
|
|
|
InterruptCh <-chan struct{}
|
2016-08-08 16:39:38 +00:00
|
|
|
|
|
|
|
// Maximum number of concurrent series.
|
|
|
|
MaxSeriesN int
|
2017-08-22 18:18:41 +00:00
|
|
|
|
|
|
|
// Maximum number of buckets for a statement.
|
|
|
|
MaxBucketsN int
|
|
|
|
}
|
|
|
|
|
|
|
|
// ShardMapper retrieves and maps shards into an IteratorCreator that can later be
|
|
|
|
// used for executing queries.
|
|
|
|
type ShardMapper interface {
|
2017-08-23 20:54:07 +00:00
|
|
|
MapShards(sources influxql.Sources, t influxql.TimeRange, opt SelectOptions) (ShardGroup, error)
|
2017-08-22 18:18:41 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// ShardGroup represents a shard or a collection of shards that can be accessed
|
|
|
|
// for creating iterators.
|
2017-08-28 14:34:36 +00:00
|
|
|
// When creating iterators, the resource used for reading the iterators should be
|
|
|
|
// separate from the resource used to map the shards. When the ShardGroup is closed,
|
|
|
|
// it should not close any resources associated with the created Iterator. Those
|
|
|
|
// resources belong to the Iterator and will be closed when the Iterator itself is
|
|
|
|
// closed.
|
|
|
|
// The query engine operates under this assumption and will close the shard group
|
|
|
|
// after creating the iterators, but before the iterators are actually read.
|
2017-08-22 18:18:41 +00:00
|
|
|
type ShardGroup interface {
|
|
|
|
IteratorCreator
|
|
|
|
influxql.FieldMapper
|
|
|
|
io.Closer
|
2016-02-11 20:11:04 +00:00
|
|
|
}
|
|
|
|
|
2017-08-23 18:20:51 +00:00
|
|
|
// Select is a prepared statement that is ready to be executed.
|
|
|
|
type PreparedStatement interface {
|
|
|
|
// Select creates the Iterators that will be used to read the query.
|
2017-10-11 14:08:31 +00:00
|
|
|
Select(ctx context.Context) ([]Iterator, []string, error)
|
2017-08-28 14:34:36 +00:00
|
|
|
|
2017-08-24 16:27:29 +00:00
|
|
|
// Explain outputs the explain plan for this statement.
|
|
|
|
Explain() (string, error)
|
|
|
|
|
2017-08-28 14:34:36 +00:00
|
|
|
// Close closes the resources associated with this prepared statement.
|
|
|
|
// This must be called as the mapped shards may hold open resources such
|
|
|
|
// as network connections.
|
|
|
|
Close() error
|
2017-08-23 18:20:51 +00:00
|
|
|
}
|
2017-08-22 18:18:41 +00:00
|
|
|
|
2017-08-23 18:20:51 +00:00
|
|
|
// Prepare will compile the statement with the default compile options and
|
|
|
|
// then prepare the query.
|
|
|
|
func Prepare(stmt *influxql.SelectStatement, shardMapper ShardMapper, opt SelectOptions) (PreparedStatement, error) {
|
|
|
|
c, err := Compile(stmt, CompileOptions{})
|
2017-08-22 18:18:41 +00:00
|
|
|
if err != nil {
|
2017-08-23 18:20:51 +00:00
|
|
|
return nil, err
|
2017-08-22 18:18:41 +00:00
|
|
|
}
|
2017-08-23 18:20:51 +00:00
|
|
|
return c.Prepare(shardMapper, opt)
|
|
|
|
}
|
2017-08-22 18:18:41 +00:00
|
|
|
|
2017-08-23 18:20:51 +00:00
|
|
|
// Select compiles, prepares, and then initiates execution of the query using the
|
|
|
|
// default compile options.
|
2017-10-11 14:08:31 +00:00
|
|
|
func Select(ctx context.Context, stmt *influxql.SelectStatement, shardMapper ShardMapper, opt SelectOptions) ([]Iterator, []string, error) {
|
2017-08-23 18:20:51 +00:00
|
|
|
s, err := Prepare(stmt, shardMapper, opt)
|
2015-11-04 21:06:06 +00:00
|
|
|
if err != nil {
|
2017-08-22 18:18:41 +00:00
|
|
|
return nil, nil, err
|
|
|
|
}
|
2017-08-28 14:34:36 +00:00
|
|
|
// Must be deferred so it runs after Select.
|
|
|
|
defer s.Close()
|
2017-10-11 14:08:31 +00:00
|
|
|
return s.Select(ctx)
|
2017-08-23 18:20:51 +00:00
|
|
|
}
|
2017-08-22 18:18:41 +00:00
|
|
|
|
2017-08-23 18:20:51 +00:00
|
|
|
type preparedStatement struct {
|
2017-08-28 14:34:36 +00:00
|
|
|
stmt *influxql.SelectStatement
|
|
|
|
opt IteratorOptions
|
|
|
|
ic interface {
|
|
|
|
IteratorCreator
|
|
|
|
io.Closer
|
|
|
|
}
|
2017-08-23 18:20:51 +00:00
|
|
|
columns []string
|
|
|
|
}
|
2017-08-22 18:18:41 +00:00
|
|
|
|
2017-10-11 14:08:31 +00:00
|
|
|
func (p *preparedStatement) Select(ctx context.Context) ([]Iterator, []string, error) {
|
|
|
|
itrs, err := buildIterators(ctx, p.stmt, p.ic, p.opt)
|
2017-08-22 18:18:41 +00:00
|
|
|
if err != nil {
|
|
|
|
return nil, nil, err
|
2015-11-04 21:06:06 +00:00
|
|
|
}
|
2017-08-23 18:20:51 +00:00
|
|
|
return itrs, p.columns, nil
|
2016-11-23 20:32:42 +00:00
|
|
|
}
|
2015-11-04 21:06:06 +00:00
|
|
|
|
2017-08-28 14:34:36 +00:00
|
|
|
func (p *preparedStatement) Close() error {
|
|
|
|
return p.ic.Close()
|
|
|
|
}
|
|
|
|
|
2017-10-11 14:08:31 +00:00
|
|
|
func buildIterators(ctx context.Context, stmt *influxql.SelectStatement, ic IteratorCreator, opt IteratorOptions) ([]Iterator, error) {
|
|
|
|
span := tracing.SpanFromContext(ctx)
|
2015-11-04 21:06:06 +00:00
|
|
|
// Retrieve refs for each call and var ref.
|
|
|
|
info := newSelectInfo(stmt)
|
|
|
|
if len(info.calls) > 1 && len(info.refs) > 0 {
|
|
|
|
return nil, errors.New("cannot select fields when selecting multiple aggregates")
|
|
|
|
}
|
|
|
|
|
2016-01-08 18:02:14 +00:00
|
|
|
// Determine auxiliary fields to be selected.
|
2017-08-15 19:24:22 +00:00
|
|
|
opt.Aux = make([]influxql.VarRef, 0, len(info.refs))
|
2015-11-04 21:06:06 +00:00
|
|
|
for ref := range info.refs {
|
2016-05-16 16:08:28 +00:00
|
|
|
opt.Aux = append(opt.Aux, *ref)
|
2015-11-04 21:06:06 +00:00
|
|
|
}
|
2017-08-15 19:24:22 +00:00
|
|
|
sort.Sort(influxql.VarRefs(opt.Aux))
|
2015-11-04 21:06:06 +00:00
|
|
|
|
|
|
|
// If there are multiple auxilary fields and no calls then construct an aux iterator.
|
|
|
|
if len(info.calls) == 0 && len(info.refs) > 0 {
|
2017-10-11 14:08:31 +00:00
|
|
|
if span != nil {
|
|
|
|
span = span.StartSpan("auxiliary_iterators")
|
|
|
|
defer span.Finish()
|
|
|
|
|
|
|
|
span.SetLabels("statement", stmt.String())
|
|
|
|
ctx = tracing.NewContextWithSpan(ctx, span)
|
|
|
|
}
|
|
|
|
return buildAuxIterators(ctx, stmt.Fields, ic, stmt.Sources, opt)
|
|
|
|
}
|
|
|
|
|
|
|
|
if span != nil {
|
|
|
|
span = span.StartSpan("field_iterators")
|
|
|
|
defer span.Finish()
|
|
|
|
|
|
|
|
span.SetLabels("statement", stmt.String())
|
|
|
|
ctx = tracing.NewContextWithSpan(ctx, span)
|
2015-11-04 21:06:06 +00:00
|
|
|
}
|
|
|
|
|
2017-05-17 16:10:51 +00:00
|
|
|
// Include auxiliary fields from top() and bottom() when not writing the results.
|
|
|
|
fields := stmt.Fields
|
|
|
|
if stmt.Target == nil {
|
|
|
|
extraFields := 0
|
|
|
|
for call := range info.calls {
|
|
|
|
if call.Name == "top" || call.Name == "bottom" {
|
|
|
|
for i := 1; i < len(call.Args)-1; i++ {
|
2017-08-15 19:24:22 +00:00
|
|
|
ref := call.Args[i].(*influxql.VarRef)
|
2017-05-17 16:10:51 +00:00
|
|
|
opt.Aux = append(opt.Aux, *ref)
|
|
|
|
extraFields++
|
|
|
|
}
|
2016-01-08 18:02:14 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2017-05-17 16:10:51 +00:00
|
|
|
if extraFields > 0 {
|
|
|
|
// Rebuild the list of fields if any extra fields are being implicitly added
|
2017-08-15 19:24:22 +00:00
|
|
|
fields = make([]*influxql.Field, 0, len(stmt.Fields)+extraFields)
|
2017-05-17 16:10:51 +00:00
|
|
|
for _, f := range stmt.Fields {
|
|
|
|
fields = append(fields, f)
|
|
|
|
switch expr := f.Expr.(type) {
|
2017-08-15 19:24:22 +00:00
|
|
|
case *influxql.Call:
|
2017-05-17 16:10:51 +00:00
|
|
|
if expr.Name == "top" || expr.Name == "bottom" {
|
|
|
|
for i := 1; i < len(expr.Args)-1; i++ {
|
2017-08-15 19:24:22 +00:00
|
|
|
fields = append(fields, &influxql.Field{Expr: expr.Args[i]})
|
2017-05-17 16:10:51 +00:00
|
|
|
}
|
2016-01-08 18:02:14 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2016-04-29 15:25:25 +00:00
|
|
|
// Determine if there is one call and it is a selector.
|
|
|
|
selector := false
|
|
|
|
if len(info.calls) == 1 {
|
|
|
|
for call := range info.calls {
|
2017-08-15 19:24:22 +00:00
|
|
|
selector = influxql.IsSelector(call)
|
2016-04-29 15:25:25 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2017-10-11 14:08:31 +00:00
|
|
|
return buildFieldIterators(ctx, fields, ic, stmt.Sources, opt, selector, stmt.Target != nil)
|
2015-11-04 21:06:06 +00:00
|
|
|
}
|
|
|
|
|
2016-12-30 01:39:24 +00:00
|
|
|
// buildAuxIterators creates a set of iterators from a single combined auxiliary iterator.
|
2017-10-11 14:08:31 +00:00
|
|
|
func buildAuxIterators(ctx context.Context, fields influxql.Fields, ic IteratorCreator, sources influxql.Sources, opt IteratorOptions) ([]Iterator, error) {
|
2016-11-23 20:32:42 +00:00
|
|
|
// Create the auxiliary iterators for each source.
|
|
|
|
inputs := make([]Iterator, 0, len(sources))
|
|
|
|
if err := func() error {
|
|
|
|
for _, source := range sources {
|
|
|
|
switch source := source.(type) {
|
2017-08-15 19:24:22 +00:00
|
|
|
case *influxql.Measurement:
|
2017-10-11 14:08:31 +00:00
|
|
|
input, err := ic.CreateIterator(ctx, source, opt)
|
2016-11-23 20:32:42 +00:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
inputs = append(inputs, input)
|
2017-08-15 19:24:22 +00:00
|
|
|
case *influxql.SubQuery:
|
2017-02-28 21:40:43 +00:00
|
|
|
b := subqueryBuilder{
|
|
|
|
ic: ic,
|
|
|
|
stmt: source.Statement,
|
2017-02-13 22:23:10 +00:00
|
|
|
}
|
|
|
|
|
2017-10-11 14:08:31 +00:00
|
|
|
input, err := b.buildAuxIterator(ctx, opt)
|
2016-11-23 20:32:42 +00:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
inputs = append(inputs, input)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}(); err != nil {
|
|
|
|
Iterators(inputs).Close()
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
|
|
|
// Merge iterators to read auxilary fields.
|
|
|
|
input, err := Iterators(inputs).Merge(opt)
|
2015-11-04 21:06:06 +00:00
|
|
|
if err != nil {
|
2016-11-23 20:32:42 +00:00
|
|
|
Iterators(inputs).Close()
|
2015-11-04 21:06:06 +00:00
|
|
|
return nil, err
|
2016-05-03 13:20:20 +00:00
|
|
|
} else if input == nil {
|
|
|
|
input = &nilFloatIterator{}
|
2015-11-04 21:06:06 +00:00
|
|
|
}
|
|
|
|
|
2016-02-05 17:23:35 +00:00
|
|
|
// Filter out duplicate rows, if required.
|
|
|
|
if opt.Dedupe {
|
2016-06-03 03:03:57 +00:00
|
|
|
// If there is no group by and it is a float iterator, see if we can use a fast dedupe.
|
|
|
|
if itr, ok := input.(FloatIterator); ok && len(opt.Dimensions) == 0 {
|
|
|
|
if sz := len(fields); sz > 0 && sz < 3 {
|
|
|
|
input = newFloatFastDedupeIterator(itr)
|
|
|
|
} else {
|
|
|
|
input = NewDedupeIterator(itr)
|
|
|
|
}
|
2016-05-02 15:17:08 +00:00
|
|
|
} else {
|
|
|
|
input = NewDedupeIterator(input)
|
|
|
|
}
|
2016-02-05 17:23:35 +00:00
|
|
|
}
|
2015-12-24 04:42:10 +00:00
|
|
|
// Apply limit & offset.
|
|
|
|
if opt.Limit > 0 || opt.Offset > 0 {
|
|
|
|
input = NewLimitIterator(input, opt)
|
|
|
|
}
|
|
|
|
|
2016-11-23 20:32:42 +00:00
|
|
|
// Wrap in an auxiliary iterator to separate the fields.
|
2016-05-16 16:08:28 +00:00
|
|
|
aitr := NewAuxIterator(input, opt)
|
2017-10-25 20:08:55 +00:00
|
|
|
tryAddAuxIteratorToContext(ctx, aitr)
|
2015-11-04 21:06:06 +00:00
|
|
|
|
|
|
|
// Generate iterators for each field.
|
|
|
|
itrs := make([]Iterator, len(fields))
|
2016-04-07 14:52:32 +00:00
|
|
|
if err := func() error {
|
|
|
|
for i, f := range fields {
|
2017-08-15 19:24:22 +00:00
|
|
|
expr := influxql.Reduce(f.Expr, nil)
|
2016-11-23 20:32:42 +00:00
|
|
|
itr, err := buildAuxIterator(expr, aitr, opt)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
2015-11-04 21:06:06 +00:00
|
|
|
}
|
2016-11-23 20:32:42 +00:00
|
|
|
itrs[i] = itr
|
2015-11-04 21:06:06 +00:00
|
|
|
}
|
2016-04-07 14:52:32 +00:00
|
|
|
return nil
|
|
|
|
}(); err != nil {
|
|
|
|
Iterators(Iterators(itrs).filterNonNil()).Close()
|
|
|
|
aitr.Close()
|
|
|
|
return nil, err
|
2015-11-04 21:06:06 +00:00
|
|
|
}
|
|
|
|
|
2016-03-13 16:05:47 +00:00
|
|
|
// Background the primary iterator since there is no reader for it.
|
|
|
|
aitr.Background()
|
2015-11-04 21:06:06 +00:00
|
|
|
|
|
|
|
return itrs, nil
|
|
|
|
}
|
|
|
|
|
2016-11-23 20:32:42 +00:00
|
|
|
// buildAuxIterator constructs an Iterator for an expression from an AuxIterator.
|
2017-08-15 19:24:22 +00:00
|
|
|
func buildAuxIterator(expr influxql.Expr, aitr AuxIterator, opt IteratorOptions) (Iterator, error) {
|
2016-11-23 20:32:42 +00:00
|
|
|
switch expr := expr.(type) {
|
2017-08-15 19:24:22 +00:00
|
|
|
case *influxql.VarRef:
|
2016-11-23 20:32:42 +00:00
|
|
|
return aitr.Iterator(expr.Val, expr.Type), nil
|
2017-08-15 19:24:22 +00:00
|
|
|
case *influxql.BinaryExpr:
|
|
|
|
if rhs, ok := expr.RHS.(influxql.Literal); ok {
|
2016-11-23 20:32:42 +00:00
|
|
|
// The right hand side is a literal. It is more common to have the RHS be a literal,
|
|
|
|
// so we check that one first and have this be the happy path.
|
2017-08-15 19:24:22 +00:00
|
|
|
if lhs, ok := expr.LHS.(influxql.Literal); ok {
|
2016-11-23 20:32:42 +00:00
|
|
|
// We have two literals that couldn't be combined by Reduce.
|
|
|
|
return nil, fmt.Errorf("unable to construct an iterator from two literals: LHS: %T, RHS: %T", lhs, rhs)
|
|
|
|
}
|
|
|
|
|
|
|
|
lhs, err := buildAuxIterator(expr.LHS, aitr, opt)
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
return buildRHSTransformIterator(lhs, rhs, expr.Op, opt)
|
2017-08-15 19:24:22 +00:00
|
|
|
} else if lhs, ok := expr.LHS.(influxql.Literal); ok {
|
2016-11-23 20:32:42 +00:00
|
|
|
rhs, err := buildAuxIterator(expr.RHS, aitr, opt)
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
return buildLHSTransformIterator(lhs, rhs, expr.Op, opt)
|
|
|
|
} else {
|
|
|
|
// We have two iterators. Combine them into a single iterator.
|
|
|
|
lhs, err := buildAuxIterator(expr.LHS, aitr, opt)
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
rhs, err := buildAuxIterator(expr.RHS, aitr, opt)
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
return buildTransformIterator(lhs, rhs, expr.Op, opt)
|
|
|
|
}
|
2017-08-15 19:24:22 +00:00
|
|
|
case *influxql.ParenExpr:
|
2016-11-23 20:32:42 +00:00
|
|
|
return buildAuxIterator(expr.Expr, aitr, opt)
|
2017-08-15 19:24:22 +00:00
|
|
|
case *influxql.NilLiteral:
|
2016-11-23 20:32:42 +00:00
|
|
|
return &nilFloatIterator{}, nil
|
|
|
|
default:
|
|
|
|
return nil, fmt.Errorf("invalid expression type: %T", expr)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2016-01-25 17:36:04 +00:00
|
|
|
// buildFieldIterators creates an iterator for each field expression.
|
2017-10-11 14:08:31 +00:00
|
|
|
func buildFieldIterators(ctx context.Context, fields influxql.Fields, ic IteratorCreator, sources influxql.Sources, opt IteratorOptions, selector, writeMode bool) ([]Iterator, error) {
|
2015-11-04 21:06:06 +00:00
|
|
|
// Create iterators from fields against the iterator creator.
|
2015-11-04 21:06:06 +00:00
|
|
|
itrs := make([]Iterator, len(fields))
|
2017-10-11 14:08:31 +00:00
|
|
|
span := tracing.SpanFromContext(ctx)
|
2015-11-04 21:06:06 +00:00
|
|
|
|
2015-11-04 21:06:06 +00:00
|
|
|
if err := func() error {
|
2015-11-04 21:06:06 +00:00
|
|
|
hasAuxFields := false
|
2016-01-25 17:36:04 +00:00
|
|
|
|
2015-11-04 21:06:06 +00:00
|
|
|
var input Iterator
|
|
|
|
for i, f := range fields {
|
|
|
|
// Build iterators for calls first and save the iterator.
|
|
|
|
// We do this so we can keep the ordering provided by the user, but
|
|
|
|
// still build the Call's iterator first.
|
2017-08-15 19:24:22 +00:00
|
|
|
if influxql.ContainsVarRef(f.Expr) {
|
2015-11-04 21:06:06 +00:00
|
|
|
hasAuxFields = true
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
|
2017-10-11 14:08:31 +00:00
|
|
|
var localSpan *tracing.Span
|
|
|
|
localContext := ctx
|
|
|
|
|
|
|
|
if span != nil {
|
|
|
|
localSpan = span.StartSpan("expression")
|
|
|
|
localSpan.SetLabels("expr", f.Expr.String())
|
|
|
|
localContext = tracing.NewContextWithSpan(ctx, localSpan)
|
|
|
|
}
|
|
|
|
|
2017-08-15 19:24:22 +00:00
|
|
|
expr := influxql.Reduce(f.Expr, nil)
|
2017-10-11 14:08:31 +00:00
|
|
|
itr, err := buildExprIterator(localContext, expr, ic, sources, opt, selector, writeMode)
|
|
|
|
|
|
|
|
if localSpan != nil {
|
|
|
|
localSpan.Finish()
|
|
|
|
}
|
|
|
|
|
2015-11-04 21:06:06 +00:00
|
|
|
if err != nil {
|
|
|
|
return err
|
2016-11-23 20:32:42 +00:00
|
|
|
} else if itr == nil {
|
|
|
|
itr = &nilFloatIterator{}
|
2015-11-04 21:06:06 +00:00
|
|
|
}
|
2017-04-05 16:47:57 +00:00
|
|
|
|
|
|
|
// If there is a limit or offset then apply it.
|
|
|
|
if opt.Limit > 0 || opt.Offset > 0 {
|
|
|
|
itr = NewLimitIterator(itr, opt)
|
|
|
|
}
|
2015-11-04 21:06:06 +00:00
|
|
|
itrs[i] = itr
|
|
|
|
input = itr
|
|
|
|
}
|
|
|
|
|
2016-01-18 22:48:49 +00:00
|
|
|
if input == nil || !hasAuxFields {
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2015-11-04 21:06:06 +00:00
|
|
|
// Build the aux iterators. Previous validation should ensure that only one
|
|
|
|
// call was present so we build an AuxIterator from that input.
|
2016-05-16 16:08:28 +00:00
|
|
|
aitr := NewAuxIterator(input, opt)
|
2017-10-25 20:08:55 +00:00
|
|
|
tryAddAuxIteratorToContext(ctx, aitr)
|
|
|
|
|
2016-01-18 22:48:49 +00:00
|
|
|
for i, f := range fields {
|
|
|
|
if itrs[i] != nil {
|
|
|
|
itrs[i] = aitr
|
|
|
|
continue
|
|
|
|
}
|
2015-11-04 21:06:06 +00:00
|
|
|
|
2017-08-15 19:24:22 +00:00
|
|
|
expr := influxql.Reduce(f.Expr, nil)
|
2016-11-23 20:32:42 +00:00
|
|
|
itr, err := buildAuxIterator(expr, aitr, opt)
|
2016-01-18 22:48:49 +00:00
|
|
|
if err != nil {
|
|
|
|
return err
|
2016-11-23 20:32:42 +00:00
|
|
|
} else if itr == nil {
|
|
|
|
itr = &nilFloatIterator{}
|
2015-11-04 21:06:06 +00:00
|
|
|
}
|
2016-01-18 22:48:49 +00:00
|
|
|
itrs[i] = itr
|
2015-11-04 21:06:06 +00:00
|
|
|
}
|
2016-02-09 23:03:44 +00:00
|
|
|
aitr.Start()
|
2015-11-04 21:06:06 +00:00
|
|
|
return nil
|
|
|
|
|
|
|
|
}(); err != nil {
|
2015-11-04 21:06:06 +00:00
|
|
|
Iterators(Iterators(itrs).filterNonNil()).Close()
|
2015-11-04 21:06:06 +00:00
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
|
|
|
return itrs, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
// buildExprIterator creates an iterator for an expression.
|
2017-10-11 14:08:31 +00:00
|
|
|
func buildExprIterator(ctx context.Context, expr influxql.Expr, ic IteratorCreator, sources influxql.Sources, opt IteratorOptions, selector, writeMode bool) (Iterator, error) {
|
2015-11-04 21:06:06 +00:00
|
|
|
opt.Expr = expr
|
2016-11-23 20:32:42 +00:00
|
|
|
b := exprIteratorBuilder{
|
2017-05-17 16:10:51 +00:00
|
|
|
ic: ic,
|
|
|
|
sources: sources,
|
|
|
|
opt: opt,
|
|
|
|
selector: selector,
|
|
|
|
writeMode: writeMode,
|
2016-11-23 20:32:42 +00:00
|
|
|
}
|
2015-11-04 21:06:06 +00:00
|
|
|
|
|
|
|
switch expr := expr.(type) {
|
2017-08-15 19:24:22 +00:00
|
|
|
case *influxql.VarRef:
|
2017-10-11 14:08:31 +00:00
|
|
|
return b.buildVarRefIterator(ctx, expr)
|
2017-08-15 19:24:22 +00:00
|
|
|
case *influxql.Call:
|
2017-10-11 14:08:31 +00:00
|
|
|
return b.buildCallIterator(ctx, expr)
|
2017-08-15 19:24:22 +00:00
|
|
|
case *influxql.BinaryExpr:
|
2017-10-11 14:08:31 +00:00
|
|
|
return b.buildBinaryExprIterator(ctx, expr)
|
2017-08-15 19:24:22 +00:00
|
|
|
case *influxql.ParenExpr:
|
2017-10-11 14:08:31 +00:00
|
|
|
return buildExprIterator(ctx, expr.Expr, ic, sources, opt, selector, writeMode)
|
2017-08-15 19:24:22 +00:00
|
|
|
case *influxql.NilLiteral:
|
2016-11-23 20:32:42 +00:00
|
|
|
return &nilFloatIterator{}, nil
|
|
|
|
default:
|
|
|
|
return nil, fmt.Errorf("invalid expression type: %T", expr)
|
|
|
|
}
|
|
|
|
}
|
2015-11-04 21:06:06 +00:00
|
|
|
|
2016-11-23 20:32:42 +00:00
|
|
|
type exprIteratorBuilder struct {
|
2017-05-17 16:10:51 +00:00
|
|
|
ic IteratorCreator
|
2017-08-15 19:24:22 +00:00
|
|
|
sources influxql.Sources
|
2017-05-17 16:10:51 +00:00
|
|
|
opt IteratorOptions
|
|
|
|
selector bool
|
|
|
|
writeMode bool
|
2016-11-23 20:32:42 +00:00
|
|
|
}
|
2016-10-04 21:20:35 +00:00
|
|
|
|
2017-10-11 14:08:31 +00:00
|
|
|
func (b *exprIteratorBuilder) buildVarRefIterator(ctx context.Context, expr *influxql.VarRef) (Iterator, error) {
|
2016-11-23 20:32:42 +00:00
|
|
|
inputs := make([]Iterator, 0, len(b.sources))
|
|
|
|
if err := func() error {
|
|
|
|
for _, source := range b.sources {
|
|
|
|
switch source := source.(type) {
|
2017-08-15 19:24:22 +00:00
|
|
|
case *influxql.Measurement:
|
2017-10-11 14:08:31 +00:00
|
|
|
input, err := b.ic.CreateIterator(ctx, source, b.opt)
|
2016-11-23 20:32:42 +00:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
inputs = append(inputs, input)
|
2017-08-15 19:24:22 +00:00
|
|
|
case *influxql.SubQuery:
|
2017-02-28 21:40:43 +00:00
|
|
|
subquery := subqueryBuilder{
|
|
|
|
ic: b.ic,
|
|
|
|
stmt: source.Statement,
|
2016-11-23 20:32:42 +00:00
|
|
|
}
|
2016-05-12 21:11:19 +00:00
|
|
|
|
2017-10-11 14:08:31 +00:00
|
|
|
input, err := subquery.buildVarRefIterator(ctx, expr, b.opt)
|
2017-02-28 21:40:43 +00:00
|
|
|
if err != nil {
|
2016-11-23 20:32:42 +00:00
|
|
|
return err
|
|
|
|
}
|
2017-02-28 21:40:43 +00:00
|
|
|
inputs = append(inputs, input)
|
2016-11-23 20:32:42 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}(); err != nil {
|
|
|
|
Iterators(inputs).Close()
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
|
|
|
// Variable references in this section will always go into some call
|
|
|
|
// iterator. Combine it with a merge iterator.
|
|
|
|
itr := NewMergeIterator(inputs, b.opt)
|
|
|
|
if itr == nil {
|
|
|
|
itr = &nilFloatIterator{}
|
|
|
|
}
|
|
|
|
|
|
|
|
if b.opt.InterruptCh != nil {
|
|
|
|
itr = NewInterruptIterator(itr, b.opt.InterruptCh)
|
|
|
|
}
|
|
|
|
return itr, nil
|
|
|
|
}
|
|
|
|
|
2017-10-11 14:08:31 +00:00
|
|
|
func (b *exprIteratorBuilder) buildCallIterator(ctx context.Context, expr *influxql.Call) (Iterator, error) {
|
2016-11-23 20:32:42 +00:00
|
|
|
// TODO(jsternberg): Refactor this. This section needs to die in a fire.
|
2017-04-05 16:47:57 +00:00
|
|
|
opt := b.opt
|
|
|
|
// Eliminate limits and offsets if they were previously set. These are handled by the caller.
|
|
|
|
opt.Limit, opt.Offset = 0, 0
|
2016-11-23 20:32:42 +00:00
|
|
|
switch expr.Name {
|
|
|
|
case "distinct":
|
2017-03-27 21:34:53 +00:00
|
|
|
opt.Ordered = true
|
2017-10-11 14:08:31 +00:00
|
|
|
input, err := buildExprIterator(ctx, expr.Args[0].(*influxql.VarRef), b.ic, b.sources, opt, b.selector, false)
|
2016-11-23 20:32:42 +00:00
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
2017-03-27 21:34:53 +00:00
|
|
|
input, err = NewDistinctIterator(input, opt)
|
2016-11-23 20:32:42 +00:00
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
2017-03-27 21:34:53 +00:00
|
|
|
return NewIntervalIterator(input, opt), nil
|
2016-11-23 20:32:42 +00:00
|
|
|
case "sample":
|
2017-03-27 21:34:53 +00:00
|
|
|
opt.Ordered = true
|
2017-10-11 14:08:31 +00:00
|
|
|
input, err := buildExprIterator(ctx, expr.Args[0], b.ic, b.sources, opt, b.selector, false)
|
2016-11-23 20:32:42 +00:00
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
2017-08-15 19:24:22 +00:00
|
|
|
size := expr.Args[1].(*influxql.IntegerLiteral)
|
2016-11-23 20:32:42 +00:00
|
|
|
|
2017-03-27 21:34:53 +00:00
|
|
|
return newSampleIterator(input, opt, int(size.Val))
|
2016-11-23 20:32:42 +00:00
|
|
|
case "holt_winters", "holt_winters_with_fit":
|
2017-03-27 21:34:53 +00:00
|
|
|
opt.Ordered = true
|
2017-10-11 14:08:31 +00:00
|
|
|
input, err := buildExprIterator(ctx, expr.Args[0], b.ic, b.sources, opt, b.selector, false)
|
2016-11-23 20:32:42 +00:00
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
2017-08-15 19:24:22 +00:00
|
|
|
h := expr.Args[1].(*influxql.IntegerLiteral)
|
|
|
|
m := expr.Args[2].(*influxql.IntegerLiteral)
|
2016-11-23 20:32:42 +00:00
|
|
|
|
|
|
|
includeFitData := "holt_winters_with_fit" == expr.Name
|
|
|
|
|
2017-04-05 16:47:57 +00:00
|
|
|
interval := opt.Interval.Duration
|
2016-11-23 20:32:42 +00:00
|
|
|
// Redefine interval to be unbounded to capture all aggregate results
|
2017-08-15 19:24:22 +00:00
|
|
|
opt.StartTime = influxql.MinTime
|
|
|
|
opt.EndTime = influxql.MaxTime
|
2016-11-23 20:32:42 +00:00
|
|
|
opt.Interval = Interval{}
|
|
|
|
|
|
|
|
return newHoltWintersIterator(input, opt, int(h.Val), int(m.Val), includeFitData, interval)
|
2017-03-31 02:26:14 +00:00
|
|
|
case "derivative", "non_negative_derivative", "difference", "non_negative_difference", "moving_average", "elapsed":
|
2016-11-23 20:32:42 +00:00
|
|
|
if !opt.Interval.IsZero() {
|
|
|
|
if opt.Ascending {
|
|
|
|
opt.StartTime -= int64(opt.Interval.Duration)
|
|
|
|
} else {
|
|
|
|
opt.EndTime += int64(opt.Interval.Duration)
|
|
|
|
}
|
|
|
|
}
|
2017-03-27 21:34:53 +00:00
|
|
|
opt.Ordered = true
|
2016-11-23 20:32:42 +00:00
|
|
|
|
2017-10-11 14:08:31 +00:00
|
|
|
input, err := buildExprIterator(ctx, expr.Args[0], b.ic, b.sources, opt, b.selector, false)
|
2016-11-23 20:32:42 +00:00
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
|
|
|
switch expr.Name {
|
|
|
|
case "derivative", "non_negative_derivative":
|
|
|
|
interval := opt.DerivativeInterval()
|
|
|
|
isNonNegative := (expr.Name == "non_negative_derivative")
|
|
|
|
return newDerivativeIterator(input, opt, interval, isNonNegative)
|
|
|
|
case "elapsed":
|
|
|
|
interval := opt.ElapsedInterval()
|
|
|
|
return newElapsedIterator(input, opt, interval)
|
2017-03-31 02:26:14 +00:00
|
|
|
case "difference", "non_negative_difference":
|
|
|
|
isNonNegative := (expr.Name == "non_negative_difference")
|
|
|
|
return newDifferenceIterator(input, opt, isNonNegative)
|
2016-11-23 20:32:42 +00:00
|
|
|
case "moving_average":
|
2017-08-15 19:24:22 +00:00
|
|
|
n := expr.Args[1].(*influxql.IntegerLiteral)
|
2017-04-05 16:47:57 +00:00
|
|
|
if n.Val > 1 && !opt.Interval.IsZero() {
|
2016-11-23 20:32:42 +00:00
|
|
|
if opt.Ascending {
|
|
|
|
opt.StartTime -= int64(opt.Interval.Duration) * (n.Val - 1)
|
|
|
|
} else {
|
|
|
|
opt.EndTime += int64(opt.Interval.Duration) * (n.Val - 1)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return newMovingAverageIterator(input, int(n.Val), opt)
|
|
|
|
}
|
|
|
|
panic(fmt.Sprintf("invalid series aggregate function: %s", expr.Name))
|
|
|
|
case "cumulative_sum":
|
2017-03-27 21:34:53 +00:00
|
|
|
opt.Ordered = true
|
2017-10-11 14:08:31 +00:00
|
|
|
input, err := buildExprIterator(ctx, expr.Args[0], b.ic, b.sources, opt, b.selector, false)
|
2016-11-23 20:32:42 +00:00
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
2017-03-27 21:34:53 +00:00
|
|
|
return newCumulativeSumIterator(input, opt)
|
2017-03-23 20:18:03 +00:00
|
|
|
case "integral":
|
|
|
|
opt.Ordered = true
|
2017-10-11 14:08:31 +00:00
|
|
|
input, err := buildExprIterator(ctx, expr.Args[0].(*influxql.VarRef), b.ic, b.sources, opt, false, false)
|
2017-03-23 20:18:03 +00:00
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
interval := opt.IntegralInterval()
|
|
|
|
return newIntegralIterator(input, opt, interval)
|
2017-04-11 15:30:06 +00:00
|
|
|
case "top":
|
|
|
|
if len(expr.Args) < 2 {
|
|
|
|
return nil, fmt.Errorf("top() requires 2 or more arguments, got %d", len(expr.Args))
|
Optimize top() and bottom() using an incremental aggregator
The previous version of `top()` and `bottom()` would gather all of the
points to use in a slice, filter them (if necessary), then use a
slightly modified heap sort to retrieve the top or bottom values.
This performed horrendously from the standpoint of memory. Since it
consumed so much memory and spent so much time in allocations (along
with sorting a potentially very large slice), this affected speed too.
These calls have now been modified so they keep the top or bottom points
in a min or max heap. For `top()`, a new point will read the minimum
value from the heap. If the new point is greater than the minimum point,
it will replace the minimum point and fix the heap with the new value.
If the new point is smaller, it discards that point. For `bottom()`, the
process is the opposite.
It will then sort the final result to ensure the correct ordering of the
selected points.
When `top()` or `bottom()` contain a tag to select, they have now been
modified so this query:
SELECT top(value, host, 2) FROM cpu
Essentially becomes this query:
SELECT top(value, 2), host FROM (
SELECT max(value) FROM cpu GROUP BY host
)
This should drastically increase the performance of all `top()` and
`bottom()` queries.
2017-05-16 17:37:39 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
var input Iterator
|
|
|
|
if len(expr.Args) > 2 {
|
|
|
|
// Create a max iterator using the groupings in the arguments.
|
2017-05-17 16:10:51 +00:00
|
|
|
dims := make(map[string]struct{}, len(expr.Args)-2+len(opt.GroupBy))
|
2017-04-11 15:30:06 +00:00
|
|
|
for i := 1; i < len(expr.Args)-1; i++ {
|
2017-08-15 19:24:22 +00:00
|
|
|
ref := expr.Args[i].(*influxql.VarRef)
|
Optimize top() and bottom() using an incremental aggregator
The previous version of `top()` and `bottom()` would gather all of the
points to use in a slice, filter them (if necessary), then use a
slightly modified heap sort to retrieve the top or bottom values.
This performed horrendously from the standpoint of memory. Since it
consumed so much memory and spent so much time in allocations (along
with sorting a potentially very large slice), this affected speed too.
These calls have now been modified so they keep the top or bottom points
in a min or max heap. For `top()`, a new point will read the minimum
value from the heap. If the new point is greater than the minimum point,
it will replace the minimum point and fix the heap with the new value.
If the new point is smaller, it discards that point. For `bottom()`, the
process is the opposite.
It will then sort the final result to ensure the correct ordering of the
selected points.
When `top()` or `bottom()` contain a tag to select, they have now been
modified so this query:
SELECT top(value, host, 2) FROM cpu
Essentially becomes this query:
SELECT top(value, 2), host FROM (
SELECT max(value) FROM cpu GROUP BY host
)
This should drastically increase the performance of all `top()` and
`bottom()` queries.
2017-05-16 17:37:39 +00:00
|
|
|
dims[ref.Val] = struct{}{}
|
|
|
|
}
|
|
|
|
for dim := range opt.GroupBy {
|
|
|
|
dims[dim] = struct{}{}
|
2017-04-11 15:30:06 +00:00
|
|
|
}
|
|
|
|
|
2017-08-15 19:24:22 +00:00
|
|
|
call := &influxql.Call{
|
Optimize top() and bottom() using an incremental aggregator
The previous version of `top()` and `bottom()` would gather all of the
points to use in a slice, filter them (if necessary), then use a
slightly modified heap sort to retrieve the top or bottom values.
This performed horrendously from the standpoint of memory. Since it
consumed so much memory and spent so much time in allocations (along
with sorting a potentially very large slice), this affected speed too.
These calls have now been modified so they keep the top or bottom points
in a min or max heap. For `top()`, a new point will read the minimum
value from the heap. If the new point is greater than the minimum point,
it will replace the minimum point and fix the heap with the new value.
If the new point is smaller, it discards that point. For `bottom()`, the
process is the opposite.
It will then sort the final result to ensure the correct ordering of the
selected points.
When `top()` or `bottom()` contain a tag to select, they have now been
modified so this query:
SELECT top(value, host, 2) FROM cpu
Essentially becomes this query:
SELECT top(value, 2), host FROM (
SELECT max(value) FROM cpu GROUP BY host
)
This should drastically increase the performance of all `top()` and
`bottom()` queries.
2017-05-16 17:37:39 +00:00
|
|
|
Name: "max",
|
|
|
|
Args: expr.Args[:1],
|
|
|
|
}
|
|
|
|
callOpt := opt
|
|
|
|
callOpt.Expr = call
|
|
|
|
callOpt.GroupBy = dims
|
2017-08-15 19:24:22 +00:00
|
|
|
callOpt.Fill = influxql.NoFill
|
Optimize top() and bottom() using an incremental aggregator
The previous version of `top()` and `bottom()` would gather all of the
points to use in a slice, filter them (if necessary), then use a
slightly modified heap sort to retrieve the top or bottom values.
This performed horrendously from the standpoint of memory. Since it
consumed so much memory and spent so much time in allocations (along
with sorting a potentially very large slice), this affected speed too.
These calls have now been modified so they keep the top or bottom points
in a min or max heap. For `top()`, a new point will read the minimum
value from the heap. If the new point is greater than the minimum point,
it will replace the minimum point and fix the heap with the new value.
If the new point is smaller, it discards that point. For `bottom()`, the
process is the opposite.
It will then sort the final result to ensure the correct ordering of the
selected points.
When `top()` or `bottom()` contain a tag to select, they have now been
modified so this query:
SELECT top(value, host, 2) FROM cpu
Essentially becomes this query:
SELECT top(value, 2), host FROM (
SELECT max(value) FROM cpu GROUP BY host
)
This should drastically increase the performance of all `top()` and
`bottom()` queries.
2017-05-16 17:37:39 +00:00
|
|
|
|
|
|
|
builder := *b
|
|
|
|
builder.opt = callOpt
|
|
|
|
builder.selector = true
|
2017-05-17 16:10:51 +00:00
|
|
|
builder.writeMode = false
|
Optimize top() and bottom() using an incremental aggregator
The previous version of `top()` and `bottom()` would gather all of the
points to use in a slice, filter them (if necessary), then use a
slightly modified heap sort to retrieve the top or bottom values.
This performed horrendously from the standpoint of memory. Since it
consumed so much memory and spent so much time in allocations (along
with sorting a potentially very large slice), this affected speed too.
These calls have now been modified so they keep the top or bottom points
in a min or max heap. For `top()`, a new point will read the minimum
value from the heap. If the new point is greater than the minimum point,
it will replace the minimum point and fix the heap with the new value.
If the new point is smaller, it discards that point. For `bottom()`, the
process is the opposite.
It will then sort the final result to ensure the correct ordering of the
selected points.
When `top()` or `bottom()` contain a tag to select, they have now been
modified so this query:
SELECT top(value, host, 2) FROM cpu
Essentially becomes this query:
SELECT top(value, 2), host FROM (
SELECT max(value) FROM cpu GROUP BY host
)
This should drastically increase the performance of all `top()` and
`bottom()` queries.
2017-05-16 17:37:39 +00:00
|
|
|
|
2017-10-11 14:08:31 +00:00
|
|
|
i, err := builder.callIterator(ctx, call, callOpt)
|
Optimize top() and bottom() using an incremental aggregator
The previous version of `top()` and `bottom()` would gather all of the
points to use in a slice, filter them (if necessary), then use a
slightly modified heap sort to retrieve the top or bottom values.
This performed horrendously from the standpoint of memory. Since it
consumed so much memory and spent so much time in allocations (along
with sorting a potentially very large slice), this affected speed too.
These calls have now been modified so they keep the top or bottom points
in a min or max heap. For `top()`, a new point will read the minimum
value from the heap. If the new point is greater than the minimum point,
it will replace the minimum point and fix the heap with the new value.
If the new point is smaller, it discards that point. For `bottom()`, the
process is the opposite.
It will then sort the final result to ensure the correct ordering of the
selected points.
When `top()` or `bottom()` contain a tag to select, they have now been
modified so this query:
SELECT top(value, host, 2) FROM cpu
Essentially becomes this query:
SELECT top(value, 2), host FROM (
SELECT max(value) FROM cpu GROUP BY host
)
This should drastically increase the performance of all `top()` and
`bottom()` queries.
2017-05-16 17:37:39 +00:00
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
input = i
|
|
|
|
} else {
|
|
|
|
// There are no arguments so do not organize the points by tags.
|
|
|
|
builder := *b
|
|
|
|
builder.opt.Expr = expr.Args[0]
|
|
|
|
builder.selector = true
|
2017-05-17 16:10:51 +00:00
|
|
|
builder.writeMode = false
|
Optimize top() and bottom() using an incremental aggregator
The previous version of `top()` and `bottom()` would gather all of the
points to use in a slice, filter them (if necessary), then use a
slightly modified heap sort to retrieve the top or bottom values.
This performed horrendously from the standpoint of memory. Since it
consumed so much memory and spent so much time in allocations (along
with sorting a potentially very large slice), this affected speed too.
These calls have now been modified so they keep the top or bottom points
in a min or max heap. For `top()`, a new point will read the minimum
value from the heap. If the new point is greater than the minimum point,
it will replace the minimum point and fix the heap with the new value.
If the new point is smaller, it discards that point. For `bottom()`, the
process is the opposite.
It will then sort the final result to ensure the correct ordering of the
selected points.
When `top()` or `bottom()` contain a tag to select, they have now been
modified so this query:
SELECT top(value, host, 2) FROM cpu
Essentially becomes this query:
SELECT top(value, 2), host FROM (
SELECT max(value) FROM cpu GROUP BY host
)
This should drastically increase the performance of all `top()` and
`bottom()` queries.
2017-05-16 17:37:39 +00:00
|
|
|
|
2017-08-15 19:24:22 +00:00
|
|
|
ref := expr.Args[0].(*influxql.VarRef)
|
2017-10-11 14:08:31 +00:00
|
|
|
i, err := builder.buildVarRefIterator(ctx, ref)
|
Optimize top() and bottom() using an incremental aggregator
The previous version of `top()` and `bottom()` would gather all of the
points to use in a slice, filter them (if necessary), then use a
slightly modified heap sort to retrieve the top or bottom values.
This performed horrendously from the standpoint of memory. Since it
consumed so much memory and spent so much time in allocations (along
with sorting a potentially very large slice), this affected speed too.
These calls have now been modified so they keep the top or bottom points
in a min or max heap. For `top()`, a new point will read the minimum
value from the heap. If the new point is greater than the minimum point,
it will replace the minimum point and fix the heap with the new value.
If the new point is smaller, it discards that point. For `bottom()`, the
process is the opposite.
It will then sort the final result to ensure the correct ordering of the
selected points.
When `top()` or `bottom()` contain a tag to select, they have now been
modified so this query:
SELECT top(value, host, 2) FROM cpu
Essentially becomes this query:
SELECT top(value, 2), host FROM (
SELECT max(value) FROM cpu GROUP BY host
)
This should drastically increase the performance of all `top()` and
`bottom()` queries.
2017-05-16 17:37:39 +00:00
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
input = i
|
2017-04-11 15:30:06 +00:00
|
|
|
}
|
Optimize top() and bottom() using an incremental aggregator
The previous version of `top()` and `bottom()` would gather all of the
points to use in a slice, filter them (if necessary), then use a
slightly modified heap sort to retrieve the top or bottom values.
This performed horrendously from the standpoint of memory. Since it
consumed so much memory and spent so much time in allocations (along
with sorting a potentially very large slice), this affected speed too.
These calls have now been modified so they keep the top or bottom points
in a min or max heap. For `top()`, a new point will read the minimum
value from the heap. If the new point is greater than the minimum point,
it will replace the minimum point and fix the heap with the new value.
If the new point is smaller, it discards that point. For `bottom()`, the
process is the opposite.
It will then sort the final result to ensure the correct ordering of the
selected points.
When `top()` or `bottom()` contain a tag to select, they have now been
modified so this query:
SELECT top(value, host, 2) FROM cpu
Essentially becomes this query:
SELECT top(value, 2), host FROM (
SELECT max(value) FROM cpu GROUP BY host
)
This should drastically increase the performance of all `top()` and
`bottom()` queries.
2017-05-16 17:37:39 +00:00
|
|
|
|
2017-08-15 19:24:22 +00:00
|
|
|
n := expr.Args[len(expr.Args)-1].(*influxql.IntegerLiteral)
|
2017-05-17 16:10:51 +00:00
|
|
|
return newTopIterator(input, opt, int(n.Val), b.writeMode)
|
2017-04-11 15:30:06 +00:00
|
|
|
case "bottom":
|
|
|
|
if len(expr.Args) < 2 {
|
|
|
|
return nil, fmt.Errorf("bottom() requires 2 or more arguments, got %d", len(expr.Args))
|
Optimize top() and bottom() using an incremental aggregator
The previous version of `top()` and `bottom()` would gather all of the
points to use in a slice, filter them (if necessary), then use a
slightly modified heap sort to retrieve the top or bottom values.
This performed horrendously from the standpoint of memory. Since it
consumed so much memory and spent so much time in allocations (along
with sorting a potentially very large slice), this affected speed too.
These calls have now been modified so they keep the top or bottom points
in a min or max heap. For `top()`, a new point will read the minimum
value from the heap. If the new point is greater than the minimum point,
it will replace the minimum point and fix the heap with the new value.
If the new point is smaller, it discards that point. For `bottom()`, the
process is the opposite.
It will then sort the final result to ensure the correct ordering of the
selected points.
When `top()` or `bottom()` contain a tag to select, they have now been
modified so this query:
SELECT top(value, host, 2) FROM cpu
Essentially becomes this query:
SELECT top(value, 2), host FROM (
SELECT max(value) FROM cpu GROUP BY host
)
This should drastically increase the performance of all `top()` and
`bottom()` queries.
2017-05-16 17:37:39 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
var input Iterator
|
|
|
|
if len(expr.Args) > 2 {
|
|
|
|
// Create a max iterator using the groupings in the arguments.
|
|
|
|
dims := make(map[string]struct{}, len(expr.Args)-2)
|
2017-04-11 15:30:06 +00:00
|
|
|
for i := 1; i < len(expr.Args)-1; i++ {
|
2017-08-15 19:24:22 +00:00
|
|
|
ref := expr.Args[i].(*influxql.VarRef)
|
Optimize top() and bottom() using an incremental aggregator
The previous version of `top()` and `bottom()` would gather all of the
points to use in a slice, filter them (if necessary), then use a
slightly modified heap sort to retrieve the top or bottom values.
This performed horrendously from the standpoint of memory. Since it
consumed so much memory and spent so much time in allocations (along
with sorting a potentially very large slice), this affected speed too.
These calls have now been modified so they keep the top or bottom points
in a min or max heap. For `top()`, a new point will read the minimum
value from the heap. If the new point is greater than the minimum point,
it will replace the minimum point and fix the heap with the new value.
If the new point is smaller, it discards that point. For `bottom()`, the
process is the opposite.
It will then sort the final result to ensure the correct ordering of the
selected points.
When `top()` or `bottom()` contain a tag to select, they have now been
modified so this query:
SELECT top(value, host, 2) FROM cpu
Essentially becomes this query:
SELECT top(value, 2), host FROM (
SELECT max(value) FROM cpu GROUP BY host
)
This should drastically increase the performance of all `top()` and
`bottom()` queries.
2017-05-16 17:37:39 +00:00
|
|
|
dims[ref.Val] = struct{}{}
|
|
|
|
}
|
|
|
|
for dim := range opt.GroupBy {
|
|
|
|
dims[dim] = struct{}{}
|
2017-04-11 15:30:06 +00:00
|
|
|
}
|
|
|
|
|
2017-08-15 19:24:22 +00:00
|
|
|
call := &influxql.Call{
|
Optimize top() and bottom() using an incremental aggregator
The previous version of `top()` and `bottom()` would gather all of the
points to use in a slice, filter them (if necessary), then use a
slightly modified heap sort to retrieve the top or bottom values.
This performed horrendously from the standpoint of memory. Since it
consumed so much memory and spent so much time in allocations (along
with sorting a potentially very large slice), this affected speed too.
These calls have now been modified so they keep the top or bottom points
in a min or max heap. For `top()`, a new point will read the minimum
value from the heap. If the new point is greater than the minimum point,
it will replace the minimum point and fix the heap with the new value.
If the new point is smaller, it discards that point. For `bottom()`, the
process is the opposite.
It will then sort the final result to ensure the correct ordering of the
selected points.
When `top()` or `bottom()` contain a tag to select, they have now been
modified so this query:
SELECT top(value, host, 2) FROM cpu
Essentially becomes this query:
SELECT top(value, 2), host FROM (
SELECT max(value) FROM cpu GROUP BY host
)
This should drastically increase the performance of all `top()` and
`bottom()` queries.
2017-05-16 17:37:39 +00:00
|
|
|
Name: "min",
|
|
|
|
Args: expr.Args[:1],
|
|
|
|
}
|
|
|
|
callOpt := opt
|
|
|
|
callOpt.Expr = call
|
|
|
|
callOpt.GroupBy = dims
|
2017-08-15 19:24:22 +00:00
|
|
|
callOpt.Fill = influxql.NoFill
|
Optimize top() and bottom() using an incremental aggregator
The previous version of `top()` and `bottom()` would gather all of the
points to use in a slice, filter them (if necessary), then use a
slightly modified heap sort to retrieve the top or bottom values.
This performed horrendously from the standpoint of memory. Since it
consumed so much memory and spent so much time in allocations (along
with sorting a potentially very large slice), this affected speed too.
These calls have now been modified so they keep the top or bottom points
in a min or max heap. For `top()`, a new point will read the minimum
value from the heap. If the new point is greater than the minimum point,
it will replace the minimum point and fix the heap with the new value.
If the new point is smaller, it discards that point. For `bottom()`, the
process is the opposite.
It will then sort the final result to ensure the correct ordering of the
selected points.
When `top()` or `bottom()` contain a tag to select, they have now been
modified so this query:
SELECT top(value, host, 2) FROM cpu
Essentially becomes this query:
SELECT top(value, 2), host FROM (
SELECT max(value) FROM cpu GROUP BY host
)
This should drastically increase the performance of all `top()` and
`bottom()` queries.
2017-05-16 17:37:39 +00:00
|
|
|
|
|
|
|
builder := *b
|
|
|
|
builder.opt = callOpt
|
|
|
|
builder.selector = true
|
2017-05-17 16:10:51 +00:00
|
|
|
builder.writeMode = false
|
Optimize top() and bottom() using an incremental aggregator
The previous version of `top()` and `bottom()` would gather all of the
points to use in a slice, filter them (if necessary), then use a
slightly modified heap sort to retrieve the top or bottom values.
This performed horrendously from the standpoint of memory. Since it
consumed so much memory and spent so much time in allocations (along
with sorting a potentially very large slice), this affected speed too.
These calls have now been modified so they keep the top or bottom points
in a min or max heap. For `top()`, a new point will read the minimum
value from the heap. If the new point is greater than the minimum point,
it will replace the minimum point and fix the heap with the new value.
If the new point is smaller, it discards that point. For `bottom()`, the
process is the opposite.
It will then sort the final result to ensure the correct ordering of the
selected points.
When `top()` or `bottom()` contain a tag to select, they have now been
modified so this query:
SELECT top(value, host, 2) FROM cpu
Essentially becomes this query:
SELECT top(value, 2), host FROM (
SELECT max(value) FROM cpu GROUP BY host
)
This should drastically increase the performance of all `top()` and
`bottom()` queries.
2017-05-16 17:37:39 +00:00
|
|
|
|
2017-10-11 14:08:31 +00:00
|
|
|
i, err := builder.callIterator(ctx, call, callOpt)
|
Optimize top() and bottom() using an incremental aggregator
The previous version of `top()` and `bottom()` would gather all of the
points to use in a slice, filter them (if necessary), then use a
slightly modified heap sort to retrieve the top or bottom values.
This performed horrendously from the standpoint of memory. Since it
consumed so much memory and spent so much time in allocations (along
with sorting a potentially very large slice), this affected speed too.
These calls have now been modified so they keep the top or bottom points
in a min or max heap. For `top()`, a new point will read the minimum
value from the heap. If the new point is greater than the minimum point,
it will replace the minimum point and fix the heap with the new value.
If the new point is smaller, it discards that point. For `bottom()`, the
process is the opposite.
It will then sort the final result to ensure the correct ordering of the
selected points.
When `top()` or `bottom()` contain a tag to select, they have now been
modified so this query:
SELECT top(value, host, 2) FROM cpu
Essentially becomes this query:
SELECT top(value, 2), host FROM (
SELECT max(value) FROM cpu GROUP BY host
)
This should drastically increase the performance of all `top()` and
`bottom()` queries.
2017-05-16 17:37:39 +00:00
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
input = i
|
|
|
|
} else {
|
|
|
|
// There are no arguments so do not organize the points by tags.
|
|
|
|
builder := *b
|
2017-05-30 15:56:16 +00:00
|
|
|
builder.opt.Expr = expr.Args[0]
|
Optimize top() and bottom() using an incremental aggregator
The previous version of `top()` and `bottom()` would gather all of the
points to use in a slice, filter them (if necessary), then use a
slightly modified heap sort to retrieve the top or bottom values.
This performed horrendously from the standpoint of memory. Since it
consumed so much memory and spent so much time in allocations (along
with sorting a potentially very large slice), this affected speed too.
These calls have now been modified so they keep the top or bottom points
in a min or max heap. For `top()`, a new point will read the minimum
value from the heap. If the new point is greater than the minimum point,
it will replace the minimum point and fix the heap with the new value.
If the new point is smaller, it discards that point. For `bottom()`, the
process is the opposite.
It will then sort the final result to ensure the correct ordering of the
selected points.
When `top()` or `bottom()` contain a tag to select, they have now been
modified so this query:
SELECT top(value, host, 2) FROM cpu
Essentially becomes this query:
SELECT top(value, 2), host FROM (
SELECT max(value) FROM cpu GROUP BY host
)
This should drastically increase the performance of all `top()` and
`bottom()` queries.
2017-05-16 17:37:39 +00:00
|
|
|
builder.selector = true
|
2017-05-17 16:10:51 +00:00
|
|
|
builder.writeMode = false
|
Optimize top() and bottom() using an incremental aggregator
The previous version of `top()` and `bottom()` would gather all of the
points to use in a slice, filter them (if necessary), then use a
slightly modified heap sort to retrieve the top or bottom values.
This performed horrendously from the standpoint of memory. Since it
consumed so much memory and spent so much time in allocations (along
with sorting a potentially very large slice), this affected speed too.
These calls have now been modified so they keep the top or bottom points
in a min or max heap. For `top()`, a new point will read the minimum
value from the heap. If the new point is greater than the minimum point,
it will replace the minimum point and fix the heap with the new value.
If the new point is smaller, it discards that point. For `bottom()`, the
process is the opposite.
It will then sort the final result to ensure the correct ordering of the
selected points.
When `top()` or `bottom()` contain a tag to select, they have now been
modified so this query:
SELECT top(value, host, 2) FROM cpu
Essentially becomes this query:
SELECT top(value, 2), host FROM (
SELECT max(value) FROM cpu GROUP BY host
)
This should drastically increase the performance of all `top()` and
`bottom()` queries.
2017-05-16 17:37:39 +00:00
|
|
|
|
2017-08-15 19:24:22 +00:00
|
|
|
ref := expr.Args[0].(*influxql.VarRef)
|
2017-10-11 14:08:31 +00:00
|
|
|
i, err := builder.buildVarRefIterator(ctx, ref)
|
Optimize top() and bottom() using an incremental aggregator
The previous version of `top()` and `bottom()` would gather all of the
points to use in a slice, filter them (if necessary), then use a
slightly modified heap sort to retrieve the top or bottom values.
This performed horrendously from the standpoint of memory. Since it
consumed so much memory and spent so much time in allocations (along
with sorting a potentially very large slice), this affected speed too.
These calls have now been modified so they keep the top or bottom points
in a min or max heap. For `top()`, a new point will read the minimum
value from the heap. If the new point is greater than the minimum point,
it will replace the minimum point and fix the heap with the new value.
If the new point is smaller, it discards that point. For `bottom()`, the
process is the opposite.
It will then sort the final result to ensure the correct ordering of the
selected points.
When `top()` or `bottom()` contain a tag to select, they have now been
modified so this query:
SELECT top(value, host, 2) FROM cpu
Essentially becomes this query:
SELECT top(value, 2), host FROM (
SELECT max(value) FROM cpu GROUP BY host
)
This should drastically increase the performance of all `top()` and
`bottom()` queries.
2017-05-16 17:37:39 +00:00
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
input = i
|
2017-04-11 15:30:06 +00:00
|
|
|
}
|
Optimize top() and bottom() using an incremental aggregator
The previous version of `top()` and `bottom()` would gather all of the
points to use in a slice, filter them (if necessary), then use a
slightly modified heap sort to retrieve the top or bottom values.
This performed horrendously from the standpoint of memory. Since it
consumed so much memory and spent so much time in allocations (along
with sorting a potentially very large slice), this affected speed too.
These calls have now been modified so they keep the top or bottom points
in a min or max heap. For `top()`, a new point will read the minimum
value from the heap. If the new point is greater than the minimum point,
it will replace the minimum point and fix the heap with the new value.
If the new point is smaller, it discards that point. For `bottom()`, the
process is the opposite.
It will then sort the final result to ensure the correct ordering of the
selected points.
When `top()` or `bottom()` contain a tag to select, they have now been
modified so this query:
SELECT top(value, host, 2) FROM cpu
Essentially becomes this query:
SELECT top(value, 2), host FROM (
SELECT max(value) FROM cpu GROUP BY host
)
This should drastically increase the performance of all `top()` and
`bottom()` queries.
2017-05-16 17:37:39 +00:00
|
|
|
|
2017-08-15 19:24:22 +00:00
|
|
|
n := expr.Args[len(expr.Args)-1].(*influxql.IntegerLiteral)
|
2017-05-17 16:10:51 +00:00
|
|
|
return newBottomIterator(input, b.opt, int(n.Val), b.writeMode)
|
2016-11-23 20:32:42 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
itr, err := func() (Iterator, error) {
|
|
|
|
switch expr.Name {
|
|
|
|
case "count":
|
|
|
|
switch arg0 := expr.Args[0].(type) {
|
2017-08-15 19:24:22 +00:00
|
|
|
case *influxql.Call:
|
2016-11-23 20:32:42 +00:00
|
|
|
if arg0.Name == "distinct" {
|
2017-10-11 14:08:31 +00:00
|
|
|
input, err := buildExprIterator(ctx, arg0, b.ic, b.sources, opt, b.selector, false)
|
2016-02-22 17:51:45 +00:00
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
2017-04-05 16:47:57 +00:00
|
|
|
return newCountIterator(input, opt)
|
2016-11-23 20:32:42 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
fallthrough
|
|
|
|
case "min", "max", "sum", "first", "last", "mean":
|
2017-10-11 14:08:31 +00:00
|
|
|
return b.callIterator(ctx, expr, opt)
|
2016-11-23 20:32:42 +00:00
|
|
|
case "median":
|
|
|
|
opt.Ordered = true
|
2017-10-11 14:08:31 +00:00
|
|
|
input, err := buildExprIterator(ctx, expr.Args[0].(*influxql.VarRef), b.ic, b.sources, opt, false, false)
|
2016-11-23 20:32:42 +00:00
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
2016-02-22 17:51:45 +00:00
|
|
|
}
|
2016-11-23 20:32:42 +00:00
|
|
|
return newMedianIterator(input, opt)
|
|
|
|
case "mode":
|
2017-10-11 14:08:31 +00:00
|
|
|
input, err := buildExprIterator(ctx, expr.Args[0].(*influxql.VarRef), b.ic, b.sources, opt, false, false)
|
2016-11-23 20:32:42 +00:00
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
2016-03-21 16:00:07 +00:00
|
|
|
}
|
2017-04-05 16:47:57 +00:00
|
|
|
return NewModeIterator(input, opt)
|
2016-11-23 20:32:42 +00:00
|
|
|
case "stddev":
|
2017-10-11 14:08:31 +00:00
|
|
|
input, err := buildExprIterator(ctx, expr.Args[0].(*influxql.VarRef), b.ic, b.sources, opt, false, false)
|
2016-11-23 20:32:42 +00:00
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
2015-11-04 21:06:06 +00:00
|
|
|
}
|
2017-04-05 16:47:57 +00:00
|
|
|
return newStddevIterator(input, opt)
|
2016-11-23 20:32:42 +00:00
|
|
|
case "spread":
|
|
|
|
// OPTIMIZE(benbjohnson): convert to map/reduce
|
2017-10-11 14:08:31 +00:00
|
|
|
input, err := buildExprIterator(ctx, expr.Args[0].(*influxql.VarRef), b.ic, b.sources, opt, false, false)
|
2015-11-04 21:06:06 +00:00
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
2017-04-05 16:47:57 +00:00
|
|
|
return newSpreadIterator(input, opt)
|
2016-11-23 20:32:42 +00:00
|
|
|
case "percentile":
|
|
|
|
opt.Ordered = true
|
2017-10-11 14:08:31 +00:00
|
|
|
input, err := buildExprIterator(ctx, expr.Args[0].(*influxql.VarRef), b.ic, b.sources, opt, false, false)
|
2015-11-04 21:06:06 +00:00
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
2016-11-23 20:32:42 +00:00
|
|
|
var percentile float64
|
|
|
|
switch arg := expr.Args[1].(type) {
|
2017-08-15 19:24:22 +00:00
|
|
|
case *influxql.NumberLiteral:
|
2016-11-23 20:32:42 +00:00
|
|
|
percentile = arg.Val
|
2017-08-15 19:24:22 +00:00
|
|
|
case *influxql.IntegerLiteral:
|
2016-11-23 20:32:42 +00:00
|
|
|
percentile = float64(arg.Val)
|
|
|
|
}
|
|
|
|
return newPercentileIterator(input, opt, percentile)
|
|
|
|
default:
|
|
|
|
return nil, fmt.Errorf("unsupported call: %s", expr.Name)
|
2015-11-04 21:06:06 +00:00
|
|
|
}
|
2016-11-23 20:32:42 +00:00
|
|
|
}()
|
|
|
|
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
2017-04-05 16:47:57 +00:00
|
|
|
if !b.selector || !opt.Interval.IsZero() {
|
|
|
|
itr = NewIntervalIterator(itr, opt)
|
2017-08-15 19:24:22 +00:00
|
|
|
if !opt.Interval.IsZero() && opt.Fill != influxql.NoFill {
|
2017-04-05 16:47:57 +00:00
|
|
|
itr = NewFillIterator(itr, expr, opt)
|
2016-11-23 20:32:42 +00:00
|
|
|
}
|
|
|
|
}
|
2017-04-05 16:47:57 +00:00
|
|
|
if opt.InterruptCh != nil {
|
|
|
|
itr = NewInterruptIterator(itr, opt.InterruptCh)
|
2016-11-23 20:32:42 +00:00
|
|
|
}
|
|
|
|
return itr, nil
|
|
|
|
}
|
|
|
|
|
2017-10-11 14:08:31 +00:00
|
|
|
func (b *exprIteratorBuilder) buildBinaryExprIterator(ctx context.Context, expr *influxql.BinaryExpr) (Iterator, error) {
|
2017-08-15 19:24:22 +00:00
|
|
|
if rhs, ok := expr.RHS.(influxql.Literal); ok {
|
2016-11-23 20:32:42 +00:00
|
|
|
// The right hand side is a literal. It is more common to have the RHS be a literal,
|
|
|
|
// so we check that one first and have this be the happy path.
|
2017-08-15 19:24:22 +00:00
|
|
|
if lhs, ok := expr.LHS.(influxql.Literal); ok {
|
2016-11-23 20:32:42 +00:00
|
|
|
// We have two literals that couldn't be combined by Reduce.
|
|
|
|
return nil, fmt.Errorf("unable to construct an iterator from two literals: LHS: %T, RHS: %T", lhs, rhs)
|
|
|
|
}
|
|
|
|
|
2017-10-11 14:08:31 +00:00
|
|
|
lhs, err := buildExprIterator(ctx, expr.LHS, b.ic, b.sources, b.opt, b.selector, false)
|
2016-11-23 20:32:42 +00:00
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
return buildRHSTransformIterator(lhs, rhs, expr.Op, b.opt)
|
2017-08-15 19:24:22 +00:00
|
|
|
} else if lhs, ok := expr.LHS.(influxql.Literal); ok {
|
2017-10-11 14:08:31 +00:00
|
|
|
rhs, err := buildExprIterator(ctx, expr.RHS, b.ic, b.sources, b.opt, b.selector, false)
|
2016-11-23 20:32:42 +00:00
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
return buildLHSTransformIterator(lhs, rhs, expr.Op, b.opt)
|
|
|
|
} else {
|
|
|
|
// We have two iterators. Combine them into a single iterator.
|
2017-10-11 14:08:31 +00:00
|
|
|
lhs, err := buildExprIterator(ctx, expr.LHS, b.ic, b.sources, b.opt, false, false)
|
2016-11-23 20:32:42 +00:00
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
2017-10-11 14:08:31 +00:00
|
|
|
rhs, err := buildExprIterator(ctx, expr.RHS, b.ic, b.sources, b.opt, false, false)
|
2016-11-23 20:32:42 +00:00
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
return buildTransformIterator(lhs, rhs, expr.Op, b.opt)
|
2015-11-04 21:06:06 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2017-10-11 14:08:31 +00:00
|
|
|
func (b *exprIteratorBuilder) callIterator(ctx context.Context, expr *influxql.Call, opt IteratorOptions) (Iterator, error) {
|
Optimize top() and bottom() using an incremental aggregator
The previous version of `top()` and `bottom()` would gather all of the
points to use in a slice, filter them (if necessary), then use a
slightly modified heap sort to retrieve the top or bottom values.
This performed horrendously from the standpoint of memory. Since it
consumed so much memory and spent so much time in allocations (along
with sorting a potentially very large slice), this affected speed too.
These calls have now been modified so they keep the top or bottom points
in a min or max heap. For `top()`, a new point will read the minimum
value from the heap. If the new point is greater than the minimum point,
it will replace the minimum point and fix the heap with the new value.
If the new point is smaller, it discards that point. For `bottom()`, the
process is the opposite.
It will then sort the final result to ensure the correct ordering of the
selected points.
When `top()` or `bottom()` contain a tag to select, they have now been
modified so this query:
SELECT top(value, host, 2) FROM cpu
Essentially becomes this query:
SELECT top(value, 2), host FROM (
SELECT max(value) FROM cpu GROUP BY host
)
This should drastically increase the performance of all `top()` and
`bottom()` queries.
2017-05-16 17:37:39 +00:00
|
|
|
inputs := make([]Iterator, 0, len(b.sources))
|
|
|
|
if err := func() error {
|
|
|
|
for _, source := range b.sources {
|
|
|
|
switch source := source.(type) {
|
2017-08-15 19:24:22 +00:00
|
|
|
case *influxql.Measurement:
|
2017-10-11 14:08:31 +00:00
|
|
|
input, err := b.ic.CreateIterator(ctx, source, opt)
|
Optimize top() and bottom() using an incremental aggregator
The previous version of `top()` and `bottom()` would gather all of the
points to use in a slice, filter them (if necessary), then use a
slightly modified heap sort to retrieve the top or bottom values.
This performed horrendously from the standpoint of memory. Since it
consumed so much memory and spent so much time in allocations (along
with sorting a potentially very large slice), this affected speed too.
These calls have now been modified so they keep the top or bottom points
in a min or max heap. For `top()`, a new point will read the minimum
value from the heap. If the new point is greater than the minimum point,
it will replace the minimum point and fix the heap with the new value.
If the new point is smaller, it discards that point. For `bottom()`, the
process is the opposite.
It will then sort the final result to ensure the correct ordering of the
selected points.
When `top()` or `bottom()` contain a tag to select, they have now been
modified so this query:
SELECT top(value, host, 2) FROM cpu
Essentially becomes this query:
SELECT top(value, 2), host FROM (
SELECT max(value) FROM cpu GROUP BY host
)
This should drastically increase the performance of all `top()` and
`bottom()` queries.
2017-05-16 17:37:39 +00:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
inputs = append(inputs, input)
|
2017-08-15 19:24:22 +00:00
|
|
|
case *influxql.SubQuery:
|
Optimize top() and bottom() using an incremental aggregator
The previous version of `top()` and `bottom()` would gather all of the
points to use in a slice, filter them (if necessary), then use a
slightly modified heap sort to retrieve the top or bottom values.
This performed horrendously from the standpoint of memory. Since it
consumed so much memory and spent so much time in allocations (along
with sorting a potentially very large slice), this affected speed too.
These calls have now been modified so they keep the top or bottom points
in a min or max heap. For `top()`, a new point will read the minimum
value from the heap. If the new point is greater than the minimum point,
it will replace the minimum point and fix the heap with the new value.
If the new point is smaller, it discards that point. For `bottom()`, the
process is the opposite.
It will then sort the final result to ensure the correct ordering of the
selected points.
When `top()` or `bottom()` contain a tag to select, they have now been
modified so this query:
SELECT top(value, host, 2) FROM cpu
Essentially becomes this query:
SELECT top(value, 2), host FROM (
SELECT max(value) FROM cpu GROUP BY host
)
This should drastically increase the performance of all `top()` and
`bottom()` queries.
2017-05-16 17:37:39 +00:00
|
|
|
// Identify the name of the field we are using.
|
2017-08-15 19:24:22 +00:00
|
|
|
arg0 := expr.Args[0].(*influxql.VarRef)
|
Optimize top() and bottom() using an incremental aggregator
The previous version of `top()` and `bottom()` would gather all of the
points to use in a slice, filter them (if necessary), then use a
slightly modified heap sort to retrieve the top or bottom values.
This performed horrendously from the standpoint of memory. Since it
consumed so much memory and spent so much time in allocations (along
with sorting a potentially very large slice), this affected speed too.
These calls have now been modified so they keep the top or bottom points
in a min or max heap. For `top()`, a new point will read the minimum
value from the heap. If the new point is greater than the minimum point,
it will replace the minimum point and fix the heap with the new value.
If the new point is smaller, it discards that point. For `bottom()`, the
process is the opposite.
It will then sort the final result to ensure the correct ordering of the
selected points.
When `top()` or `bottom()` contain a tag to select, they have now been
modified so this query:
SELECT top(value, host, 2) FROM cpu
Essentially becomes this query:
SELECT top(value, 2), host FROM (
SELECT max(value) FROM cpu GROUP BY host
)
This should drastically increase the performance of all `top()` and
`bottom()` queries.
2017-05-16 17:37:39 +00:00
|
|
|
|
2017-10-11 14:08:31 +00:00
|
|
|
input, err := buildExprIterator(ctx, arg0, b.ic, []influxql.Source{source}, opt, b.selector, false)
|
Optimize top() and bottom() using an incremental aggregator
The previous version of `top()` and `bottom()` would gather all of the
points to use in a slice, filter them (if necessary), then use a
slightly modified heap sort to retrieve the top or bottom values.
This performed horrendously from the standpoint of memory. Since it
consumed so much memory and spent so much time in allocations (along
with sorting a potentially very large slice), this affected speed too.
These calls have now been modified so they keep the top or bottom points
in a min or max heap. For `top()`, a new point will read the minimum
value from the heap. If the new point is greater than the minimum point,
it will replace the minimum point and fix the heap with the new value.
If the new point is smaller, it discards that point. For `bottom()`, the
process is the opposite.
It will then sort the final result to ensure the correct ordering of the
selected points.
When `top()` or `bottom()` contain a tag to select, they have now been
modified so this query:
SELECT top(value, host, 2) FROM cpu
Essentially becomes this query:
SELECT top(value, 2), host FROM (
SELECT max(value) FROM cpu GROUP BY host
)
This should drastically increase the performance of all `top()` and
`bottom()` queries.
2017-05-16 17:37:39 +00:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
|
|
|
// Wrap the result in a call iterator.
|
|
|
|
i, err := NewCallIterator(input, opt)
|
|
|
|
if err != nil {
|
|
|
|
input.Close()
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
inputs = append(inputs, i)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}(); err != nil {
|
|
|
|
Iterators(inputs).Close()
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
|
|
|
itr, err := Iterators(inputs).Merge(opt)
|
|
|
|
if err != nil {
|
|
|
|
Iterators(inputs).Close()
|
|
|
|
return nil, err
|
|
|
|
} else if itr == nil {
|
|
|
|
itr = &nilFloatIterator{}
|
|
|
|
}
|
|
|
|
return itr, nil
|
|
|
|
}
|
|
|
|
|
2017-08-15 19:24:22 +00:00
|
|
|
func buildRHSTransformIterator(lhs Iterator, rhs influxql.Literal, op influxql.Token, opt IteratorOptions) (Iterator, error) {
|
2017-10-02 17:31:07 +00:00
|
|
|
itrType, litType := iteratorDataType(lhs), literalDataType(rhs)
|
|
|
|
if litType == influxql.Unsigned && itrType == influxql.Integer {
|
|
|
|
// If the literal is unsigned but the iterator is an integer, return
|
|
|
|
// an error since we cannot add an unsigned to an integer.
|
|
|
|
return nil, fmt.Errorf("cannot use %s with an integer and unsigned", op)
|
|
|
|
}
|
|
|
|
|
2016-02-18 21:37:03 +00:00
|
|
|
fn := binaryExprFunc(iteratorDataType(lhs), literalDataType(rhs), op)
|
2015-11-04 21:06:06 +00:00
|
|
|
switch fn := fn.(type) {
|
|
|
|
case func(float64, float64) float64:
|
2016-02-18 21:37:03 +00:00
|
|
|
var input FloatIterator
|
|
|
|
switch lhs := lhs.(type) {
|
|
|
|
case FloatIterator:
|
|
|
|
input = lhs
|
|
|
|
case IntegerIterator:
|
|
|
|
input = &integerFloatCastIterator{input: lhs}
|
2017-10-02 17:31:07 +00:00
|
|
|
case UnsignedIterator:
|
|
|
|
input = &unsignedFloatCastIterator{input: lhs}
|
2016-02-18 21:37:03 +00:00
|
|
|
default:
|
|
|
|
return nil, fmt.Errorf("type mismatch on LHS, unable to use %T as a FloatIterator", lhs)
|
2015-11-04 21:06:06 +00:00
|
|
|
}
|
2016-02-18 21:37:03 +00:00
|
|
|
|
2016-02-19 04:06:41 +00:00
|
|
|
var val float64
|
|
|
|
switch rhs := rhs.(type) {
|
2017-08-15 19:24:22 +00:00
|
|
|
case *influxql.NumberLiteral:
|
2016-02-19 04:06:41 +00:00
|
|
|
val = rhs.Val
|
2017-08-15 19:24:22 +00:00
|
|
|
case *influxql.IntegerLiteral:
|
2016-02-19 04:06:41 +00:00
|
|
|
val = float64(rhs.Val)
|
2017-10-02 17:31:07 +00:00
|
|
|
case *influxql.UnsignedLiteral:
|
|
|
|
val = float64(rhs.Val)
|
2016-02-19 04:06:41 +00:00
|
|
|
default:
|
|
|
|
return nil, fmt.Errorf("type mismatch on RHS, unable to use %T as a NumberLiteral", rhs)
|
2015-11-04 21:06:06 +00:00
|
|
|
}
|
|
|
|
return &floatTransformIterator{
|
|
|
|
input: input,
|
|
|
|
fn: func(p *FloatPoint) *FloatPoint {
|
|
|
|
if p == nil {
|
|
|
|
return nil
|
2016-03-11 15:58:16 +00:00
|
|
|
} else if p.Nil {
|
|
|
|
return p
|
2015-11-04 21:06:06 +00:00
|
|
|
}
|
2016-02-19 04:06:41 +00:00
|
|
|
p.Value = fn(p.Value, val)
|
2015-11-04 21:06:06 +00:00
|
|
|
return p
|
|
|
|
},
|
|
|
|
}, nil
|
2016-02-19 04:06:41 +00:00
|
|
|
case func(int64, int64) float64:
|
|
|
|
input, ok := lhs.(IntegerIterator)
|
|
|
|
if !ok {
|
|
|
|
return nil, fmt.Errorf("type mismatch on LHS, unable to use %T as a IntegerIterator", lhs)
|
|
|
|
}
|
|
|
|
|
|
|
|
var val int64
|
|
|
|
switch rhs := rhs.(type) {
|
2017-08-15 19:24:22 +00:00
|
|
|
case *influxql.IntegerLiteral:
|
2016-02-19 04:06:41 +00:00
|
|
|
val = rhs.Val
|
|
|
|
default:
|
|
|
|
return nil, fmt.Errorf("type mismatch on RHS, unable to use %T as a IntegerLiteral", rhs)
|
|
|
|
}
|
|
|
|
return &integerFloatTransformIterator{
|
|
|
|
input: input,
|
|
|
|
fn: func(p *IntegerPoint) *FloatPoint {
|
|
|
|
if p == nil {
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
fp := &FloatPoint{
|
|
|
|
Name: p.Name,
|
|
|
|
Tags: p.Tags,
|
|
|
|
Time: p.Time,
|
|
|
|
Aux: p.Aux,
|
|
|
|
}
|
|
|
|
if p.Nil {
|
|
|
|
fp.Nil = true
|
|
|
|
} else {
|
|
|
|
fp.Value = fn(p.Value, val)
|
|
|
|
}
|
|
|
|
return fp
|
|
|
|
},
|
|
|
|
}, nil
|
2015-11-04 21:06:06 +00:00
|
|
|
case func(float64, float64) bool:
|
2016-02-18 21:37:03 +00:00
|
|
|
var input FloatIterator
|
|
|
|
switch lhs := lhs.(type) {
|
|
|
|
case FloatIterator:
|
|
|
|
input = lhs
|
|
|
|
case IntegerIterator:
|
|
|
|
input = &integerFloatCastIterator{input: lhs}
|
2017-10-02 17:31:07 +00:00
|
|
|
case UnsignedIterator:
|
|
|
|
input = &unsignedFloatCastIterator{input: lhs}
|
2016-02-18 21:37:03 +00:00
|
|
|
default:
|
|
|
|
return nil, fmt.Errorf("type mismatch on LHS, unable to use %T as a FloatIterator", lhs)
|
2015-11-04 21:06:06 +00:00
|
|
|
}
|
2016-02-18 21:37:03 +00:00
|
|
|
|
2016-02-19 04:06:41 +00:00
|
|
|
var val float64
|
|
|
|
switch rhs := rhs.(type) {
|
2017-08-15 19:24:22 +00:00
|
|
|
case *influxql.NumberLiteral:
|
2016-02-19 04:06:41 +00:00
|
|
|
val = rhs.Val
|
2017-08-15 19:24:22 +00:00
|
|
|
case *influxql.IntegerLiteral:
|
2016-02-19 04:06:41 +00:00
|
|
|
val = float64(rhs.Val)
|
2017-10-02 17:31:07 +00:00
|
|
|
case *influxql.UnsignedLiteral:
|
|
|
|
val = float64(rhs.Val)
|
2016-02-19 04:06:41 +00:00
|
|
|
default:
|
|
|
|
return nil, fmt.Errorf("type mismatch on RHS, unable to use %T as a NumberLiteral", rhs)
|
2015-11-04 21:06:06 +00:00
|
|
|
}
|
|
|
|
return &floatBoolTransformIterator{
|
|
|
|
input: input,
|
|
|
|
fn: func(p *FloatPoint) *BooleanPoint {
|
|
|
|
if p == nil {
|
|
|
|
return nil
|
|
|
|
}
|
2016-03-11 15:58:16 +00:00
|
|
|
|
|
|
|
bp := &BooleanPoint{
|
|
|
|
Name: p.Name,
|
|
|
|
Tags: p.Tags,
|
|
|
|
Time: p.Time,
|
|
|
|
Aux: p.Aux,
|
|
|
|
}
|
|
|
|
if p.Nil {
|
|
|
|
bp.Nil = true
|
|
|
|
} else {
|
2016-02-19 04:06:41 +00:00
|
|
|
bp.Value = fn(p.Value, val)
|
|
|
|
}
|
|
|
|
return bp
|
|
|
|
},
|
|
|
|
}, nil
|
|
|
|
case func(int64, int64) int64:
|
|
|
|
var input IntegerIterator
|
|
|
|
switch lhs := lhs.(type) {
|
|
|
|
case IntegerIterator:
|
|
|
|
input = lhs
|
|
|
|
default:
|
|
|
|
return nil, fmt.Errorf("type mismatch on LHS, unable to use %T as an IntegerIterator", lhs)
|
|
|
|
}
|
|
|
|
|
|
|
|
var val int64
|
|
|
|
switch rhs := rhs.(type) {
|
2017-08-15 19:24:22 +00:00
|
|
|
case *influxql.IntegerLiteral:
|
2016-02-19 04:06:41 +00:00
|
|
|
val = rhs.Val
|
|
|
|
default:
|
|
|
|
return nil, fmt.Errorf("type mismatch on RHS, unable to use %T as an IntegerLiteral", rhs)
|
|
|
|
}
|
|
|
|
return &integerTransformIterator{
|
|
|
|
input: input,
|
|
|
|
fn: func(p *IntegerPoint) *IntegerPoint {
|
|
|
|
if p == nil {
|
|
|
|
return nil
|
|
|
|
} else if p.Nil {
|
|
|
|
return p
|
|
|
|
}
|
|
|
|
p.Value = fn(p.Value, val)
|
|
|
|
return p
|
|
|
|
},
|
|
|
|
}, nil
|
|
|
|
case func(int64, int64) bool:
|
|
|
|
var input IntegerIterator
|
|
|
|
switch lhs := lhs.(type) {
|
|
|
|
case IntegerIterator:
|
|
|
|
input = lhs
|
|
|
|
default:
|
|
|
|
return nil, fmt.Errorf("type mismatch on LHS, unable to use %T as an IntegerIterator", lhs)
|
|
|
|
}
|
|
|
|
|
|
|
|
var val int64
|
|
|
|
switch rhs := rhs.(type) {
|
2017-08-15 19:24:22 +00:00
|
|
|
case *influxql.IntegerLiteral:
|
2016-02-19 04:06:41 +00:00
|
|
|
val = rhs.Val
|
|
|
|
default:
|
|
|
|
return nil, fmt.Errorf("type mismatch on RHS, unable to use %T as an IntegerLiteral", rhs)
|
|
|
|
}
|
|
|
|
return &integerBoolTransformIterator{
|
|
|
|
input: input,
|
|
|
|
fn: func(p *IntegerPoint) *BooleanPoint {
|
|
|
|
if p == nil {
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2017-10-02 17:31:07 +00:00
|
|
|
bp := &BooleanPoint{
|
|
|
|
Name: p.Name,
|
|
|
|
Tags: p.Tags,
|
|
|
|
Time: p.Time,
|
|
|
|
Aux: p.Aux,
|
|
|
|
}
|
|
|
|
if p.Nil {
|
|
|
|
bp.Nil = true
|
|
|
|
} else {
|
|
|
|
bp.Value = fn(p.Value, val)
|
|
|
|
}
|
|
|
|
return bp
|
|
|
|
},
|
|
|
|
}, nil
|
|
|
|
case func(uint64, uint64) uint64:
|
|
|
|
var input UnsignedIterator
|
|
|
|
switch lhs := lhs.(type) {
|
|
|
|
case UnsignedIterator:
|
|
|
|
input = lhs
|
|
|
|
default:
|
|
|
|
return nil, fmt.Errorf("type mismatch on LHS, unable to use %T as a FloatIterator", lhs)
|
|
|
|
}
|
|
|
|
|
|
|
|
var val uint64
|
|
|
|
switch rhs := rhs.(type) {
|
|
|
|
case *influxql.IntegerLiteral:
|
|
|
|
if rhs.Val < 0 {
|
|
|
|
return nil, fmt.Errorf("cannot use negative integer '%s' in math with unsigned", rhs)
|
|
|
|
}
|
|
|
|
val = uint64(rhs.Val)
|
|
|
|
case *influxql.UnsignedLiteral:
|
|
|
|
val = rhs.Val
|
|
|
|
default:
|
|
|
|
return nil, fmt.Errorf("type mismatch on RHS, unable to use %T as a NumberLiteral", rhs)
|
|
|
|
}
|
|
|
|
return &unsignedTransformIterator{
|
|
|
|
input: input,
|
|
|
|
fn: func(p *UnsignedPoint) *UnsignedPoint {
|
|
|
|
if p == nil {
|
|
|
|
return nil
|
|
|
|
} else if p.Nil {
|
|
|
|
return p
|
|
|
|
}
|
|
|
|
p.Value = fn(p.Value, val)
|
|
|
|
return p
|
|
|
|
},
|
|
|
|
}, nil
|
|
|
|
case func(uint64, uint64) bool:
|
|
|
|
var input UnsignedIterator
|
|
|
|
switch lhs := lhs.(type) {
|
|
|
|
case UnsignedIterator:
|
|
|
|
input = lhs
|
|
|
|
default:
|
|
|
|
return nil, fmt.Errorf("type mismatch on LHS, unable to use %T as a FloatIterator", lhs)
|
|
|
|
}
|
|
|
|
|
|
|
|
var val uint64
|
|
|
|
switch rhs := rhs.(type) {
|
|
|
|
case *influxql.IntegerLiteral:
|
|
|
|
if rhs.Val < 0 {
|
|
|
|
return nil, fmt.Errorf("cannot use negative integer '%s' in math with unsigned", rhs)
|
|
|
|
}
|
|
|
|
val = uint64(rhs.Val)
|
|
|
|
case *influxql.UnsignedLiteral:
|
|
|
|
val = rhs.Val
|
|
|
|
default:
|
|
|
|
return nil, fmt.Errorf("type mismatch on RHS, unable to use %T as a NumberLiteral", rhs)
|
|
|
|
}
|
|
|
|
return &unsignedBoolTransformIterator{
|
|
|
|
input: input,
|
|
|
|
fn: func(p *UnsignedPoint) *BooleanPoint {
|
|
|
|
if p == nil {
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2017-03-19 09:58:09 +00:00
|
|
|
bp := &BooleanPoint{
|
|
|
|
Name: p.Name,
|
|
|
|
Tags: p.Tags,
|
|
|
|
Time: p.Time,
|
|
|
|
Aux: p.Aux,
|
|
|
|
}
|
|
|
|
if p.Nil {
|
|
|
|
bp.Nil = true
|
|
|
|
} else {
|
|
|
|
bp.Value = fn(p.Value, val)
|
|
|
|
}
|
|
|
|
return bp
|
|
|
|
},
|
|
|
|
}, nil
|
|
|
|
case func(bool, bool) bool:
|
|
|
|
var input BooleanIterator
|
|
|
|
switch lhs := lhs.(type) {
|
|
|
|
case BooleanIterator:
|
|
|
|
input = lhs
|
|
|
|
default:
|
|
|
|
return nil, fmt.Errorf("type mismatch on LHS, unable to use %T as an BooleanIterator", lhs)
|
|
|
|
}
|
|
|
|
|
|
|
|
var val bool
|
|
|
|
switch rhs := rhs.(type) {
|
2017-08-15 19:24:22 +00:00
|
|
|
case *influxql.BooleanLiteral:
|
2017-03-19 09:58:09 +00:00
|
|
|
val = rhs.Val
|
|
|
|
default:
|
|
|
|
return nil, fmt.Errorf("type mismatch on RHS, unable to use %T as an BooleanLiteral", rhs)
|
|
|
|
}
|
|
|
|
return &booleanTransformIterator{
|
|
|
|
input: input,
|
|
|
|
fn: func(p *BooleanPoint) *BooleanPoint {
|
|
|
|
if p == nil {
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2016-02-19 04:06:41 +00:00
|
|
|
bp := &BooleanPoint{
|
|
|
|
Name: p.Name,
|
|
|
|
Tags: p.Tags,
|
|
|
|
Time: p.Time,
|
|
|
|
Aux: p.Aux,
|
|
|
|
}
|
|
|
|
if p.Nil {
|
|
|
|
bp.Nil = true
|
|
|
|
} else {
|
|
|
|
bp.Value = fn(p.Value, val)
|
2015-11-04 21:06:06 +00:00
|
|
|
}
|
2016-03-11 15:58:16 +00:00
|
|
|
return bp
|
2015-11-04 21:06:06 +00:00
|
|
|
},
|
|
|
|
}, nil
|
|
|
|
}
|
2016-01-18 22:48:49 +00:00
|
|
|
return nil, fmt.Errorf("unable to construct rhs transform iterator from %T and %T", lhs, rhs)
|
2015-11-04 21:06:06 +00:00
|
|
|
}
|
|
|
|
|
2017-08-15 19:24:22 +00:00
|
|
|
func buildLHSTransformIterator(lhs influxql.Literal, rhs Iterator, op influxql.Token, opt IteratorOptions) (Iterator, error) {
|
2017-10-02 17:31:07 +00:00
|
|
|
litType, itrType := literalDataType(lhs), iteratorDataType(rhs)
|
|
|
|
if litType == influxql.Unsigned && itrType == influxql.Integer {
|
|
|
|
// If the literal is unsigned but the iterator is an integer, return
|
|
|
|
// an error since we cannot add an unsigned to an integer.
|
|
|
|
return nil, fmt.Errorf("cannot use %s with unsigned and an integer", op)
|
|
|
|
}
|
|
|
|
|
|
|
|
fn := binaryExprFunc(litType, itrType, op)
|
2015-11-04 21:06:06 +00:00
|
|
|
switch fn := fn.(type) {
|
|
|
|
case func(float64, float64) float64:
|
2016-02-18 21:37:03 +00:00
|
|
|
var input FloatIterator
|
|
|
|
switch rhs := rhs.(type) {
|
|
|
|
case FloatIterator:
|
|
|
|
input = rhs
|
|
|
|
case IntegerIterator:
|
|
|
|
input = &integerFloatCastIterator{input: rhs}
|
2017-10-02 17:31:07 +00:00
|
|
|
case UnsignedIterator:
|
|
|
|
input = &unsignedFloatCastIterator{input: rhs}
|
2016-02-18 21:37:03 +00:00
|
|
|
default:
|
|
|
|
return nil, fmt.Errorf("type mismatch on RHS, unable to use %T as a FloatIterator", rhs)
|
2015-11-04 21:06:06 +00:00
|
|
|
}
|
2016-02-18 21:37:03 +00:00
|
|
|
|
2016-02-19 04:06:41 +00:00
|
|
|
var val float64
|
|
|
|
switch lhs := lhs.(type) {
|
2017-08-15 19:24:22 +00:00
|
|
|
case *influxql.NumberLiteral:
|
2016-02-19 04:06:41 +00:00
|
|
|
val = lhs.Val
|
2017-08-15 19:24:22 +00:00
|
|
|
case *influxql.IntegerLiteral:
|
2016-02-19 04:06:41 +00:00
|
|
|
val = float64(lhs.Val)
|
2017-10-02 17:31:07 +00:00
|
|
|
case *influxql.UnsignedLiteral:
|
|
|
|
val = float64(lhs.Val)
|
2016-02-19 04:06:41 +00:00
|
|
|
default:
|
2016-02-18 21:37:03 +00:00
|
|
|
return nil, fmt.Errorf("type mismatch on LHS, unable to use %T as a NumberLiteral", lhs)
|
2015-11-04 21:06:06 +00:00
|
|
|
}
|
|
|
|
return &floatTransformIterator{
|
|
|
|
input: input,
|
|
|
|
fn: func(p *FloatPoint) *FloatPoint {
|
|
|
|
if p == nil {
|
|
|
|
return nil
|
2016-03-11 15:58:16 +00:00
|
|
|
} else if p.Nil {
|
|
|
|
return p
|
2015-11-04 21:06:06 +00:00
|
|
|
}
|
2016-02-19 04:06:41 +00:00
|
|
|
p.Value = fn(val, p.Value)
|
2015-11-04 21:06:06 +00:00
|
|
|
return p
|
|
|
|
},
|
|
|
|
}, nil
|
2016-02-19 04:06:41 +00:00
|
|
|
case func(int64, int64) float64:
|
|
|
|
input, ok := rhs.(IntegerIterator)
|
|
|
|
if !ok {
|
|
|
|
return nil, fmt.Errorf("type mismatch on RHS, unable to use %T as a IntegerIterator", lhs)
|
|
|
|
}
|
|
|
|
|
|
|
|
var val int64
|
|
|
|
switch lhs := lhs.(type) {
|
2017-08-15 19:24:22 +00:00
|
|
|
case *influxql.IntegerLiteral:
|
2016-02-19 04:06:41 +00:00
|
|
|
val = lhs.Val
|
|
|
|
default:
|
|
|
|
return nil, fmt.Errorf("type mismatch on LHS, unable to use %T as a IntegerLiteral", rhs)
|
|
|
|
}
|
|
|
|
return &integerFloatTransformIterator{
|
|
|
|
input: input,
|
|
|
|
fn: func(p *IntegerPoint) *FloatPoint {
|
|
|
|
if p == nil {
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
fp := &FloatPoint{
|
|
|
|
Name: p.Name,
|
|
|
|
Tags: p.Tags,
|
|
|
|
Time: p.Time,
|
|
|
|
Aux: p.Aux,
|
|
|
|
}
|
|
|
|
if p.Nil {
|
|
|
|
fp.Nil = true
|
|
|
|
} else {
|
|
|
|
fp.Value = fn(val, p.Value)
|
|
|
|
}
|
|
|
|
return fp
|
|
|
|
},
|
|
|
|
}, nil
|
2015-11-04 21:06:06 +00:00
|
|
|
case func(float64, float64) bool:
|
2016-02-18 21:37:03 +00:00
|
|
|
var input FloatIterator
|
|
|
|
switch rhs := rhs.(type) {
|
|
|
|
case FloatIterator:
|
|
|
|
input = rhs
|
|
|
|
case IntegerIterator:
|
|
|
|
input = &integerFloatCastIterator{input: rhs}
|
2017-10-02 17:31:07 +00:00
|
|
|
case UnsignedIterator:
|
|
|
|
input = &unsignedFloatCastIterator{input: rhs}
|
2016-02-18 21:37:03 +00:00
|
|
|
default:
|
|
|
|
return nil, fmt.Errorf("type mismatch on RHS, unable to use %T as a FloatIterator", rhs)
|
2015-11-04 21:06:06 +00:00
|
|
|
}
|
2016-02-18 21:37:03 +00:00
|
|
|
|
2016-02-19 04:06:41 +00:00
|
|
|
var val float64
|
|
|
|
switch lhs := lhs.(type) {
|
2017-08-15 19:24:22 +00:00
|
|
|
case *influxql.NumberLiteral:
|
2016-02-19 04:06:41 +00:00
|
|
|
val = lhs.Val
|
2017-08-15 19:24:22 +00:00
|
|
|
case *influxql.IntegerLiteral:
|
2016-02-19 04:06:41 +00:00
|
|
|
val = float64(lhs.Val)
|
2017-10-02 17:31:07 +00:00
|
|
|
case *influxql.UnsignedLiteral:
|
|
|
|
val = float64(lhs.Val)
|
2016-02-19 04:06:41 +00:00
|
|
|
default:
|
2016-02-18 21:37:03 +00:00
|
|
|
return nil, fmt.Errorf("type mismatch on LHS, unable to use %T as a NumberLiteral", lhs)
|
2015-11-04 21:06:06 +00:00
|
|
|
}
|
|
|
|
return &floatBoolTransformIterator{
|
|
|
|
input: input,
|
|
|
|
fn: func(p *FloatPoint) *BooleanPoint {
|
|
|
|
if p == nil {
|
|
|
|
return nil
|
|
|
|
}
|
2016-03-11 15:58:16 +00:00
|
|
|
|
|
|
|
bp := &BooleanPoint{
|
|
|
|
Name: p.Name,
|
|
|
|
Tags: p.Tags,
|
|
|
|
Time: p.Time,
|
|
|
|
Aux: p.Aux,
|
2015-11-04 21:06:06 +00:00
|
|
|
}
|
2016-03-11 15:58:16 +00:00
|
|
|
if p.Nil {
|
|
|
|
bp.Nil = true
|
|
|
|
} else {
|
2016-02-19 04:06:41 +00:00
|
|
|
bp.Value = fn(val, p.Value)
|
|
|
|
}
|
|
|
|
return bp
|
|
|
|
},
|
|
|
|
}, nil
|
|
|
|
case func(int64, int64) int64:
|
|
|
|
var input IntegerIterator
|
|
|
|
switch rhs := rhs.(type) {
|
|
|
|
case IntegerIterator:
|
|
|
|
input = rhs
|
|
|
|
default:
|
|
|
|
return nil, fmt.Errorf("type mismatch on RHS, unable to use %T as an IntegerIterator", rhs)
|
|
|
|
}
|
|
|
|
|
|
|
|
var val int64
|
|
|
|
switch lhs := lhs.(type) {
|
2017-08-15 19:24:22 +00:00
|
|
|
case *influxql.IntegerLiteral:
|
2016-02-19 04:06:41 +00:00
|
|
|
val = lhs.Val
|
|
|
|
default:
|
|
|
|
return nil, fmt.Errorf("type mismatch on LHS, unable to use %T as an IntegerLiteral", lhs)
|
|
|
|
}
|
|
|
|
return &integerTransformIterator{
|
|
|
|
input: input,
|
|
|
|
fn: func(p *IntegerPoint) *IntegerPoint {
|
|
|
|
if p == nil {
|
|
|
|
return nil
|
|
|
|
} else if p.Nil {
|
|
|
|
return p
|
|
|
|
}
|
|
|
|
p.Value = fn(val, p.Value)
|
|
|
|
return p
|
|
|
|
},
|
|
|
|
}, nil
|
|
|
|
case func(int64, int64) bool:
|
|
|
|
var input IntegerIterator
|
|
|
|
switch rhs := rhs.(type) {
|
|
|
|
case IntegerIterator:
|
|
|
|
input = rhs
|
|
|
|
default:
|
|
|
|
return nil, fmt.Errorf("type mismatch on RHS, unable to use %T as an IntegerIterator", rhs)
|
|
|
|
}
|
|
|
|
|
|
|
|
var val int64
|
|
|
|
switch lhs := lhs.(type) {
|
2017-08-15 19:24:22 +00:00
|
|
|
case *influxql.IntegerLiteral:
|
2016-02-19 04:06:41 +00:00
|
|
|
val = lhs.Val
|
|
|
|
default:
|
|
|
|
return nil, fmt.Errorf("type mismatch on LHS, unable to use %T as an IntegerLiteral", lhs)
|
|
|
|
}
|
|
|
|
return &integerBoolTransformIterator{
|
|
|
|
input: input,
|
|
|
|
fn: func(p *IntegerPoint) *BooleanPoint {
|
|
|
|
if p == nil {
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2017-10-02 17:31:07 +00:00
|
|
|
bp := &BooleanPoint{
|
|
|
|
Name: p.Name,
|
|
|
|
Tags: p.Tags,
|
|
|
|
Time: p.Time,
|
|
|
|
Aux: p.Aux,
|
|
|
|
}
|
|
|
|
if p.Nil {
|
|
|
|
bp.Nil = true
|
|
|
|
} else {
|
|
|
|
bp.Value = fn(val, p.Value)
|
|
|
|
}
|
|
|
|
return bp
|
|
|
|
},
|
|
|
|
}, nil
|
|
|
|
case func(uint64, uint64) uint64:
|
|
|
|
var input UnsignedIterator
|
|
|
|
switch rhs := rhs.(type) {
|
|
|
|
case UnsignedIterator:
|
|
|
|
input = rhs
|
|
|
|
default:
|
|
|
|
return nil, fmt.Errorf("type mismatch on LHS, unable to use %T as a FloatIterator", lhs)
|
|
|
|
}
|
|
|
|
|
|
|
|
var val uint64
|
|
|
|
switch lhs := lhs.(type) {
|
|
|
|
case *influxql.IntegerLiteral:
|
|
|
|
if lhs.Val < 0 {
|
|
|
|
return nil, fmt.Errorf("cannot use negative integer '%s' in math with unsigned", rhs)
|
|
|
|
}
|
|
|
|
val = uint64(lhs.Val)
|
|
|
|
case *influxql.UnsignedLiteral:
|
|
|
|
val = lhs.Val
|
|
|
|
default:
|
|
|
|
return nil, fmt.Errorf("type mismatch on RHS, unable to use %T as a NumberLiteral", rhs)
|
|
|
|
}
|
|
|
|
return &unsignedTransformIterator{
|
|
|
|
input: input,
|
|
|
|
fn: func(p *UnsignedPoint) *UnsignedPoint {
|
|
|
|
if p == nil {
|
|
|
|
return nil
|
|
|
|
} else if p.Nil {
|
|
|
|
return p
|
|
|
|
}
|
|
|
|
p.Value = fn(val, p.Value)
|
|
|
|
return p
|
|
|
|
},
|
|
|
|
}, nil
|
|
|
|
case func(uint64, uint64) bool:
|
|
|
|
var input UnsignedIterator
|
|
|
|
switch rhs := rhs.(type) {
|
|
|
|
case UnsignedIterator:
|
|
|
|
input = rhs
|
|
|
|
default:
|
|
|
|
return nil, fmt.Errorf("type mismatch on LHS, unable to use %T as a FloatIterator", lhs)
|
|
|
|
}
|
|
|
|
|
|
|
|
var val uint64
|
|
|
|
switch lhs := lhs.(type) {
|
|
|
|
case *influxql.IntegerLiteral:
|
|
|
|
if lhs.Val < 0 {
|
|
|
|
return nil, fmt.Errorf("cannot use negative integer '%s' in math with unsigned", rhs)
|
|
|
|
}
|
|
|
|
val = uint64(lhs.Val)
|
|
|
|
case *influxql.UnsignedLiteral:
|
|
|
|
val = lhs.Val
|
|
|
|
default:
|
|
|
|
return nil, fmt.Errorf("type mismatch on RHS, unable to use %T as a NumberLiteral", rhs)
|
|
|
|
}
|
|
|
|
return &unsignedBoolTransformIterator{
|
|
|
|
input: input,
|
|
|
|
fn: func(p *UnsignedPoint) *BooleanPoint {
|
|
|
|
if p == nil {
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2017-03-19 09:58:09 +00:00
|
|
|
bp := &BooleanPoint{
|
|
|
|
Name: p.Name,
|
|
|
|
Tags: p.Tags,
|
|
|
|
Time: p.Time,
|
|
|
|
Aux: p.Aux,
|
|
|
|
}
|
|
|
|
if p.Nil {
|
|
|
|
bp.Nil = true
|
|
|
|
} else {
|
|
|
|
bp.Value = fn(val, p.Value)
|
|
|
|
}
|
|
|
|
return bp
|
|
|
|
},
|
|
|
|
}, nil
|
|
|
|
case func(bool, bool) bool:
|
|
|
|
var input BooleanIterator
|
|
|
|
switch rhs := rhs.(type) {
|
|
|
|
case BooleanIterator:
|
|
|
|
input = rhs
|
|
|
|
default:
|
|
|
|
return nil, fmt.Errorf("type mismatch on RHS, unable to use %T as an BooleanIterator", rhs)
|
|
|
|
}
|
|
|
|
|
|
|
|
var val bool
|
|
|
|
switch lhs := lhs.(type) {
|
2017-08-15 19:24:22 +00:00
|
|
|
case *influxql.BooleanLiteral:
|
2017-03-19 09:58:09 +00:00
|
|
|
val = lhs.Val
|
|
|
|
default:
|
|
|
|
return nil, fmt.Errorf("type mismatch on LHS, unable to use %T as a BooleanLiteral", lhs)
|
|
|
|
}
|
|
|
|
return &booleanTransformIterator{
|
|
|
|
input: input,
|
|
|
|
fn: func(p *BooleanPoint) *BooleanPoint {
|
|
|
|
if p == nil {
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2016-02-19 04:06:41 +00:00
|
|
|
bp := &BooleanPoint{
|
|
|
|
Name: p.Name,
|
|
|
|
Tags: p.Tags,
|
|
|
|
Time: p.Time,
|
|
|
|
Aux: p.Aux,
|
|
|
|
}
|
|
|
|
if p.Nil {
|
|
|
|
bp.Nil = true
|
|
|
|
} else {
|
|
|
|
bp.Value = fn(val, p.Value)
|
2016-03-11 15:58:16 +00:00
|
|
|
}
|
|
|
|
return bp
|
2015-11-04 21:06:06 +00:00
|
|
|
},
|
|
|
|
}, nil
|
|
|
|
}
|
2016-01-18 22:48:49 +00:00
|
|
|
return nil, fmt.Errorf("unable to construct lhs transform iterator from %T and %T", lhs, rhs)
|
2015-11-04 21:06:06 +00:00
|
|
|
}
|
|
|
|
|
2017-08-15 19:24:22 +00:00
|
|
|
func buildTransformIterator(lhs Iterator, rhs Iterator, op influxql.Token, opt IteratorOptions) (Iterator, error) {
|
2017-10-02 17:31:07 +00:00
|
|
|
lhsType, rhsType := iteratorDataType(lhs), iteratorDataType(rhs)
|
|
|
|
if lhsType == influxql.Integer && rhsType == influxql.Unsigned {
|
|
|
|
return nil, fmt.Errorf("cannot use %s between an integer and unsigned, an explicit cast is required", op)
|
|
|
|
} else if lhsType == influxql.Unsigned && rhsType == influxql.Integer {
|
|
|
|
return nil, fmt.Errorf("cannot use %s between unsigned and an integer, an explicit cast is required", op)
|
|
|
|
}
|
|
|
|
|
|
|
|
fn := binaryExprFunc(lhsType, rhsType, op)
|
2015-11-04 21:06:06 +00:00
|
|
|
switch fn := fn.(type) {
|
|
|
|
case func(float64, float64) float64:
|
2016-02-18 21:37:03 +00:00
|
|
|
var left FloatIterator
|
|
|
|
switch lhs := lhs.(type) {
|
|
|
|
case FloatIterator:
|
|
|
|
left = lhs
|
|
|
|
case IntegerIterator:
|
|
|
|
left = &integerFloatCastIterator{input: lhs}
|
2017-10-02 17:31:07 +00:00
|
|
|
case UnsignedIterator:
|
|
|
|
left = &unsignedFloatCastIterator{input: lhs}
|
2016-02-18 21:37:03 +00:00
|
|
|
default:
|
|
|
|
return nil, fmt.Errorf("type mismatch on LHS, unable to use %T as a FloatIterator", lhs)
|
2015-11-04 21:06:06 +00:00
|
|
|
}
|
2016-02-18 21:37:03 +00:00
|
|
|
|
|
|
|
var right FloatIterator
|
|
|
|
switch rhs := rhs.(type) {
|
|
|
|
case FloatIterator:
|
|
|
|
right = rhs
|
|
|
|
case IntegerIterator:
|
|
|
|
right = &integerFloatCastIterator{input: rhs}
|
2017-10-02 17:31:07 +00:00
|
|
|
case UnsignedIterator:
|
|
|
|
right = &unsignedFloatCastIterator{input: rhs}
|
2016-02-18 21:37:03 +00:00
|
|
|
default:
|
|
|
|
return nil, fmt.Errorf("type mismatch on RHS, unable to use %T as a FloatIterator", rhs)
|
2015-11-04 21:06:06 +00:00
|
|
|
}
|
2016-10-18 18:30:59 +00:00
|
|
|
return newFloatExprIterator(left, right, opt, fn), nil
|
2016-03-03 13:15:54 +00:00
|
|
|
case func(int64, int64) float64:
|
|
|
|
left, ok := lhs.(IntegerIterator)
|
|
|
|
if !ok {
|
|
|
|
return nil, fmt.Errorf("type mismatch on LHS, unable to use %T as a IntegerIterator", lhs)
|
|
|
|
}
|
|
|
|
right, ok := rhs.(IntegerIterator)
|
|
|
|
if !ok {
|
|
|
|
return nil, fmt.Errorf("type mismatch on RHS, unable to use %T as a IntegerIterator", rhs)
|
|
|
|
}
|
2016-10-18 18:30:59 +00:00
|
|
|
return newIntegerFloatExprIterator(left, right, opt, fn), nil
|
2016-01-18 22:48:49 +00:00
|
|
|
case func(int64, int64) int64:
|
|
|
|
left, ok := lhs.(IntegerIterator)
|
|
|
|
if !ok {
|
2016-02-18 21:37:03 +00:00
|
|
|
return nil, fmt.Errorf("type mismatch on LHS, unable to use %T as a IntegerIterator", lhs)
|
2016-01-18 22:48:49 +00:00
|
|
|
}
|
|
|
|
right, ok := rhs.(IntegerIterator)
|
|
|
|
if !ok {
|
2016-02-18 21:37:03 +00:00
|
|
|
return nil, fmt.Errorf("type mismatch on RHS, unable to use %T as a IntegerIterator", rhs)
|
2016-01-18 22:48:49 +00:00
|
|
|
}
|
2016-10-18 18:30:59 +00:00
|
|
|
return newIntegerExprIterator(left, right, opt, fn), nil
|
2017-10-02 17:31:07 +00:00
|
|
|
case func(uint64, uint64) uint64:
|
|
|
|
left, ok := lhs.(UnsignedIterator)
|
|
|
|
if !ok {
|
|
|
|
return nil, fmt.Errorf("type mismatch on LHS, unable to use %T as an UnsignedIterator", lhs)
|
|
|
|
}
|
|
|
|
right, ok := rhs.(UnsignedIterator)
|
|
|
|
if !ok {
|
|
|
|
return nil, fmt.Errorf("type mismatch on RHS, unable to use %T as an UnsignedIterator", lhs)
|
|
|
|
}
|
|
|
|
return newUnsignedExprIterator(left, right, opt, fn), nil
|
2015-11-04 21:06:06 +00:00
|
|
|
case func(float64, float64) bool:
|
2016-02-18 21:37:03 +00:00
|
|
|
var left FloatIterator
|
|
|
|
switch lhs := lhs.(type) {
|
|
|
|
case FloatIterator:
|
|
|
|
left = lhs
|
|
|
|
case IntegerIterator:
|
|
|
|
left = &integerFloatCastIterator{input: lhs}
|
2017-10-02 17:31:07 +00:00
|
|
|
case UnsignedIterator:
|
|
|
|
left = &unsignedFloatCastIterator{input: lhs}
|
2016-02-18 21:37:03 +00:00
|
|
|
default:
|
|
|
|
return nil, fmt.Errorf("type mismatch on LHS, unable to use %T as a FloatIterator", lhs)
|
2015-11-04 21:06:06 +00:00
|
|
|
}
|
2016-02-18 21:37:03 +00:00
|
|
|
|
|
|
|
var right FloatIterator
|
|
|
|
switch rhs := rhs.(type) {
|
|
|
|
case FloatIterator:
|
|
|
|
right = rhs
|
|
|
|
case IntegerIterator:
|
|
|
|
right = &integerFloatCastIterator{input: rhs}
|
2017-10-02 17:31:07 +00:00
|
|
|
case UnsignedIterator:
|
|
|
|
right = &unsignedFloatCastIterator{input: rhs}
|
2016-02-18 21:37:03 +00:00
|
|
|
default:
|
|
|
|
return nil, fmt.Errorf("type mismatch on RHS, unable to use %T as a FloatIterator", rhs)
|
2015-11-04 21:06:06 +00:00
|
|
|
}
|
2016-10-18 18:30:59 +00:00
|
|
|
return newFloatBooleanExprIterator(left, right, opt, fn), nil
|
2016-01-18 22:48:49 +00:00
|
|
|
case func(int64, int64) bool:
|
|
|
|
left, ok := lhs.(IntegerIterator)
|
|
|
|
if !ok {
|
2016-02-18 21:37:03 +00:00
|
|
|
return nil, fmt.Errorf("type mismatch on LHS, unable to use %T as a IntegerIterator", lhs)
|
2016-01-18 22:48:49 +00:00
|
|
|
}
|
|
|
|
right, ok := rhs.(IntegerIterator)
|
|
|
|
if !ok {
|
2017-03-19 09:58:09 +00:00
|
|
|
return nil, fmt.Errorf("type mismatch on RHS, unable to use %T as a IntegerIterator", rhs)
|
2016-01-18 22:48:49 +00:00
|
|
|
}
|
2016-10-18 18:30:59 +00:00
|
|
|
return newIntegerBooleanExprIterator(left, right, opt, fn), nil
|
2017-10-02 17:31:07 +00:00
|
|
|
case func(uint64, uint64) bool:
|
|
|
|
left, ok := lhs.(UnsignedIterator)
|
|
|
|
if !ok {
|
|
|
|
return nil, fmt.Errorf("type mismatch on LHS, unable to use %T as an UnsignedIterator", lhs)
|
|
|
|
}
|
|
|
|
right, ok := rhs.(UnsignedIterator)
|
|
|
|
if !ok {
|
|
|
|
return nil, fmt.Errorf("type mismatch on RHS, unable to use %T as an UnsignedIterator", lhs)
|
|
|
|
}
|
|
|
|
return newUnsignedBooleanExprIterator(left, right, opt, fn), nil
|
2017-03-19 09:58:09 +00:00
|
|
|
case func(bool, bool) bool:
|
|
|
|
left, ok := lhs.(BooleanIterator)
|
|
|
|
if !ok {
|
|
|
|
return nil, fmt.Errorf("type mismatch on LHS, unable to use %T as a BooleanIterator", lhs)
|
|
|
|
}
|
|
|
|
right, ok := rhs.(BooleanIterator)
|
|
|
|
if !ok {
|
|
|
|
return nil, fmt.Errorf("type mismatch on RHS, unable to use %T as a BooleanIterator", rhs)
|
|
|
|
}
|
|
|
|
return newBooleanExprIterator(left, right, opt, fn), nil
|
2015-11-04 21:06:06 +00:00
|
|
|
}
|
|
|
|
return nil, fmt.Errorf("unable to construct transform iterator from %T and %T", lhs, rhs)
|
|
|
|
}
|
|
|
|
|
2017-08-15 19:24:22 +00:00
|
|
|
func iteratorDataType(itr Iterator) influxql.DataType {
|
2015-11-04 21:06:06 +00:00
|
|
|
switch itr.(type) {
|
|
|
|
case FloatIterator:
|
2017-08-15 19:24:22 +00:00
|
|
|
return influxql.Float
|
2016-01-18 22:48:49 +00:00
|
|
|
case IntegerIterator:
|
2017-08-15 19:24:22 +00:00
|
|
|
return influxql.Integer
|
2017-10-02 17:31:07 +00:00
|
|
|
case UnsignedIterator:
|
|
|
|
return influxql.Unsigned
|
2015-11-04 21:06:06 +00:00
|
|
|
case StringIterator:
|
2017-08-15 19:24:22 +00:00
|
|
|
return influxql.String
|
2015-11-04 21:06:06 +00:00
|
|
|
case BooleanIterator:
|
2017-08-15 19:24:22 +00:00
|
|
|
return influxql.Boolean
|
2015-11-04 21:06:06 +00:00
|
|
|
default:
|
2017-08-15 19:24:22 +00:00
|
|
|
return influxql.Unknown
|
2015-11-04 21:06:06 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2017-08-15 19:24:22 +00:00
|
|
|
func literalDataType(lit influxql.Literal) influxql.DataType {
|
2016-02-18 21:37:03 +00:00
|
|
|
switch lit.(type) {
|
2017-08-15 19:24:22 +00:00
|
|
|
case *influxql.NumberLiteral:
|
|
|
|
return influxql.Float
|
|
|
|
case *influxql.IntegerLiteral:
|
|
|
|
return influxql.Integer
|
2017-10-02 17:31:07 +00:00
|
|
|
case *influxql.UnsignedLiteral:
|
|
|
|
return influxql.Unsigned
|
2017-08-15 19:24:22 +00:00
|
|
|
case *influxql.StringLiteral:
|
|
|
|
return influxql.String
|
|
|
|
case *influxql.BooleanLiteral:
|
|
|
|
return influxql.Boolean
|
2016-02-18 21:37:03 +00:00
|
|
|
default:
|
2017-08-15 19:24:22 +00:00
|
|
|
return influxql.Unknown
|
2016-02-18 21:37:03 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2017-08-15 19:24:22 +00:00
|
|
|
func binaryExprFunc(typ1 influxql.DataType, typ2 influxql.DataType, op influxql.Token) interface{} {
|
2016-01-18 22:48:49 +00:00
|
|
|
var fn interface{}
|
2016-02-18 21:37:03 +00:00
|
|
|
switch typ1 {
|
2017-08-15 19:24:22 +00:00
|
|
|
case influxql.Float:
|
2016-01-18 22:48:49 +00:00
|
|
|
fn = floatBinaryExprFunc(op)
|
2017-08-15 19:24:22 +00:00
|
|
|
case influxql.Integer:
|
2016-02-18 21:37:03 +00:00
|
|
|
switch typ2 {
|
2017-08-15 19:24:22 +00:00
|
|
|
case influxql.Float:
|
2016-02-18 21:37:03 +00:00
|
|
|
fn = floatBinaryExprFunc(op)
|
2017-10-02 17:31:07 +00:00
|
|
|
case influxql.Unsigned:
|
|
|
|
// Special case for LT, LTE, GT, and GTE.
|
|
|
|
fn = unsignedBinaryExprFunc(op)
|
2016-02-18 21:37:03 +00:00
|
|
|
default:
|
|
|
|
fn = integerBinaryExprFunc(op)
|
|
|
|
}
|
2017-10-02 17:31:07 +00:00
|
|
|
case influxql.Unsigned:
|
|
|
|
switch typ2 {
|
|
|
|
case influxql.Float:
|
|
|
|
fn = floatBinaryExprFunc(op)
|
|
|
|
case influxql.Integer:
|
|
|
|
// Special case for LT, LTE, GT, and GTE.
|
|
|
|
// Since the RHS is an integer, we need to check if it is less than
|
|
|
|
// zero for the comparison operators to not be subject to overflow.
|
|
|
|
switch op {
|
|
|
|
case influxql.LT:
|
|
|
|
return func(lhs, rhs uint64) bool {
|
|
|
|
if int64(rhs) < 0 {
|
|
|
|
return false
|
|
|
|
}
|
|
|
|
return lhs < rhs
|
|
|
|
}
|
|
|
|
case influxql.LTE:
|
|
|
|
return func(lhs, rhs uint64) bool {
|
|
|
|
if int64(rhs) < 0 {
|
|
|
|
return false
|
|
|
|
}
|
|
|
|
return lhs <= rhs
|
|
|
|
}
|
|
|
|
case influxql.GT:
|
|
|
|
return func(lhs, rhs uint64) bool {
|
|
|
|
if int64(rhs) < 0 {
|
|
|
|
return true
|
|
|
|
}
|
|
|
|
return lhs > rhs
|
|
|
|
}
|
|
|
|
case influxql.GTE:
|
|
|
|
return func(lhs, rhs uint64) bool {
|
|
|
|
if int64(rhs) < 0 {
|
|
|
|
return true
|
|
|
|
}
|
|
|
|
return lhs >= rhs
|
|
|
|
}
|
|
|
|
}
|
|
|
|
fallthrough
|
|
|
|
default:
|
|
|
|
fn = unsignedBinaryExprFunc(op)
|
|
|
|
}
|
2017-08-15 19:24:22 +00:00
|
|
|
case influxql.Boolean:
|
2017-03-19 09:58:09 +00:00
|
|
|
fn = booleanBinaryExprFunc(op)
|
2016-01-18 22:48:49 +00:00
|
|
|
}
|
|
|
|
return fn
|
|
|
|
}
|
|
|
|
|
2017-08-15 19:24:22 +00:00
|
|
|
func floatBinaryExprFunc(op influxql.Token) interface{} {
|
2016-01-18 22:48:49 +00:00
|
|
|
switch op {
|
2017-08-15 19:24:22 +00:00
|
|
|
case influxql.ADD:
|
2016-01-18 22:48:49 +00:00
|
|
|
return func(lhs, rhs float64) float64 { return lhs + rhs }
|
2017-08-15 19:24:22 +00:00
|
|
|
case influxql.SUB:
|
2016-01-18 22:48:49 +00:00
|
|
|
return func(lhs, rhs float64) float64 { return lhs - rhs }
|
2017-08-15 19:24:22 +00:00
|
|
|
case influxql.MUL:
|
2016-01-18 22:48:49 +00:00
|
|
|
return func(lhs, rhs float64) float64 { return lhs * rhs }
|
2017-08-15 19:24:22 +00:00
|
|
|
case influxql.DIV:
|
2016-01-18 22:48:49 +00:00
|
|
|
return func(lhs, rhs float64) float64 {
|
|
|
|
if rhs == 0 {
|
|
|
|
return float64(0)
|
|
|
|
}
|
|
|
|
return lhs / rhs
|
|
|
|
}
|
2017-08-15 19:24:22 +00:00
|
|
|
case influxql.MOD:
|
2016-12-27 23:12:11 +00:00
|
|
|
return func(lhs, rhs float64) float64 { return math.Mod(lhs, rhs) }
|
2017-08-15 19:24:22 +00:00
|
|
|
case influxql.EQ:
|
2016-01-18 22:48:49 +00:00
|
|
|
return func(lhs, rhs float64) bool { return lhs == rhs }
|
2017-08-15 19:24:22 +00:00
|
|
|
case influxql.NEQ:
|
2016-01-18 22:48:49 +00:00
|
|
|
return func(lhs, rhs float64) bool { return lhs != rhs }
|
2017-08-15 19:24:22 +00:00
|
|
|
case influxql.LT:
|
2016-01-18 22:48:49 +00:00
|
|
|
return func(lhs, rhs float64) bool { return lhs < rhs }
|
2017-08-15 19:24:22 +00:00
|
|
|
case influxql.LTE:
|
2016-01-18 22:48:49 +00:00
|
|
|
return func(lhs, rhs float64) bool { return lhs <= rhs }
|
2017-08-15 19:24:22 +00:00
|
|
|
case influxql.GT:
|
2016-01-18 22:48:49 +00:00
|
|
|
return func(lhs, rhs float64) bool { return lhs > rhs }
|
2017-08-15 19:24:22 +00:00
|
|
|
case influxql.GTE:
|
2016-01-18 22:48:49 +00:00
|
|
|
return func(lhs, rhs float64) bool { return lhs >= rhs }
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2017-08-15 19:24:22 +00:00
|
|
|
func integerBinaryExprFunc(op influxql.Token) interface{} {
|
2016-01-18 22:48:49 +00:00
|
|
|
switch op {
|
2017-08-15 19:24:22 +00:00
|
|
|
case influxql.ADD:
|
2016-01-18 22:48:49 +00:00
|
|
|
return func(lhs, rhs int64) int64 { return lhs + rhs }
|
2017-08-15 19:24:22 +00:00
|
|
|
case influxql.SUB:
|
2016-01-18 22:48:49 +00:00
|
|
|
return func(lhs, rhs int64) int64 { return lhs - rhs }
|
2017-08-15 19:24:22 +00:00
|
|
|
case influxql.MUL:
|
2016-01-18 22:48:49 +00:00
|
|
|
return func(lhs, rhs int64) int64 { return lhs * rhs }
|
2017-08-15 19:24:22 +00:00
|
|
|
case influxql.DIV:
|
2016-03-03 13:15:54 +00:00
|
|
|
return func(lhs, rhs int64) float64 {
|
2016-01-18 22:48:49 +00:00
|
|
|
if rhs == 0 {
|
2016-03-03 13:15:54 +00:00
|
|
|
return float64(0)
|
2016-01-18 22:48:49 +00:00
|
|
|
}
|
2016-03-03 13:15:54 +00:00
|
|
|
return float64(lhs) / float64(rhs)
|
2015-11-04 21:06:06 +00:00
|
|
|
}
|
2017-08-15 19:24:22 +00:00
|
|
|
case influxql.MOD:
|
2016-12-27 23:12:11 +00:00
|
|
|
return func(lhs, rhs int64) int64 {
|
|
|
|
if rhs == 0 {
|
|
|
|
return int64(0)
|
|
|
|
}
|
|
|
|
return lhs % rhs
|
|
|
|
}
|
2017-08-15 19:24:22 +00:00
|
|
|
case influxql.BITWISE_AND:
|
2017-03-19 09:58:09 +00:00
|
|
|
return func(lhs, rhs int64) int64 { return lhs & rhs }
|
2017-08-15 19:24:22 +00:00
|
|
|
case influxql.BITWISE_OR:
|
2017-03-19 09:58:09 +00:00
|
|
|
return func(lhs, rhs int64) int64 { return lhs | rhs }
|
2017-08-15 19:24:22 +00:00
|
|
|
case influxql.BITWISE_XOR:
|
2017-03-19 09:58:09 +00:00
|
|
|
return func(lhs, rhs int64) int64 { return lhs ^ rhs }
|
2017-08-15 19:24:22 +00:00
|
|
|
case influxql.EQ:
|
2016-01-18 22:48:49 +00:00
|
|
|
return func(lhs, rhs int64) bool { return lhs == rhs }
|
2017-08-15 19:24:22 +00:00
|
|
|
case influxql.NEQ:
|
2016-01-18 22:48:49 +00:00
|
|
|
return func(lhs, rhs int64) bool { return lhs != rhs }
|
2017-08-15 19:24:22 +00:00
|
|
|
case influxql.LT:
|
2016-01-18 22:48:49 +00:00
|
|
|
return func(lhs, rhs int64) bool { return lhs < rhs }
|
2017-08-15 19:24:22 +00:00
|
|
|
case influxql.LTE:
|
2016-01-18 22:48:49 +00:00
|
|
|
return func(lhs, rhs int64) bool { return lhs <= rhs }
|
2017-08-15 19:24:22 +00:00
|
|
|
case influxql.GT:
|
2016-01-18 22:48:49 +00:00
|
|
|
return func(lhs, rhs int64) bool { return lhs > rhs }
|
2017-08-15 19:24:22 +00:00
|
|
|
case influxql.GTE:
|
2016-01-18 22:48:49 +00:00
|
|
|
return func(lhs, rhs int64) bool { return lhs >= rhs }
|
2015-11-04 21:06:06 +00:00
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2017-10-02 17:31:07 +00:00
|
|
|
func unsignedBinaryExprFunc(op influxql.Token) interface{} {
|
|
|
|
switch op {
|
|
|
|
case influxql.ADD:
|
|
|
|
return func(lhs, rhs uint64) uint64 { return lhs + rhs }
|
|
|
|
case influxql.SUB:
|
|
|
|
return func(lhs, rhs uint64) uint64 { return lhs - rhs }
|
|
|
|
case influxql.MUL:
|
|
|
|
return func(lhs, rhs uint64) uint64 { return lhs * rhs }
|
|
|
|
case influxql.DIV:
|
|
|
|
return func(lhs, rhs uint64) uint64 {
|
|
|
|
if rhs == 0 {
|
|
|
|
return uint64(0)
|
|
|
|
}
|
|
|
|
return lhs / rhs
|
|
|
|
}
|
|
|
|
case influxql.MOD:
|
|
|
|
return func(lhs, rhs uint64) uint64 {
|
|
|
|
if rhs == 0 {
|
|
|
|
return uint64(0)
|
|
|
|
}
|
|
|
|
return lhs % rhs
|
|
|
|
}
|
|
|
|
case influxql.BITWISE_AND:
|
|
|
|
return func(lhs, rhs uint64) uint64 { return lhs & rhs }
|
|
|
|
case influxql.BITWISE_OR:
|
|
|
|
return func(lhs, rhs uint64) uint64 { return lhs | rhs }
|
|
|
|
case influxql.BITWISE_XOR:
|
|
|
|
return func(lhs, rhs uint64) uint64 { return lhs ^ rhs }
|
|
|
|
case influxql.EQ:
|
|
|
|
return func(lhs, rhs uint64) bool { return lhs == rhs }
|
|
|
|
case influxql.NEQ:
|
|
|
|
return func(lhs, rhs uint64) bool { return lhs != rhs }
|
|
|
|
case influxql.LT:
|
|
|
|
return func(lhs, rhs uint64) bool { return lhs < rhs }
|
|
|
|
case influxql.LTE:
|
|
|
|
return func(lhs, rhs uint64) bool { return lhs <= rhs }
|
|
|
|
case influxql.GT:
|
|
|
|
return func(lhs, rhs uint64) bool { return lhs > rhs }
|
|
|
|
case influxql.GTE:
|
|
|
|
return func(lhs, rhs uint64) bool { return lhs >= rhs }
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2017-08-15 19:24:22 +00:00
|
|
|
func booleanBinaryExprFunc(op influxql.Token) interface{} {
|
2017-03-19 09:58:09 +00:00
|
|
|
switch op {
|
2017-08-15 19:24:22 +00:00
|
|
|
case influxql.BITWISE_AND:
|
2017-03-19 09:58:09 +00:00
|
|
|
return func(lhs, rhs bool) bool { return lhs && rhs }
|
2017-08-15 19:24:22 +00:00
|
|
|
case influxql.BITWISE_OR:
|
2017-03-19 09:58:09 +00:00
|
|
|
return func(lhs, rhs bool) bool { return lhs || rhs }
|
2017-08-15 19:24:22 +00:00
|
|
|
case influxql.BITWISE_XOR:
|
2017-03-19 09:58:09 +00:00
|
|
|
return func(lhs, rhs bool) bool { return lhs != rhs }
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|