From 414f5e3dd2622d09b2b44472d5e8066e3a2cb12c Mon Sep 17 00:00:00 2001 From: Nathaniel Cook Date: Mon, 21 May 2018 17:02:42 -0600 Subject: [PATCH] move types around to avoid cyclic imports --- control/controller.go | 10 +- csv/result.go | 134 ++++----- csv/result_test.go | 178 +++++------ execute/aggergate_test.go | 227 +++++++------- execute/aggregate.go | 52 ++-- execute/block.go | 412 ++++++++++---------------- execute/dataset.go | 22 +- execute/executetest/aggregate.go | 21 +- execute/executetest/block.go | 41 +-- execute/executetest/dataset.go | 4 +- execute/executetest/result.go | 6 +- execute/executetest/selector.go | 17 +- execute/executetest/transformation.go | 3 +- execute/executor.go | 10 +- execute/executor_test.go | 86 +++--- execute/format.go | 44 +-- execute/partition_lookup.go | 26 +- execute/result.go | 16 +- execute/row_fn.go | 62 ++-- execute/selector.go | 102 +++---- execute/selector_test.go | 143 ++++----- execute/transformation.go | 6 +- execute/transport.go | 18 +- execute/trigger.go | 2 +- functions/count.go | 4 +- functions/covariance.go | 16 +- functions/covariance_test.go | 124 ++++---- functions/cumulative_sum.go | 20 +- functions/cumulative_sum_test.go | 80 ++--- functions/data_test.go | 22 +- functions/derivative.go | 22 +- functions/derivative_test.go | 198 ++++++------- functions/difference.go | 32 +- functions/difference_test.go | 156 +++++----- functions/distinct.go | 46 +-- functions/filter.go | 10 +- functions/filter_test.go | 76 ++--- functions/first_test.go | 12 +- functions/from_csv.go | 6 +- functions/group.go | 8 +- functions/group_test.go | 204 ++++++------- functions/integral.go | 14 +- functions/integral_test.go | 98 +++--- functions/join.go | 50 ++-- functions/join_test.go | 206 ++++++------- functions/keys.go | 10 +- functions/keys_test.go | 75 ++--- functions/last.go | 12 +- functions/last_test.go | 12 +- functions/limit.go | 8 +- functions/limit_test.go | 52 ++-- functions/map.go | 10 +- functions/map_test.go | 62 ++-- functions/max.go | 8 +- functions/max_test.go | 32 +- functions/mean.go | 4 +- functions/min.go | 8 +- functions/min_test.go | 32 +- functions/percentile.go | 8 +- functions/range.go | 4 +- functions/sample_test.go | 84 +++--- functions/set.go | 12 +- functions/set_test.go | 124 ++++---- functions/shift.go | 12 +- functions/shift_test.go | 16 +- functions/skew.go | 4 +- functions/sort.go | 6 +- functions/sort_test.go | 136 ++++----- functions/spread.go | 12 +- functions/state_tracking.go | 16 +- functions/state_tracking_test.go | 56 ++-- functions/stddev.go | 4 +- functions/storage/pb/reader.go | 115 +++---- functions/storage/storage.go | 9 +- functions/sum.go | 12 +- functions/unique.go | 30 +- functions/unique_test.go | 54 ++-- functions/window.go | 32 +- functions/window_test.go | 246 +++++++-------- influxql/influxql_encoder_test.go | 18 +- influxql/result.go | 26 +- query.go | 273 ++++++++--------- query_test/query_test.go | 12 +- repl/repl.go | 8 +- result.go | 116 +++++++- spec.go | 176 +++++++++++ transpiler.go | 5 +- 87 files changed, 2595 insertions(+), 2402 deletions(-) create mode 100644 spec.go diff --git a/control/controller.go b/control/controller.go index 0650c4677d..4783d942d6 100644 --- a/control/controller.go +++ b/control/controller.go @@ -7,9 +7,9 @@ import ( "sync" "time" - "github.com/influxdata/platform/query/id" "github.com/influxdata/platform/query" "github.com/influxdata/platform/query/execute" + "github.com/influxdata/platform/query/id" "github.com/influxdata/platform/query/plan" opentracing "github.com/opentracing/opentracing-go" "github.com/pkg/errors" @@ -90,7 +90,7 @@ func (c *Controller) Query(ctx context.Context, orgID id.ID, qSpec *query.Spec) func (c *Controller) createQuery(ctx context.Context, orgID id.ID) *Query { id := c.nextID() cctx, cancel := context.WithCancel(ctx) - ready := make(chan map[string]execute.Result, 1) + ready := make(chan map[string]query.Result, 1) return &Query{ id: id, orgID: orgID, @@ -276,7 +276,7 @@ type Query struct { err error - ready chan map[string]execute.Result + ready chan map[string]query.Result mu sync.Mutex state State @@ -335,7 +335,7 @@ func (q *Query) Cancel() { // Ready returns a channel that will deliver the query results. // Its possible that the channel is closed before any results arrive, in which case the query should be // inspected for an error using Err(). -func (q *Query) Ready() <-chan map[string]execute.Result { +func (q *Query) Ready() <-chan map[string]query.Result { return q.ready } @@ -414,7 +414,7 @@ func (q *Query) setErr(err error) { q.state = Errored } -func (q *Query) setResults(r map[string]execute.Result) { +func (q *Query) setResults(r map[string]query.Result) { q.mu.Lock() if q.state == Executing { q.ready <- r diff --git a/csv/result.go b/csv/result.go index 09f22f9d2f..26d8dd90dd 100644 --- a/csv/result.go +++ b/csv/result.go @@ -9,9 +9,9 @@ import ( "strings" "time" + "github.com/influxdata/platform/query" "github.com/influxdata/platform/query/execute" "github.com/influxdata/platform/query/values" - "github.com/influxdata/platform" "github.com/pkg/errors" ) @@ -67,7 +67,7 @@ type ResultDecoderConfig struct { MaxBufferCount int } -func (d *ResultDecoder) Decode(r io.Reader) (execute.Result, error) { +func (d *ResultDecoder) Decode(r io.Reader) (query.Result, error) { return newResultDecoder(r, d.c, nil), nil } @@ -85,7 +85,7 @@ func NewMultiResultDecoder(c ResultDecoderConfig) *MultiResultDecoder { } } -func (d *MultiResultDecoder) Decode(r io.Reader) (platform.ResultIterator, error) { +func (d *MultiResultDecoder) Decode(r io.Reader) (query.ResultIterator, error) { return &resultIterator{ c: d.c, r: r, @@ -112,7 +112,7 @@ func (r *resultIterator) More() bool { return false } -func (r *resultIterator) Next() (string, execute.Result) { +func (r *resultIterator) Next() (string, query.Result) { return r.next.id, r.next } @@ -149,7 +149,7 @@ func newCSVReader(r io.Reader) *csv.Reader { return csvr } -func (r *resultDecoder) Blocks() execute.BlockIterator { +func (r *resultDecoder) Blocks() query.BlockIterator { return r } @@ -157,7 +157,7 @@ func (r *resultDecoder) Abort(error) { panic("not implemented") } -func (r *resultDecoder) Do(f func(execute.Block) error) error { +func (r *resultDecoder) Do(f func(query.Block) error) error { cr := newCSVReader(r.r) var extraLine []string @@ -312,7 +312,7 @@ func readMetadata(r *csv.Reader, c ResultDecoderConfig, extraLine []string) (tab } cols[j].ColMeta.Label = label cols[j].ColMeta.Type = t - if t == execute.TTime { + if t == query.TTime { switch desc { case "RFC3339": cols[j].fmt = time.RFC3339 @@ -352,7 +352,7 @@ type blockDecoder struct { initialized bool id string - key execute.PartitionKey + key query.PartitionKey cols []colMeta builder *execute.ColListBlockBuilder @@ -385,7 +385,7 @@ func newBlock( return b, nil } -func (b *blockDecoder) Do(f func(execute.ColReader) error) (err error) { +func (b *blockDecoder) Do(f func(query.ColReader) error) (err error) { // Send off first batch from first advance call. err = f(b.builder.RawBlock()) if err != nil { @@ -480,7 +480,7 @@ func (b *blockDecoder) init(line []string) error { if len(line) != 0 { record = line[recordStartIdx:] } - keyCols := make([]execute.ColMeta, 0, len(b.meta.Cols)) + keyCols := make([]query.ColMeta, 0, len(b.meta.Cols)) keyValues := make([]interface{}, 0, len(b.meta.Cols)) for j, c := range b.meta.Cols { if b.meta.Partitions[j] { @@ -514,22 +514,22 @@ func (b *blockDecoder) appendRecord(record []string) error { for j, c := range b.meta.Cols { if record[j] == "" && b.meta.Defaults[j] != nil { switch c.Type { - case execute.TBool: + case query.TBool: v := b.meta.Defaults[j].(bool) b.builder.AppendBool(j, v) - case execute.TInt: + case query.TInt: v := b.meta.Defaults[j].(int64) b.builder.AppendInt(j, v) - case execute.TUInt: + case query.TUInt: v := b.meta.Defaults[j].(uint64) b.builder.AppendUInt(j, v) - case execute.TFloat: + case query.TFloat: v := b.meta.Defaults[j].(float64) b.builder.AppendFloat(j, v) - case execute.TString: + case query.TString: v := b.meta.Defaults[j].(string) b.builder.AppendString(j, v) - case execute.TTime: + case query.TTime: v := b.meta.Defaults[j].(execute.Time) b.builder.AppendTime(j, v) default: @@ -546,16 +546,16 @@ func (b *blockDecoder) appendRecord(record []string) error { func (b *blockDecoder) RefCount(n int) {} -func (b *blockDecoder) Key() execute.PartitionKey { +func (b *blockDecoder) Key() query.PartitionKey { return b.builder.Key() } -func (b *blockDecoder) Cols() []execute.ColMeta { +func (b *blockDecoder) Cols() []query.ColMeta { return b.builder.Cols() } type colMeta struct { - execute.ColMeta + query.ColMeta fmt string } @@ -593,12 +593,12 @@ func NewResultEncoder(c ResultEncoderConfig) *ResultEncoder { } } -func (e *ResultEncoder) Encode(w io.Writer, result execute.Result) error { +func (e *ResultEncoder) Encode(w io.Writer, result query.Result) error { tableID := 0 metaCols := []colMeta{ - {ColMeta: execute.ColMeta{Label: "", Type: execute.TInvalid}}, - {ColMeta: execute.ColMeta{Label: resultLabel, Type: execute.TString}}, - {ColMeta: execute.ColMeta{Label: tableLabel, Type: execute.TInt}}, + {ColMeta: query.ColMeta{Label: "", Type: query.TInvalid}}, + {ColMeta: query.ColMeta{Label: resultLabel, Type: query.TString}}, + {ColMeta: query.ColMeta{Label: tableLabel, Type: query.TInt}}, } writer := csv.NewWriter(w) if e.c.Delimiter != 0 { @@ -608,12 +608,12 @@ func (e *ResultEncoder) Encode(w io.Writer, result execute.Result) error { var lastCols []colMeta - return result.Blocks().Do(func(b execute.Block) error { + return result.Blocks().Do(func(b query.Block) error { // Update cols with block cols cols := metaCols for _, c := range b.Cols() { cm := colMeta{ColMeta: c} - if c.Type == execute.TTime { + if c.Type == query.TTime { cm.fmt = time.RFC3339Nano } cols = append(cols, cm) @@ -650,7 +650,7 @@ func (e *ResultEncoder) Encode(w io.Writer, result execute.Result) error { } count := 0 - err := b.Do(func(cr execute.ColReader) error { + err := b.Do(func(cr query.ColReader) error { record := row[recordStartIdx:] l := cr.Len() count += l @@ -688,7 +688,7 @@ func (e *ResultEncoder) Encode(w io.Writer, result execute.Result) error { }) } -func writeSchema(writer *csv.Writer, c *ResultEncoderConfig, row []string, cols []colMeta, useKeyDefaults bool, key execute.PartitionKey) error { +func writeSchema(writer *csv.Writer, c *ResultEncoderConfig, row []string, cols []colMeta, useKeyDefaults bool, key query.PartitionKey) error { defaults := make([]string, len(row)) for j, c := range cols { switch j { @@ -730,7 +730,7 @@ func writeSchema(writer *csv.Writer, c *ResultEncoderConfig, row []string, cols return writer.Error() } -func writeAnnotations(writer *csv.Writer, annotations []string, row, defaults []string, cols []colMeta, key execute.PartitionKey) error { +func writeAnnotations(writer *csv.Writer, annotations []string, row, defaults []string, cols []colMeta, key query.PartitionKey) error { for _, annotation := range annotations { switch annotation { case datatypeAnnotation: @@ -759,17 +759,17 @@ func writeDatatypes(writer *csv.Writer, row []string, cols []colMeta) error { continue } switch c.Type { - case execute.TBool: + case query.TBool: row[j] = boolDatatype - case execute.TInt: + case query.TInt: row[j] = intDatatype - case execute.TUInt: + case query.TUInt: row[j] = uintDatatype - case execute.TFloat: + case query.TFloat: row[j] = floatDatatype - case execute.TString: + case query.TString: row[j] = stringDatatype - case execute.TTime: + case query.TTime: row[j] = timeDataTypeWithFmt default: return fmt.Errorf("unknown column type %v", c.Type) @@ -778,7 +778,7 @@ func writeDatatypes(writer *csv.Writer, row []string, cols []colMeta) error { return writer.Write(row) } -func writePartitions(writer *csv.Writer, row []string, cols []colMeta, key execute.PartitionKey) error { +func writePartitions(writer *csv.Writer, row []string, cols []colMeta, key query.PartitionKey) error { for j, c := range cols { if j == annotationIdx { row[j] = commentPrefix + partitionAnnotation @@ -803,17 +803,17 @@ func writeDefaults(writer *csv.Writer, row, defaults []string) error { func decodeValue(value string, c colMeta) (v interface{}, err error) { switch c.Type { - case execute.TBool: + case query.TBool: v, err = strconv.ParseBool(value) - case execute.TInt: + case query.TInt: v, err = strconv.ParseInt(value, 10, 64) - case execute.TUInt: + case query.TUInt: v, err = strconv.ParseUint(value, 10, 64) - case execute.TFloat: + case query.TFloat: v, err = strconv.ParseFloat(value, 64) - case execute.TString: + case query.TString: v = value - case execute.TTime: + case query.TTime: v, err = decodeTime(value, c.fmt) default: return nil, fmt.Errorf("unsupported type %v", c.Type) @@ -823,33 +823,33 @@ func decodeValue(value string, c colMeta) (v interface{}, err error) { func decodeValueInto(j int, c colMeta, value string, builder execute.BlockBuilder) error { switch c.Type { - case execute.TBool: + case query.TBool: v, err := strconv.ParseBool(value) if err != nil { return err } builder.AppendBool(j, v) - case execute.TInt: + case query.TInt: v, err := strconv.ParseInt(value, 10, 64) if err != nil { return err } builder.AppendInt(j, v) - case execute.TUInt: + case query.TUInt: v, err := strconv.ParseUint(value, 10, 64) if err != nil { return err } builder.AppendUInt(j, v) - case execute.TFloat: + case query.TFloat: v, err := strconv.ParseFloat(value, 64) if err != nil { return err } builder.AppendFloat(j, v) - case execute.TString: + case query.TString: builder.AppendString(j, value) - case execute.TTime: + case query.TTime: t, err := decodeTime(value, c.fmt) if err != nil { return err @@ -863,36 +863,36 @@ func decodeValueInto(j int, c colMeta, value string, builder execute.BlockBuilde func encodeValue(value interface{}, c colMeta) (string, error) { switch c.Type { - case execute.TBool: + case query.TBool: return strconv.FormatBool(value.(bool)), nil - case execute.TInt: + case query.TInt: return strconv.FormatInt(value.(int64), 10), nil - case execute.TUInt: + case query.TUInt: return strconv.FormatUint(value.(uint64), 10), nil - case execute.TFloat: + case query.TFloat: return strconv.FormatFloat(value.(float64), 'f', -1, 64), nil - case execute.TString: + case query.TString: return value.(string), nil - case execute.TTime: + case query.TTime: return encodeTime(value.(execute.Time), c.fmt), nil default: return "", fmt.Errorf("unknown type %v", c.Type) } } -func encodeValueFrom(i, j int, c colMeta, cr execute.ColReader) (string, error) { +func encodeValueFrom(i, j int, c colMeta, cr query.ColReader) (string, error) { switch c.Type { - case execute.TBool: + case query.TBool: return strconv.FormatBool(cr.Bools(j)[i]), nil - case execute.TInt: + case query.TInt: return strconv.FormatInt(cr.Ints(j)[i], 10), nil - case execute.TUInt: + case query.TUInt: return strconv.FormatUint(cr.UInts(j)[i], 10), nil - case execute.TFloat: + case query.TFloat: return strconv.FormatFloat(cr.Floats(j)[i], 'f', -1, 64), nil - case execute.TString: + case query.TString: return cr.Strings(j)[i], nil - case execute.TTime: + case query.TTime: return encodeTime(cr.Times(j)[i], c.fmt), nil default: return "", fmt.Errorf("unknown type %v", c.Type) @@ -918,7 +918,7 @@ func copyLine(line []string) []string { } // decodeType returns the execute.DataType and any additional format description. -func decodeType(datatype string) (t execute.DataType, desc string, err error) { +func decodeType(datatype string) (t query.DataType, desc string, err error) { split := strings.SplitN(datatype, ":", 2) if len(split) > 1 { desc = split[1] @@ -926,17 +926,17 @@ func decodeType(datatype string) (t execute.DataType, desc string, err error) { typ := split[0] switch typ { case boolDatatype: - t = execute.TBool + t = query.TBool case intDatatype: - t = execute.TInt + t = query.TInt case uintDatatype: - t = execute.TUInt + t = query.TUInt case floatDatatype: - t = execute.TFloat + t = query.TFloat case stringDatatype: - t = execute.TString + t = query.TString case timeDatatype: - t = execute.TTime + t = query.TTime default: err = fmt.Errorf("unsupported data type %q", typ) } diff --git a/csv/result_test.go b/csv/result_test.go index 0cb1b4a0ff..bb573b20e3 100644 --- a/csv/result_test.go +++ b/csv/result_test.go @@ -8,10 +8,10 @@ import ( "github.com/andreyvit/diff" "github.com/google/go-cmp/cmp" - "github.com/influxdata/platform/query/execute" + "github.com/influxdata/platform/query" + "github.com/influxdata/platform/query/csv" "github.com/influxdata/platform/query/execute/executetest" "github.com/influxdata/platform/query/values" - "github.com/influxdata/platform/query/csv" ) type TestCase struct { @@ -36,13 +36,13 @@ var symetricalTestCases = []TestCase{ `), result: &executetest.Result{Blks: []*executetest.Block{{ KeyCols: []string{"_start", "_stop", "_measurement", "host"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_measurement", Type: execute.TString}, - {Label: "host", Type: execute.TString}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_measurement", Type: query.TString}, + {Label: "host", Type: query.TString}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ { @@ -82,13 +82,13 @@ var symetricalTestCases = []TestCase{ "cpu", "A", }, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_measurement", Type: execute.TString}, - {Label: "host", Type: execute.TString}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_measurement", Type: query.TString}, + {Label: "host", Type: query.TString}, + {Label: "_value", Type: query.TFloat}, }, }}}, }, @@ -107,13 +107,13 @@ var symetricalTestCases = []TestCase{ result: &executetest.Result{Blks: []*executetest.Block{ { KeyCols: []string{"_start", "_stop", "_measurement", "host"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_measurement", Type: execute.TString}, - {Label: "host", Type: execute.TString}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_measurement", Type: query.TString}, + {Label: "host", Type: query.TString}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ { @@ -136,13 +136,13 @@ var symetricalTestCases = []TestCase{ }, { KeyCols: []string{"_start", "_stop", "_measurement", "host"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_measurement", Type: execute.TString}, - {Label: "host", Type: execute.TString}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_measurement", Type: query.TString}, + {Label: "host", Type: query.TString}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ { @@ -189,13 +189,13 @@ var symetricalTestCases = []TestCase{ result: &executetest.Result{Blks: []*executetest.Block{ { KeyCols: []string{"_start", "_stop", "_measurement", "host"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_measurement", Type: execute.TString}, - {Label: "host", Type: execute.TString}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_measurement", Type: query.TString}, + {Label: "host", Type: query.TString}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ { @@ -218,13 +218,13 @@ var symetricalTestCases = []TestCase{ }, { KeyCols: []string{"_start", "_stop", "_measurement", "host"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_measurement", Type: execute.TString}, - {Label: "host", Type: execute.TString}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_measurement", Type: query.TString}, + {Label: "host", Type: query.TString}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ { @@ -247,14 +247,14 @@ var symetricalTestCases = []TestCase{ }, { KeyCols: []string{"_start", "_stop", "location"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "location", Type: execute.TString}, - {Label: "device", Type: execute.TString}, - {Label: "min", Type: execute.TFloat}, - {Label: "max", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "location", Type: query.TString}, + {Label: "device", Type: query.TString}, + {Label: "min", Type: query.TFloat}, + {Label: "max", Type: query.TFloat}, }, Data: [][]interface{}{ { @@ -279,14 +279,14 @@ var symetricalTestCases = []TestCase{ }, { KeyCols: []string{"_start", "_stop", "location"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "location", Type: execute.TString}, - {Label: "device", Type: execute.TString}, - {Label: "min", Type: execute.TFloat}, - {Label: "max", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "location", Type: query.TString}, + {Label: "device", Type: query.TString}, + {Label: "min", Type: query.TFloat}, + {Label: "max", Type: query.TFloat}, }, Data: [][]interface{}{ { @@ -333,13 +333,13 @@ var symetricalTestCases = []TestCase{ result: &executetest.Result{Blks: []*executetest.Block{ { KeyCols: []string{"_start", "_stop", "_measurement", "host"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_measurement", Type: execute.TString}, - {Label: "host", Type: execute.TString}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_measurement", Type: query.TString}, + {Label: "host", Type: query.TString}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ { @@ -362,13 +362,13 @@ var symetricalTestCases = []TestCase{ }, { KeyCols: []string{"_start", "_stop", "_measurement", "host"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_measurement", Type: execute.TString}, - {Label: "host", Type: execute.TString}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_measurement", Type: query.TString}, + {Label: "host", Type: query.TString}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ { @@ -397,13 +397,13 @@ var symetricalTestCases = []TestCase{ "cpu", "A", }, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_measurement", Type: execute.TString}, - {Label: "host", Type: execute.TString}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_measurement", Type: query.TString}, + {Label: "host", Type: query.TString}, + {Label: "_value", Type: query.TFloat}, }, }, }}, @@ -424,13 +424,13 @@ func TestResultDecoder(t *testing.T) { `), result: &executetest.Result{Blks: []*executetest.Block{{ KeyCols: []string{"_start", "_stop", "_measurement", "host"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_measurement", Type: execute.TString}, - {Label: "host", Type: execute.TString}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_measurement", Type: query.TString}, + {Label: "host", Type: query.TString}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ { @@ -466,7 +466,7 @@ func TestResultDecoder(t *testing.T) { t.Fatal(err) } got := new(executetest.Result) - if err := result.Blocks().Do(func(b execute.Block) error { + if err := result.Blocks().Do(func(b query.Block) error { cb, err := executetest.ConvertBlock(b) if err != nil { return err diff --git a/execute/aggergate_test.go b/execute/aggergate_test.go index d6e4005326..333f03db74 100644 --- a/execute/aggergate_test.go +++ b/execute/aggergate_test.go @@ -6,9 +6,10 @@ import ( "github.com/google/go-cmp/cmp" "github.com/google/go-cmp/cmp/cmpopts" - "github.com/influxdata/platform/query/functions" + "github.com/influxdata/platform/query" "github.com/influxdata/platform/query/execute" "github.com/influxdata/platform/query/execute/executetest" + "github.com/influxdata/platform/query/functions" ) func TestAggregate_Process(t *testing.T) { @@ -27,11 +28,11 @@ func TestAggregate_Process(t *testing.T) { agg: sumAgg, data: []*executetest.Block{{ KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(0), execute.Time(100), execute.Time(0), 0.0}, @@ -48,11 +49,11 @@ func TestAggregate_Process(t *testing.T) { }}, want: []*executetest.Block{{ KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(0), execute.Time(100), execute.Time(100), 45.0}, @@ -69,11 +70,11 @@ func TestAggregate_Process(t *testing.T) { agg: sumAgg, data: []*executetest.Block{{ KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(0), execute.Time(100), execute.Time(0), 0.0}, @@ -90,11 +91,11 @@ func TestAggregate_Process(t *testing.T) { }}, want: []*executetest.Block{{ KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(0), execute.Time(100), execute.Time(0), 45.0}, @@ -108,11 +109,11 @@ func TestAggregate_Process(t *testing.T) { data: []*executetest.Block{ { KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(0), execute.Time(100), execute.Time(0), 0.0}, @@ -129,11 +130,11 @@ func TestAggregate_Process(t *testing.T) { }, { KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(100), execute.Time(200), execute.Time(100), 10.0}, @@ -152,11 +153,11 @@ func TestAggregate_Process(t *testing.T) { want: []*executetest.Block{ { KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(0), execute.Time(100), execute.Time(100), 45.0}, @@ -164,11 +165,11 @@ func TestAggregate_Process(t *testing.T) { }, { KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(100), execute.Time(200), execute.Time(200), 145.0}, @@ -183,12 +184,12 @@ func TestAggregate_Process(t *testing.T) { data: []*executetest.Block{ { KeyCols: []string{"_start", "_stop", "t1"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "t1", Type: execute.TString}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "t1", Type: query.TString}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(0), execute.Time(100), "a", execute.Time(0), 0.0}, @@ -205,12 +206,12 @@ func TestAggregate_Process(t *testing.T) { }, { KeyCols: []string{"_start", "_stop", "t1"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "t1", Type: execute.TString}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "t1", Type: query.TString}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(0), execute.Time(100), "b", execute.Time(0), 0.3}, @@ -227,12 +228,12 @@ func TestAggregate_Process(t *testing.T) { }, { KeyCols: []string{"_start", "_stop", "t1"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "t1", Type: execute.TString}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "t1", Type: query.TString}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(100), execute.Time(200), "a", execute.Time(100), 10.0}, @@ -249,12 +250,12 @@ func TestAggregate_Process(t *testing.T) { }, { KeyCols: []string{"_start", "_stop", "t1"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "t1", Type: execute.TString}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "t1", Type: query.TString}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(100), execute.Time(200), "b", execute.Time(100), 10.3}, @@ -273,12 +274,12 @@ func TestAggregate_Process(t *testing.T) { want: []*executetest.Block{ { KeyCols: []string{"_start", "_stop", "t1"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "t1", Type: execute.TString}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "t1", Type: query.TString}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(0), execute.Time(100), "a", execute.Time(100), 45.0}, @@ -286,12 +287,12 @@ func TestAggregate_Process(t *testing.T) { }, { KeyCols: []string{"_start", "_stop", "t1"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "t1", Type: execute.TString}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "t1", Type: query.TString}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(100), execute.Time(200), "a", execute.Time(200), 145.0}, @@ -299,12 +300,12 @@ func TestAggregate_Process(t *testing.T) { }, { KeyCols: []string{"_start", "_stop", "t1"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "t1", Type: execute.TString}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "t1", Type: query.TString}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(0), execute.Time(100), "b", execute.Time(100), 48.0}, @@ -312,12 +313,12 @@ func TestAggregate_Process(t *testing.T) { }, { KeyCols: []string{"_start", "_stop", "t1"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "t1", Type: execute.TString}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "t1", Type: query.TString}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(100), execute.Time(200), "b", execute.Time(200), 148.0}, @@ -335,12 +336,12 @@ func TestAggregate_Process(t *testing.T) { agg: sumAgg, data: []*executetest.Block{{ KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "x", Type: execute.TFloat}, - {Label: "y", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "x", Type: query.TFloat}, + {Label: "y", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(0), execute.Time(100), execute.Time(0), 0.0, 0.0}, @@ -357,12 +358,12 @@ func TestAggregate_Process(t *testing.T) { }}, want: []*executetest.Block{{ KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "x", Type: execute.TFloat}, - {Label: "y", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "x", Type: query.TFloat}, + {Label: "y", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(0), execute.Time(100), execute.Time(100), 45.0, -45.0}, @@ -379,12 +380,12 @@ func TestAggregate_Process(t *testing.T) { agg: countAgg, data: []*executetest.Block{{ KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "x", Type: execute.TFloat}, - {Label: "y", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "x", Type: query.TFloat}, + {Label: "y", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(0), execute.Time(100), execute.Time(0), 0.0, 0.0}, @@ -401,12 +402,12 @@ func TestAggregate_Process(t *testing.T) { }}, want: []*executetest.Block{{ KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "x", Type: execute.TInt}, - {Label: "y", Type: execute.TInt}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "x", Type: query.TInt}, + {Label: "y", Type: query.TInt}, }, Data: [][]interface{}{ {execute.Time(0), execute.Time(100), execute.Time(100), int64(10), int64(10)}, diff --git a/execute/aggregate.go b/execute/aggregate.go index b177f49dc0..fdae642285 100644 --- a/execute/aggregate.go +++ b/execute/aggregate.go @@ -3,8 +3,8 @@ package execute import ( "fmt" - "github.com/influxdata/platform/query/interpreter" "github.com/influxdata/platform/query" + "github.com/influxdata/platform/query/interpreter" "github.com/influxdata/platform/query/semantic" "github.com/pkg/errors" ) @@ -84,21 +84,21 @@ func NewAggregateTransformationAndDataset(id DatasetID, mode AccumulationMode, a return NewAggregateTransformation(d, cache, agg, config), d } -func (t *aggregateTransformation) RetractBlock(id DatasetID, key PartitionKey) error { +func (t *aggregateTransformation) RetractBlock(id DatasetID, key query.PartitionKey) error { //TODO(nathanielc): Store intermediate state for retractions return t.d.RetractBlock(key) } -func (t *aggregateTransformation) Process(id DatasetID, b Block) error { +func (t *aggregateTransformation) Process(id DatasetID, b query.Block) error { builder, new := t.cache.BlockBuilder(b.Key()) if !new { return fmt.Errorf("aggregate found duplicate block with key: %v", b.Key()) } AddBlockKeyCols(b.Key(), builder) - builder.AddCol(ColMeta{ + builder.AddCol(query.ColMeta{ Label: t.config.TimeDst, - Type: TTime, + Type: query.TTime, }) builderColMap := make([]int, len(t.config.Columns)) @@ -123,21 +123,21 @@ func (t *aggregateTransformation) Process(id DatasetID, b Block) error { } var vf ValueFunc switch c.Type { - case TBool: + case query.TBool: vf = t.agg.NewBoolAgg() - case TInt: + case query.TInt: vf = t.agg.NewIntAgg() - case TUInt: + case query.TUInt: vf = t.agg.NewUIntAgg() - case TFloat: + case query.TFloat: vf = t.agg.NewFloatAgg() - case TString: + case query.TString: vf = t.agg.NewStringAgg() default: return fmt.Errorf("unsupported aggregate column type %v", c.Type) } aggregates[j] = vf - builderColMap[j] = builder.AddCol(ColMeta{ + builderColMap[j] = builder.AddCol(query.ColMeta{ Label: c.Label, Type: vf.Type(), }) @@ -147,7 +147,7 @@ func (t *aggregateTransformation) Process(id DatasetID, b Block) error { return err } - b.Do(func(cr ColReader) error { + b.Do(func(cr query.ColReader) error { for j := range t.config.Columns { vf := aggregates[j] @@ -155,15 +155,15 @@ func (t *aggregateTransformation) Process(id DatasetID, b Block) error { c := b.Cols()[tj] switch c.Type { - case TBool: + case query.TBool: vf.(DoBoolAgg).DoBool(cr.Bools(tj)) - case TInt: + case query.TInt: vf.(DoIntAgg).DoInt(cr.Ints(tj)) - case TUInt: + case query.TUInt: vf.(DoUIntAgg).DoUInt(cr.UInts(tj)) - case TFloat: + case query.TFloat: vf.(DoFloatAgg).DoFloat(cr.Floats(tj)) - case TString: + case query.TString: vf.(DoStringAgg).DoString(cr.Strings(tj)) default: return fmt.Errorf("unsupport aggregate type %v", c.Type) @@ -175,15 +175,15 @@ func (t *aggregateTransformation) Process(id DatasetID, b Block) error { bj := builderColMap[j] // Append aggregated value switch vf.Type() { - case TBool: + case query.TBool: builder.AppendBool(bj, vf.(BoolValueFunc).ValueBool()) - case TInt: + case query.TInt: builder.AppendInt(bj, vf.(IntValueFunc).ValueInt()) - case TUInt: + case query.TUInt: builder.AppendUInt(bj, vf.(UIntValueFunc).ValueUInt()) - case TFloat: + case query.TFloat: builder.AppendFloat(bj, vf.(FloatValueFunc).ValueFloat()) - case TString: + case query.TString: builder.AppendString(bj, vf.(StringValueFunc).ValueString()) } } @@ -203,13 +203,13 @@ func (t *aggregateTransformation) Finish(id DatasetID, err error) { t.d.Finish(err) } -func AppendAggregateTime(srcTime, dstTime string, key PartitionKey, builder BlockBuilder) error { +func AppendAggregateTime(srcTime, dstTime string, key query.PartitionKey, builder BlockBuilder) error { srcTimeIdx := ColIdx(srcTime, key.Cols()) if srcTimeIdx < 0 { return fmt.Errorf("timeValue column %q does not exist", srcTime) } srcTimeCol := key.Cols()[srcTimeIdx] - if srcTimeCol.Type != TTime { + if srcTimeCol.Type != query.TTime { return fmt.Errorf("timeValue column %q does not have type time", srcTime) } @@ -218,7 +218,7 @@ func AppendAggregateTime(srcTime, dstTime string, key PartitionKey, builder Bloc return fmt.Errorf("timeValue column %q does not exist", dstTime) } dstTimeCol := builder.Cols()[dstTimeIdx] - if dstTimeCol.Type != TTime { + if dstTimeCol.Type != query.TTime { return fmt.Errorf("timeValue column %q does not have type time", dstTime) } @@ -235,7 +235,7 @@ type Aggregate interface { } type ValueFunc interface { - Type() DataType + Type() query.DataType } type DoBoolAgg interface { ValueFunc diff --git a/execute/block.go b/execute/block.go index d1512bec30..8bf6e6382f 100644 --- a/execute/block.go +++ b/execute/block.go @@ -17,45 +17,21 @@ const ( DefaultValueColLabel = "_value" ) -type PartitionKey interface { - Cols() []ColMeta - - HasCol(label string) bool - - ValueBool(j int) bool - ValueUInt(j int) uint64 - ValueInt(j int) int64 - ValueFloat(j int) float64 - ValueString(j int) string - ValueDuration(j int) Duration - ValueTime(j int) Time - Value(j int) interface{} - - // Intersect returns a new PartitionKey with only columns in the list of labels. - Intersect(labels []string) PartitionKey - // Diff returns the labels that exist in list of labels but not in the key's columns. - Diff(labels []string) []string - Hash() uint64 - Equal(o PartitionKey) bool - Less(o PartitionKey) bool - String() string -} - type partitionKey struct { - cols []ColMeta + cols []query.ColMeta values []interface{} hasHash bool hash uint64 } -func NewPartitionKey(cols []ColMeta, values []interface{}) PartitionKey { +func NewPartitionKey(cols []query.ColMeta, values []interface{}) query.PartitionKey { return &partitionKey{ cols: cols, values: values, } } -func (k *partitionKey) Cols() []ColMeta { +func (k *partitionKey) Cols() []query.ColMeta { return k.cols } func (k *partitionKey) HasCol(label string) bool { @@ -86,9 +62,9 @@ func (k *partitionKey) ValueTime(j int) Time { return k.values[j].(Time) } -func (k *partitionKey) Intersect(keys []string) PartitionKey { +func (k *partitionKey) Intersect(keys []string) query.PartitionKey { nk := &partitionKey{ - cols: make([]ColMeta, 0, len(k.cols)), + cols: make([]query.ColMeta, 0, len(k.cols)), values: make([]interface{}, 0, len(k.values)), } for i, c := range k.cols { @@ -125,15 +101,15 @@ func (k *partitionKey) Hash() uint64 { return k.hash } -func (k *partitionKey) Equal(o PartitionKey) bool { +func (k *partitionKey) Equal(o query.PartitionKey) bool { return partitionKeyEqual(k, o) } -func (k *partitionKey) Less(o PartitionKey) bool { +func (k *partitionKey) Less(o query.PartitionKey) bool { return partitionKeyLess(k, o) } -func partitionKeyEqual(a, b PartitionKey) bool { +func partitionKeyEqual(a, b query.PartitionKey) bool { if a.Hash() != b.Hash() { return false } @@ -147,27 +123,27 @@ func partitionKeyEqual(a, b PartitionKey) bool { return false } switch c.Type { - case TBool: + case query.TBool: if a.ValueBool(j) != b.ValueBool(j) { return false } - case TInt: + case query.TInt: if a.ValueInt(j) != b.ValueInt(j) { return false } - case TUInt: + case query.TUInt: if a.ValueUInt(j) != b.ValueUInt(j) { return false } - case TFloat: + case query.TFloat: if a.ValueFloat(j) != b.ValueFloat(j) { return false } - case TString: + case query.TString: if a.ValueString(j) != b.ValueString(j) { return false } - case TTime: + case query.TTime: if a.ValueTime(j) != b.ValueTime(j) { return false } @@ -176,7 +152,7 @@ func partitionKeyEqual(a, b PartitionKey) bool { return true } -func partitionKeyLess(a, b PartitionKey) bool { +func partitionKeyLess(a, b query.PartitionKey) bool { aCols := a.Cols() bCols := b.Cols() if av, bv := len(aCols), len(bCols); av != bv { @@ -187,27 +163,27 @@ func partitionKeyLess(a, b PartitionKey) bool { return aCols[j].Label < bCols[j].Label } switch c.Type { - case TBool: + case query.TBool: if av, bv := a.ValueBool(j), b.ValueBool(j); av != bv { return av } - case TInt: + case query.TInt: if av, bv := a.ValueInt(j), b.ValueInt(j); av != bv { return av < bv } - case TUInt: + case query.TUInt: if av, bv := a.ValueUInt(j), b.ValueUInt(j); av != bv { return av < bv } - case TFloat: + case query.TFloat: if av, bv := a.ValueFloat(j), b.ValueFloat(j); av != bv { return av < bv } - case TString: + case query.TString: if av, bv := a.ValueString(j), b.ValueString(j); av != bv { return av < bv } - case TTime: + case query.TTime: if av, bv := a.ValueTime(j), b.ValueTime(j); av != bv { return av < bv } @@ -229,10 +205,10 @@ func (k *partitionKey) String() string { return b.String() } -func PartitionKeyForRow(i int, cr ColReader) PartitionKey { +func PartitionKeyForRow(i int, cr query.ColReader) query.PartitionKey { key := cr.Key() cols := cr.Cols() - colsCpy := make([]ColMeta, 0, len(cols)) + colsCpy := make([]query.ColMeta, 0, len(cols)) values := make([]interface{}, 0, len(cols)) for j, c := range cols { if !key.HasCol(c.Label) { @@ -240,17 +216,17 @@ func PartitionKeyForRow(i int, cr ColReader) PartitionKey { } colsCpy = append(colsCpy, c) switch c.Type { - case TBool: + case query.TBool: values = append(values, cr.Bools(j)[i]) - case TInt: + case query.TInt: values = append(values, cr.Ints(j)[i]) - case TUInt: + case query.TUInt: values = append(values, cr.UInts(j)[i]) - case TFloat: + case query.TFloat: values = append(values, cr.Floats(j)[i]) - case TString: + case query.TString: values = append(values, cr.Strings(j)[i]) - case TTime: + case query.TTime: values = append(values, cr.Times(j)[i]) } } @@ -260,8 +236,8 @@ func PartitionKeyForRow(i int, cr ColReader) PartitionKey { } } -func PartitionKeyForRowOn(i int, cr ColReader, on map[string]bool) PartitionKey { - cols := make([]ColMeta, 0, len(on)) +func PartitionKeyForRowOn(i int, cr query.ColReader, on map[string]bool) query.PartitionKey { + cols := make([]query.ColMeta, 0, len(on)) values := make([]interface{}, 0, len(on)) for j, c := range cr.Cols() { if !on[c.Label] { @@ -269,48 +245,34 @@ func PartitionKeyForRowOn(i int, cr ColReader, on map[string]bool) PartitionKey } cols = append(cols, c) switch c.Type { - case TBool: + case query.TBool: values = append(values, cr.Bools(j)[i]) - case TInt: + case query.TInt: values = append(values, cr.Ints(j)[i]) - case TUInt: + case query.TUInt: values = append(values, cr.UInts(j)[i]) - case TFloat: + case query.TFloat: values = append(values, cr.Floats(j)[i]) - case TString: + case query.TString: values = append(values, cr.Strings(j)[i]) - case TTime: + case query.TTime: values = append(values, cr.Times(j)[i]) } } return NewPartitionKey(cols, values) } -type Block interface { - Key() PartitionKey - - Cols() []ColMeta - - // Do calls f to process the data contained within the block. - // The function f will be called zero or more times. - Do(f func(ColReader) error) error - - // RefCount modifies the reference count on the block by n. - // When the RefCount goes to zero, the block is freed. - RefCount(n int) -} - // OneTimeBlock is a Block that permits reading data only once. // Specifically the ValueIterator may only be consumed once from any of the columns. type OneTimeBlock interface { - Block + query.Block onetime() } // CacheOneTimeBlock returns a block that can be read multiple times. // If the block is not a OneTimeBlock it is returned directly. // Otherwise its contents are read into a new block. -func CacheOneTimeBlock(b Block, a *Allocator) Block { +func CacheOneTimeBlock(b query.Block, a *Allocator) query.Block { _, ok := b.(OneTimeBlock) if !ok { return b @@ -319,7 +281,7 @@ func CacheOneTimeBlock(b Block, a *Allocator) Block { } // CopyBlock returns a copy of the block and is OneTimeBlock safe. -func CopyBlock(b Block, a *Allocator) Block { +func CopyBlock(b query.Block, a *Allocator) query.Block { builder := NewColListBlockBuilder(b.Key(), a) cols := b.Cols() @@ -336,14 +298,14 @@ func CopyBlock(b Block, a *Allocator) Block { } // AddBlockCols adds the columns of b onto builder. -func AddBlockCols(b Block, builder BlockBuilder) { +func AddBlockCols(b query.Block, builder BlockBuilder) { cols := b.Cols() for _, c := range cols { builder.AddCol(c) } } -func AddBlockKeyCols(key PartitionKey, builder BlockBuilder) { +func AddBlockKeyCols(key query.PartitionKey, builder BlockBuilder) { for _, c := range key.Cols() { builder.AddCol(c) } @@ -351,7 +313,7 @@ func AddBlockKeyCols(key PartitionKey, builder BlockBuilder) { // AddNewCols adds the columns of b onto builder that did not already exist. // Returns the mapping of builder cols to block cols. -func AddNewCols(b Block, builder BlockBuilder) []int { +func AddNewCols(b query.Block, builder BlockBuilder) []int { cols := b.Cols() existing := builder.Cols() colMap := make([]int, len(existing)) @@ -374,12 +336,12 @@ func AddNewCols(b Block, builder BlockBuilder) []int { // AppendBlock append data from block b onto builder. // The colMap is a map of builder column index to block column index. -func AppendBlock(b Block, builder BlockBuilder, colMap []int) { +func AppendBlock(b query.Block, builder BlockBuilder, colMap []int) { if len(b.Cols()) == 0 { return } - b.Do(func(cr ColReader) error { + b.Do(func(cr query.ColReader) error { AppendCols(cr, builder, colMap) return nil }) @@ -387,7 +349,7 @@ func AppendBlock(b Block, builder BlockBuilder, colMap []int) { // AppendCols appends all columns from cr onto builder. // The colMap is a map of builder column index to cr column index. -func AppendCols(cr ColReader, builder BlockBuilder, colMap []int) { +func AppendCols(cr query.ColReader, builder BlockBuilder, colMap []int) { for j := range builder.Cols() { AppendCol(j, colMap[j], cr, builder) } @@ -395,20 +357,20 @@ func AppendCols(cr ColReader, builder BlockBuilder, colMap []int) { // AppendCol append a column from cr onto builder // The indexes bj and cj are builder and col reader indexes respectively. -func AppendCol(bj, cj int, cr ColReader, builder BlockBuilder) { +func AppendCol(bj, cj int, cr query.ColReader, builder BlockBuilder) { c := cr.Cols()[cj] switch c.Type { - case TBool: + case query.TBool: builder.AppendBools(bj, cr.Bools(cj)) - case TInt: + case query.TInt: builder.AppendInts(bj, cr.Ints(cj)) - case TUInt: + case query.TUInt: builder.AppendUInts(bj, cr.UInts(cj)) - case TFloat: + case query.TFloat: builder.AppendFloats(bj, cr.Floats(cj)) - case TString: + case query.TString: builder.AppendStrings(bj, cr.Strings(cj)) - case TTime: + case query.TTime: builder.AppendTimes(bj, cr.Times(cj)) default: PanicUnknownType(c.Type) @@ -416,20 +378,20 @@ func AppendCol(bj, cj int, cr ColReader, builder BlockBuilder) { } // AppendMappedRecord appends the record from cr onto builder assuming matching columns. -func AppendRecord(i int, cr ColReader, builder BlockBuilder) { +func AppendRecord(i int, cr query.ColReader, builder BlockBuilder) { for j, c := range builder.Cols() { switch c.Type { - case TBool: + case query.TBool: builder.AppendBool(j, cr.Bools(j)[i]) - case TInt: + case query.TInt: builder.AppendInt(j, cr.Ints(j)[i]) - case TUInt: + case query.TUInt: builder.AppendUInt(j, cr.UInts(j)[i]) - case TFloat: + case query.TFloat: builder.AppendFloat(j, cr.Floats(j)[i]) - case TString: + case query.TString: builder.AppendString(j, cr.Strings(j)[i]) - case TTime: + case query.TTime: builder.AppendTime(j, cr.Times(j)[i]) default: PanicUnknownType(c.Type) @@ -438,20 +400,20 @@ func AppendRecord(i int, cr ColReader, builder BlockBuilder) { } // AppendMappedRecord appends the records from cr onto builder, using colMap as a map of builder index to cr index. -func AppendMappedRecord(i int, cr ColReader, builder BlockBuilder, colMap []int) { +func AppendMappedRecord(i int, cr query.ColReader, builder BlockBuilder, colMap []int) { for j, c := range builder.Cols() { switch c.Type { - case TBool: + case query.TBool: builder.AppendBool(j, cr.Bools(colMap[j])[i]) - case TInt: + case query.TInt: builder.AppendInt(j, cr.Ints(colMap[j])[i]) - case TUInt: + case query.TUInt: builder.AppendUInt(j, cr.UInts(colMap[j])[i]) - case TFloat: + case query.TFloat: builder.AppendFloat(j, cr.Floats(colMap[j])[i]) - case TString: + case query.TString: builder.AppendString(j, cr.Strings(colMap[j])[i]) - case TTime: + case query.TTime: builder.AppendTime(j, cr.Times(colMap[j])[i]) default: PanicUnknownType(c.Type) @@ -460,20 +422,20 @@ func AppendMappedRecord(i int, cr ColReader, builder BlockBuilder, colMap []int) } // AppendRecordForCols appends the only the columns provided from cr onto builder. -func AppendRecordForCols(i int, cr ColReader, builder BlockBuilder, cols []ColMeta) { +func AppendRecordForCols(i int, cr query.ColReader, builder BlockBuilder, cols []query.ColMeta) { for j, c := range cols { switch c.Type { - case TBool: + case query.TBool: builder.AppendBool(j, cr.Bools(j)[i]) - case TInt: + case query.TInt: builder.AppendInt(j, cr.Ints(j)[i]) - case TUInt: + case query.TUInt: builder.AppendUInt(j, cr.UInts(j)[i]) - case TFloat: + case query.TFloat: builder.AppendFloat(j, cr.Floats(j)[i]) - case TString: + case query.TString: builder.AppendString(j, cr.Strings(j)[i]) - case TTime: + case query.TTime: builder.AppendTime(j, cr.Times(j)[i]) default: PanicUnknownType(c.Type) @@ -481,21 +443,21 @@ func AppendRecordForCols(i int, cr ColReader, builder BlockBuilder, cols []ColMe } } -func AppendKeyValues(key PartitionKey, builder BlockBuilder) { +func AppendKeyValues(key query.PartitionKey, builder BlockBuilder) { for j, c := range key.Cols() { idx := ColIdx(c.Label, builder.Cols()) switch c.Type { - case TBool: + case query.TBool: builder.AppendBool(idx, key.ValueBool(j)) - case TInt: + case query.TInt: builder.AppendInt(idx, key.ValueInt(j)) - case TUInt: + case query.TUInt: builder.AppendUInt(idx, key.ValueUInt(j)) - case TFloat: + case query.TFloat: builder.AppendFloat(idx, key.ValueFloat(j)) - case TString: + case query.TString: builder.AppendString(idx, key.ValueString(j)) - case TTime: + case query.TTime: builder.AppendTime(idx, key.ValueTime(j)) default: PanicUnknownType(c.Type) @@ -512,7 +474,7 @@ func ContainsStr(strs []string, str string) bool { return false } -func ColIdx(label string, cols []ColMeta) int { +func ColIdx(label string, cols []query.ColMeta) int { for j, c := range cols { if c.Label == label { return j @@ -520,21 +482,21 @@ func ColIdx(label string, cols []ColMeta) int { } return -1 } -func HasCol(label string, cols []ColMeta) bool { +func HasCol(label string, cols []query.ColMeta) bool { return ColIdx(label, cols) >= 0 } // BlockBuilder builds blocks that can be used multiple times type BlockBuilder interface { - Key() PartitionKey + Key() query.PartitionKey NRows() int NCols() int - Cols() []ColMeta + Cols() []query.ColMeta // AddCol increases the size of the block by one column. // The index of the column is returned. - AddCol(ColMeta) int + AddCol(query.ColMeta) int // Set sets the value at the specified coordinates // The rows and columns must exist before calling set, otherwise Set panics. @@ -567,67 +529,7 @@ type BlockBuilder interface { // Block returns the block that has been built. // Further modifications of the builder will not effect the returned block. - Block() (Block, error) -} - -type DataType int - -const ( - TInvalid DataType = iota - TBool - TInt - TUInt - TFloat - TString - TTime -) - -func (t DataType) String() string { - switch t { - case TInvalid: - return "invalid" - case TBool: - return "bool" - case TInt: - return "int" - case TUInt: - return "uint" - case TFloat: - return "float" - case TString: - return "string" - case TTime: - return "time" - default: - return "unknown" - } -} - -type ColMeta struct { - Label string - Type DataType -} - -type BlockIterator interface { - Do(f func(Block) error) error -} - -// ColReader allows access to reading slices of column data. -// All data the ColReader exposes is guaranteed to be in memory. -// Once a ColReader goes out of scope all slices are considered invalid. -type ColReader interface { - Key() PartitionKey - // Cols returns a list of column metadata. - Cols() []ColMeta - // Len returns the length of the slices. - // All slices will have the same length. - Len() int - Bools(j int) []bool - Ints(j int) []int64 - UInts(j int) []uint64 - Floats(j int) []float64 - Strings(j int) []string - Times(j int) []Time + Block() (query.Block, error) } type ColListBlockBuilder struct { @@ -635,14 +537,14 @@ type ColListBlockBuilder struct { alloc *Allocator } -func NewColListBlockBuilder(key PartitionKey, a *Allocator) *ColListBlockBuilder { +func NewColListBlockBuilder(key query.PartitionKey, a *Allocator) *ColListBlockBuilder { return &ColListBlockBuilder{ blk: &ColListBlock{key: key}, alloc: a, } } -func (b ColListBlockBuilder) Key() PartitionKey { +func (b ColListBlockBuilder) Key() query.PartitionKey { return b.blk.Key() } @@ -652,39 +554,39 @@ func (b ColListBlockBuilder) NRows() int { func (b ColListBlockBuilder) NCols() int { return len(b.blk.cols) } -func (b ColListBlockBuilder) Cols() []ColMeta { +func (b ColListBlockBuilder) Cols() []query.ColMeta { return b.blk.colMeta } -func (b ColListBlockBuilder) AddCol(c ColMeta) int { +func (b ColListBlockBuilder) AddCol(c query.ColMeta) int { var col column switch c.Type { - case TBool: + case query.TBool: col = &boolColumn{ ColMeta: c, alloc: b.alloc, } - case TInt: + case query.TInt: col = &intColumn{ ColMeta: c, alloc: b.alloc, } - case TUInt: + case query.TUInt: col = &uintColumn{ ColMeta: c, alloc: b.alloc, } - case TFloat: + case query.TFloat: col = &floatColumn{ ColMeta: c, alloc: b.alloc, } - case TString: + case query.TString: col = &stringColumn{ ColMeta: c, alloc: b.alloc, } - case TTime: + case query.TTime: col = &timeColumn{ ColMeta: c, alloc: b.alloc, @@ -698,123 +600,123 @@ func (b ColListBlockBuilder) AddCol(c ColMeta) int { } func (b ColListBlockBuilder) SetBool(i int, j int, value bool) { - b.checkColType(j, TBool) + b.checkColType(j, query.TBool) b.blk.cols[j].(*boolColumn).data[i] = value } func (b ColListBlockBuilder) AppendBool(j int, value bool) { - b.checkColType(j, TBool) + b.checkColType(j, query.TBool) col := b.blk.cols[j].(*boolColumn) col.data = b.alloc.AppendBools(col.data, value) b.blk.nrows = len(col.data) } func (b ColListBlockBuilder) AppendBools(j int, values []bool) { - b.checkColType(j, TBool) + b.checkColType(j, query.TBool) col := b.blk.cols[j].(*boolColumn) col.data = b.alloc.AppendBools(col.data, values...) b.blk.nrows = len(col.data) } func (b ColListBlockBuilder) SetInt(i int, j int, value int64) { - b.checkColType(j, TInt) + b.checkColType(j, query.TInt) b.blk.cols[j].(*intColumn).data[i] = value } func (b ColListBlockBuilder) AppendInt(j int, value int64) { - b.checkColType(j, TInt) + b.checkColType(j, query.TInt) col := b.blk.cols[j].(*intColumn) col.data = b.alloc.AppendInts(col.data, value) b.blk.nrows = len(col.data) } func (b ColListBlockBuilder) AppendInts(j int, values []int64) { - b.checkColType(j, TInt) + b.checkColType(j, query.TInt) col := b.blk.cols[j].(*intColumn) col.data = b.alloc.AppendInts(col.data, values...) b.blk.nrows = len(col.data) } func (b ColListBlockBuilder) SetUInt(i int, j int, value uint64) { - b.checkColType(j, TUInt) + b.checkColType(j, query.TUInt) b.blk.cols[j].(*uintColumn).data[i] = value } func (b ColListBlockBuilder) AppendUInt(j int, value uint64) { - b.checkColType(j, TUInt) + b.checkColType(j, query.TUInt) col := b.blk.cols[j].(*uintColumn) col.data = b.alloc.AppendUInts(col.data, value) b.blk.nrows = len(col.data) } func (b ColListBlockBuilder) AppendUInts(j int, values []uint64) { - b.checkColType(j, TUInt) + b.checkColType(j, query.TUInt) col := b.blk.cols[j].(*uintColumn) col.data = b.alloc.AppendUInts(col.data, values...) b.blk.nrows = len(col.data) } func (b ColListBlockBuilder) SetFloat(i int, j int, value float64) { - b.checkColType(j, TFloat) + b.checkColType(j, query.TFloat) b.blk.cols[j].(*floatColumn).data[i] = value } func (b ColListBlockBuilder) AppendFloat(j int, value float64) { - b.checkColType(j, TFloat) + b.checkColType(j, query.TFloat) col := b.blk.cols[j].(*floatColumn) col.data = b.alloc.AppendFloats(col.data, value) b.blk.nrows = len(col.data) } func (b ColListBlockBuilder) AppendFloats(j int, values []float64) { - b.checkColType(j, TFloat) + b.checkColType(j, query.TFloat) col := b.blk.cols[j].(*floatColumn) col.data = b.alloc.AppendFloats(col.data, values...) b.blk.nrows = len(col.data) } func (b ColListBlockBuilder) SetString(i int, j int, value string) { - b.checkColType(j, TString) + b.checkColType(j, query.TString) b.blk.cols[j].(*stringColumn).data[i] = value } func (b ColListBlockBuilder) AppendString(j int, value string) { meta := b.blk.cols[j].Meta() - CheckColType(meta, TString) + CheckColType(meta, query.TString) col := b.blk.cols[j].(*stringColumn) col.data = b.alloc.AppendStrings(col.data, value) b.blk.nrows = len(col.data) } func (b ColListBlockBuilder) AppendStrings(j int, values []string) { - b.checkColType(j, TString) + b.checkColType(j, query.TString) col := b.blk.cols[j].(*stringColumn) col.data = b.alloc.AppendStrings(col.data, values...) b.blk.nrows = len(col.data) } func (b ColListBlockBuilder) SetTime(i int, j int, value Time) { - b.checkColType(j, TTime) + b.checkColType(j, query.TTime) b.blk.cols[j].(*timeColumn).data[i] = value } func (b ColListBlockBuilder) AppendTime(j int, value Time) { - b.checkColType(j, TTime) + b.checkColType(j, query.TTime) col := b.blk.cols[j].(*timeColumn) col.data = b.alloc.AppendTimes(col.data, value) b.blk.nrows = len(col.data) } func (b ColListBlockBuilder) AppendTimes(j int, values []Time) { - b.checkColType(j, TTime) + b.checkColType(j, query.TTime) col := b.blk.cols[j].(*timeColumn) col.data = b.alloc.AppendTimes(col.data, values...) b.blk.nrows = len(col.data) } -func (b ColListBlockBuilder) checkColType(j int, typ DataType) { +func (b ColListBlockBuilder) checkColType(j int, typ query.DataType) { CheckColType(b.blk.colMeta[j], typ) } -func CheckColType(col ColMeta, typ DataType) { +func CheckColType(col query.ColMeta, typ query.DataType) { if col.Type != typ { panic(fmt.Errorf("column %s is not of type %v", col.Label, typ)) } } -func PanicUnknownType(typ DataType) { +func PanicUnknownType(typ query.DataType) { panic(fmt.Errorf("unknown type %v", typ)) } -func (b ColListBlockBuilder) Block() (Block, error) { +func (b ColListBlockBuilder) Block() (query.Block, error) { // Create copy in mutable state return b.blk.Copy(), nil } @@ -851,8 +753,8 @@ func (b ColListBlockBuilder) Sort(cols []string, desc bool) { // All data for the block is stored in RAM. // As a result At* methods are provided directly on the block for easy access. type ColListBlock struct { - key PartitionKey - colMeta []ColMeta + key query.PartitionKey + colMeta []query.ColMeta cols []column nrows int @@ -868,10 +770,10 @@ func (b *ColListBlock) RefCount(n int) { } } -func (b *ColListBlock) Key() PartitionKey { +func (b *ColListBlock) Key() query.PartitionKey { return b.key } -func (b *ColListBlock) Cols() []ColMeta { +func (b *ColListBlock) Cols() []query.ColMeta { return b.colMeta } func (b *ColListBlock) NRows() int { @@ -882,33 +784,33 @@ func (b *ColListBlock) Len() int { return b.nrows } -func (b *ColListBlock) Do(f func(ColReader) error) error { +func (b *ColListBlock) Do(f func(query.ColReader) error) error { return f(b) } func (b *ColListBlock) Bools(j int) []bool { - CheckColType(b.colMeta[j], TBool) + CheckColType(b.colMeta[j], query.TBool) return b.cols[j].(*boolColumn).data } func (b *ColListBlock) Ints(j int) []int64 { - CheckColType(b.colMeta[j], TInt) + CheckColType(b.colMeta[j], query.TInt) return b.cols[j].(*intColumn).data } func (b *ColListBlock) UInts(j int) []uint64 { - CheckColType(b.colMeta[j], TUInt) + CheckColType(b.colMeta[j], query.TUInt) return b.cols[j].(*uintColumn).data } func (b *ColListBlock) Floats(j int) []float64 { - CheckColType(b.colMeta[j], TFloat) + CheckColType(b.colMeta[j], query.TFloat) return b.cols[j].(*floatColumn).data } func (b *ColListBlock) Strings(j int) []string { meta := b.colMeta[j] - CheckColType(meta, TString) + CheckColType(meta, query.TString) return b.cols[j].(*stringColumn).data } func (b *ColListBlock) Times(j int) []Time { - CheckColType(b.colMeta[j], TTime) + CheckColType(b.colMeta[j], query.TTime) return b.cols[j].(*timeColumn).data } @@ -917,7 +819,7 @@ func (b *ColListBlock) Copy() *ColListBlock { cpy.key = b.key cpy.nrows = b.nrows - cpy.colMeta = make([]ColMeta, len(b.colMeta)) + cpy.colMeta = make([]query.ColMeta, len(b.colMeta)) copy(cpy.colMeta, b.colMeta) cpy.cols = make([]column, len(b.cols)) @@ -958,7 +860,7 @@ func (c colListBlockSorter) Swap(x int, y int) { } type column interface { - Meta() ColMeta + Meta() query.ColMeta Clear() Copy() column Equal(i, j int) bool @@ -967,12 +869,12 @@ type column interface { } type boolColumn struct { - ColMeta + query.ColMeta data []bool alloc *Allocator } -func (c *boolColumn) Meta() ColMeta { +func (c *boolColumn) Meta() query.ColMeta { return c.ColMeta } @@ -1004,12 +906,12 @@ func (c *boolColumn) Swap(i, j int) { } type intColumn struct { - ColMeta + query.ColMeta data []int64 alloc *Allocator } -func (c *intColumn) Meta() ColMeta { +func (c *intColumn) Meta() query.ColMeta { return c.ColMeta } @@ -1038,12 +940,12 @@ func (c *intColumn) Swap(i, j int) { } type uintColumn struct { - ColMeta + query.ColMeta data []uint64 alloc *Allocator } -func (c *uintColumn) Meta() ColMeta { +func (c *uintColumn) Meta() query.ColMeta { return c.ColMeta } @@ -1072,12 +974,12 @@ func (c *uintColumn) Swap(i, j int) { } type floatColumn struct { - ColMeta + query.ColMeta data []float64 alloc *Allocator } -func (c *floatColumn) Meta() ColMeta { +func (c *floatColumn) Meta() query.ColMeta { return c.ColMeta } @@ -1106,12 +1008,12 @@ func (c *floatColumn) Swap(i, j int) { } type stringColumn struct { - ColMeta + query.ColMeta data []string alloc *Allocator } -func (c *stringColumn) Meta() ColMeta { +func (c *stringColumn) Meta() query.ColMeta { return c.ColMeta } @@ -1141,12 +1043,12 @@ func (c *stringColumn) Swap(i, j int) { } type timeColumn struct { - ColMeta + query.ColMeta data []Time alloc *Allocator } -func (c *timeColumn) Meta() ColMeta { +func (c *timeColumn) Meta() query.ColMeta { return c.ColMeta } @@ -1177,8 +1079,8 @@ func (c *timeColumn) Swap(i, j int) { type BlockBuilderCache interface { // BlockBuilder returns an existing or new BlockBuilder for the given meta data. // The boolean return value indicates if BlockBuilder is new. - BlockBuilder(key PartitionKey) (BlockBuilder, bool) - ForEachBuilder(f func(PartitionKey, BlockBuilder)) + BlockBuilder(key query.PartitionKey) (BlockBuilder, bool) + ForEachBuilder(f func(query.PartitionKey, BlockBuilder)) } type blockBuilderCache struct { @@ -1204,7 +1106,7 @@ func (d *blockBuilderCache) SetTriggerSpec(ts query.TriggerSpec) { d.triggerSpec = ts } -func (d *blockBuilderCache) Block(key PartitionKey) (Block, error) { +func (d *blockBuilderCache) Block(key query.PartitionKey) (query.Block, error) { b, ok := d.lookupState(key) if !ok { return nil, errors.New("block not found") @@ -1212,7 +1114,7 @@ func (d *blockBuilderCache) Block(key PartitionKey) (Block, error) { return b.builder.Block() } -func (d *blockBuilderCache) lookupState(key PartitionKey) (blockState, bool) { +func (d *blockBuilderCache) lookupState(key query.PartitionKey) (blockState, bool) { v, ok := d.blocks.Lookup(key) if !ok { return blockState{}, false @@ -1222,7 +1124,7 @@ func (d *blockBuilderCache) lookupState(key PartitionKey) (blockState, bool) { // BlockBuilder will return the builder for the specified block. // If no builder exists, one will be created. -func (d *blockBuilderCache) BlockBuilder(key PartitionKey) (BlockBuilder, bool) { +func (d *blockBuilderCache) BlockBuilder(key query.PartitionKey) (BlockBuilder, bool) { b, ok := d.lookupState(key) if !ok { builder := NewColListBlockBuilder(key, d.alloc) @@ -1236,34 +1138,34 @@ func (d *blockBuilderCache) BlockBuilder(key PartitionKey) (BlockBuilder, bool) return b.builder, !ok } -func (d *blockBuilderCache) ForEachBuilder(f func(PartitionKey, BlockBuilder)) { - d.blocks.Range(func(key PartitionKey, value interface{}) { +func (d *blockBuilderCache) ForEachBuilder(f func(query.PartitionKey, BlockBuilder)) { + d.blocks.Range(func(key query.PartitionKey, value interface{}) { f(key, value.(blockState).builder) }) } -func (d *blockBuilderCache) DiscardBlock(key PartitionKey) { +func (d *blockBuilderCache) DiscardBlock(key query.PartitionKey) { b, ok := d.lookupState(key) if ok { b.builder.ClearData() } } -func (d *blockBuilderCache) ExpireBlock(key PartitionKey) { +func (d *blockBuilderCache) ExpireBlock(key query.PartitionKey) { b, ok := d.blocks.Delete(key) if ok { b.(blockState).builder.ClearData() } } -func (d *blockBuilderCache) ForEach(f func(PartitionKey)) { - d.blocks.Range(func(key PartitionKey, value interface{}) { +func (d *blockBuilderCache) ForEach(f func(query.PartitionKey)) { + d.blocks.Range(func(key query.PartitionKey, value interface{}) { f(key) }) } -func (d *blockBuilderCache) ForEachWithContext(f func(PartitionKey, Trigger, BlockContext)) { - d.blocks.Range(func(key PartitionKey, value interface{}) { +func (d *blockBuilderCache) ForEachWithContext(f func(query.PartitionKey, Trigger, BlockContext)) { + d.blocks.Range(func(key query.PartitionKey, value interface{}) { b := value.(blockState) f(key, b.trigger, BlockContext{ Key: key, diff --git a/execute/dataset.go b/execute/dataset.go index b7d615dbde..c9cc1d0e19 100644 --- a/execute/dataset.go +++ b/execute/dataset.go @@ -9,7 +9,7 @@ import ( type Dataset interface { Node - RetractBlock(key PartitionKey) error + RetractBlock(key query.PartitionKey) error UpdateProcessingTime(t Time) error UpdateWatermark(mark Time) error Finish(error) @@ -19,13 +19,13 @@ type Dataset interface { // DataCache holds all working data for a transformation. type DataCache interface { - Block(PartitionKey) (Block, error) + Block(query.PartitionKey) (query.Block, error) - ForEach(func(PartitionKey)) - ForEachWithContext(func(PartitionKey, Trigger, BlockContext)) + ForEach(func(query.PartitionKey)) + ForEachWithContext(func(query.PartitionKey, Trigger, BlockContext)) - DiscardBlock(PartitionKey) - ExpireBlock(PartitionKey) + DiscardBlock(query.PartitionKey) + ExpireBlock(query.PartitionKey) SetTriggerSpec(t query.TriggerSpec) } @@ -105,7 +105,7 @@ func (d *dataset) UpdateProcessingTime(time Time) error { } func (d *dataset) evalTriggers() (err error) { - d.cache.ForEachWithContext(func(key PartitionKey, trigger Trigger, bc BlockContext) { + d.cache.ForEachWithContext(func(key query.PartitionKey, trigger Trigger, bc BlockContext) { if err != nil { // Skip the rest once we have encountered an error return @@ -126,7 +126,7 @@ func (d *dataset) evalTriggers() (err error) { return err } -func (d *dataset) triggerBlock(key PartitionKey) error { +func (d *dataset) triggerBlock(key query.PartitionKey) error { b, err := d.cache.Block(key) if err != nil { return err @@ -157,11 +157,11 @@ func (d *dataset) triggerBlock(key PartitionKey) error { return nil } -func (d *dataset) expireBlock(key PartitionKey) { +func (d *dataset) expireBlock(key query.PartitionKey) { d.cache.ExpireBlock(key) } -func (d *dataset) RetractBlock(key PartitionKey) error { +func (d *dataset) RetractBlock(key query.PartitionKey) error { d.cache.DiscardBlock(key) for _, t := range d.ts { if err := t.RetractBlock(d.id, key); err != nil { @@ -174,7 +174,7 @@ func (d *dataset) RetractBlock(key PartitionKey) error { func (d *dataset) Finish(err error) { if err == nil { // Only trigger blocks we if we not finishing because of an error. - d.cache.ForEach(func(bk PartitionKey) { + d.cache.ForEach(func(bk query.PartitionKey) { if err != nil { return } diff --git a/execute/executetest/aggregate.go b/execute/executetest/aggregate.go index c580f928a8..973628f3a6 100644 --- a/execute/executetest/aggregate.go +++ b/execute/executetest/aggregate.go @@ -5,6 +5,7 @@ import ( "github.com/google/go-cmp/cmp" "github.com/google/go-cmp/cmp/cmpopts" + "github.com/influxdata/platform/query" "github.com/influxdata/platform/query/execute" ) @@ -23,15 +24,15 @@ func AggFuncTestHelper(t *testing.T, agg execute.Aggregate, data []float64, want var got interface{} switch vf.Type() { - case execute.TBool: + case query.TBool: got = vf.(execute.BoolValueFunc).ValueBool() - case execute.TInt: + case query.TInt: got = vf.(execute.IntValueFunc).ValueInt() - case execute.TUInt: + case query.TUInt: got = vf.(execute.UIntValueFunc).ValueUInt() - case execute.TFloat: + case query.TFloat: got = vf.(execute.FloatValueFunc).ValueFloat() - case execute.TString: + case query.TString: got = vf.(execute.StringValueFunc).ValueString() } @@ -49,15 +50,15 @@ func AggFuncBenchmarkHelper(b *testing.B, agg execute.Aggregate, data []float64, vf.DoFloat(data) var got interface{} switch vf.Type() { - case execute.TBool: + case query.TBool: got = vf.(execute.BoolValueFunc).ValueBool() - case execute.TInt: + case query.TInt: got = vf.(execute.IntValueFunc).ValueInt() - case execute.TUInt: + case query.TUInt: got = vf.(execute.UIntValueFunc).ValueUInt() - case execute.TFloat: + case query.TFloat: got = vf.(execute.FloatValueFunc).ValueFloat() - case execute.TString: + case query.TString: got = vf.(execute.StringValueFunc).ValueString() } if !cmp.Equal(want, got) { diff --git a/execute/executetest/block.go b/execute/executetest/block.go index 3d6cd5b6f4..a0b55d82e6 100644 --- a/execute/executetest/block.go +++ b/execute/executetest/block.go @@ -3,6 +3,7 @@ package executetest import ( "fmt" + "github.com/influxdata/platform/query" "github.com/influxdata/platform/query/execute" ) @@ -12,7 +13,7 @@ import ( // Use Normalize to ensure that all fields are set before equality comparisons. type Block struct { // PartitionKey of the block. Does not need to be set explicitly. - PartitionKey execute.PartitionKey + PartitionKey query.PartitionKey // KeyCols is a list of column that are part of the partition key. // The column type is deduced from the ColMeta slice. KeyCols []string @@ -20,7 +21,7 @@ type Block struct { // Only needs to be set when no data is present on the Block. KeyValues []interface{} // ColMeta is a list of columns of the block. - ColMeta []execute.ColMeta + ColMeta []query.ColMeta // Data is a list of rows, i.e. Data[row][col] // Each row must be a list with length equal to len(ColMeta) Data [][]interface{} @@ -29,7 +30,7 @@ type Block struct { // Normalize ensures all fields of the Block are set correctly. func (b *Block) Normalize() { if b.PartitionKey == nil { - cols := make([]execute.ColMeta, len(b.KeyCols)) + cols := make([]query.ColMeta, len(b.KeyCols)) if len(b.KeyValues) != len(b.KeyCols) { b.KeyValues = make([]interface{}, len(b.KeyCols)) } @@ -49,16 +50,16 @@ func (b *Block) Normalize() { func (b *Block) RefCount(n int) {} -func (b *Block) Cols() []execute.ColMeta { +func (b *Block) Cols() []query.ColMeta { return b.ColMeta } -func (b *Block) Key() execute.PartitionKey { +func (b *Block) Key() query.PartitionKey { b.Normalize() return b.PartitionKey } -func (b *Block) Do(f func(execute.ColReader) error) error { +func (b *Block) Do(f func(query.ColReader) error) error { for _, r := range b.Data { if err := f(ColReader{ key: b.Key(), @@ -72,16 +73,16 @@ func (b *Block) Do(f func(execute.ColReader) error) error { } type ColReader struct { - key execute.PartitionKey - cols []execute.ColMeta + key query.PartitionKey + cols []query.ColMeta row []interface{} } -func (cr ColReader) Cols() []execute.ColMeta { +func (cr ColReader) Cols() []query.ColMeta { return cr.cols } -func (cr ColReader) Key() execute.PartitionKey { +func (cr ColReader) Key() query.PartitionKey { return cr.key } func (cr ColReader) Len() int { @@ -113,11 +114,11 @@ func (cr ColReader) Times(j int) []execute.Time { } func BlocksFromCache(c execute.DataCache) (blocks []*Block, err error) { - c.ForEach(func(key execute.PartitionKey) { + c.ForEach(func(key query.PartitionKey) { if err != nil { return } - var b execute.Block + var b query.Block b, err = c.Block(key) if err != nil { return @@ -132,7 +133,7 @@ func BlocksFromCache(c execute.DataCache) (blocks []*Block, err error) { return blocks, nil } -func ConvertBlock(b execute.Block) (*Block, error) { +func ConvertBlock(b query.Block) (*Block, error) { key := b.Key() blk := &Block{ PartitionKey: key, @@ -149,24 +150,24 @@ func ConvertBlock(b execute.Block) (*Block, error) { } } - err := b.Do(func(cr execute.ColReader) error { + err := b.Do(func(cr query.ColReader) error { l := cr.Len() for i := 0; i < l; i++ { row := make([]interface{}, len(blk.ColMeta)) for j, c := range blk.ColMeta { var v interface{} switch c.Type { - case execute.TBool: + case query.TBool: v = cr.Bools(j)[i] - case execute.TInt: + case query.TInt: v = cr.Ints(j)[i] - case execute.TUInt: + case query.TUInt: v = cr.UInts(j)[i] - case execute.TFloat: + case query.TFloat: v = cr.Floats(j)[i] - case execute.TString: + case query.TString: v = cr.Strings(j)[i] - case execute.TTime: + case query.TTime: v = cr.Times(j)[i] default: panic(fmt.Errorf("unknown column type %s", c.Type)) diff --git a/execute/executetest/dataset.go b/execute/executetest/dataset.go index 7e74778183..11fb7fe1fa 100644 --- a/execute/executetest/dataset.go +++ b/execute/executetest/dataset.go @@ -15,7 +15,7 @@ func RandomDatasetID() execute.DatasetID { type Dataset struct { ID execute.DatasetID - Retractions []execute.PartitionKey + Retractions []query.PartitionKey ProcessingTimeUpdates []execute.Time WatermarkUpdates []execute.Time Finished bool @@ -32,7 +32,7 @@ func (d *Dataset) AddTransformation(t execute.Transformation) { panic("not implemented") } -func (d *Dataset) RetractBlock(key execute.PartitionKey) error { +func (d *Dataset) RetractBlock(key query.PartitionKey) error { d.Retractions = append(d.Retractions, key) return nil } diff --git a/execute/executetest/result.go b/execute/executetest/result.go index dadef07cde..980de34e42 100644 --- a/execute/executetest/result.go +++ b/execute/executetest/result.go @@ -1,6 +1,6 @@ package executetest -import "github.com/influxdata/platform/query/execute" +import "github.com/influxdata/platform/query" type Result struct { Blks []*Block @@ -10,7 +10,7 @@ func NewResult(blocks []*Block) *Result { return &Result{Blks: blocks} } -func (r *Result) Blocks() execute.BlockIterator { +func (r *Result) Blocks() query.BlockIterator { return &BlockIterator{ r.Blks, } @@ -24,7 +24,7 @@ type BlockIterator struct { blocks []*Block } -func (bi *BlockIterator) Do(f func(execute.Block) error) error { +func (bi *BlockIterator) Do(f func(query.Block) error) error { for _, b := range bi.blocks { if err := f(b); err != nil { return err diff --git a/execute/executetest/selector.go b/execute/executetest/selector.go index 72066959f2..37b7826ce2 100644 --- a/execute/executetest/selector.go +++ b/execute/executetest/selector.go @@ -4,10 +4,11 @@ import ( "testing" "github.com/google/go-cmp/cmp" + "github.com/influxdata/platform/query" "github.com/influxdata/platform/query/execute" ) -func RowSelectorFuncTestHelper(t *testing.T, selector execute.RowSelector, data execute.Block, want []execute.Row) { +func RowSelectorFuncTestHelper(t *testing.T, selector execute.RowSelector, data query.Block, want []execute.Row) { t.Helper() s := selector.NewFloatSelector() @@ -15,7 +16,7 @@ func RowSelectorFuncTestHelper(t *testing.T, selector execute.RowSelector, data if valueIdx < 0 { t.Fatal("no _value column found") } - data.Do(func(cr execute.ColReader) error { + data.Do(func(cr query.ColReader) error { s.DoFloat(cr.Floats(valueIdx), cr) return nil }) @@ -29,7 +30,7 @@ func RowSelectorFuncTestHelper(t *testing.T, selector execute.RowSelector, data var rows []execute.Row -func RowSelectorFuncBenchmarkHelper(b *testing.B, selector execute.RowSelector, data execute.Block) { +func RowSelectorFuncBenchmarkHelper(b *testing.B, selector execute.RowSelector, data query.Block) { b.Helper() valueIdx := execute.ColIdx(execute.DefaultValueColLabel, data.Cols()) @@ -40,7 +41,7 @@ func RowSelectorFuncBenchmarkHelper(b *testing.B, selector execute.RowSelector, b.ResetTimer() for n := 0; n < b.N; n++ { s := selector.NewFloatSelector() - data.Do(func(cr execute.ColReader) error { + data.Do(func(cr query.ColReader) error { s.DoFloat(cr.Floats(valueIdx), cr) return nil }) @@ -48,7 +49,7 @@ func RowSelectorFuncBenchmarkHelper(b *testing.B, selector execute.RowSelector, } } -func IndexSelectorFuncTestHelper(t *testing.T, selector execute.IndexSelector, data execute.Block, want [][]int) { +func IndexSelectorFuncTestHelper(t *testing.T, selector execute.IndexSelector, data query.Block, want [][]int) { t.Helper() var got [][]int @@ -57,7 +58,7 @@ func IndexSelectorFuncTestHelper(t *testing.T, selector execute.IndexSelector, d if valueIdx < 0 { t.Fatal("no _value column found") } - data.Do(func(cr execute.ColReader) error { + data.Do(func(cr query.ColReader) error { var cpy []int selected := s.DoFloat(cr.Floats(valueIdx)) if len(selected) > 0 { @@ -73,7 +74,7 @@ func IndexSelectorFuncTestHelper(t *testing.T, selector execute.IndexSelector, d } } -func IndexSelectorFuncBenchmarkHelper(b *testing.B, selector execute.IndexSelector, data execute.Block) { +func IndexSelectorFuncBenchmarkHelper(b *testing.B, selector execute.IndexSelector, data query.Block) { b.Helper() valueIdx := execute.ColIdx(execute.DefaultValueColLabel, data.Cols()) @@ -85,7 +86,7 @@ func IndexSelectorFuncBenchmarkHelper(b *testing.B, selector execute.IndexSelect var got [][]int for n := 0; n < b.N; n++ { s := selector.NewFloatSelector() - data.Do(func(cr execute.ColReader) error { + data.Do(func(cr query.ColReader) error { got = append(got, s.DoFloat(cr.Floats(valueIdx))) return nil }) diff --git a/execute/executetest/transformation.go b/execute/executetest/transformation.go index 2a62cb63aa..01762d3f02 100644 --- a/execute/executetest/transformation.go +++ b/execute/executetest/transformation.go @@ -6,12 +6,13 @@ import ( "github.com/google/go-cmp/cmp" "github.com/google/go-cmp/cmp/cmpopts" + "github.com/influxdata/platform/query" "github.com/influxdata/platform/query/execute" ) func ProcessTestHelper( t *testing.T, - data []execute.Block, + data []query.Block, want []*Block, create func(d execute.Dataset, c execute.BlockBuilderCache) execute.Transformation, ) { diff --git a/execute/executor.go b/execute/executor.go index c6a1415f14..aaa1c6fb79 100644 --- a/execute/executor.go +++ b/execute/executor.go @@ -5,14 +5,14 @@ import ( "fmt" "runtime/debug" - "github.com/influxdata/platform/query/id" "github.com/influxdata/platform/query" + "github.com/influxdata/platform/query/id" "github.com/influxdata/platform/query/plan" "github.com/pkg/errors" ) type Executor interface { - Execute(ctx context.Context, orgID id.ID, p *plan.PlanSpec) (map[string]Result, error) + Execute(ctx context.Context, orgID id.ID, p *plan.PlanSpec) (map[string]query.Result, error) } type executor struct { @@ -38,7 +38,7 @@ type executionState struct { bounds Bounds - results map[string]Result + results map[string]query.Result sources []Source transports []Transport @@ -46,7 +46,7 @@ type executionState struct { dispatcher *poolDispatcher } -func (e *executor) Execute(ctx context.Context, orgID id.ID, p *plan.PlanSpec) (map[string]Result, error) { +func (e *executor) Execute(ctx context.Context, orgID id.ID, p *plan.PlanSpec) (map[string]query.Result, error) { es, err := e.createExecutionState(ctx, orgID, p) if err != nil { return nil, errors.Wrap(err, "failed to initialize execute state") @@ -74,7 +74,7 @@ func (e *executor) createExecutionState(ctx context.Context, orgID id.ID, p *pla Limit: p.Resources.MemoryBytesQuota, }, resources: p.Resources, - results: make(map[string]Result, len(p.Results)), + results: make(map[string]query.Result, len(p.Results)), // TODO(nathanielc): Have the planner specify the dispatcher throughput dispatcher: newPoolDispatcher(10), bounds: Bounds{ diff --git a/execute/executor_test.go b/execute/executor_test.go index cb4480dd09..b6a0eea4cb 100644 --- a/execute/executor_test.go +++ b/execute/executor_test.go @@ -7,12 +7,12 @@ import ( "time" "github.com/google/go-cmp/cmp" - "github.com/influxdata/platform/query/ast" - "github.com/influxdata/platform/query/functions" - "github.com/influxdata/platform/query/id" "github.com/influxdata/platform/query" + "github.com/influxdata/platform/query/ast" "github.com/influxdata/platform/query/execute" "github.com/influxdata/platform/query/execute/executetest" + "github.com/influxdata/platform/query/functions" + "github.com/influxdata/platform/query/id" "github.com/influxdata/platform/query/plan" "github.com/influxdata/platform/query/semantic" uuid "github.com/satori/go.uuid" @@ -47,13 +47,13 @@ func TestExecutor_Execute(t *testing.T) { plan.ProcedureIDFromOperationID("from"): { ID: plan.ProcedureIDFromOperationID("from"), Spec: &testFromProcedureSource{ - data: []execute.Block{&executetest.Block{ + data: []query.Block{&executetest.Block{ KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(0), execute.Time(5), execute.Time(0), 1.0}, @@ -85,11 +85,11 @@ func TestExecutor_Execute(t *testing.T) { want: map[string][]*executetest.Block{ plan.DefaultYieldName: []*executetest.Block{{ KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(0), execute.Time(5), execute.Time(5), 15.0}, @@ -113,13 +113,13 @@ func TestExecutor_Execute(t *testing.T) { plan.ProcedureIDFromOperationID("from"): { ID: plan.ProcedureIDFromOperationID("from"), Spec: &testFromProcedureSource{ - data: []execute.Block{&executetest.Block{ + data: []query.Block{&executetest.Block{ KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TInt}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TInt}, }, Data: [][]interface{}{ {execute.Time(0), execute.Time(5), execute.Time(0), int64(1)}, @@ -243,11 +243,11 @@ func TestExecutor_Execute(t *testing.T) { want: map[string][]*executetest.Block{ plan.DefaultYieldName: []*executetest.Block{{ KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TInt}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TInt}, }, Data: [][]interface{}{ {execute.Time(0), execute.Time(5), execute.Time(5), int64(3)}, @@ -271,13 +271,13 @@ func TestExecutor_Execute(t *testing.T) { plan.ProcedureIDFromOperationID("from"): { ID: plan.ProcedureIDFromOperationID("from"), Spec: &testFromProcedureSource{ - data: []execute.Block{&executetest.Block{ + data: []query.Block{&executetest.Block{ KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(0), execute.Time(5), execute.Time(0), 1.0}, @@ -323,11 +323,11 @@ func TestExecutor_Execute(t *testing.T) { want: map[string][]*executetest.Block{ "sum": []*executetest.Block{{ KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(0), execute.Time(5), execute.Time(5), 15.0}, @@ -335,11 +335,11 @@ func TestExecutor_Execute(t *testing.T) { }}, "mean": []*executetest.Block{{ KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(0), execute.Time(5), execute.Time(5), 3.0}, @@ -359,7 +359,7 @@ func TestExecutor_Execute(t *testing.T) { } got := make(map[string][]*executetest.Block, len(results)) for name, r := range results { - if err := r.Blocks().Do(func(b execute.Block) error { + if err := r.Blocks().Do(func(b query.Block) error { cb, err := executetest.ConvertBlock(b) if err != nil { return err @@ -386,7 +386,7 @@ func TestExecutor_Execute(t *testing.T) { } type testFromProcedureSource struct { - data []execute.Block + data []query.Block ts []execute.Transformation } diff --git a/execute/format.go b/execute/format.go index b9e688d9c4..e58e14a286 100644 --- a/execute/format.go +++ b/execute/format.go @@ -5,13 +5,15 @@ import ( "sort" "strconv" "strings" + + "github.com/influxdata/platform/query" ) const fixedWidthTimeFmt = "2006-01-02T15:04:05.000000000Z" // Formatter writes a block to a Writer. type Formatter struct { - b Block + b query.Block widths []int maxWidth int newWidths []int @@ -38,7 +40,7 @@ var eol = []byte{'\n'} // NewFormatter creates a Formatter for a given block. // If opts is nil, the DefaultFormatOptions are used. -func NewFormatter(b Block, opts *FormatOptions) *Formatter { +func NewFormatter(b query.Block, opts *FormatOptions) *Formatter { if opts == nil { opts = DefaultFormatOptions() } @@ -63,14 +65,14 @@ func (w *writeToHelper) write(data []byte) { w.err = err } -var minWidthsByType = map[DataType]int{ - TBool: 12, - TInt: 26, - TUInt: 27, - TFloat: 28, - TString: 22, - TTime: len(fixedWidthTimeFmt), - TInvalid: 10, +var minWidthsByType = map[query.DataType]int{ + query.TBool: 12, + query.TInt: 26, + query.TUInt: 27, + query.TFloat: 28, + query.TString: 22, + query.TTime: len(fixedWidthTimeFmt), + query.TInvalid: 10, } // WriteTo writes the formatted block data to w. @@ -115,7 +117,7 @@ func (f *Formatter) WriteTo(out io.Writer) (int64, error) { // Write rows r := 0 - f.b.Do(func(cr ColReader) error { + f.b.Do(func(cr query.ColReader) error { if r == 0 { l := cr.Len() for i := 0; i < l; i++ { @@ -210,20 +212,20 @@ func (f *Formatter) writeHeaderSeparator(w *writeToHelper) { w.write(eol) } -func (f *Formatter) valueBuf(i, j int, typ DataType, cr ColReader) (buf []byte) { +func (f *Formatter) valueBuf(i, j int, typ query.DataType, cr query.ColReader) (buf []byte) { switch typ { - case TBool: + case query.TBool: buf = strconv.AppendBool(f.fmtBuf[0:0], cr.Bools(j)[i]) - case TInt: + case query.TInt: buf = strconv.AppendInt(f.fmtBuf[0:0], cr.Ints(j)[i], 10) - case TUInt: + case query.TUInt: buf = strconv.AppendUint(f.fmtBuf[0:0], cr.UInts(j)[i], 10) - case TFloat: + case query.TFloat: // TODO allow specifying format and precision buf = strconv.AppendFloat(f.fmtBuf[0:0], cr.Floats(j)[i], 'f', -1, 64) - case TString: + case query.TString: buf = []byte(cr.Strings(j)[i]) - case TTime: + case query.TTime: buf = []byte(cr.Times(j)[i].String()) } return @@ -238,15 +240,15 @@ func (f *Formatter) valueBuf(i, j int, typ DataType, cr ColReader) (buf []byte) // type orderedCols struct { indexMap []int - cols []ColMeta + cols []query.ColMeta } -func newOrderedCols(cols []ColMeta) orderedCols { +func newOrderedCols(cols []query.ColMeta) orderedCols { indexMap := make([]int, len(cols)) for i := range indexMap { indexMap[i] = i } - cpy := make([]ColMeta, len(cols)) + cpy := make([]query.ColMeta, len(cols)) copy(cpy, cols) return orderedCols{ indexMap: indexMap, diff --git a/execute/partition_lookup.go b/execute/partition_lookup.go index 80aa463ca9..0ca7f2aa4f 100644 --- a/execute/partition_lookup.go +++ b/execute/partition_lookup.go @@ -4,6 +4,8 @@ import ( "encoding/binary" "hash/fnv" "math" + + "github.com/influxdata/platform/query" ) type PartitionLookup struct { @@ -11,7 +13,7 @@ type PartitionLookup struct { } type partitionEntry struct { - key PartitionKey + key query.PartitionKey value interface{} } @@ -21,7 +23,7 @@ func NewPartitionLookup() *PartitionLookup { } } -func (l *PartitionLookup) Lookup(key PartitionKey) (interface{}, bool) { +func (l *PartitionLookup) Lookup(key query.PartitionKey) (interface{}, bool) { if key == nil { return nil, false } @@ -39,7 +41,7 @@ func (l *PartitionLookup) Lookup(key PartitionKey) (interface{}, bool) { return nil, false } -func (l *PartitionLookup) Set(key PartitionKey, value interface{}) { +func (l *PartitionLookup) Set(key query.PartitionKey, value interface{}) { h := key.Hash() entries := l.partitions[h] l.partitions[h] = append(entries, partitionEntry{ @@ -48,7 +50,7 @@ func (l *PartitionLookup) Set(key PartitionKey, value interface{}) { }) } -func (l *PartitionLookup) Delete(key PartitionKey) (interface{}, bool) { +func (l *PartitionLookup) Delete(key query.PartitionKey) (interface{}, bool) { if key == nil { return nil, false } @@ -67,7 +69,7 @@ func (l *PartitionLookup) Delete(key PartitionKey) (interface{}, bool) { return nil, false } -func (l *PartitionLookup) Range(f func(key PartitionKey, value interface{})) { +func (l *PartitionLookup) Range(f func(key query.PartitionKey, value interface{})) { for _, entries := range l.partitions { for _, entry := range entries { f(entry.key, entry.value) @@ -75,26 +77,26 @@ func (l *PartitionLookup) Range(f func(key PartitionKey, value interface{})) { } } -func computeKeyHash(key PartitionKey) uint64 { +func computeKeyHash(key query.PartitionKey) uint64 { h := fnv.New64() for j, c := range key.Cols() { h.Write([]byte(c.Label)) switch c.Type { - case TBool: + case query.TBool: if key.ValueBool(j) { h.Write([]byte{1}) } else { h.Write([]byte{0}) } - case TInt: + case query.TInt: binary.Write(h, binary.BigEndian, key.ValueInt(j)) - case TUInt: + case query.TUInt: binary.Write(h, binary.BigEndian, key.ValueUInt(j)) - case TFloat: + case query.TFloat: binary.Write(h, binary.BigEndian, math.Float64bits(key.ValueFloat(j))) - case TString: + case query.TString: h.Write([]byte(key.ValueString(j))) - case TTime: + case query.TTime: binary.Write(h, binary.BigEndian, uint64(key.ValueTime(j))) } } diff --git a/execute/result.go b/execute/result.go index 910044551d..0fc7ad0757 100644 --- a/execute/result.go +++ b/execute/result.go @@ -3,14 +3,10 @@ package execute import ( "sync" + "github.com/influxdata/platform/query" "github.com/influxdata/platform/query/plan" ) -type Result interface { - // Blocks returns a BlockIterator for iterating through results - Blocks() BlockIterator -} - // result implements both the Transformation and Result interfaces, // mapping the pushed based Transformation API to the pull based Result interface. type result struct { @@ -22,7 +18,7 @@ type result struct { } type resultMessage struct { - block Block + block query.Block err error } @@ -35,12 +31,12 @@ func newResult(plan.YieldSpec) *result { } } -func (s *result) RetractBlock(DatasetID, PartitionKey) error { +func (s *result) RetractBlock(DatasetID, query.PartitionKey) error { //TODO implement return nil } -func (s *result) Process(id DatasetID, b Block) error { +func (s *result) Process(id DatasetID, b query.Block) error { select { case s.blocks <- resultMessage{ block: b, @@ -50,11 +46,11 @@ func (s *result) Process(id DatasetID, b Block) error { return nil } -func (s *result) Blocks() BlockIterator { +func (s *result) Blocks() query.BlockIterator { return s } -func (s *result) Do(f func(Block) error) error { +func (s *result) Do(f func(query.Block) error) error { for { select { case err := <-s.abortErr: diff --git a/execute/row_fn.go b/execute/row_fn.go index c82535bc6b..56adfd9e69 100644 --- a/execute/row_fn.go +++ b/execute/row_fn.go @@ -4,8 +4,8 @@ import ( "fmt" "regexp" - "github.com/influxdata/platform/query/compiler" "github.com/influxdata/platform/query" + "github.com/influxdata/platform/query/compiler" "github.com/influxdata/platform/query/semantic" "github.com/influxdata/platform/query/values" "github.com/pkg/errors" @@ -39,7 +39,7 @@ func newRowFn(fn *semantic.FunctionExpression) (rowFn, error) { }, nil } -func (f *rowFn) prepare(cols []ColMeta) error { +func (f *rowFn) prepare(cols []query.ColMeta) error { // Prepare types and recordCols propertyTypes := make(map[string]semantic.Type, len(f.references)) for _, r := range f.references { @@ -68,51 +68,51 @@ func (f *rowFn) prepare(cols []ColMeta) error { return nil } -func ConvertToKind(t DataType) semantic.Kind { +func ConvertToKind(t query.DataType) semantic.Kind { // TODO make this an array lookup. switch t { - case TInvalid: + case query.TInvalid: return semantic.Invalid - case TBool: + case query.TBool: return semantic.Bool - case TInt: + case query.TInt: return semantic.Int - case TUInt: + case query.TUInt: return semantic.UInt - case TFloat: + case query.TFloat: return semantic.Float - case TString: + case query.TString: return semantic.String - case TTime: + case query.TTime: return semantic.Time default: return semantic.Invalid } } -func ConvertFromKind(k semantic.Kind) DataType { +func ConvertFromKind(k semantic.Kind) query.DataType { // TODO make this an array lookup. switch k { case semantic.Invalid: - return TInvalid + return query.TInvalid case semantic.Bool: - return TBool + return query.TBool case semantic.Int: - return TInt + return query.TInt case semantic.UInt: - return TUInt + return query.TUInt case semantic.Float: - return TFloat + return query.TFloat case semantic.String: - return TString + return query.TString case semantic.Time: - return TTime + return query.TTime default: - return TInvalid + return query.TInvalid } } -func (f *rowFn) eval(row int, cr ColReader) (values.Value, error) { +func (f *rowFn) eval(row int, cr query.ColReader) (values.Value, error) { for _, r := range f.references { f.record.Set(r, ValueForRow(row, f.recordCols[r], cr)) } @@ -134,7 +134,7 @@ func NewRowPredicateFn(fn *semantic.FunctionExpression) (*RowPredicateFn, error) }, nil } -func (f *RowPredicateFn) Prepare(cols []ColMeta) error { +func (f *RowPredicateFn) Prepare(cols []query.ColMeta) error { err := f.rowFn.prepare(cols) if err != nil { return err @@ -145,7 +145,7 @@ func (f *RowPredicateFn) Prepare(cols []ColMeta) error { return nil } -func (f *RowPredicateFn) Eval(row int, cr ColReader) (bool, error) { +func (f *RowPredicateFn) Eval(row int, cr query.ColReader) (bool, error) { v, err := f.rowFn.eval(row, cr) if err != nil { return false, err @@ -170,7 +170,7 @@ func NewRowMapFn(fn *semantic.FunctionExpression) (*RowMapFn, error) { }, nil } -func (f *RowMapFn) Prepare(cols []ColMeta) error { +func (f *RowMapFn) Prepare(cols []query.ColMeta) error { err := f.rowFn.prepare(cols) if err != nil { return err @@ -192,7 +192,7 @@ func (f *RowMapFn) Type() semantic.Type { return f.preparedFn.Type() } -func (f *RowMapFn) Eval(row int, cr ColReader) (values.Object, error) { +func (f *RowMapFn) Eval(row int, cr query.ColReader) (values.Object, error) { v, err := f.rowFn.eval(row, cr) if err != nil { return nil, err @@ -204,20 +204,20 @@ func (f *RowMapFn) Eval(row int, cr ColReader) (values.Object, error) { return v.Object(), nil } -func ValueForRow(i, j int, cr ColReader) values.Value { +func ValueForRow(i, j int, cr query.ColReader) values.Value { t := cr.Cols()[j].Type switch t { - case TString: + case query.TString: return values.NewStringValue(cr.Strings(j)[i]) - case TInt: + case query.TInt: return values.NewIntValue(cr.Ints(j)[i]) - case TUInt: + case query.TUInt: return values.NewUIntValue(cr.UInts(j)[i]) - case TFloat: + case query.TFloat: return values.NewFloatValue(cr.Floats(j)[i]) - case TBool: + case query.TBool: return values.NewBoolValue(cr.Bools(j)[i]) - case TTime: + case query.TTime: return values.NewTimeValue(cr.Times(j)[i]) default: PanicUnknownType(t) diff --git a/execute/selector.go b/execute/selector.go index 137326be81..1b6ef0594f 100644 --- a/execute/selector.go +++ b/execute/selector.go @@ -70,7 +70,7 @@ func newSelectorTransformation(d Dataset, c BlockBuilderCache, config SelectorCo } } -func (t *selectorTransformation) RetractBlock(id DatasetID, key PartitionKey) error { +func (t *selectorTransformation) RetractBlock(id DatasetID, key query.PartitionKey) error { //TODO(nathanielc): Store intermediate state for retractions return t.d.RetractBlock(key) } @@ -84,7 +84,7 @@ func (t *selectorTransformation) Finish(id DatasetID, err error) { t.d.Finish(err) } -func (t *selectorTransformation) setupBuilder(b Block) (BlockBuilder, int, error) { +func (t *selectorTransformation) setupBuilder(b query.Block) (BlockBuilder, int, error) { builder, new := t.cache.BlockBuilder(b.Key()) if !new { return nil, 0, fmt.Errorf("found duplicate block with key: %v", b.Key()) @@ -99,7 +99,7 @@ func (t *selectorTransformation) setupBuilder(b Block) (BlockBuilder, int, error return builder, valueIdx, nil } -func (t *indexSelectorTransformation) Process(id DatasetID, b Block) error { +func (t *indexSelectorTransformation) Process(id DatasetID, b query.Block) error { builder, valueIdx, err := t.setupBuilder(b) if err != nil { return err @@ -108,35 +108,35 @@ func (t *indexSelectorTransformation) Process(id DatasetID, b Block) error { var s interface{} switch valueCol.Type { - case TBool: + case query.TBool: s = t.selector.NewBoolSelector() - case TInt: + case query.TInt: s = t.selector.NewIntSelector() - case TUInt: + case query.TUInt: s = t.selector.NewUIntSelector() - case TFloat: + case query.TFloat: s = t.selector.NewFloatSelector() - case TString: + case query.TString: s = t.selector.NewStringSelector() default: return fmt.Errorf("unsupported selector type %v", valueCol.Type) } - return b.Do(func(cr ColReader) error { + return b.Do(func(cr query.ColReader) error { switch valueCol.Type { - case TBool: + case query.TBool: selected := s.(DoBoolIndexSelector).DoBool(cr.Bools(valueIdx)) t.appendSelected(selected, builder, cr) - case TInt: + case query.TInt: selected := s.(DoIntIndexSelector).DoInt(cr.Ints(valueIdx)) t.appendSelected(selected, builder, cr) - case TUInt: + case query.TUInt: selected := s.(DoUIntIndexSelector).DoUInt(cr.UInts(valueIdx)) t.appendSelected(selected, builder, cr) - case TFloat: + case query.TFloat: selected := s.(DoFloatIndexSelector).DoFloat(cr.Floats(valueIdx)) t.appendSelected(selected, builder, cr) - case TString: + case query.TString: selected := s.(DoStringIndexSelector).DoString(cr.Strings(valueIdx)) t.appendSelected(selected, builder, cr) default: @@ -146,7 +146,7 @@ func (t *indexSelectorTransformation) Process(id DatasetID, b Block) error { }) } -func (t *rowSelectorTransformation) Process(id DatasetID, b Block) error { +func (t *rowSelectorTransformation) Process(id DatasetID, b query.Block) error { builder, valueIdx, err := t.setupBuilder(b) if err != nil { return err @@ -155,31 +155,31 @@ func (t *rowSelectorTransformation) Process(id DatasetID, b Block) error { var rower Rower switch valueCol.Type { - case TBool: + case query.TBool: rower = t.selector.NewBoolSelector() - case TInt: + case query.TInt: rower = t.selector.NewIntSelector() - case TUInt: + case query.TUInt: rower = t.selector.NewUIntSelector() - case TFloat: + case query.TFloat: rower = t.selector.NewFloatSelector() - case TString: + case query.TString: rower = t.selector.NewStringSelector() default: return fmt.Errorf("unsupported selector type %v", valueCol.Type) } - b.Do(func(cr ColReader) error { + b.Do(func(cr query.ColReader) error { switch valueCol.Type { - case TBool: + case query.TBool: rower.(DoBoolRowSelector).DoBool(cr.Bools(valueIdx), cr) - case TInt: + case query.TInt: rower.(DoIntRowSelector).DoInt(cr.Ints(valueIdx), cr) - case TUInt: + case query.TUInt: rower.(DoUIntRowSelector).DoUInt(cr.UInts(valueIdx), cr) - case TFloat: + case query.TFloat: rower.(DoFloatRowSelector).DoFloat(cr.Floats(valueIdx), cr) - case TString: + case query.TString: rower.(DoStringRowSelector).DoString(cr.Strings(valueIdx), cr) default: return fmt.Errorf("unsupported selector type %v", valueCol.Type) @@ -191,7 +191,7 @@ func (t *rowSelectorTransformation) Process(id DatasetID, b Block) error { return nil } -func (t *indexSelectorTransformation) appendSelected(selected []int, builder BlockBuilder, cr ColReader) { +func (t *indexSelectorTransformation) appendSelected(selected []int, builder BlockBuilder, cr query.ColReader) { if len(selected) == 0 { return } @@ -199,17 +199,17 @@ func (t *indexSelectorTransformation) appendSelected(selected []int, builder Blo for j, c := range cols { for _, i := range selected { switch c.Type { - case TBool: + case query.TBool: builder.AppendBool(j, cr.Bools(j)[i]) - case TInt: + case query.TInt: builder.AppendInt(j, cr.Ints(j)[i]) - case TUInt: + case query.TUInt: builder.AppendUInt(j, cr.UInts(j)[i]) - case TFloat: + case query.TFloat: builder.AppendFloat(j, cr.Floats(j)[i]) - case TString: + case query.TString: builder.AppendString(j, cr.Strings(j)[i]) - case TTime: + case query.TTime: builder.AppendTime(j, cr.Times(j)[i]) default: PanicUnknownType(c.Type) @@ -224,17 +224,17 @@ func (t *rowSelectorTransformation) appendRows(builder BlockBuilder, rows []Row) for _, row := range rows { v := row.Values[j] switch c.Type { - case TBool: + case query.TBool: builder.AppendBool(j, v.(bool)) - case TInt: + case query.TInt: builder.AppendInt(j, v.(int64)) - case TUInt: + case query.TUInt: builder.AppendUInt(j, v.(uint64)) - case TFloat: + case query.TFloat: builder.AppendFloat(j, v.(float64)) - case TString: + case query.TString: builder.AppendString(j, v.(string)) - case TTime: + case query.TTime: builder.AppendTime(j, v.(Time)) default: PanicUnknownType(c.Type) @@ -280,45 +280,45 @@ type Rower interface { type DoBoolRowSelector interface { Rower - DoBool(vs []bool, cr ColReader) + DoBool(vs []bool, cr query.ColReader) } type DoIntRowSelector interface { Rower - DoInt(vs []int64, cr ColReader) + DoInt(vs []int64, cr query.ColReader) } type DoUIntRowSelector interface { Rower - DoUInt(vs []uint64, cr ColReader) + DoUInt(vs []uint64, cr query.ColReader) } type DoFloatRowSelector interface { Rower - DoFloat(vs []float64, cr ColReader) + DoFloat(vs []float64, cr query.ColReader) } type DoStringRowSelector interface { Rower - DoString(vs []string, cr ColReader) + DoString(vs []string, cr query.ColReader) } type Row struct { Values []interface{} } -func ReadRow(i int, cr ColReader) (row Row) { +func ReadRow(i int, cr query.ColReader) (row Row) { cols := cr.Cols() row.Values = make([]interface{}, len(cols)) for j, c := range cols { switch c.Type { - case TBool: + case query.TBool: row.Values[j] = cr.Bools(j)[i] - case TInt: + case query.TInt: row.Values[j] = cr.Ints(j)[i] - case TUInt: + case query.TUInt: row.Values[j] = cr.UInts(j)[i] - case TFloat: + case query.TFloat: row.Values[j] = cr.Floats(j)[i] - case TString: + case query.TString: row.Values[j] = cr.Strings(j)[i] - case TTime: + case query.TTime: row.Values[j] = cr.Times(j)[i] } } diff --git a/execute/selector_test.go b/execute/selector_test.go index 15657107a4..a14b9a44f4 100644 --- a/execute/selector_test.go +++ b/execute/selector_test.go @@ -6,9 +6,10 @@ import ( "github.com/google/go-cmp/cmp" "github.com/google/go-cmp/cmp/cmpopts" - "github.com/influxdata/platform/query/functions" + "github.com/influxdata/platform/query" "github.com/influxdata/platform/query/execute" "github.com/influxdata/platform/query/execute/executetest" + "github.com/influxdata/platform/query/functions" ) func TestRowSelector_Process(t *testing.T) { @@ -26,11 +27,11 @@ func TestRowSelector_Process(t *testing.T) { }, data: []*executetest.Block{{ KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(0), execute.Time(100), execute.Time(1), 0.0}, @@ -47,11 +48,11 @@ func TestRowSelector_Process(t *testing.T) { }}, want: []*executetest.Block{{ KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(0), execute.Time(100), execute.Time(1), 0.0}, @@ -65,11 +66,11 @@ func TestRowSelector_Process(t *testing.T) { }, data: []*executetest.Block{{ KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "x", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "x", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(0), execute.Time(100), execute.Time(1), 0.0}, @@ -86,11 +87,11 @@ func TestRowSelector_Process(t *testing.T) { }}, want: []*executetest.Block{{ KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "x", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "x", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(0), execute.Time(100), execute.Time(1), 0.0}, @@ -105,11 +106,11 @@ func TestRowSelector_Process(t *testing.T) { data: []*executetest.Block{ { KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(0), execute.Time(100), execute.Time(1), 0.0}, @@ -126,11 +127,11 @@ func TestRowSelector_Process(t *testing.T) { }, { KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(100), execute.Time(200), execute.Time(101), 10.0}, @@ -149,11 +150,11 @@ func TestRowSelector_Process(t *testing.T) { want: []*executetest.Block{ { KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(0), execute.Time(100), execute.Time(1), 0.0}, @@ -161,11 +162,11 @@ func TestRowSelector_Process(t *testing.T) { }, { KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(100), execute.Time(200), execute.Time(101), 10.0}, @@ -223,11 +224,11 @@ func TestIndexSelector_Process(t *testing.T) { }, data: []*executetest.Block{{ KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(0), execute.Time(100), execute.Time(1), 0.0}, @@ -244,11 +245,11 @@ func TestIndexSelector_Process(t *testing.T) { }}, want: []*executetest.Block{{ KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(0), execute.Time(100), execute.Time(1), 0.0}, @@ -263,11 +264,11 @@ func TestIndexSelector_Process(t *testing.T) { data: []*executetest.Block{ { KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(0), execute.Time(100), execute.Time(1), 0.0}, @@ -284,11 +285,11 @@ func TestIndexSelector_Process(t *testing.T) { }, { KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(100), execute.Time(200), execute.Time(101), 10.0}, @@ -307,11 +308,11 @@ func TestIndexSelector_Process(t *testing.T) { want: []*executetest.Block{ { KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(0), execute.Time(100), execute.Time(1), 0.0}, @@ -319,11 +320,11 @@ func TestIndexSelector_Process(t *testing.T) { }, { KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(100), execute.Time(200), execute.Time(101), 10.0}, diff --git a/execute/transformation.go b/execute/transformation.go index 4c42ab2b2d..668819899c 100644 --- a/execute/transformation.go +++ b/execute/transformation.go @@ -3,14 +3,14 @@ package execute import ( "fmt" - "github.com/influxdata/platform/query/id" "github.com/influxdata/platform/query" + "github.com/influxdata/platform/query/id" "github.com/influxdata/platform/query/plan" ) type Transformation interface { - RetractBlock(id DatasetID, key PartitionKey) error - Process(id DatasetID, b Block) error + RetractBlock(id DatasetID, key query.PartitionKey) error + Process(id DatasetID, b query.Block) error UpdateWatermark(id DatasetID, t Time) error UpdateProcessingTime(id DatasetID, t Time) error Finish(id DatasetID, err error) diff --git a/execute/transport.go b/execute/transport.go index 377322466c..965d381160 100644 --- a/execute/transport.go +++ b/execute/transport.go @@ -3,6 +3,8 @@ package execute import ( "sync" "sync/atomic" + + "github.com/influxdata/platform/query" ) type Transport interface { @@ -52,7 +54,7 @@ func (t *consecutiveTransport) Finished() <-chan struct{} { return t.finished } -func (t *consecutiveTransport) RetractBlock(id DatasetID, key PartitionKey) error { +func (t *consecutiveTransport) RetractBlock(id DatasetID, key query.PartitionKey) error { select { case <-t.finished: return t.err() @@ -65,7 +67,7 @@ func (t *consecutiveTransport) RetractBlock(id DatasetID, key PartitionKey) erro return nil } -func (t *consecutiveTransport) Process(id DatasetID, b Block) error { +func (t *consecutiveTransport) Process(id DatasetID, b query.Block) error { select { case <-t.finished: return t.err() @@ -230,35 +232,35 @@ func (m srcMessage) SrcDatasetID() DatasetID { type RetractBlockMsg interface { Message - Key() PartitionKey + Key() query.PartitionKey } type retractBlockMsg struct { srcMessage - key PartitionKey + key query.PartitionKey } func (m *retractBlockMsg) Type() MessageType { return RetractBlockType } -func (m *retractBlockMsg) Key() PartitionKey { +func (m *retractBlockMsg) Key() query.PartitionKey { return m.key } type ProcessMsg interface { Message - Block() Block + Block() query.Block } type processMsg struct { srcMessage - block Block + block query.Block } func (m *processMsg) Type() MessageType { return ProcessType } -func (m *processMsg) Block() Block { +func (m *processMsg) Block() query.Block { return m.block } diff --git a/execute/trigger.go b/execute/trigger.go index 788af37925..dbe11687ee 100644 --- a/execute/trigger.go +++ b/execute/trigger.go @@ -19,7 +19,7 @@ type TriggerContext struct { } type BlockContext struct { - Key PartitionKey + Key query.PartitionKey Count int } diff --git a/functions/count.go b/functions/count.go index ed94613057..20252815f3 100644 --- a/functions/count.go +++ b/functions/count.go @@ -143,8 +143,8 @@ func (a *CountAgg) DoString(vs []string) { a.count += int64(len(vs)) } -func (a *CountAgg) Type() execute.DataType { - return execute.TInt +func (a *CountAgg) Type() query.DataType { + return query.TInt } func (a *CountAgg) ValueInt() int64 { return a.count diff --git a/functions/covariance.go b/functions/covariance.go index 0d2feea45f..f35d94c90d 100644 --- a/functions/covariance.go +++ b/functions/covariance.go @@ -151,24 +151,24 @@ func NewCovarianceTransformation(d execute.Dataset, cache execute.BlockBuilderCa } } -func (t *CovarianceTransformation) RetractBlock(id execute.DatasetID, key execute.PartitionKey) error { +func (t *CovarianceTransformation) RetractBlock(id execute.DatasetID, key query.PartitionKey) error { return t.d.RetractBlock(key) } -func (t *CovarianceTransformation) Process(id execute.DatasetID, b execute.Block) error { +func (t *CovarianceTransformation) Process(id execute.DatasetID, b query.Block) error { cols := b.Cols() builder, created := t.cache.BlockBuilder(b.Key()) if !created { return fmt.Errorf("covariance found duplicate block with key: %v", b.Key()) } execute.AddBlockKeyCols(b.Key(), builder) - builder.AddCol(execute.ColMeta{ + builder.AddCol(query.ColMeta{ Label: t.spec.TimeDst, - Type: execute.TTime, + Type: query.TTime, }) - valueIdx := builder.AddCol(execute.ColMeta{ + valueIdx := builder.AddCol(query.ColMeta{ Label: t.spec.ValueLabel, - Type: execute.TFloat, + Type: query.TFloat, }) xIdx := execute.ColIdx(t.spec.Columns[0], cols) yIdx := execute.ColIdx(t.spec.Columns[1], cols) @@ -181,9 +181,9 @@ func (t *CovarianceTransformation) Process(id execute.DatasetID, b execute.Block } t.reset() - b.Do(func(cr execute.ColReader) error { + b.Do(func(cr query.ColReader) error { switch typ := cols[xIdx].Type; typ { - case execute.TFloat: + case query.TFloat: t.DoFloat(cr.Floats(xIdx), cr.Floats(yIdx)) default: return fmt.Errorf("covariance does not support %v", typ) diff --git a/functions/covariance_test.go b/functions/covariance_test.go index 1d8886f21c..70ecfa6671 100644 --- a/functions/covariance_test.go +++ b/functions/covariance_test.go @@ -3,10 +3,10 @@ package functions_test import ( "testing" - "github.com/influxdata/platform/query/functions" "github.com/influxdata/platform/query" "github.com/influxdata/platform/query/execute" "github.com/influxdata/platform/query/execute/executetest" + "github.com/influxdata/platform/query/functions" "github.com/influxdata/platform/query/querytest" "github.com/influxdata/platform/query/semantic" ) @@ -177,7 +177,7 @@ func TestCovariance_Process(t *testing.T) { testCases := []struct { name string spec *functions.CovarianceProcedureSpec - data []execute.Block + data []query.Block want []*executetest.Block }{ { @@ -190,14 +190,14 @@ func TestCovariance_Process(t *testing.T) { Columns: []string{"x", "y"}, }, }, - data: []execute.Block{&executetest.Block{ + data: []query.Block{&executetest.Block{ KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "x", Type: execute.TFloat}, - {Label: "y", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "x", Type: query.TFloat}, + {Label: "y", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(0), execute.Time(5), execute.Time(0), 1.0, 1.0}, @@ -209,11 +209,11 @@ func TestCovariance_Process(t *testing.T) { }}, want: []*executetest.Block{{ KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(0), execute.Time(5), execute.Time(5), 2.5}, @@ -230,14 +230,14 @@ func TestCovariance_Process(t *testing.T) { Columns: []string{"x", "y"}, }, }, - data: []execute.Block{&executetest.Block{ + data: []query.Block{&executetest.Block{ KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "x", Type: execute.TFloat}, - {Label: "y", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "x", Type: query.TFloat}, + {Label: "y", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(0), execute.Time(5), execute.Time(0), 1.0, 5.0}, @@ -249,11 +249,11 @@ func TestCovariance_Process(t *testing.T) { }}, want: []*executetest.Block{{ KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(0), execute.Time(5), execute.Time(5), -2.5}, @@ -270,14 +270,14 @@ func TestCovariance_Process(t *testing.T) { Columns: []string{"x", "y"}, }, }, - data: []execute.Block{&executetest.Block{ + data: []query.Block{&executetest.Block{ KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "x", Type: execute.TFloat}, - {Label: "y", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "x", Type: query.TFloat}, + {Label: "y", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(0), execute.Time(5), execute.Time(0), 1.0, 1.0}, @@ -289,11 +289,11 @@ func TestCovariance_Process(t *testing.T) { }}, want: []*executetest.Block{{ KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(0), execute.Time(5), execute.Time(5), 0.5}, @@ -311,14 +311,14 @@ func TestCovariance_Process(t *testing.T) { Columns: []string{"x", "y"}, }, }, - data: []execute.Block{&executetest.Block{ + data: []query.Block{&executetest.Block{ KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "x", Type: execute.TFloat}, - {Label: "y", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "x", Type: query.TFloat}, + {Label: "y", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(0), execute.Time(5), execute.Time(0), 1.0, 1.0}, @@ -330,11 +330,11 @@ func TestCovariance_Process(t *testing.T) { }}, want: []*executetest.Block{{ KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(0), execute.Time(5), execute.Time(5), 1.0}, @@ -352,14 +352,14 @@ func TestCovariance_Process(t *testing.T) { Columns: []string{"x", "y"}, }, }, - data: []execute.Block{&executetest.Block{ + data: []query.Block{&executetest.Block{ KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "x", Type: execute.TFloat}, - {Label: "y", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "x", Type: query.TFloat}, + {Label: "y", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(0), execute.Time(5), execute.Time(0), 1.0, 5.0}, @@ -371,11 +371,11 @@ func TestCovariance_Process(t *testing.T) { }}, want: []*executetest.Block{{ KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(0), execute.Time(5), execute.Time(5), -1.0}, diff --git a/functions/cumulative_sum.go b/functions/cumulative_sum.go index 1e72902e61..765592f3e6 100644 --- a/functions/cumulative_sum.go +++ b/functions/cumulative_sum.go @@ -3,9 +3,9 @@ package functions import ( "fmt" - "github.com/influxdata/platform/query/interpreter" "github.com/influxdata/platform/query" "github.com/influxdata/platform/query/execute" + "github.com/influxdata/platform/query/interpreter" "github.com/influxdata/platform/query/plan" "github.com/influxdata/platform/query/semantic" ) @@ -106,11 +106,11 @@ func NewCumulativeSumTransformation(d execute.Dataset, cache execute.BlockBuilde } } -func (t *cumulativeSumTransformation) RetractBlock(id execute.DatasetID, key execute.PartitionKey) error { +func (t *cumulativeSumTransformation) RetractBlock(id execute.DatasetID, key query.PartitionKey) error { return t.d.RetractBlock(key) } -func (t *cumulativeSumTransformation) Process(id execute.DatasetID, b execute.Block) error { +func (t *cumulativeSumTransformation) Process(id execute.DatasetID, b query.Block) error { builder, created := t.cache.BlockBuilder(b.Key()) if !created { return fmt.Errorf("cumulative sum found duplicate block with key: %v", b.Key()) @@ -127,13 +127,13 @@ func (t *cumulativeSumTransformation) Process(id execute.DatasetID, b execute.Bl } } } - return b.Do(func(cr execute.ColReader) error { + return b.Do(func(cr query.ColReader) error { l := cr.Len() for j, c := range cols { switch c.Type { - case execute.TBool: + case query.TBool: builder.AppendBools(j, cr.Bools(j)) - case execute.TInt: + case query.TInt: if sumers[j] != nil { for i := 0; i < l; i++ { builder.AppendInt(j, sumers[j].sumInt(cr.Ints(j)[i])) @@ -141,7 +141,7 @@ func (t *cumulativeSumTransformation) Process(id execute.DatasetID, b execute.Bl } else { builder.AppendInts(j, cr.Ints(j)) } - case execute.TUInt: + case query.TUInt: if sumers[j] != nil { for i := 0; i < l; i++ { builder.AppendUInt(j, sumers[j].sumUInt(cr.UInts(j)[i])) @@ -149,7 +149,7 @@ func (t *cumulativeSumTransformation) Process(id execute.DatasetID, b execute.Bl } else { builder.AppendUInts(j, cr.UInts(j)) } - case execute.TFloat: + case query.TFloat: if sumers[j] != nil { for i := 0; i < l; i++ { builder.AppendFloat(j, sumers[j].sumFloat(cr.Floats(j)[i])) @@ -157,9 +157,9 @@ func (t *cumulativeSumTransformation) Process(id execute.DatasetID, b execute.Bl } else { builder.AppendFloats(j, cr.Floats(j)) } - case execute.TString: + case query.TString: builder.AppendStrings(j, cr.Strings(j)) - case execute.TTime: + case query.TTime: builder.AppendTimes(j, cr.Times(j)) } } diff --git a/functions/cumulative_sum_test.go b/functions/cumulative_sum_test.go index 1d7db25a73..b46bf0fe58 100644 --- a/functions/cumulative_sum_test.go +++ b/functions/cumulative_sum_test.go @@ -3,10 +3,10 @@ package functions_test import ( "testing" - "github.com/influxdata/platform/query/functions" "github.com/influxdata/platform/query" "github.com/influxdata/platform/query/execute" "github.com/influxdata/platform/query/execute/executetest" + "github.com/influxdata/platform/query/functions" "github.com/influxdata/platform/query/querytest" ) @@ -34,7 +34,7 @@ func TestCumulativeSum_Process(t *testing.T) { testCases := []struct { name string spec *functions.CumulativeSumProcedureSpec - data []execute.Block + data []query.Block want []*executetest.Block }{ { @@ -42,10 +42,10 @@ func TestCumulativeSum_Process(t *testing.T) { spec: &functions.CumulativeSumProcedureSpec{ Columns: []string{execute.DefaultValueColLabel}, }, - data: []execute.Block{&executetest.Block{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + data: []query.Block{&executetest.Block{ + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(0), 2.0}, @@ -61,9 +61,9 @@ func TestCumulativeSum_Process(t *testing.T) { }, }}, want: []*executetest.Block{{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(0), 2.0}, @@ -84,12 +84,12 @@ func TestCumulativeSum_Process(t *testing.T) { spec: &functions.CumulativeSumProcedureSpec{ Columns: []string{"int", "uint", "float"}, }, - data: []execute.Block{&executetest.Block{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "int", Type: execute.TInt}, - {Label: "uint", Type: execute.TUInt}, - {Label: "float", Type: execute.TFloat}, + data: []query.Block{&executetest.Block{ + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "int", Type: query.TInt}, + {Label: "uint", Type: query.TUInt}, + {Label: "float", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(0), int64(2), uint64(1), 1.0}, @@ -105,11 +105,11 @@ func TestCumulativeSum_Process(t *testing.T) { }, }}, want: []*executetest.Block{{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "int", Type: execute.TInt}, - {Label: "uint", Type: execute.TUInt}, - {Label: "float", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "int", Type: query.TInt}, + {Label: "uint", Type: query.TUInt}, + {Label: "float", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(0), int64(2), uint64(1), 1.0}, @@ -130,12 +130,12 @@ func TestCumulativeSum_Process(t *testing.T) { spec: &functions.CumulativeSumProcedureSpec{ Columns: []string{"int", "float"}, }, - data: []execute.Block{&executetest.Block{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "time2", Type: execute.TTime}, - {Label: "int", Type: execute.TInt}, - {Label: "float", Type: execute.TFloat}, + data: []query.Block{&executetest.Block{ + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "time2", Type: query.TTime}, + {Label: "int", Type: query.TInt}, + {Label: "float", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(0), execute.Time(0), int64(2), 1.0}, @@ -151,11 +151,11 @@ func TestCumulativeSum_Process(t *testing.T) { }, }}, want: []*executetest.Block{{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "time2", Type: execute.TTime}, - {Label: "int", Type: execute.TInt}, - {Label: "float", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "time2", Type: query.TTime}, + {Label: "int", Type: query.TInt}, + {Label: "float", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(0), execute.Time(0), int64(2), 1.0}, @@ -176,11 +176,11 @@ func TestCumulativeSum_Process(t *testing.T) { spec: &functions.CumulativeSumProcedureSpec{ Columns: []string{"int"}, }, - data: []execute.Block{&executetest.Block{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "int", Type: execute.TInt}, - {Label: "t", Type: execute.TString}, + data: []query.Block{&executetest.Block{ + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "int", Type: query.TInt}, + {Label: "t", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(0), int64(2), "tag0"}, @@ -196,10 +196,10 @@ func TestCumulativeSum_Process(t *testing.T) { }, }}, want: []*executetest.Block{{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "int", Type: execute.TInt}, - {Label: "t", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "int", Type: query.TInt}, + {Label: "t", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(0), int64(2), "tag0"}, diff --git a/functions/data_test.go b/functions/data_test.go index 77be4d64c5..0cb9352eb9 100644 --- a/functions/data_test.go +++ b/functions/data_test.go @@ -26,7 +26,7 @@ func init() { var NormalData []float64 // NormalBlock is a block of data whose value col is NormalData. -var NormalBlock execute.Block +var NormalBlock query.Block func init() { dist := distuv.Normal{ @@ -42,10 +42,10 @@ func init() { stop := execute.Time(time.Date(2017, 10, 10, 0, 0, 0, 0, time.UTC).UnixNano()) t1Value := "a" key := execute.NewPartitionKey( - []execute.ColMeta{ - {Label: execute.DefaultStartColLabel, Type: execute.TTime}, - {Label: execute.DefaultStopColLabel, Type: execute.TTime}, - {Label: "t1", Type: execute.TString}, + []query.ColMeta{ + {Label: execute.DefaultStartColLabel, Type: query.TTime}, + {Label: execute.DefaultStopColLabel, Type: query.TTime}, + {Label: "t1", Type: query.TString}, }, []interface{}{ start, @@ -55,12 +55,12 @@ func init() { ) normalBlockBuilder := execute.NewColListBlockBuilder(key, executetest.UnlimitedAllocator) - normalBlockBuilder.AddCol(execute.ColMeta{Label: execute.DefaultTimeColLabel, Type: execute.TTime}) - normalBlockBuilder.AddCol(execute.ColMeta{Label: execute.DefaultStartColLabel, Type: execute.TTime}) - normalBlockBuilder.AddCol(execute.ColMeta{Label: execute.DefaultStopColLabel, Type: execute.TTime}) - normalBlockBuilder.AddCol(execute.ColMeta{Label: execute.DefaultValueColLabel, Type: execute.TFloat}) - normalBlockBuilder.AddCol(execute.ColMeta{Label: "t1", Type: execute.TString}) - normalBlockBuilder.AddCol(execute.ColMeta{Label: "t2", Type: execute.TString}) + normalBlockBuilder.AddCol(query.ColMeta{Label: execute.DefaultTimeColLabel, Type: query.TTime}) + normalBlockBuilder.AddCol(query.ColMeta{Label: execute.DefaultStartColLabel, Type: query.TTime}) + normalBlockBuilder.AddCol(query.ColMeta{Label: execute.DefaultStopColLabel, Type: query.TTime}) + normalBlockBuilder.AddCol(query.ColMeta{Label: execute.DefaultValueColLabel, Type: query.TFloat}) + normalBlockBuilder.AddCol(query.ColMeta{Label: "t1", Type: query.TString}) + normalBlockBuilder.AddCol(query.ColMeta{Label: "t2", Type: query.TString}) times := make([]execute.Time, N) startTimes := make([]execute.Time, N) diff --git a/functions/derivative.go b/functions/derivative.go index e66f850d92..2ebb1bfb33 100644 --- a/functions/derivative.go +++ b/functions/derivative.go @@ -5,9 +5,9 @@ import ( "math" "time" - "github.com/influxdata/platform/query/interpreter" "github.com/influxdata/platform/query" "github.com/influxdata/platform/query/execute" + "github.com/influxdata/platform/query/interpreter" "github.com/influxdata/platform/query/plan" "github.com/influxdata/platform/query/semantic" ) @@ -152,11 +152,11 @@ func NewDerivativeTransformation(d execute.Dataset, cache execute.BlockBuilderCa } } -func (t *derivativeTransformation) RetractBlock(id execute.DatasetID, key execute.PartitionKey) error { +func (t *derivativeTransformation) RetractBlock(id execute.DatasetID, key query.PartitionKey) error { return t.d.RetractBlock(key) } -func (t *derivativeTransformation) Process(id execute.DatasetID, b execute.Block) error { +func (t *derivativeTransformation) Process(id execute.DatasetID, b query.Block) error { builder, created := t.cache.BlockBuilder(b.Key()) if !created { return fmt.Errorf("derivative found duplicate block with key: %v", b.Key()) @@ -179,7 +179,7 @@ func (t *derivativeTransformation) Process(id execute.DatasetID, b execute.Block if found { dc := c // Derivative always results in a float - dc.Type = execute.TFloat + dc.Type = query.TFloat builder.AddCol(dc) derivatives[j] = newDerivative(j, t.unit, t.nonNegative) } else { @@ -192,14 +192,14 @@ func (t *derivativeTransformation) Process(id execute.DatasetID, b execute.Block // We need to drop the first row since its derivative is undefined firstIdx := 1 - return b.Do(func(cr execute.ColReader) error { + return b.Do(func(cr query.ColReader) error { l := cr.Len() for j, c := range cols { d := derivatives[j] switch c.Type { - case execute.TBool: + case query.TBool: builder.AppendBools(j, cr.Bools(j)[firstIdx:]) - case execute.TInt: + case query.TInt: if d != nil { for i := 0; i < l; i++ { time := cr.Times(timeIdx)[i] @@ -211,7 +211,7 @@ func (t *derivativeTransformation) Process(id execute.DatasetID, b execute.Block } else { builder.AppendInts(j, cr.Ints(j)[firstIdx:]) } - case execute.TUInt: + case query.TUInt: if d != nil { for i := 0; i < l; i++ { time := cr.Times(timeIdx)[i] @@ -223,7 +223,7 @@ func (t *derivativeTransformation) Process(id execute.DatasetID, b execute.Block } else { builder.AppendUInts(j, cr.UInts(j)[firstIdx:]) } - case execute.TFloat: + case query.TFloat: if d != nil { for i := 0; i < l; i++ { time := cr.Times(timeIdx)[i] @@ -235,9 +235,9 @@ func (t *derivativeTransformation) Process(id execute.DatasetID, b execute.Block } else { builder.AppendFloats(j, cr.Floats(j)[firstIdx:]) } - case execute.TString: + case query.TString: builder.AppendStrings(j, cr.Strings(j)[firstIdx:]) - case execute.TTime: + case query.TTime: builder.AppendTimes(j, cr.Times(j)[firstIdx:]) } } diff --git a/functions/derivative_test.go b/functions/derivative_test.go index 7adce71fba..12318a7e17 100644 --- a/functions/derivative_test.go +++ b/functions/derivative_test.go @@ -4,10 +4,10 @@ import ( "testing" "time" - "github.com/influxdata/platform/query/functions" "github.com/influxdata/platform/query" "github.com/influxdata/platform/query/execute" "github.com/influxdata/platform/query/execute/executetest" + "github.com/influxdata/platform/query/functions" "github.com/influxdata/platform/query/querytest" ) @@ -38,7 +38,7 @@ func TestDerivative_Process(t *testing.T) { testCases := []struct { name string spec *functions.DerivativeProcedureSpec - data []execute.Block + data []query.Block want []*executetest.Block }{ { @@ -48,10 +48,10 @@ func TestDerivative_Process(t *testing.T) { TimeCol: execute.DefaultTimeColLabel, Unit: 1, }, - data: []execute.Block{&executetest.Block{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + data: []query.Block{&executetest.Block{ + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(1), 2.0}, @@ -59,9 +59,9 @@ func TestDerivative_Process(t *testing.T) { }, }}, want: []*executetest.Block{{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(2), -1.0}, @@ -75,10 +75,10 @@ func TestDerivative_Process(t *testing.T) { TimeCol: execute.DefaultTimeColLabel, Unit: query.Duration(time.Second), }, - data: []execute.Block{&executetest.Block{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + data: []query.Block{&executetest.Block{ + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(1 * time.Second), 2.0}, @@ -86,9 +86,9 @@ func TestDerivative_Process(t *testing.T) { }, }}, want: []*executetest.Block{{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(3 * time.Second), -0.5}, @@ -102,10 +102,10 @@ func TestDerivative_Process(t *testing.T) { TimeCol: execute.DefaultTimeColLabel, Unit: 1, }, - data: []execute.Block{&executetest.Block{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TInt}, + data: []query.Block{&executetest.Block{ + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TInt}, }, Data: [][]interface{}{ {execute.Time(1), int64(20)}, @@ -113,9 +113,9 @@ func TestDerivative_Process(t *testing.T) { }, }}, want: []*executetest.Block{{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(2), -10.0}, @@ -129,10 +129,10 @@ func TestDerivative_Process(t *testing.T) { TimeCol: execute.DefaultTimeColLabel, Unit: query.Duration(time.Second), }, - data: []execute.Block{&executetest.Block{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TInt}, + data: []query.Block{&executetest.Block{ + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TInt}, }, Data: [][]interface{}{ {execute.Time(1 * time.Second), int64(20)}, @@ -140,9 +140,9 @@ func TestDerivative_Process(t *testing.T) { }, }}, want: []*executetest.Block{{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(3 * time.Second), -5.0}, @@ -157,10 +157,10 @@ func TestDerivative_Process(t *testing.T) { Unit: 1, NonNegative: true, }, - data: []execute.Block{&executetest.Block{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TInt}, + data: []query.Block{&executetest.Block{ + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TInt}, }, Data: [][]interface{}{ {execute.Time(1), int64(20)}, @@ -169,9 +169,9 @@ func TestDerivative_Process(t *testing.T) { }, }}, want: []*executetest.Block{{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(2), 10.0}, @@ -186,10 +186,10 @@ func TestDerivative_Process(t *testing.T) { TimeCol: execute.DefaultTimeColLabel, Unit: 1, }, - data: []execute.Block{&executetest.Block{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TUInt}, + data: []query.Block{&executetest.Block{ + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TUInt}, }, Data: [][]interface{}{ {execute.Time(1), uint64(10)}, @@ -197,9 +197,9 @@ func TestDerivative_Process(t *testing.T) { }, }}, want: []*executetest.Block{{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(2), 10.0}, @@ -213,10 +213,10 @@ func TestDerivative_Process(t *testing.T) { TimeCol: execute.DefaultTimeColLabel, Unit: 1, }, - data: []execute.Block{&executetest.Block{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TUInt}, + data: []query.Block{&executetest.Block{ + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TUInt}, }, Data: [][]interface{}{ {execute.Time(1), uint64(20)}, @@ -224,9 +224,9 @@ func TestDerivative_Process(t *testing.T) { }, }}, want: []*executetest.Block{{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(2), -10.0}, @@ -241,10 +241,10 @@ func TestDerivative_Process(t *testing.T) { Unit: 1, NonNegative: true, }, - data: []execute.Block{&executetest.Block{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TUInt}, + data: []query.Block{&executetest.Block{ + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TUInt}, }, Data: [][]interface{}{ {execute.Time(1), uint64(20)}, @@ -253,9 +253,9 @@ func TestDerivative_Process(t *testing.T) { }, }}, want: []*executetest.Block{{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(2), 10.0}, @@ -270,10 +270,10 @@ func TestDerivative_Process(t *testing.T) { TimeCol: execute.DefaultTimeColLabel, Unit: query.Duration(time.Second), }, - data: []execute.Block{&executetest.Block{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TUInt}, + data: []query.Block{&executetest.Block{ + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TUInt}, }, Data: [][]interface{}{ {execute.Time(1 * time.Second), uint64(20)}, @@ -281,9 +281,9 @@ func TestDerivative_Process(t *testing.T) { }, }}, want: []*executetest.Block{{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(3 * time.Second), -5.0}, @@ -298,10 +298,10 @@ func TestDerivative_Process(t *testing.T) { Unit: 1, NonNegative: true, }, - data: []execute.Block{&executetest.Block{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + data: []query.Block{&executetest.Block{ + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(1), 2.0}, @@ -310,9 +310,9 @@ func TestDerivative_Process(t *testing.T) { }, }}, want: []*executetest.Block{{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(2), 1.0}, @@ -327,11 +327,11 @@ func TestDerivative_Process(t *testing.T) { TimeCol: execute.DefaultTimeColLabel, Unit: 1, }, - data: []execute.Block{&executetest.Block{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t", Type: execute.TString}, + data: []query.Block{&executetest.Block{ + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(1), 2.0, "a"}, @@ -339,10 +339,10 @@ func TestDerivative_Process(t *testing.T) { }, }}, want: []*executetest.Block{{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(2), -1.0, "b"}, @@ -356,11 +356,11 @@ func TestDerivative_Process(t *testing.T) { TimeCol: execute.DefaultTimeColLabel, Unit: 1, }, - data: []execute.Block{&executetest.Block{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "x", Type: execute.TFloat}, - {Label: "y", Type: execute.TFloat}, + data: []query.Block{&executetest.Block{ + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "x", Type: query.TFloat}, + {Label: "y", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(1), 2.0, 20.0}, @@ -368,10 +368,10 @@ func TestDerivative_Process(t *testing.T) { }, }}, want: []*executetest.Block{{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "x", Type: execute.TFloat}, - {Label: "y", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "x", Type: query.TFloat}, + {Label: "y", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(2), -1.0, -10.0}, @@ -386,11 +386,11 @@ func TestDerivative_Process(t *testing.T) { Unit: 1, NonNegative: true, }, - data: []execute.Block{&executetest.Block{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "x", Type: execute.TFloat}, - {Label: "y", Type: execute.TFloat}, + data: []query.Block{&executetest.Block{ + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "x", Type: query.TFloat}, + {Label: "y", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(1), 2.0, 20.0}, @@ -399,10 +399,10 @@ func TestDerivative_Process(t *testing.T) { }, }}, want: []*executetest.Block{{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "x", Type: execute.TFloat}, - {Label: "y", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "x", Type: query.TFloat}, + {Label: "y", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(2), 1.0, 10.0}, diff --git a/functions/difference.go b/functions/difference.go index b94fea9fcd..c55e3b8b43 100644 --- a/functions/difference.go +++ b/functions/difference.go @@ -4,9 +4,9 @@ import ( "fmt" "math" - "github.com/influxdata/platform/query/interpreter" "github.com/influxdata/platform/query" "github.com/influxdata/platform/query/execute" + "github.com/influxdata/platform/query/interpreter" "github.com/influxdata/platform/query/plan" "github.com/influxdata/platform/query/semantic" ) @@ -129,11 +129,11 @@ func NewDifferenceTransformation(d execute.Dataset, cache execute.BlockBuilderCa } } -func (t *differenceTransformation) RetractBlock(id execute.DatasetID, key execute.PartitionKey) error { +func (t *differenceTransformation) RetractBlock(id execute.DatasetID, key query.PartitionKey) error { return t.d.RetractBlock(key) } -func (t *differenceTransformation) Process(id execute.DatasetID, b execute.Block) error { +func (t *differenceTransformation) Process(id execute.DatasetID, b query.Block) error { builder, created := t.cache.BlockBuilder(b.Key()) if !created { return fmt.Errorf("difference found duplicate block with key: %v", b.Key()) @@ -150,14 +150,14 @@ func (t *differenceTransformation) Process(id execute.DatasetID, b execute.Block } if found { - var typ execute.DataType + var typ query.DataType switch c.Type { - case execute.TInt, execute.TUInt: - typ = execute.TInt - case execute.TFloat: - typ = execute.TFloat + case query.TInt, query.TUInt: + typ = query.TInt + case query.TFloat: + typ = query.TFloat } - builder.AddCol(execute.ColMeta{ + builder.AddCol(query.ColMeta{ Label: c.Label, Type: typ, }) @@ -169,14 +169,14 @@ func (t *differenceTransformation) Process(id execute.DatasetID, b execute.Block // We need to drop the first row since its derivative is undefined firstIdx := 1 - return b.Do(func(cr execute.ColReader) error { + return b.Do(func(cr query.ColReader) error { l := cr.Len() for j, c := range cols { d := differences[j] switch c.Type { - case execute.TBool: + case query.TBool: builder.AppendBools(j, cr.Bools(j)[firstIdx:]) - case execute.TInt: + case query.TInt: if d != nil { for i := 0; i < l; i++ { v := d.updateInt(cr.Ints(j)[i]) @@ -187,7 +187,7 @@ func (t *differenceTransformation) Process(id execute.DatasetID, b execute.Block } else { builder.AppendInts(j, cr.Ints(j)[firstIdx:]) } - case execute.TUInt: + case query.TUInt: if d != nil { for i := 0; i < l; i++ { v := d.updateUInt(cr.UInts(j)[i]) @@ -198,7 +198,7 @@ func (t *differenceTransformation) Process(id execute.DatasetID, b execute.Block } else { builder.AppendUInts(j, cr.UInts(j)[firstIdx:]) } - case execute.TFloat: + case query.TFloat: if d != nil { for i := 0; i < l; i++ { v := d.updateFloat(cr.Floats(j)[i]) @@ -209,9 +209,9 @@ func (t *differenceTransformation) Process(id execute.DatasetID, b execute.Block } else { builder.AppendFloats(j, cr.Floats(j)[firstIdx:]) } - case execute.TString: + case query.TString: builder.AppendStrings(j, cr.Strings(j)[firstIdx:]) - case execute.TTime: + case query.TTime: builder.AppendTimes(j, cr.Times(j)[firstIdx:]) } } diff --git a/functions/difference_test.go b/functions/difference_test.go index be2b1319ba..06c6bd83f2 100644 --- a/functions/difference_test.go +++ b/functions/difference_test.go @@ -3,10 +3,10 @@ package functions_test import ( "testing" - "github.com/influxdata/platform/query/functions" "github.com/influxdata/platform/query" "github.com/influxdata/platform/query/execute" "github.com/influxdata/platform/query/execute/executetest" + "github.com/influxdata/platform/query/functions" "github.com/influxdata/platform/query/querytest" ) @@ -36,7 +36,7 @@ func TestDifference_Process(t *testing.T) { testCases := []struct { name string spec *functions.DifferenceProcedureSpec - data []execute.Block + data []query.Block want []*executetest.Block }{ { @@ -44,10 +44,10 @@ func TestDifference_Process(t *testing.T) { spec: &functions.DifferenceProcedureSpec{ Columns: []string{execute.DefaultValueColLabel}, }, - data: []execute.Block{&executetest.Block{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + data: []query.Block{&executetest.Block{ + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(1), 2.0}, @@ -55,9 +55,9 @@ func TestDifference_Process(t *testing.T) { }, }}, want: []*executetest.Block{{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(2), -1.0}, @@ -69,10 +69,10 @@ func TestDifference_Process(t *testing.T) { spec: &functions.DifferenceProcedureSpec{ Columns: []string{execute.DefaultValueColLabel}, }, - data: []execute.Block{&executetest.Block{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TInt}, + data: []query.Block{&executetest.Block{ + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TInt}, }, Data: [][]interface{}{ {execute.Time(1), int64(20)}, @@ -80,9 +80,9 @@ func TestDifference_Process(t *testing.T) { }, }}, want: []*executetest.Block{{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TInt}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TInt}, }, Data: [][]interface{}{ {execute.Time(2), int64(-10)}, @@ -95,10 +95,10 @@ func TestDifference_Process(t *testing.T) { Columns: []string{execute.DefaultValueColLabel}, NonNegative: true, }, - data: []execute.Block{&executetest.Block{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TInt}, + data: []query.Block{&executetest.Block{ + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TInt}, }, Data: [][]interface{}{ {execute.Time(1), int64(20)}, @@ -107,9 +107,9 @@ func TestDifference_Process(t *testing.T) { }, }}, want: []*executetest.Block{{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TInt}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TInt}, }, Data: [][]interface{}{ {execute.Time(2), int64(10)}, @@ -122,10 +122,10 @@ func TestDifference_Process(t *testing.T) { spec: &functions.DifferenceProcedureSpec{ Columns: []string{execute.DefaultValueColLabel}, }, - data: []execute.Block{&executetest.Block{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TUInt}, + data: []query.Block{&executetest.Block{ + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TUInt}, }, Data: [][]interface{}{ {execute.Time(1), uint64(10)}, @@ -133,9 +133,9 @@ func TestDifference_Process(t *testing.T) { }, }}, want: []*executetest.Block{{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TInt}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TInt}, }, Data: [][]interface{}{ {execute.Time(2), int64(10)}, @@ -147,10 +147,10 @@ func TestDifference_Process(t *testing.T) { spec: &functions.DifferenceProcedureSpec{ Columns: []string{execute.DefaultValueColLabel}, }, - data: []execute.Block{&executetest.Block{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TUInt}, + data: []query.Block{&executetest.Block{ + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TUInt}, }, Data: [][]interface{}{ {execute.Time(1), uint64(20)}, @@ -158,9 +158,9 @@ func TestDifference_Process(t *testing.T) { }, }}, want: []*executetest.Block{{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TInt}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TInt}, }, Data: [][]interface{}{ {execute.Time(2), int64(-10)}, @@ -173,10 +173,10 @@ func TestDifference_Process(t *testing.T) { Columns: []string{execute.DefaultValueColLabel}, NonNegative: true, }, - data: []execute.Block{&executetest.Block{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TUInt}, + data: []query.Block{&executetest.Block{ + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TUInt}, }, Data: [][]interface{}{ {execute.Time(1), uint64(20)}, @@ -185,9 +185,9 @@ func TestDifference_Process(t *testing.T) { }, }}, want: []*executetest.Block{{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TInt}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TInt}, }, Data: [][]interface{}{ {execute.Time(2), int64(10)}, @@ -201,10 +201,10 @@ func TestDifference_Process(t *testing.T) { Columns: []string{execute.DefaultValueColLabel}, NonNegative: true, }, - data: []execute.Block{&executetest.Block{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + data: []query.Block{&executetest.Block{ + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(1), 2.0}, @@ -213,9 +213,9 @@ func TestDifference_Process(t *testing.T) { }, }}, want: []*executetest.Block{{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(2), 1.0}, @@ -228,11 +228,11 @@ func TestDifference_Process(t *testing.T) { spec: &functions.DifferenceProcedureSpec{ Columns: []string{execute.DefaultValueColLabel}, }, - data: []execute.Block{&executetest.Block{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t", Type: execute.TString}, + data: []query.Block{&executetest.Block{ + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(1), 2.0, "a"}, @@ -240,10 +240,10 @@ func TestDifference_Process(t *testing.T) { }, }}, want: []*executetest.Block{{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(2), -1.0, "b"}, @@ -255,11 +255,11 @@ func TestDifference_Process(t *testing.T) { spec: &functions.DifferenceProcedureSpec{ Columns: []string{"x", "y"}, }, - data: []execute.Block{&executetest.Block{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "x", Type: execute.TFloat}, - {Label: "y", Type: execute.TFloat}, + data: []query.Block{&executetest.Block{ + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "x", Type: query.TFloat}, + {Label: "y", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(1), 2.0, 20.0}, @@ -267,10 +267,10 @@ func TestDifference_Process(t *testing.T) { }, }}, want: []*executetest.Block{{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "x", Type: execute.TFloat}, - {Label: "y", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "x", Type: query.TFloat}, + {Label: "y", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(2), -1.0, -10.0}, @@ -283,11 +283,11 @@ func TestDifference_Process(t *testing.T) { Columns: []string{"x", "y"}, NonNegative: true, }, - data: []execute.Block{&executetest.Block{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "x", Type: execute.TFloat}, - {Label: "y", Type: execute.TFloat}, + data: []query.Block{&executetest.Block{ + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "x", Type: query.TFloat}, + {Label: "y", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(1), 2.0, 20.0}, @@ -296,10 +296,10 @@ func TestDifference_Process(t *testing.T) { }, }}, want: []*executetest.Block{{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "x", Type: execute.TFloat}, - {Label: "y", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "x", Type: query.TFloat}, + {Label: "y", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(2), 1.0, 10.0}, diff --git a/functions/distinct.go b/functions/distinct.go index 930f596d5b..68094bbc49 100644 --- a/functions/distinct.go +++ b/functions/distinct.go @@ -138,11 +138,11 @@ func NewDistinctTransformation(d execute.Dataset, cache execute.BlockBuilderCach } } -func (t *distinctTransformation) RetractBlock(id execute.DatasetID, key execute.PartitionKey) error { +func (t *distinctTransformation) RetractBlock(id execute.DatasetID, key query.PartitionKey) error { return t.d.RetractBlock(key) } -func (t *distinctTransformation) Process(id execute.DatasetID, b execute.Block) error { +func (t *distinctTransformation) Process(id execute.DatasetID, b query.Block) error { builder, created := t.cache.BlockBuilder(b.Key()) if !created { return fmt.Errorf("distinct found duplicate block with key: %v", b.Key()) @@ -155,7 +155,7 @@ func (t *distinctTransformation) Process(id execute.DatasetID, b execute.Block) col := b.Cols()[colIdx] execute.AddBlockKeyCols(b.Key(), builder) - colIdx = builder.AddCol(execute.ColMeta{ + colIdx = builder.AddCol(query.ColMeta{ Label: execute.DefaultValueColLabel, Type: col.Type, }) @@ -163,23 +163,23 @@ func (t *distinctTransformation) Process(id execute.DatasetID, b execute.Block) if b.Key().HasCol(t.column) { j := execute.ColIdx(t.column, b.Key().Cols()) switch col.Type { - case execute.TBool: + case query.TBool: builder.AppendBool(colIdx, b.Key().ValueBool(j)) - case execute.TInt: + case query.TInt: builder.AppendInt(colIdx, b.Key().ValueInt(j)) - case execute.TUInt: + case query.TUInt: builder.AppendUInt(colIdx, b.Key().ValueUInt(j)) - case execute.TFloat: + case query.TFloat: builder.AppendFloat(colIdx, b.Key().ValueFloat(j)) - case execute.TString: + case query.TString: builder.AppendString(colIdx, b.Key().ValueString(j)) - case execute.TTime: + case query.TTime: builder.AppendTime(colIdx, b.Key().ValueTime(j)) } execute.AppendKeyValues(b.Key(), builder) // TODO: this is a hack - return b.Do(func(execute.ColReader) error { + return b.Do(func(query.ColReader) error { return nil }) } @@ -193,61 +193,61 @@ func (t *distinctTransformation) Process(id execute.DatasetID, b execute.Block) timeDistinct map[execute.Time]bool ) switch col.Type { - case execute.TBool: + case query.TBool: boolDistinct = make(map[bool]bool) - case execute.TInt: + case query.TInt: intDistinct = make(map[int64]bool) - case execute.TUInt: + case query.TUInt: uintDistinct = make(map[uint64]bool) - case execute.TFloat: + case query.TFloat: floatDistinct = make(map[float64]bool) - case execute.TString: + case query.TString: stringDistinct = make(map[string]bool) - case execute.TTime: + case query.TTime: timeDistinct = make(map[execute.Time]bool) } - return b.Do(func(cr execute.ColReader) error { + return b.Do(func(cr query.ColReader) error { l := cr.Len() for i := 0; i < l; i++ { // Check distinct switch col.Type { - case execute.TBool: + case query.TBool: v := cr.Bools(colIdx)[i] if boolDistinct[v] { continue } boolDistinct[v] = true builder.AppendBool(colIdx, v) - case execute.TInt: + case query.TInt: v := cr.Ints(colIdx)[i] if intDistinct[v] { continue } intDistinct[v] = true builder.AppendInt(colIdx, v) - case execute.TUInt: + case query.TUInt: v := cr.UInts(colIdx)[i] if uintDistinct[v] { continue } uintDistinct[v] = true builder.AppendUInt(colIdx, v) - case execute.TFloat: + case query.TFloat: v := cr.Floats(colIdx)[i] if floatDistinct[v] { continue } floatDistinct[v] = true builder.AppendFloat(colIdx, v) - case execute.TString: + case query.TString: v := cr.Strings(colIdx)[i] if stringDistinct[v] { continue } stringDistinct[v] = true builder.AppendString(colIdx, v) - case execute.TTime: + case query.TTime: v := cr.Times(colIdx)[i] if timeDistinct[v] { continue diff --git a/functions/filter.go b/functions/filter.go index 3ba56b74ee..58198b1c88 100644 --- a/functions/filter.go +++ b/functions/filter.go @@ -4,10 +4,10 @@ import ( "fmt" "log" - "github.com/influxdata/platform/query/ast" - "github.com/influxdata/platform/query/interpreter" "github.com/influxdata/platform/query" + "github.com/influxdata/platform/query/ast" "github.com/influxdata/platform/query/execute" + "github.com/influxdata/platform/query/interpreter" "github.com/influxdata/platform/query/plan" "github.com/influxdata/platform/query/semantic" ) @@ -217,11 +217,11 @@ func NewFilterTransformation(d execute.Dataset, cache execute.BlockBuilderCache, }, nil } -func (t *filterTransformation) RetractBlock(id execute.DatasetID, key execute.PartitionKey) error { +func (t *filterTransformation) RetractBlock(id execute.DatasetID, key query.PartitionKey) error { return t.d.RetractBlock(key) } -func (t *filterTransformation) Process(id execute.DatasetID, b execute.Block) error { +func (t *filterTransformation) Process(id execute.DatasetID, b query.Block) error { builder, created := t.cache.BlockBuilder(b.Key()) if !created { return fmt.Errorf("filter found duplicate block with key: %v", b.Key()) @@ -236,7 +236,7 @@ func (t *filterTransformation) Process(id execute.DatasetID, b execute.Block) er } // Append only matching rows to block - return b.Do(func(cr execute.ColReader) error { + return b.Do(func(cr query.ColReader) error { l := cr.Len() for i := 0; i < l; i++ { if pass, err := t.fn.Eval(i, cr); err != nil { diff --git a/functions/filter_test.go b/functions/filter_test.go index c635dda357..d4cc9a3665 100644 --- a/functions/filter_test.go +++ b/functions/filter_test.go @@ -5,11 +5,11 @@ import ( "testing" "time" - "github.com/influxdata/platform/query/ast" - "github.com/influxdata/platform/query/functions" "github.com/influxdata/platform/query" + "github.com/influxdata/platform/query/ast" "github.com/influxdata/platform/query/execute" "github.com/influxdata/platform/query/execute/executetest" + "github.com/influxdata/platform/query/functions" "github.com/influxdata/platform/query/plan" "github.com/influxdata/platform/query/plan/plantest" "github.com/influxdata/platform/query/querytest" @@ -530,7 +530,7 @@ func TestFilter_Process(t *testing.T) { testCases := []struct { name string spec *functions.FilterProcedureSpec - data []execute.Block + data []query.Block want []*executetest.Block }{ { @@ -548,10 +548,10 @@ func TestFilter_Process(t *testing.T) { }, }, }, - data: []execute.Block{&executetest.Block{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + data: []query.Block{&executetest.Block{ + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(1), 1.0}, @@ -559,9 +559,9 @@ func TestFilter_Process(t *testing.T) { }, }}, want: []*executetest.Block{{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(2), 6.0}, @@ -585,13 +585,13 @@ func TestFilter_Process(t *testing.T) { }, }, }, - data: []execute.Block{ + data: []query.Block{ &executetest.Block{ KeyCols: []string{"t1"}, - ColMeta: []execute.ColMeta{ - {Label: "t1", Type: execute.TString}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "t1", Type: query.TString}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {"a", execute.Time(1), 3.0}, @@ -601,10 +601,10 @@ func TestFilter_Process(t *testing.T) { }, &executetest.Block{ KeyCols: []string{"t1"}, - ColMeta: []execute.ColMeta{ - {Label: "t1", Type: execute.TString}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "t1", Type: query.TString}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {"b", execute.Time(3), 3.0}, @@ -616,10 +616,10 @@ func TestFilter_Process(t *testing.T) { want: []*executetest.Block{ { KeyCols: []string{"t1"}, - ColMeta: []execute.ColMeta{ - {Label: "t1", Type: execute.TString}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "t1", Type: query.TString}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {"a", execute.Time(2), 6.0}, @@ -627,10 +627,10 @@ func TestFilter_Process(t *testing.T) { }, { KeyCols: []string{"t1"}, - ColMeta: []execute.ColMeta{ - {Label: "t1", Type: execute.TString}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "t1", Type: query.TString}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {"b", execute.Time(4), 8.0}, @@ -681,12 +681,12 @@ func TestFilter_Process(t *testing.T) { }, }, }, - data: []execute.Block{&executetest.Block{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, - {Label: "t2", Type: execute.TString}, + data: []query.Block{&executetest.Block{ + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, + {Label: "t2", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(1), 1.0, "a", "x"}, @@ -695,11 +695,11 @@ func TestFilter_Process(t *testing.T) { }, }}, want: []*executetest.Block{{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, - {Label: "t2", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, + {Label: "t2", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(3), 8.0, "a", "y"}, diff --git a/functions/first_test.go b/functions/first_test.go index 7bc5fa1f66..4d7f9978bf 100644 --- a/functions/first_test.go +++ b/functions/first_test.go @@ -3,10 +3,10 @@ package functions_test import ( "testing" - "github.com/influxdata/platform/query/functions" "github.com/influxdata/platform/query" "github.com/influxdata/platform/query/execute" "github.com/influxdata/platform/query/execute/executetest" + "github.com/influxdata/platform/query/functions" "github.com/influxdata/platform/query/plan" "github.com/influxdata/platform/query/plan/plantest" "github.com/influxdata/platform/query/querytest" @@ -35,11 +35,11 @@ func TestFirst_Process(t *testing.T) { { name: "first", data: &executetest.Block{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, - {Label: "t2", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, + {Label: "t2", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(0), 0.0, "a", "y"}, diff --git a/functions/from_csv.go b/functions/from_csv.go index cd84c0f2ba..1233bca765 100644 --- a/functions/from_csv.go +++ b/functions/from_csv.go @@ -7,10 +7,10 @@ import ( "strings" "github.com/influxdata/platform/query" + "github.com/influxdata/platform/query/csv" "github.com/influxdata/platform/query/execute" "github.com/influxdata/platform/query/plan" "github.com/influxdata/platform/query/semantic" - "github.com/influxdata/platform/query/csv" "github.com/pkg/errors" ) @@ -102,7 +102,7 @@ func createFromCSVSource(prSpec plan.ProcedureSpec, dsid execute.DatasetID, a ex type CSVSource struct { id execute.DatasetID - data execute.Result + data query.Result ts []execute.Transformation } @@ -113,7 +113,7 @@ func (c *CSVSource) AddTransformation(t execute.Transformation) { func (c *CSVSource) Run(ctx context.Context) { var err error var max execute.Time - err = c.data.Blocks().Do(func(b execute.Block) error { + err = c.data.Blocks().Do(func(b query.Block) error { for _, t := range c.ts { err := t.Process(c.id, b) if err != nil { diff --git a/functions/group.go b/functions/group.go index 23a563cb3b..0129861376 100644 --- a/functions/group.go +++ b/functions/group.go @@ -5,9 +5,9 @@ import ( "fmt" "sort" - "github.com/influxdata/platform/query/interpreter" "github.com/influxdata/platform/query" "github.com/influxdata/platform/query/execute" + "github.com/influxdata/platform/query/interpreter" "github.com/influxdata/platform/query/plan" "github.com/influxdata/platform/query/semantic" ) @@ -216,7 +216,7 @@ func NewGroupTransformation(d execute.Dataset, cache execute.BlockBuilderCache, return t } -func (t *groupTransformation) RetractBlock(id execute.DatasetID, key execute.PartitionKey) (err error) { +func (t *groupTransformation) RetractBlock(id execute.DatasetID, key query.PartitionKey) (err error) { //TODO(nathanielc): Investigate if this can be smarter and not retract all blocks with the same time bounds. panic("not implemented") //t.cache.ForEachBuilder(func(bk execute.BlockKey, builder execute.BlockBuilder) { @@ -230,7 +230,7 @@ func (t *groupTransformation) RetractBlock(id execute.DatasetID, key execute.Par //return } -func (t *groupTransformation) Process(id execute.DatasetID, b execute.Block) error { +func (t *groupTransformation) Process(id execute.DatasetID, b query.Block) error { cols := b.Cols() on := make(map[string]bool, len(cols)) if len(t.keys) > 0 { @@ -248,7 +248,7 @@ func (t *groupTransformation) Process(id execute.DatasetID, b execute.Block) err on[c.Label] = true } } - return b.Do(func(cr execute.ColReader) error { + return b.Do(func(cr query.ColReader) error { l := cr.Len() for i := 0; i < l; i++ { key := execute.PartitionKeyForRowOn(i, cr, on) diff --git a/functions/group_test.go b/functions/group_test.go index 67f99b180f..c2b1663b88 100644 --- a/functions/group_test.go +++ b/functions/group_test.go @@ -3,10 +3,10 @@ package functions_test import ( "testing" - "github.com/influxdata/platform/query/functions" "github.com/influxdata/platform/query" "github.com/influxdata/platform/query/execute" "github.com/influxdata/platform/query/execute/executetest" + "github.com/influxdata/platform/query/functions" "github.com/influxdata/platform/query/plan" "github.com/influxdata/platform/query/plan/plantest" "github.com/influxdata/platform/query/querytest" @@ -27,7 +27,7 @@ func TestGroup_Process(t *testing.T) { testCases := []struct { name string spec *functions.GroupProcedureSpec - data []execute.Block + data []query.Block want []*executetest.Block }{ { @@ -35,14 +35,14 @@ func TestGroup_Process(t *testing.T) { spec: &functions.GroupProcedureSpec{ By: []string{"t1"}, }, - data: []execute.Block{ + data: []query.Block{ &executetest.Block{ KeyCols: []string{"t1", "t2"}, - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, - {Label: "t2", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, + {Label: "t2", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(1), 2.0, "a", "x"}, @@ -50,11 +50,11 @@ func TestGroup_Process(t *testing.T) { }, &executetest.Block{ KeyCols: []string{"t1", "t2"}, - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, - {Label: "t2", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, + {Label: "t2", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(2), 1.0, "a", "y"}, @@ -62,11 +62,11 @@ func TestGroup_Process(t *testing.T) { }, &executetest.Block{ KeyCols: []string{"t1", "t2"}, - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, - {Label: "t2", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, + {Label: "t2", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(1), 4.0, "b", "x"}, @@ -74,11 +74,11 @@ func TestGroup_Process(t *testing.T) { }, &executetest.Block{ KeyCols: []string{"t1", "t2"}, - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, - {Label: "t2", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, + {Label: "t2", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(2), 7.0, "b", "y"}, @@ -88,11 +88,11 @@ func TestGroup_Process(t *testing.T) { want: []*executetest.Block{ { KeyCols: []string{"t1"}, - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, - {Label: "t2", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, + {Label: "t2", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(1), 2.0, "a", "x"}, @@ -101,11 +101,11 @@ func TestGroup_Process(t *testing.T) { }, { KeyCols: []string{"t1"}, - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, - {Label: "t2", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, + {Label: "t2", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(1), 4.0, "b", "x"}, @@ -119,15 +119,15 @@ func TestGroup_Process(t *testing.T) { spec: &functions.GroupProcedureSpec{ Except: []string{"_time", "_value", "t2"}, }, - data: []execute.Block{ + data: []query.Block{ &executetest.Block{ KeyCols: []string{"t1", "t2", "t3"}, - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, - {Label: "t2", Type: execute.TString}, - {Label: "t3", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, + {Label: "t2", Type: query.TString}, + {Label: "t3", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(1), 2.0, "a", "m", "x"}, @@ -135,12 +135,12 @@ func TestGroup_Process(t *testing.T) { }, &executetest.Block{ KeyCols: []string{"t1", "t2", "t3"}, - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, - {Label: "t2", Type: execute.TString}, - {Label: "t3", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, + {Label: "t2", Type: query.TString}, + {Label: "t3", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(2), 1.0, "a", "n", "x"}, @@ -148,12 +148,12 @@ func TestGroup_Process(t *testing.T) { }, &executetest.Block{ KeyCols: []string{"t1", "t2", "t3"}, - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, - {Label: "t2", Type: execute.TString}, - {Label: "t3", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, + {Label: "t2", Type: query.TString}, + {Label: "t3", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(1), 4.0, "b", "m", "x"}, @@ -161,12 +161,12 @@ func TestGroup_Process(t *testing.T) { }, &executetest.Block{ KeyCols: []string{"t1", "t2", "t3"}, - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, - {Label: "t2", Type: execute.TString}, - {Label: "t3", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, + {Label: "t2", Type: query.TString}, + {Label: "t3", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(2), 7.0, "b", "n", "x"}, @@ -176,12 +176,12 @@ func TestGroup_Process(t *testing.T) { want: []*executetest.Block{ { KeyCols: []string{"t1", "t3"}, - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, - {Label: "t2", Type: execute.TString}, - {Label: "t3", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, + {Label: "t2", Type: query.TString}, + {Label: "t3", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(1), 2.0, "a", "m", "x"}, @@ -190,12 +190,12 @@ func TestGroup_Process(t *testing.T) { }, { KeyCols: []string{"t1", "t3"}, - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, - {Label: "t2", Type: execute.TString}, - {Label: "t3", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, + {Label: "t2", Type: query.TString}, + {Label: "t3", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(1), 4.0, "b", "m", "x"}, @@ -209,11 +209,11 @@ func TestGroup_Process(t *testing.T) { spec: &functions.GroupProcedureSpec{ By: []string{"t1"}, }, - data: []execute.Block{&executetest.Block{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, + data: []query.Block{&executetest.Block{ + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(1), 2.0, "a"}, @@ -223,10 +223,10 @@ func TestGroup_Process(t *testing.T) { want: []*executetest.Block{ { KeyCols: []string{"t1"}, - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(1), 2.0, "a"}, @@ -234,10 +234,10 @@ func TestGroup_Process(t *testing.T) { }, { KeyCols: []string{"t1"}, - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(2), 1.0, "b"}, @@ -250,13 +250,13 @@ func TestGroup_Process(t *testing.T) { spec: &functions.GroupProcedureSpec{ Except: []string{"_time", "_value", "t2"}, }, - data: []execute.Block{&executetest.Block{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, - {Label: "t2", Type: execute.TString}, - {Label: "t3", Type: execute.TString}, + data: []query.Block{&executetest.Block{ + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, + {Label: "t2", Type: query.TString}, + {Label: "t3", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(1), 2.0, "a", "m", "x"}, @@ -266,12 +266,12 @@ func TestGroup_Process(t *testing.T) { want: []*executetest.Block{ { KeyCols: []string{"t1", "t3"}, - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, - {Label: "t2", Type: execute.TString}, - {Label: "t3", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, + {Label: "t2", Type: query.TString}, + {Label: "t3", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(1), 2.0, "a", "m", "x"}, @@ -279,12 +279,12 @@ func TestGroup_Process(t *testing.T) { }, { KeyCols: []string{"t1", "t3"}, - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, - {Label: "t2", Type: execute.TString}, - {Label: "t3", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, + {Label: "t2", Type: query.TString}, + {Label: "t3", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(2), 1.0, "a", "n", "y"}, diff --git a/functions/integral.go b/functions/integral.go index 07381533c1..5294c20e9c 100644 --- a/functions/integral.go +++ b/functions/integral.go @@ -113,20 +113,20 @@ func NewIntegralTransformation(d execute.Dataset, cache execute.BlockBuilderCach } } -func (t *integralTransformation) RetractBlock(id execute.DatasetID, key execute.PartitionKey) error { +func (t *integralTransformation) RetractBlock(id execute.DatasetID, key query.PartitionKey) error { return t.d.RetractBlock(key) } -func (t *integralTransformation) Process(id execute.DatasetID, b execute.Block) error { +func (t *integralTransformation) Process(id execute.DatasetID, b query.Block) error { builder, created := t.cache.BlockBuilder(b.Key()) if !created { return fmt.Errorf("integral found duplicate block with key: %v", b.Key()) } execute.AddBlockKeyCols(b.Key(), builder) - builder.AddCol(execute.ColMeta{ + builder.AddCol(query.ColMeta{ Label: t.spec.TimeDst, - Type: execute.TTime, + Type: query.TTime, }) cols := b.Cols() integrals := make([]*integral, len(cols)) @@ -134,9 +134,9 @@ func (t *integralTransformation) Process(id execute.DatasetID, b execute.Block) for j, c := range cols { if execute.ContainsStr(t.spec.Columns, c.Label) { integrals[j] = newIntegral(time.Duration(t.spec.Unit)) - colMap[j] = builder.AddCol(execute.ColMeta{ + colMap[j] = builder.AddCol(query.ColMeta{ Label: c.Label, - Type: execute.TFloat, + Type: query.TFloat, }) } } @@ -149,7 +149,7 @@ func (t *integralTransformation) Process(id execute.DatasetID, b execute.Block) if timeIdx < 0 { return fmt.Errorf("no column %q exists", t.spec.TimeSrc) } - err := b.Do(func(cr execute.ColReader) error { + err := b.Do(func(cr query.ColReader) error { for j, in := range integrals { if in == nil { continue diff --git a/functions/integral_test.go b/functions/integral_test.go index e6803cc922..d66d01d5c9 100644 --- a/functions/integral_test.go +++ b/functions/integral_test.go @@ -4,10 +4,10 @@ import ( "testing" "time" - "github.com/influxdata/platform/query/functions" "github.com/influxdata/platform/query" "github.com/influxdata/platform/query/execute" "github.com/influxdata/platform/query/execute/executetest" + "github.com/influxdata/platform/query/functions" "github.com/influxdata/platform/query/querytest" ) @@ -37,7 +37,7 @@ func TestIntegral_Process(t *testing.T) { testCases := []struct { name string spec *functions.IntegralProcedureSpec - data []execute.Block + data []query.Block want []*executetest.Block }{ { @@ -46,13 +46,13 @@ func TestIntegral_Process(t *testing.T) { Unit: 1, AggregateConfig: execute.DefaultAggregateConfig, }, - data: []execute.Block{&executetest.Block{ + data: []query.Block{&executetest.Block{ KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(1), execute.Time(3), execute.Time(1), 2.0}, @@ -61,11 +61,11 @@ func TestIntegral_Process(t *testing.T) { }}, want: []*executetest.Block{{ KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(1), execute.Time(3), execute.Time(3), 1.5}, @@ -78,13 +78,13 @@ func TestIntegral_Process(t *testing.T) { Unit: query.Duration(time.Second), AggregateConfig: execute.DefaultAggregateConfig, }, - data: []execute.Block{&executetest.Block{ + data: []query.Block{&executetest.Block{ KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(1 * time.Second), execute.Time(4 * time.Second), execute.Time(1 * time.Second), 2.0}, @@ -93,11 +93,11 @@ func TestIntegral_Process(t *testing.T) { }}, want: []*executetest.Block{{ KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(1 * time.Second), execute.Time(4 * time.Second), execute.Time(4 * time.Second), 3.0}, @@ -110,14 +110,14 @@ func TestIntegral_Process(t *testing.T) { Unit: 1, AggregateConfig: execute.DefaultAggregateConfig, }, - data: []execute.Block{&executetest.Block{ + data: []query.Block{&executetest.Block{ KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(1), execute.Time(3), execute.Time(1), 2.0, "a"}, @@ -126,11 +126,11 @@ func TestIntegral_Process(t *testing.T) { }}, want: []*executetest.Block{{ KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(1), execute.Time(3), execute.Time(3), 1.5}, @@ -147,14 +147,14 @@ func TestIntegral_Process(t *testing.T) { Columns: []string{"x", "y"}, }, }, - data: []execute.Block{&executetest.Block{ + data: []query.Block{&executetest.Block{ KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "x", Type: execute.TFloat}, - {Label: "y", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "x", Type: query.TFloat}, + {Label: "y", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(1), execute.Time(5), execute.Time(1), 2.0, 20.0}, @@ -165,12 +165,12 @@ func TestIntegral_Process(t *testing.T) { }}, want: []*executetest.Block{{ KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "x", Type: execute.TFloat}, - {Label: "y", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "x", Type: query.TFloat}, + {Label: "y", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(1), execute.Time(5), execute.Time(5), 4.5, 45.0}, diff --git a/functions/join.go b/functions/join.go index 1e41c61a56..1115807792 100644 --- a/functions/join.go +++ b/functions/join.go @@ -6,10 +6,10 @@ import ( "sort" "sync" - "github.com/influxdata/platform/query/compiler" - "github.com/influxdata/platform/query/interpreter" "github.com/influxdata/platform/query" + "github.com/influxdata/platform/query/compiler" "github.com/influxdata/platform/query/execute" + "github.com/influxdata/platform/query/interpreter" "github.com/influxdata/platform/query/plan" "github.com/influxdata/platform/query/semantic" "github.com/influxdata/platform/query/values" @@ -225,7 +225,7 @@ type mergeJoinParentState struct { finished bool } -func (t *mergeJoinTransformation) RetractBlock(id execute.DatasetID, key execute.PartitionKey) error { +func (t *mergeJoinTransformation) RetractBlock(id execute.DatasetID, key query.PartitionKey) error { panic("not implemented") //t.mu.Lock() //defer t.mu.Unlock() @@ -237,7 +237,7 @@ func (t *mergeJoinTransformation) RetractBlock(id execute.DatasetID, key execute //return t.d.RetractBlock(execute.ToBlockKey(bm)) } -func (t *mergeJoinTransformation) Process(id execute.DatasetID, b execute.Block) error { +func (t *mergeJoinTransformation) Process(id execute.DatasetID, b query.Block) error { t.mu.Lock() defer t.mu.Unlock() @@ -342,7 +342,7 @@ func (t *mergeJoinTransformation) Finish(id execute.DatasetID, err error) { } type MergeJoinCache interface { - Tables(execute.PartitionKey) *joinTables + Tables(query.PartitionKey) *joinTables } type mergeJoinCache struct { @@ -375,7 +375,7 @@ func NewMergeJoinCache(joinFn *joinFunc, a *execute.Allocator, leftName, rightNa } } -func (c *mergeJoinCache) Block(key execute.PartitionKey) (execute.Block, error) { +func (c *mergeJoinCache) Block(key query.PartitionKey) (query.Block, error) { t, ok := c.lookup(key) if !ok { return nil, errors.New("block not found") @@ -383,14 +383,14 @@ func (c *mergeJoinCache) Block(key execute.PartitionKey) (execute.Block, error) return t.Join() } -func (c *mergeJoinCache) ForEach(f func(execute.PartitionKey)) { - c.data.Range(func(key execute.PartitionKey, value interface{}) { +func (c *mergeJoinCache) ForEach(f func(query.PartitionKey)) { + c.data.Range(func(key query.PartitionKey, value interface{}) { f(key) }) } -func (c *mergeJoinCache) ForEachWithContext(f func(execute.PartitionKey, execute.Trigger, execute.BlockContext)) { - c.data.Range(func(key execute.PartitionKey, value interface{}) { +func (c *mergeJoinCache) ForEachWithContext(f func(query.PartitionKey, execute.Trigger, execute.BlockContext)) { + c.data.Range(func(key query.PartitionKey, value interface{}) { tables := value.(*joinTables) bc := execute.BlockContext{ Key: key, @@ -400,14 +400,14 @@ func (c *mergeJoinCache) ForEachWithContext(f func(execute.PartitionKey, execute }) } -func (c *mergeJoinCache) DiscardBlock(key execute.PartitionKey) { +func (c *mergeJoinCache) DiscardBlock(key query.PartitionKey) { t, ok := c.lookup(key) if ok { t.ClearData() } } -func (c *mergeJoinCache) ExpireBlock(key execute.PartitionKey) { +func (c *mergeJoinCache) ExpireBlock(key query.PartitionKey) { v, ok := c.data.Delete(key) if ok { v.(*joinTables).ClearData() @@ -418,7 +418,7 @@ func (c *mergeJoinCache) SetTriggerSpec(spec query.TriggerSpec) { c.triggerSpec = spec } -func (c *mergeJoinCache) lookup(key execute.PartitionKey) (*joinTables, bool) { +func (c *mergeJoinCache) lookup(key query.PartitionKey) (*joinTables, bool) { v, ok := c.data.Lookup(key) if !ok { return nil, false @@ -426,7 +426,7 @@ func (c *mergeJoinCache) lookup(key execute.PartitionKey) (*joinTables, bool) { return v.(*joinTables), true } -func (c *mergeJoinCache) Tables(key execute.PartitionKey) *joinTables { +func (c *mergeJoinCache) Tables(key query.PartitionKey) *joinTables { tables, ok := c.lookup(key) if !ok { tables = &joinTables{ @@ -449,7 +449,7 @@ func (c *mergeJoinCache) Tables(key execute.PartitionKey) *joinTables { type joinTables struct { keys []string on map[string]bool - key execute.PartitionKey + key query.PartitionKey alloc *execute.Allocator @@ -471,7 +471,7 @@ func (t *joinTables) ClearData() { } // Join performs a sort-merge join -func (t *joinTables) Join() (execute.Block, error) { +func (t *joinTables) Join() (query.Block, error) { // First prepare the join function left := t.left.RawBlock() right := t.right.RawBlock() @@ -493,7 +493,7 @@ func (t *joinTables) Join() (execute.Block, error) { } sort.Strings(keys) for _, k := range keys { - builder.AddCol(execute.ColMeta{ + builder.AddCol(query.ColMeta{ Label: k, Type: execute.ConvertFromKind(properties[k].Kind()), }) @@ -513,7 +513,7 @@ func (t *joinTables) Join() (execute.Block, error) { var ( leftSet, rightSet subset - leftKey, rightKey execute.PartitionKey + leftKey, rightKey query.PartitionKey ) rows := map[string]int{ @@ -552,7 +552,7 @@ func (t *joinTables) Join() (execute.Block, error) { return builder.Block() } -func (t *joinTables) advance(offset int, table *execute.ColListBlock) (subset, execute.PartitionKey) { +func (t *joinTables) advance(offset int, table *execute.ColListBlock) (subset, query.PartitionKey) { if n := table.NRows(); n == offset { return subset{Start: n, Stop: n}, nil } @@ -582,27 +582,27 @@ func equalRowKeys(x, y int, table *execute.ColListBlock, on map[string]bool) boo continue } switch c.Type { - case execute.TBool: + case query.TBool: if xv, yv := table.Bools(j)[x], table.Bools(j)[y]; xv != yv { return false } - case execute.TInt: + case query.TInt: if xv, yv := table.Ints(j)[x], table.Ints(j)[y]; xv != yv { return false } - case execute.TUInt: + case query.TUInt: if xv, yv := table.UInts(j)[x], table.UInts(j)[y]; xv != yv { return false } - case execute.TFloat: + case query.TFloat: if xv, yv := table.Floats(j)[x], table.Floats(j)[y]; xv != yv { return false } - case execute.TString: + case query.TString: if xv, yv := table.Strings(j)[x], table.Strings(j)[y]; xv != yv { return false } - case execute.TTime: + case query.TTime: if xv, yv := table.Times(j)[x], table.Times(j)[y]; xv != yv { return false } diff --git a/functions/join_test.go b/functions/join_test.go index c33bb0eb4e..701f8e2771 100644 --- a/functions/join_test.go +++ b/functions/join_test.go @@ -6,11 +6,11 @@ import ( "time" "github.com/google/go-cmp/cmp" - "github.com/influxdata/platform/query/ast" - "github.com/influxdata/platform/query/functions" "github.com/influxdata/platform/query" + "github.com/influxdata/platform/query/ast" "github.com/influxdata/platform/query/execute" "github.com/influxdata/platform/query/execute/executetest" + "github.com/influxdata/platform/query/functions" "github.com/influxdata/platform/query/plan" "github.com/influxdata/platform/query/plan/plantest" "github.com/influxdata/platform/query/querytest" @@ -569,9 +569,9 @@ func TestMergeJoin_Process(t *testing.T) { }, data0: []*executetest.Block{ { - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(1), 1.0}, @@ -582,9 +582,9 @@ func TestMergeJoin_Process(t *testing.T) { }, data1: []*executetest.Block{ { - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(1), 10.0}, @@ -595,9 +595,9 @@ func TestMergeJoin_Process(t *testing.T) { }, want: []*executetest.Block{ { - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(1), 11.0}, @@ -616,9 +616,9 @@ func TestMergeJoin_Process(t *testing.T) { }, data0: []*executetest.Block{ { - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TInt}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TInt}, }, Data: [][]interface{}{ {execute.Time(1), int64(1)}, @@ -629,9 +629,9 @@ func TestMergeJoin_Process(t *testing.T) { }, data1: []*executetest.Block{ { - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TInt}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TInt}, }, Data: [][]interface{}{ {execute.Time(1), int64(10)}, @@ -642,9 +642,9 @@ func TestMergeJoin_Process(t *testing.T) { }, want: []*executetest.Block{ { - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TInt}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TInt}, }, Data: [][]interface{}{ {execute.Time(1), int64(11)}, @@ -663,9 +663,9 @@ func TestMergeJoin_Process(t *testing.T) { }, data0: []*executetest.Block{ { - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(1), 1.0}, @@ -676,9 +676,9 @@ func TestMergeJoin_Process(t *testing.T) { }, data1: []*executetest.Block{ { - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(1), 10.0}, @@ -688,9 +688,9 @@ func TestMergeJoin_Process(t *testing.T) { }, want: []*executetest.Block{ { - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(1), 11.0}, @@ -708,9 +708,9 @@ func TestMergeJoin_Process(t *testing.T) { }, data0: []*executetest.Block{ { - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(1), 1.0}, @@ -721,9 +721,9 @@ func TestMergeJoin_Process(t *testing.T) { }, data1: []*executetest.Block{ { - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(1), 10.0}, @@ -736,9 +736,9 @@ func TestMergeJoin_Process(t *testing.T) { }, want: []*executetest.Block{ { - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(1), 11.0}, @@ -760,10 +760,10 @@ func TestMergeJoin_Process(t *testing.T) { data0: []*executetest.Block{ { KeyCols: []string{"t1"}, - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(1), 1.0, "a"}, @@ -775,10 +775,10 @@ func TestMergeJoin_Process(t *testing.T) { data1: []*executetest.Block{ { KeyCols: []string{"t1"}, - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(1), 10.0, "a"}, @@ -790,10 +790,10 @@ func TestMergeJoin_Process(t *testing.T) { want: []*executetest.Block{ { KeyCols: []string{"t1"}, - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(1), 11.0, "a"}, @@ -812,10 +812,10 @@ func TestMergeJoin_Process(t *testing.T) { }, data0: []*executetest.Block{ { - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(1), 1.0, "a"}, @@ -829,10 +829,10 @@ func TestMergeJoin_Process(t *testing.T) { }, data1: []*executetest.Block{ { - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(1), 10.0, "a"}, @@ -846,10 +846,10 @@ func TestMergeJoin_Process(t *testing.T) { }, want: []*executetest.Block{ { - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(1), 11.0, "a"}, @@ -872,11 +872,11 @@ func TestMergeJoin_Process(t *testing.T) { data0: []*executetest.Block{ { KeyCols: []string{"t1"}, - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, - {Label: "t2", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, + {Label: "t2", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(1), 1.0, "a", "x"}, @@ -891,11 +891,11 @@ func TestMergeJoin_Process(t *testing.T) { data1: []*executetest.Block{ { KeyCols: []string{"t1"}, - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, - {Label: "t2", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, + {Label: "t2", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(1), 10.0, "a", "x"}, @@ -910,11 +910,11 @@ func TestMergeJoin_Process(t *testing.T) { want: []*executetest.Block{ { KeyCols: []string{"t1"}, - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, - {Label: "t2", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, + {Label: "t2", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(1), 11.0, "a", "x"}, @@ -936,9 +936,9 @@ func TestMergeJoin_Process(t *testing.T) { }, data0: []*executetest.Block{ { - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(1), 1.0}, @@ -949,9 +949,9 @@ func TestMergeJoin_Process(t *testing.T) { }, data1: []*executetest.Block{ { - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(1), 10.0}, @@ -962,10 +962,10 @@ func TestMergeJoin_Process(t *testing.T) { }, want: []*executetest.Block{ { - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "a", Type: execute.TFloat}, - {Label: "b", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "a", Type: query.TFloat}, + {Label: "b", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(1), 1.0, 10.0}, @@ -985,11 +985,11 @@ func TestMergeJoin_Process(t *testing.T) { data0: []*executetest.Block{ { KeyCols: []string{"t1"}, - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, - {Label: "t2", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, + {Label: "t2", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(1), 1.0, "a", "x"}, @@ -1004,11 +1004,11 @@ func TestMergeJoin_Process(t *testing.T) { data1: []*executetest.Block{ { KeyCols: []string{"t1"}, - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, - {Label: "t2", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, + {Label: "t2", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(1), 10.0, "a", "x"}, @@ -1023,12 +1023,12 @@ func TestMergeJoin_Process(t *testing.T) { want: []*executetest.Block{ { KeyCols: []string{"t1"}, - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "a", Type: execute.TFloat}, - {Label: "b", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, - {Label: "t2", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "a", Type: query.TFloat}, + {Label: "b", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, + {Label: "t2", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(1), 1.0, 10.0, "a", "x"}, diff --git a/functions/keys.go b/functions/keys.go index 427c16ae56..050b77fe98 100644 --- a/functions/keys.go +++ b/functions/keys.go @@ -5,9 +5,9 @@ import ( "sort" "strings" - "github.com/influxdata/platform/query/interpreter" "github.com/influxdata/platform/query" "github.com/influxdata/platform/query/execute" + "github.com/influxdata/platform/query/interpreter" "github.com/influxdata/platform/query/plan" "github.com/influxdata/platform/query/semantic" ) @@ -151,18 +151,18 @@ func NewKeysTransformation(d execute.Dataset, cache execute.BlockBuilderCache, s } } -func (t *keysTransformation) RetractBlock(id execute.DatasetID, key execute.PartitionKey) error { +func (t *keysTransformation) RetractBlock(id execute.DatasetID, key query.PartitionKey) error { return t.d.RetractBlock(key) } -func (t *keysTransformation) Process(id execute.DatasetID, b execute.Block) error { +func (t *keysTransformation) Process(id execute.DatasetID, b query.Block) error { builder, created := t.cache.BlockBuilder(b.Key()) if !created { return fmt.Errorf("keys found duplicate block with key: %v", b.Key()) } execute.AddBlockKeyCols(b.Key(), builder) - colIdx := builder.AddCol(execute.ColMeta{Label: execute.DefaultValueColLabel, Type: execute.TString}) + colIdx := builder.AddCol(query.ColMeta{Label: execute.DefaultValueColLabel, Type: query.TString}) cols := b.Cols() sort.Slice(cols, func(i, j int) bool { @@ -194,7 +194,7 @@ func (t *keysTransformation) Process(id execute.DatasetID, b execute.Block) erro } // TODO: this is a hack - return b.Do(func(execute.ColReader) error { + return b.Do(func(query.ColReader) error { return nil }) } diff --git a/functions/keys_test.go b/functions/keys_test.go index 014e2b2939..a154cb4cc2 100644 --- a/functions/keys_test.go +++ b/functions/keys_test.go @@ -3,28 +3,29 @@ package functions_test import ( "testing" - "github.com/influxdata/platform/query/functions" + "github.com/influxdata/platform/query" "github.com/influxdata/platform/query/execute" "github.com/influxdata/platform/query/execute/executetest" + "github.com/influxdata/platform/query/functions" ) func TestKeys_Process(t *testing.T) { testCases := []struct { name string spec *functions.KeysProcedureSpec - data []execute.Block + data []query.Block want []*executetest.Block }{ { name: "one block", spec: &functions.KeysProcedureSpec{}, - data: []execute.Block{ + data: []query.Block{ &executetest.Block{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "tag0", Type: execute.TString}, - {Label: "tag1", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "tag0", Type: query.TString}, + {Label: "tag1", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(1), 2.0}, @@ -32,8 +33,8 @@ func TestKeys_Process(t *testing.T) { }, }, want: []*executetest.Block{{ - ColMeta: []execute.ColMeta{ - {Label: "_value", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_value", Type: query.TString}, }, Data: [][]interface{}{ {"_time"}, @@ -46,13 +47,13 @@ func TestKeys_Process(t *testing.T) { { name: "one block except", spec: &functions.KeysProcedureSpec{Except: []string{"_value", "_time"}}, - data: []execute.Block{ + data: []query.Block{ &executetest.Block{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "tag0", Type: execute.TString}, - {Label: "tag1", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "tag0", Type: query.TString}, + {Label: "tag1", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(1), 2.0}, @@ -60,8 +61,8 @@ func TestKeys_Process(t *testing.T) { }, }, want: []*executetest.Block{{ - ColMeta: []execute.ColMeta{ - {Label: "_value", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_value", Type: query.TString}, }, Data: [][]interface{}{ {"tag0"}, @@ -72,14 +73,14 @@ func TestKeys_Process(t *testing.T) { { name: "two blocks", spec: &functions.KeysProcedureSpec{}, - data: []execute.Block{ + data: []query.Block{ &executetest.Block{ KeyCols: []string{"tag0", "tag1"}, - ColMeta: []execute.ColMeta{ - {Label: "tag0", Type: execute.TString}, - {Label: "tag1", Type: execute.TString}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "tag0", Type: query.TString}, + {Label: "tag1", Type: query.TString}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {"tag0-0", "tag1-0", execute.Time(1), 2.0}, @@ -87,11 +88,11 @@ func TestKeys_Process(t *testing.T) { }, &executetest.Block{ KeyCols: []string{"tag0", "tag2"}, - ColMeta: []execute.ColMeta{ - {Label: "tag0", Type: execute.TString}, - {Label: "tag2", Type: execute.TString}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "tag0", Type: query.TString}, + {Label: "tag2", Type: query.TString}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {"tag0-0", "tag2-0", execute.Time(1), 2.0}, @@ -101,10 +102,10 @@ func TestKeys_Process(t *testing.T) { want: []*executetest.Block{ { KeyCols: []string{"tag0", "tag1"}, - ColMeta: []execute.ColMeta{ - {Label: "tag0", Type: execute.TString}, - {Label: "tag1", Type: execute.TString}, - {Label: "_value", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "tag0", Type: query.TString}, + {Label: "tag1", Type: query.TString}, + {Label: "_value", Type: query.TString}, }, Data: [][]interface{}{ {"tag0-0", "tag1-0", "_time"}, @@ -115,10 +116,10 @@ func TestKeys_Process(t *testing.T) { }, { KeyCols: []string{"tag0", "tag2"}, - ColMeta: []execute.ColMeta{ - {Label: "tag0", Type: execute.TString}, - {Label: "tag2", Type: execute.TString}, - {Label: "_value", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "tag0", Type: query.TString}, + {Label: "tag2", Type: query.TString}, + {Label: "_value", Type: query.TString}, }, Data: [][]interface{}{ {"tag0-0", "tag2-0", "_time"}, diff --git a/functions/last.go b/functions/last.go index 5d9d0df94a..448db36cfc 100644 --- a/functions/last.go +++ b/functions/last.go @@ -153,24 +153,24 @@ func (s *LastSelector) Rows() []execute.Row { return s.rows } -func (s *LastSelector) selectLast(l int, cr execute.ColReader) { +func (s *LastSelector) selectLast(l int, cr query.ColReader) { if l > 0 { s.rows = []execute.Row{execute.ReadRow(l-1, cr)} } } -func (s *LastSelector) DoBool(vs []bool, cr execute.ColReader) { +func (s *LastSelector) DoBool(vs []bool, cr query.ColReader) { s.selectLast(len(vs), cr) } -func (s *LastSelector) DoInt(vs []int64, cr execute.ColReader) { +func (s *LastSelector) DoInt(vs []int64, cr query.ColReader) { s.selectLast(len(vs), cr) } -func (s *LastSelector) DoUInt(vs []uint64, cr execute.ColReader) { +func (s *LastSelector) DoUInt(vs []uint64, cr query.ColReader) { s.selectLast(len(vs), cr) } -func (s *LastSelector) DoFloat(vs []float64, cr execute.ColReader) { +func (s *LastSelector) DoFloat(vs []float64, cr query.ColReader) { s.selectLast(len(vs), cr) } -func (s *LastSelector) DoString(vs []string, cr execute.ColReader) { +func (s *LastSelector) DoString(vs []string, cr query.ColReader) { s.selectLast(len(vs), cr) } diff --git a/functions/last_test.go b/functions/last_test.go index e7565ee5f8..18c8e71619 100644 --- a/functions/last_test.go +++ b/functions/last_test.go @@ -3,10 +3,10 @@ package functions_test import ( "testing" - "github.com/influxdata/platform/query/functions" "github.com/influxdata/platform/query" "github.com/influxdata/platform/query/execute" "github.com/influxdata/platform/query/execute/executetest" + "github.com/influxdata/platform/query/functions" "github.com/influxdata/platform/query/plan" "github.com/influxdata/platform/query/plan/plantest" "github.com/influxdata/platform/query/querytest" @@ -36,11 +36,11 @@ func TestLast_Process(t *testing.T) { name: "last", data: &executetest.Block{ KeyCols: []string{"t1"}, - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, - {Label: "t2", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, + {Label: "t2", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(0), 0.0, "a", "y"}, diff --git a/functions/limit.go b/functions/limit.go index 0c5763281b..7d2b57dcf2 100644 --- a/functions/limit.go +++ b/functions/limit.go @@ -132,11 +132,11 @@ func NewLimitTransformation(d execute.Dataset, cache execute.BlockBuilderCache, } } -func (t *limitTransformation) RetractBlock(id execute.DatasetID, key execute.PartitionKey) error { +func (t *limitTransformation) RetractBlock(id execute.DatasetID, key query.PartitionKey) error { return t.d.RetractBlock(key) } -func (t *limitTransformation) Process(id execute.DatasetID, b execute.Block) error { +func (t *limitTransformation) Process(id execute.DatasetID, b query.Block) error { builder, created := t.cache.BlockBuilder(b.Key()) if !created { return fmt.Errorf("limit found duplicate block with key: %v", b.Key()) @@ -155,7 +155,7 @@ func (t *limitTransformation) Process(id execute.DatasetID, b execute.Block) err // AppendBlock with limit n := t.n - b.Do(func(cr execute.ColReader) error { + b.Do(func(cr query.ColReader) error { if n <= 0 { // Returning an error terminates iteration return errors.New("finished") @@ -176,7 +176,7 @@ func (t *limitTransformation) Process(id execute.DatasetID, b execute.Block) err } type limitColReader struct { - execute.ColReader + query.ColReader n int } diff --git a/functions/limit_test.go b/functions/limit_test.go index 4abb7aa416..ca16b1aa9e 100644 --- a/functions/limit_test.go +++ b/functions/limit_test.go @@ -3,10 +3,10 @@ package functions_test import ( "testing" - "github.com/influxdata/platform/query/functions" "github.com/influxdata/platform/query" "github.com/influxdata/platform/query/execute" "github.com/influxdata/platform/query/execute/executetest" + "github.com/influxdata/platform/query/functions" "github.com/influxdata/platform/query/plan" "github.com/influxdata/platform/query/plan/plantest" "github.com/influxdata/platform/query/querytest" @@ -28,7 +28,7 @@ func TestLimit_Process(t *testing.T) { testCases := []struct { name string spec *functions.LimitProcedureSpec - data []execute.Block + data []query.Block want []*executetest.Block }{ { @@ -36,10 +36,10 @@ func TestLimit_Process(t *testing.T) { spec: &functions.LimitProcedureSpec{ N: 1, }, - data: []execute.Block{&executetest.Block{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + data: []query.Block{&executetest.Block{ + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(1), 2.0}, @@ -47,9 +47,9 @@ func TestLimit_Process(t *testing.T) { }, }}, want: []*executetest.Block{{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(1), 2.0}, @@ -61,13 +61,13 @@ func TestLimit_Process(t *testing.T) { spec: &functions.LimitProcedureSpec{ N: 2, }, - data: []execute.Block{ + data: []query.Block{ &executetest.Block{ KeyCols: []string{"t1"}, - ColMeta: []execute.ColMeta{ - {Label: "t1", Type: execute.TString}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "t1", Type: query.TString}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {"a", execute.Time(1), 3.0}, @@ -77,10 +77,10 @@ func TestLimit_Process(t *testing.T) { }, &executetest.Block{ KeyCols: []string{"t1"}, - ColMeta: []execute.ColMeta{ - {Label: "t1", Type: execute.TString}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "t1", Type: query.TString}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {"b", execute.Time(3), 3.0}, @@ -92,10 +92,10 @@ func TestLimit_Process(t *testing.T) { want: []*executetest.Block{ { KeyCols: []string{"t1"}, - ColMeta: []execute.ColMeta{ - {Label: "t1", Type: execute.TString}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "t1", Type: query.TString}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {"a", execute.Time(1), 3.0}, @@ -104,10 +104,10 @@ func TestLimit_Process(t *testing.T) { }, { KeyCols: []string{"t1"}, - ColMeta: []execute.ColMeta{ - {Label: "t1", Type: execute.TString}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "t1", Type: query.TString}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {"b", execute.Time(3), 3.0}, diff --git a/functions/map.go b/functions/map.go index 68dfdc08a4..1c1898b52c 100644 --- a/functions/map.go +++ b/functions/map.go @@ -5,9 +5,9 @@ import ( "log" "sort" - "github.com/influxdata/platform/query/interpreter" "github.com/influxdata/platform/query" "github.com/influxdata/platform/query/execute" + "github.com/influxdata/platform/query/interpreter" "github.com/influxdata/platform/query/plan" "github.com/influxdata/platform/query/semantic" ) @@ -112,11 +112,11 @@ func NewMapTransformation(d execute.Dataset, cache execute.BlockBuilderCache, sp }, nil } -func (t *mapTransformation) RetractBlock(id execute.DatasetID, key execute.PartitionKey) error { +func (t *mapTransformation) RetractBlock(id execute.DatasetID, key query.PartitionKey) error { return t.d.RetractBlock(key) } -func (t *mapTransformation) Process(id execute.DatasetID, b execute.Block) error { +func (t *mapTransformation) Process(id execute.DatasetID, b query.Block) error { // Prepare the functions for the column types. cols := b.Cols() err := t.fn.Prepare(cols) @@ -125,7 +125,7 @@ func (t *mapTransformation) Process(id execute.DatasetID, b execute.Block) error return err } - return b.Do(func(cr execute.ColReader) error { + return b.Do(func(cr query.ColReader) error { l := cr.Len() for i := 0; i < l; i++ { m, err := t.fn.Eval(i, cr) @@ -144,7 +144,7 @@ func (t *mapTransformation) Process(id execute.DatasetID, b execute.Block) error } sort.Strings(keys) for _, k := range keys { - builder.AddCol(execute.ColMeta{ + builder.AddCol(query.ColMeta{ Label: k, Type: execute.ConvertFromKind(properties[k].Kind()), }) diff --git a/functions/map_test.go b/functions/map_test.go index cccbe26197..efe3d9b9ae 100644 --- a/functions/map_test.go +++ b/functions/map_test.go @@ -3,11 +3,11 @@ package functions_test import ( "testing" - "github.com/influxdata/platform/query/ast" - "github.com/influxdata/platform/query/functions" "github.com/influxdata/platform/query" + "github.com/influxdata/platform/query/ast" "github.com/influxdata/platform/query/execute" "github.com/influxdata/platform/query/execute/executetest" + "github.com/influxdata/platform/query/functions" "github.com/influxdata/platform/query/querytest" "github.com/influxdata/platform/query/semantic" ) @@ -145,7 +145,7 @@ func TestMap_Process(t *testing.T) { testCases := []struct { name string spec *functions.MapProcedureSpec - data []execute.Block + data []query.Block want []*executetest.Block }{ { @@ -183,10 +183,10 @@ func TestMap_Process(t *testing.T) { }, }, }, - data: []execute.Block{&executetest.Block{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + data: []query.Block{&executetest.Block{ + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(1), 1.0}, @@ -194,9 +194,9 @@ func TestMap_Process(t *testing.T) { }, }}, want: []*executetest.Block{{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(1), 6.0}, @@ -242,10 +242,10 @@ func TestMap_Process(t *testing.T) { }, }, }, - data: []execute.Block{&executetest.Block{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + data: []query.Block{&executetest.Block{ + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(1), 1.0}, @@ -253,9 +253,9 @@ func TestMap_Process(t *testing.T) { }, }}, want: []*executetest.Block{{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(1), 1.0}, @@ -300,10 +300,10 @@ func TestMap_Process(t *testing.T) { }, }, }, - data: []execute.Block{&executetest.Block{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TInt}, + data: []query.Block{&executetest.Block{ + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TInt}, }, Data: [][]interface{}{ {execute.Time(1), int64(1)}, @@ -311,9 +311,9 @@ func TestMap_Process(t *testing.T) { }, }}, want: []*executetest.Block{{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(1), 1.0}, @@ -358,10 +358,10 @@ func TestMap_Process(t *testing.T) { }, }, }, - data: []execute.Block{&executetest.Block{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TUInt}, + data: []query.Block{&executetest.Block{ + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TUInt}, }, Data: [][]interface{}{ {execute.Time(1), uint64(1)}, @@ -369,9 +369,9 @@ func TestMap_Process(t *testing.T) { }, }}, want: []*executetest.Block{{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(1), 1.0}, diff --git a/functions/max.go b/functions/max.go index c2fa95f53f..d307c7ff19 100644 --- a/functions/max.go +++ b/functions/max.go @@ -125,14 +125,14 @@ func (s *MaxSelector) Rows() []execute.Row { return s.rows } -func (s *MaxSelector) selectRow(idx int, cr execute.ColReader) { +func (s *MaxSelector) selectRow(idx int, cr query.ColReader) { // Capture row if idx >= 0 { s.rows = []execute.Row{execute.ReadRow(idx, cr)} } } -func (s *MaxIntSelector) DoInt(vs []int64, cr execute.ColReader) { +func (s *MaxIntSelector) DoInt(vs []int64, cr query.ColReader) { maxIdx := -1 for i, v := range vs { if !s.set || v > s.max { @@ -143,7 +143,7 @@ func (s *MaxIntSelector) DoInt(vs []int64, cr execute.ColReader) { } s.selectRow(maxIdx, cr) } -func (s *MaxUIntSelector) DoUInt(vs []uint64, cr execute.ColReader) { +func (s *MaxUIntSelector) DoUInt(vs []uint64, cr query.ColReader) { maxIdx := -1 for i, v := range vs { if !s.set || v > s.max { @@ -154,7 +154,7 @@ func (s *MaxUIntSelector) DoUInt(vs []uint64, cr execute.ColReader) { } s.selectRow(maxIdx, cr) } -func (s *MaxFloatSelector) DoFloat(vs []float64, cr execute.ColReader) { +func (s *MaxFloatSelector) DoFloat(vs []float64, cr query.ColReader) { maxIdx := -1 for i, v := range vs { if !s.set || v > s.max { diff --git a/functions/max_test.go b/functions/max_test.go index 9fdb6e51d4..830abb2b98 100644 --- a/functions/max_test.go +++ b/functions/max_test.go @@ -3,10 +3,10 @@ package functions_test import ( "testing" - "github.com/influxdata/platform/query/functions" "github.com/influxdata/platform/query" "github.com/influxdata/platform/query/execute" "github.com/influxdata/platform/query/execute/executetest" + "github.com/influxdata/platform/query/functions" "github.com/influxdata/platform/query/querytest" ) @@ -34,11 +34,11 @@ func TestMax_Process(t *testing.T) { name: "first", data: &executetest.Block{ KeyCols: []string{"t1"}, - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, - {Label: "t2", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, + {Label: "t2", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(0), 10.0, "a", "y"}, @@ -61,11 +61,11 @@ func TestMax_Process(t *testing.T) { name: "last", data: &executetest.Block{ KeyCols: []string{"t1"}, - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, - {Label: "t2", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, + {Label: "t2", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(0), 7.0, "a", "y"}, @@ -88,11 +88,11 @@ func TestMax_Process(t *testing.T) { name: "middle", data: &executetest.Block{ KeyCols: []string{"t1"}, - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, - {Label: "t2", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, + {Label: "t2", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(0), 7.0, "a", "y"}, diff --git a/functions/mean.go b/functions/mean.go index 4b881ae697..282f44d8ed 100644 --- a/functions/mean.go +++ b/functions/mean.go @@ -120,8 +120,8 @@ func (a *MeanAgg) DoFloat(vs []float64) { a.sum += v } } -func (a *MeanAgg) Type() execute.DataType { - return execute.TFloat +func (a *MeanAgg) Type() query.DataType { + return query.TFloat } func (a *MeanAgg) ValueFloat() float64 { if a.count < 1 { diff --git a/functions/min.go b/functions/min.go index 723513db45..86f40d59ad 100644 --- a/functions/min.go +++ b/functions/min.go @@ -125,14 +125,14 @@ func (s *MinSelector) Rows() []execute.Row { return s.rows } -func (s *MinSelector) selectRow(idx int, cr execute.ColReader) { +func (s *MinSelector) selectRow(idx int, cr query.ColReader) { // Capture row if idx >= 0 { s.rows = []execute.Row{execute.ReadRow(idx, cr)} } } -func (s *MinIntSelector) DoInt(vs []int64, cr execute.ColReader) { +func (s *MinIntSelector) DoInt(vs []int64, cr query.ColReader) { minIdx := -1 for i, v := range vs { if !s.set || v < s.min { @@ -143,7 +143,7 @@ func (s *MinIntSelector) DoInt(vs []int64, cr execute.ColReader) { } s.selectRow(minIdx, cr) } -func (s *MinUIntSelector) DoUInt(vs []uint64, cr execute.ColReader) { +func (s *MinUIntSelector) DoUInt(vs []uint64, cr query.ColReader) { minIdx := -1 for i, v := range vs { if !s.set || v < s.min { @@ -154,7 +154,7 @@ func (s *MinUIntSelector) DoUInt(vs []uint64, cr execute.ColReader) { } s.selectRow(minIdx, cr) } -func (s *MinFloatSelector) DoFloat(vs []float64, cr execute.ColReader) { +func (s *MinFloatSelector) DoFloat(vs []float64, cr query.ColReader) { minIdx := -1 for i, v := range vs { if !s.set || v < s.min { diff --git a/functions/min_test.go b/functions/min_test.go index 3ea35926be..f02906a9d8 100644 --- a/functions/min_test.go +++ b/functions/min_test.go @@ -3,10 +3,10 @@ package functions_test import ( "testing" - "github.com/influxdata/platform/query/functions" "github.com/influxdata/platform/query" "github.com/influxdata/platform/query/execute" "github.com/influxdata/platform/query/execute/executetest" + "github.com/influxdata/platform/query/functions" "github.com/influxdata/platform/query/querytest" ) @@ -34,11 +34,11 @@ func TestMin_Process(t *testing.T) { name: "first", data: &executetest.Block{ KeyCols: []string{"t1"}, - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, - {Label: "t2", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, + {Label: "t2", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(0), 0.0, "a", "y"}, @@ -61,11 +61,11 @@ func TestMin_Process(t *testing.T) { name: "last", data: &executetest.Block{ KeyCols: []string{"t1"}, - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, - {Label: "t2", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, + {Label: "t2", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(0), 7.0, "a", "y"}, @@ -88,11 +88,11 @@ func TestMin_Process(t *testing.T) { name: "middle", data: &executetest.Block{ KeyCols: []string{"t1"}, - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, - {Label: "t2", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, + {Label: "t2", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(0), 7.0, "a", "y"}, diff --git a/functions/percentile.go b/functions/percentile.go index fc680dc7f8..401ba78edb 100644 --- a/functions/percentile.go +++ b/functions/percentile.go @@ -195,8 +195,8 @@ func (a *PercentileAgg) DoFloat(vs []float64) { } } -func (a *PercentileAgg) Type() execute.DataType { - return execute.TFloat +func (a *PercentileAgg) Type() query.DataType { + return query.TFloat } func (a *PercentileAgg) ValueFloat() float64 { return a.digest.Quantile(a.Quantile) @@ -250,8 +250,8 @@ func (a *ExactPercentileAgg) DoFloat(vs []float64) { a.data = append(a.data, vs...) } -func (a *ExactPercentileAgg) Type() execute.DataType { - return execute.TFloat +func (a *ExactPercentileAgg) Type() query.DataType { + return query.TFloat } func (a *ExactPercentileAgg) ValueFloat() float64 { diff --git a/functions/range.go b/functions/range.go index 4b4f76f617..ef91dc32a2 100644 --- a/functions/range.go +++ b/functions/range.go @@ -145,11 +145,11 @@ func NewRangeTransformation(d execute.Dataset, cache execute.BlockBuilderCache, }, nil } -func (t *rangeTransformation) RetractBlock(id execute.DatasetID, key execute.PartitionKey) error { +func (t *rangeTransformation) RetractBlock(id execute.DatasetID, key query.PartitionKey) error { return t.d.RetractBlock(key) } -func (t *rangeTransformation) Process(id execute.DatasetID, b execute.Block) error { +func (t *rangeTransformation) Process(id execute.DatasetID, b query.Block) error { builder, created := t.cache.BlockBuilder(b.Key()) if !created { return fmt.Errorf("range found duplicate block with key: %v", b.Key()) diff --git a/functions/sample_test.go b/functions/sample_test.go index 9169e73d17..acc3ae094e 100644 --- a/functions/sample_test.go +++ b/functions/sample_test.go @@ -3,10 +3,10 @@ package functions_test import ( "testing" - "github.com/influxdata/platform/query/functions" "github.com/influxdata/platform/query" "github.com/influxdata/platform/query/execute" "github.com/influxdata/platform/query/execute/executetest" + "github.com/influxdata/platform/query/functions" "github.com/influxdata/platform/query/querytest" ) @@ -26,7 +26,7 @@ func TestSampleOperation_Marshaling(t *testing.T) { func TestSample_Process(t *testing.T) { testCases := []struct { name string - data execute.Block + data query.Block want [][]int fromor *functions.SampleSelector }{ @@ -38,11 +38,11 @@ func TestSample_Process(t *testing.T) { name: "everything in separate Do calls", data: &executetest.Block{ KeyCols: []string{"t1"}, - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, - {Label: "t2", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, + {Label: "t2", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(0), 7.0, "a", "y"}, @@ -78,11 +78,11 @@ func TestSample_Process(t *testing.T) { name: "everything in single Do call", data: execute.CopyBlock(&executetest.Block{ KeyCols: []string{"t1"}, - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, - {Label: "t2", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, + {Label: "t2", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(0), 7.0, "a", "y"}, @@ -118,11 +118,11 @@ func TestSample_Process(t *testing.T) { name: "every-other-even", data: execute.CopyBlock(&executetest.Block{ KeyCols: []string{"t1"}, - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, - {Label: "t2", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, + {Label: "t2", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(0), 7.0, "a", "y"}, @@ -153,11 +153,11 @@ func TestSample_Process(t *testing.T) { name: "every-other-odd", data: execute.CopyBlock(&executetest.Block{ KeyCols: []string{"t1"}, - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, - {Label: "t2", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, + {Label: "t2", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(0), 7.0, "a", "y"}, @@ -188,11 +188,11 @@ func TestSample_Process(t *testing.T) { name: "every-third-0", data: execute.CopyBlock(&executetest.Block{ KeyCols: []string{"t1"}, - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, - {Label: "t2", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, + {Label: "t2", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(0), 7.0, "a", "y"}, @@ -222,11 +222,11 @@ func TestSample_Process(t *testing.T) { name: "every-third-1", data: execute.CopyBlock(&executetest.Block{ KeyCols: []string{"t1"}, - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, - {Label: "t2", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, + {Label: "t2", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(0), 7.0, "a", "y"}, @@ -255,11 +255,11 @@ func TestSample_Process(t *testing.T) { name: "every-third-2", data: execute.CopyBlock(&executetest.Block{ KeyCols: []string{"t1"}, - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, - {Label: "t2", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, + {Label: "t2", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(0), 7.0, "a", "y"}, @@ -288,11 +288,11 @@ func TestSample_Process(t *testing.T) { name: "every-third-2 in separate Do calls", data: &executetest.Block{ KeyCols: []string{"t1"}, - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, - {Label: "t2", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, + {Label: "t2", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(0), 7.0, "a", "y"}, diff --git a/functions/set.go b/functions/set.go index 1914138d1a..525ca24825 100644 --- a/functions/set.go +++ b/functions/set.go @@ -114,16 +114,16 @@ func NewSetTransformation( } } -func (t *setTransformation) RetractBlock(id execute.DatasetID, key execute.PartitionKey) error { +func (t *setTransformation) RetractBlock(id execute.DatasetID, key query.PartitionKey) error { // TODO return nil } -func (t *setTransformation) Process(id execute.DatasetID, b execute.Block) error { +func (t *setTransformation) Process(id execute.DatasetID, b query.Block) error { key := b.Key() if idx := execute.ColIdx(t.key, key.Cols()); idx >= 0 { // Update key - cols := make([]execute.ColMeta, len(key.Cols())) + cols := make([]query.ColMeta, len(key.Cols())) values := make([]interface{}, len(key.Cols())) for j, c := range key.Cols() { cols[j] = c @@ -139,14 +139,14 @@ func (t *setTransformation) Process(id execute.DatasetID, b execute.Block) error if created { execute.AddBlockCols(b, builder) if !execute.HasCol(t.key, builder.Cols()) { - builder.AddCol(execute.ColMeta{ + builder.AddCol(query.ColMeta{ Label: t.key, - Type: execute.TString, + Type: query.TString, }) } } idx := execute.ColIdx(t.key, builder.Cols()) - return b.Do(func(cr execute.ColReader) error { + return b.Do(func(cr query.ColReader) error { for j := range cr.Cols() { if j == idx { continue diff --git a/functions/set_test.go b/functions/set_test.go index f26b68913b..3000abb812 100644 --- a/functions/set_test.go +++ b/functions/set_test.go @@ -3,10 +3,10 @@ package functions_test import ( "testing" - "github.com/influxdata/platform/query/functions" "github.com/influxdata/platform/query" "github.com/influxdata/platform/query/execute" "github.com/influxdata/platform/query/execute/executetest" + "github.com/influxdata/platform/query/functions" "github.com/influxdata/platform/query/querytest" ) @@ -27,7 +27,7 @@ func TestSet_Process(t *testing.T) { testCases := []struct { name string spec *functions.SetProcedureSpec - data []execute.Block + data []query.Block want []*executetest.Block }{ { @@ -36,10 +36,10 @@ func TestSet_Process(t *testing.T) { Key: "t1", Value: "bob", }, - data: []execute.Block{&executetest.Block{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + data: []query.Block{&executetest.Block{ + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(1), 2.0}, @@ -47,10 +47,10 @@ func TestSet_Process(t *testing.T) { }, }}, want: []*executetest.Block{{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(1), 2.0, "bob"}, @@ -64,11 +64,11 @@ func TestSet_Process(t *testing.T) { Key: "t1", Value: "bob", }, - data: []execute.Block{&executetest.Block{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, + data: []query.Block{&executetest.Block{ + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(1), 1.0, "jim"}, @@ -76,10 +76,10 @@ func TestSet_Process(t *testing.T) { }, }}, want: []*executetest.Block{{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(1), 1.0, "bob"}, @@ -93,13 +93,13 @@ func TestSet_Process(t *testing.T) { Key: "t1", Value: "bob", }, - data: []execute.Block{&executetest.Block{ + data: []query.Block{&executetest.Block{ KeyCols: []string{"t1"}, - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, - {Label: "t2", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, + {Label: "t2", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(1), 1.0, "alice", "a"}, @@ -108,11 +108,11 @@ func TestSet_Process(t *testing.T) { }}, want: []*executetest.Block{{ KeyCols: []string{"t1"}, - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, - {Label: "t2", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, + {Label: "t2", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(1), 1.0, "bob", "a"}, @@ -126,13 +126,13 @@ func TestSet_Process(t *testing.T) { Key: "t1", Value: "bob", }, - data: []execute.Block{ + data: []query.Block{ &executetest.Block{ KeyCols: []string{"t1"}, - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(1), 1.0, "alice"}, @@ -141,10 +141,10 @@ func TestSet_Process(t *testing.T) { }, &executetest.Block{ KeyCols: []string{"t1"}, - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(3), 3.0, "sue"}, @@ -154,10 +154,10 @@ func TestSet_Process(t *testing.T) { }, want: []*executetest.Block{{ KeyCols: []string{"t1"}, - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(1), 1.0, "bob"}, @@ -173,13 +173,13 @@ func TestSet_Process(t *testing.T) { Key: "t2", Value: "bob", }, - data: []execute.Block{ + data: []query.Block{ &executetest.Block{ KeyCols: []string{"t1"}, - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(1), 1.0, "alice"}, @@ -188,10 +188,10 @@ func TestSet_Process(t *testing.T) { }, &executetest.Block{ KeyCols: []string{"t1"}, - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(3), 3.0, "sue"}, @@ -202,11 +202,11 @@ func TestSet_Process(t *testing.T) { want: []*executetest.Block{ { KeyCols: []string{"t1"}, - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, - {Label: "t2", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, + {Label: "t2", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(1), 1.0, "alice", "bob"}, @@ -215,11 +215,11 @@ func TestSet_Process(t *testing.T) { }, { KeyCols: []string{"t1"}, - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "t1", Type: execute.TString}, - {Label: "t2", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "t1", Type: query.TString}, + {Label: "t2", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(3), 3.0, "sue", "bob"}, diff --git a/functions/shift.go b/functions/shift.go index efb2861a99..0d25493aeb 100644 --- a/functions/shift.go +++ b/functions/shift.go @@ -3,9 +3,9 @@ package functions import ( "fmt" - "github.com/influxdata/platform/query/interpreter" "github.com/influxdata/platform/query" "github.com/influxdata/platform/query/execute" + "github.com/influxdata/platform/query/interpreter" "github.com/influxdata/platform/query/plan" "github.com/influxdata/platform/query/semantic" ) @@ -126,18 +126,18 @@ func NewShiftTransformation(d execute.Dataset, cache execute.BlockBuilderCache, } } -func (t *shiftTransformation) RetractBlock(id execute.DatasetID, key execute.PartitionKey) error { +func (t *shiftTransformation) RetractBlock(id execute.DatasetID, key query.PartitionKey) error { return t.d.RetractBlock(key) } -func (t *shiftTransformation) Process(id execute.DatasetID, b execute.Block) error { +func (t *shiftTransformation) Process(id execute.DatasetID, b query.Block) error { key := b.Key() // Update key - cols := make([]execute.ColMeta, len(key.Cols())) + cols := make([]query.ColMeta, len(key.Cols())) values := make([]interface{}, len(key.Cols())) for j, c := range key.Cols() { if execute.ContainsStr(t.columns, c.Label) { - if c.Type != execute.TTime { + if c.Type != query.TTime { return fmt.Errorf("column %q is not of type time", c.Label) } cols[j] = c @@ -155,7 +155,7 @@ func (t *shiftTransformation) Process(id execute.DatasetID, b execute.Block) err } execute.AddBlockCols(b, builder) - return b.Do(func(cr execute.ColReader) error { + return b.Do(func(cr query.ColReader) error { for j, c := range cr.Cols() { if execute.ContainsStr(t.columns, c.Label) { l := cr.Len() diff --git a/functions/shift_test.go b/functions/shift_test.go index e5ed3a1036..15fb085914 100644 --- a/functions/shift_test.go +++ b/functions/shift_test.go @@ -4,10 +4,10 @@ import ( "testing" "time" - "github.com/influxdata/platform/query/functions" "github.com/influxdata/platform/query" "github.com/influxdata/platform/query/execute" "github.com/influxdata/platform/query/execute/executetest" + "github.com/influxdata/platform/query/functions" "github.com/influxdata/platform/query/querytest" ) @@ -23,16 +23,16 @@ func TestShiftOperation_Marshaling(t *testing.T) { } func TestShift_Process(t *testing.T) { - cols := []execute.ColMeta{ - {Label: "t1", Type: execute.TString}, - {Label: execute.DefaultTimeColLabel, Type: execute.TTime}, - {Label: execute.DefaultValueColLabel, Type: execute.TFloat}, + cols := []query.ColMeta{ + {Label: "t1", Type: query.TString}, + {Label: execute.DefaultTimeColLabel, Type: query.TTime}, + {Label: execute.DefaultValueColLabel, Type: query.TFloat}, } testCases := []struct { name string spec *functions.ShiftProcedureSpec - data []execute.Block + data []query.Block want []*executetest.Block }{ { @@ -41,7 +41,7 @@ func TestShift_Process(t *testing.T) { Columns: []string{execute.DefaultTimeColLabel}, Shift: query.Duration(1), }, - data: []execute.Block{ + data: []query.Block{ &executetest.Block{ KeyCols: []string{"t1"}, ColMeta: cols, @@ -68,7 +68,7 @@ func TestShift_Process(t *testing.T) { Columns: []string{execute.DefaultTimeColLabel}, Shift: query.Duration(2), }, - data: []execute.Block{ + data: []query.Block{ &executetest.Block{ KeyCols: []string{"t1"}, ColMeta: cols, diff --git a/functions/skew.go b/functions/skew.go index 906a59dfff..ba282859e9 100644 --- a/functions/skew.go +++ b/functions/skew.go @@ -147,8 +147,8 @@ func (a *SkewAgg) DoFloat(vs []float64) { a.m1 += deltaN } } -func (a *SkewAgg) Type() execute.DataType { - return execute.TFloat +func (a *SkewAgg) Type() query.DataType { + return query.TFloat } func (a *SkewAgg) ValueFloat() float64 { if a.n < 2 { diff --git a/functions/sort.go b/functions/sort.go index 2eca0b6e6b..27e83c2b0a 100644 --- a/functions/sort.go +++ b/functions/sort.go @@ -3,9 +3,9 @@ package functions import ( "fmt" - "github.com/influxdata/platform/query/interpreter" "github.com/influxdata/platform/query" "github.com/influxdata/platform/query/execute" + "github.com/influxdata/platform/query/interpreter" "github.com/influxdata/platform/query/plan" "github.com/influxdata/platform/query/semantic" ) @@ -124,11 +124,11 @@ func NewSortTransformation(d execute.Dataset, cache execute.BlockBuilderCache, s } } -func (t *sortTransformation) RetractBlock(id execute.DatasetID, key execute.PartitionKey) error { +func (t *sortTransformation) RetractBlock(id execute.DatasetID, key query.PartitionKey) error { return t.d.RetractBlock(key) } -func (t *sortTransformation) Process(id execute.DatasetID, b execute.Block) error { +func (t *sortTransformation) Process(id execute.DatasetID, b query.Block) error { builder, created := t.cache.BlockBuilder(b.Key()) if !created { return fmt.Errorf("sort found duplicate block with key: %v", b.Key()) diff --git a/functions/sort_test.go b/functions/sort_test.go index 62c0eebcdd..9200f57c10 100644 --- a/functions/sort_test.go +++ b/functions/sort_test.go @@ -3,10 +3,10 @@ package functions_test import ( "testing" - "github.com/influxdata/platform/query/functions" "github.com/influxdata/platform/query" "github.com/influxdata/platform/query/execute" "github.com/influxdata/platform/query/execute/executetest" + "github.com/influxdata/platform/query/functions" "github.com/influxdata/platform/query/querytest" ) @@ -40,7 +40,7 @@ func TestSort_Process(t *testing.T) { testCases := []struct { name string spec *functions.SortProcedureSpec - data []execute.Block + data []query.Block want []*executetest.Block }{ { @@ -49,10 +49,10 @@ func TestSort_Process(t *testing.T) { Cols: []string{"_value"}, Desc: false, }, - data: []execute.Block{&executetest.Block{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + data: []query.Block{&executetest.Block{ + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(1), 2.0}, @@ -60,9 +60,9 @@ func TestSort_Process(t *testing.T) { }, }}, want: []*executetest.Block{{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(2), 1.0}, @@ -76,10 +76,10 @@ func TestSort_Process(t *testing.T) { Cols: []string{"_value"}, Desc: true, }, - data: []execute.Block{&executetest.Block{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + data: []query.Block{&executetest.Block{ + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(1), 1.0}, @@ -87,9 +87,9 @@ func TestSort_Process(t *testing.T) { }, }}, want: []*executetest.Block{{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(2), 2.0}, @@ -103,10 +103,10 @@ func TestSort_Process(t *testing.T) { Cols: []string{"_value", "time"}, Desc: false, }, - data: []execute.Block{&executetest.Block{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + data: []query.Block{&executetest.Block{ + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(2), 2.0}, @@ -115,9 +115,9 @@ func TestSort_Process(t *testing.T) { }, }}, want: []*executetest.Block{{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(1), 1.0}, @@ -132,10 +132,10 @@ func TestSort_Process(t *testing.T) { Cols: []string{"_value", "time"}, Desc: true, }, - data: []execute.Block{&executetest.Block{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + data: []query.Block{&executetest.Block{ + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(1), 1.0}, @@ -144,9 +144,9 @@ func TestSort_Process(t *testing.T) { }, }}, want: []*executetest.Block{{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(2), 2.0}, @@ -161,13 +161,13 @@ func TestSort_Process(t *testing.T) { Cols: []string{"_value"}, Desc: false, }, - data: []execute.Block{ + data: []query.Block{ &executetest.Block{ KeyCols: []string{"t1"}, - ColMeta: []execute.ColMeta{ - {Label: "t1", Type: execute.TString}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "t1", Type: query.TString}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {"a", execute.Time(1), 3.0}, @@ -177,10 +177,10 @@ func TestSort_Process(t *testing.T) { }, &executetest.Block{ KeyCols: []string{"t1"}, - ColMeta: []execute.ColMeta{ - {Label: "t1", Type: execute.TString}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "t1", Type: query.TString}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {"b", execute.Time(3), 3.0}, @@ -192,10 +192,10 @@ func TestSort_Process(t *testing.T) { want: []*executetest.Block{ { KeyCols: []string{"t1"}, - ColMeta: []execute.ColMeta{ - {Label: "t1", Type: execute.TString}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "t1", Type: query.TString}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {"a", execute.Time(2), 1.0}, @@ -205,10 +205,10 @@ func TestSort_Process(t *testing.T) { }, { KeyCols: []string{"t1"}, - ColMeta: []execute.ColMeta{ - {Label: "t1", Type: execute.TString}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "t1", Type: query.TString}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {"b", execute.Time(4), 1.0}, @@ -224,14 +224,14 @@ func TestSort_Process(t *testing.T) { Cols: []string{"_field", "_value"}, Desc: false, }, - data: []execute.Block{ + data: []query.Block{ &executetest.Block{ KeyCols: []string{"host"}, - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "host", Type: execute.TString}, - {Label: "_field", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "host", Type: query.TString}, + {Label: "_field", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(1), 1.0, "hostA", "F1"}, @@ -244,11 +244,11 @@ func TestSort_Process(t *testing.T) { }, &executetest.Block{ KeyCols: []string{"host"}, - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "host", Type: execute.TString}, - {Label: "_field", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "host", Type: query.TString}, + {Label: "_field", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(1), 1.0, "hostB", "F1"}, @@ -263,11 +263,11 @@ func TestSort_Process(t *testing.T) { want: []*executetest.Block{ { KeyCols: []string{"host"}, - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "host", Type: execute.TString}, - {Label: "_field", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "host", Type: query.TString}, + {Label: "_field", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(1), 1.0, "hostA", "F1"}, @@ -280,11 +280,11 @@ func TestSort_Process(t *testing.T) { }, { KeyCols: []string{"host"}, - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "host", Type: execute.TString}, - {Label: "_field", Type: execute.TString}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "host", Type: query.TString}, + {Label: "_field", Type: query.TString}, }, Data: [][]interface{}{ {execute.Time(1), 1.0, "hostB", "F1"}, diff --git a/functions/spread.go b/functions/spread.go index 8da9016dfe..7785c94a8f 100644 --- a/functions/spread.go +++ b/functions/spread.go @@ -137,8 +137,8 @@ func (a *SpreadIntAgg) DoInt(vs []int64) { } } -func (a *SpreadIntAgg) Type() execute.DataType { - return execute.TInt +func (a *SpreadIntAgg) Type() query.DataType { + return query.TInt } // Value returns the difference between max and min @@ -160,8 +160,8 @@ func (a *SpreadUIntAgg) DoUInt(vs []uint64) { } } -func (a *SpreadUIntAgg) Type() execute.DataType { - return execute.TUInt +func (a *SpreadUIntAgg) Type() query.DataType { + return query.TUInt } // Value returns the difference between max and min @@ -183,8 +183,8 @@ func (a *SpreadFloatAgg) DoFloat(vs []float64) { } } -func (a *SpreadFloatAgg) Type() execute.DataType { - return execute.TFloat +func (a *SpreadFloatAgg) Type() query.DataType { + return query.TFloat } // Value returns the difference between max and min diff --git a/functions/state_tracking.go b/functions/state_tracking.go index d877fbda8e..162ec23cc3 100644 --- a/functions/state_tracking.go +++ b/functions/state_tracking.go @@ -5,9 +5,9 @@ import ( "log" "time" - "github.com/influxdata/platform/query/interpreter" "github.com/influxdata/platform/query" "github.com/influxdata/platform/query/execute" + "github.com/influxdata/platform/query/interpreter" "github.com/influxdata/platform/query/plan" "github.com/influxdata/platform/query/semantic" "github.com/pkg/errors" @@ -205,11 +205,11 @@ func NewStateTrackingTransformation(d execute.Dataset, cache execute.BlockBuilde }, nil } -func (t *stateTrackingTransformation) RetractBlock(id execute.DatasetID, key execute.PartitionKey) error { +func (t *stateTrackingTransformation) RetractBlock(id execute.DatasetID, key query.PartitionKey) error { return t.d.RetractBlock(key) } -func (t *stateTrackingTransformation) Process(id execute.DatasetID, b execute.Block) error { +func (t *stateTrackingTransformation) Process(id execute.DatasetID, b query.Block) error { builder, created := t.cache.BlockBuilder(b.Key()) if !created { return fmt.Errorf("found duplicate block with key: %v", b.Key()) @@ -228,15 +228,15 @@ func (t *stateTrackingTransformation) Process(id execute.DatasetID, b execute.Bl // Add new value columns if t.countLabel != "" { - countCol = builder.AddCol(execute.ColMeta{ + countCol = builder.AddCol(query.ColMeta{ Label: t.countLabel, - Type: execute.TInt, + Type: query.TInt, }) } if t.durationLabel != "" { - durationCol = builder.AddCol(execute.ColMeta{ + durationCol = builder.AddCol(query.ColMeta{ Label: t.durationLabel, - Type: execute.TInt, + Type: query.TInt, }) } @@ -252,7 +252,7 @@ func (t *stateTrackingTransformation) Process(id execute.DatasetID, b execute.Bl return fmt.Errorf("no column %q exists", t.timeCol) } // Append modified rows - return b.Do(func(cr execute.ColReader) error { + return b.Do(func(cr query.ColReader) error { l := cr.Len() for i := 0; i < l; i++ { tm := cr.Times(timeIdx)[i] diff --git a/functions/state_tracking_test.go b/functions/state_tracking_test.go index f82141af66..eeb182eee9 100644 --- a/functions/state_tracking_test.go +++ b/functions/state_tracking_test.go @@ -4,11 +4,11 @@ import ( "testing" "time" - "github.com/influxdata/platform/query/ast" - "github.com/influxdata/platform/query/functions" "github.com/influxdata/platform/query" + "github.com/influxdata/platform/query/ast" "github.com/influxdata/platform/query/execute" "github.com/influxdata/platform/query/execute/executetest" + "github.com/influxdata/platform/query/functions" "github.com/influxdata/platform/query/querytest" "github.com/influxdata/platform/query/semantic" ) @@ -41,7 +41,7 @@ func TestStateTracking_Process(t *testing.T) { testCases := []struct { name string spec *functions.StateTrackingProcedureSpec - data []execute.Block + data []query.Block want []*executetest.Block }{ { @@ -53,10 +53,10 @@ func TestStateTracking_Process(t *testing.T) { Fn: gt5, TimeCol: "_time", }, - data: []execute.Block{&executetest.Block{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + data: []query.Block{&executetest.Block{ + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(1), 2.0}, @@ -68,11 +68,11 @@ func TestStateTracking_Process(t *testing.T) { }, }}, want: []*executetest.Block{{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "count", Type: execute.TInt}, - {Label: "duration", Type: execute.TInt}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "count", Type: query.TInt}, + {Label: "duration", Type: query.TInt}, }, Data: [][]interface{}{ {execute.Time(1), 2.0, int64(-1), int64(-1)}, @@ -92,10 +92,10 @@ func TestStateTracking_Process(t *testing.T) { Fn: gt5, TimeCol: "_time", }, - data: []execute.Block{&executetest.Block{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + data: []query.Block{&executetest.Block{ + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(1), 2.0}, @@ -107,10 +107,10 @@ func TestStateTracking_Process(t *testing.T) { }, }}, want: []*executetest.Block{{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "duration", Type: execute.TInt}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "duration", Type: query.TInt}, }, Data: [][]interface{}{ {execute.Time(1), 2.0, int64(-1)}, @@ -129,10 +129,10 @@ func TestStateTracking_Process(t *testing.T) { Fn: gt5, TimeCol: "_time", }, - data: []execute.Block{&executetest.Block{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + data: []query.Block{&executetest.Block{ + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(1), 2.0}, @@ -144,10 +144,10 @@ func TestStateTracking_Process(t *testing.T) { }, }}, want: []*executetest.Block{{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, - {Label: "count", Type: execute.TInt}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, + {Label: "count", Type: query.TInt}, }, Data: [][]interface{}{ {execute.Time(1), 2.0, int64(-1)}, diff --git a/functions/stddev.go b/functions/stddev.go index f03eed86a3..ec996a04e2 100644 --- a/functions/stddev.go +++ b/functions/stddev.go @@ -129,8 +129,8 @@ func (a *StddevAgg) DoFloat(vs []float64) { a.m2 += delta * delta2 } } -func (a *StddevAgg) Type() execute.DataType { - return execute.TFloat +func (a *StddevAgg) Type() query.DataType { + return query.TFloat } func (a *StddevAgg) ValueFloat() float64 { if a.n < 2 { diff --git a/functions/storage/pb/reader.go b/functions/storage/pb/reader.go index ee9ade5141..c5c4aed177 100644 --- a/functions/storage/pb/reader.go +++ b/functions/storage/pb/reader.go @@ -6,8 +6,9 @@ import ( "io" "strings" - "github.com/influxdata/platform/query/functions/storage" + "github.com/influxdata/platform/query" "github.com/influxdata/platform/query/execute" + "github.com/influxdata/platform/query/functions/storage" "github.com/influxdata/yarpc" "github.com/pkg/errors" ) @@ -42,7 +43,7 @@ type connection struct { client StorageClient } -func (sr *reader) Read(ctx context.Context, trace map[string]string, readSpec storage.ReadSpec, start, stop execute.Time) (execute.BlockIterator, error) { +func (sr *reader) Read(ctx context.Context, trace map[string]string, readSpec storage.ReadSpec, start, stop execute.Time) (query.BlockIterator, error) { var predicate *Predicate if readSpec.Predicate != nil { p, err := ToStoragePredicate(readSpec.Predicate) @@ -81,7 +82,7 @@ type bockIterator struct { predicate *Predicate } -func (bi *bockIterator) Do(f func(execute.Block) error) error { +func (bi *bockIterator) Do(f func(query.Block) error) error { // Setup read request var req ReadRequest req.Database = string(bi.readSpec.BucketID) @@ -163,20 +164,20 @@ func determineAggregateMethod(agg string) (Aggregate_AggregateType, error) { return 0, fmt.Errorf("unknown aggregate type %q", agg) } -func convertDataType(t ReadResponse_DataType) execute.DataType { +func convertDataType(t ReadResponse_DataType) query.DataType { switch t { case DataTypeFloat: - return execute.TFloat + return query.TFloat case DataTypeInteger: - return execute.TInt + return query.TInt case DataTypeUnsigned: - return execute.TUInt + return query.TUInt case DataTypeBoolean: - return execute.TBool + return query.TBool case DataTypeString: - return execute.TString + return query.TString default: - return execute.TInvalid + return query.TInvalid } } @@ -187,44 +188,44 @@ const ( valueColIdx = 3 ) -func (bi *bockIterator) determineBlockCols(s *ReadResponse_SeriesFrame, typ execute.DataType) []execute.ColMeta { - cols := make([]execute.ColMeta, 4+len(s.Tags)) - cols[startColIdx] = execute.ColMeta{ +func (bi *bockIterator) determineBlockCols(s *ReadResponse_SeriesFrame, typ query.DataType) []query.ColMeta { + cols := make([]query.ColMeta, 4+len(s.Tags)) + cols[startColIdx] = query.ColMeta{ Label: execute.DefaultStartColLabel, - Type: execute.TTime, + Type: query.TTime, } - cols[stopColIdx] = execute.ColMeta{ + cols[stopColIdx] = query.ColMeta{ Label: execute.DefaultStopColLabel, - Type: execute.TTime, + Type: query.TTime, } - cols[timeColIdx] = execute.ColMeta{ + cols[timeColIdx] = query.ColMeta{ Label: execute.DefaultTimeColLabel, - Type: execute.TTime, + Type: query.TTime, } - cols[valueColIdx] = execute.ColMeta{ + cols[valueColIdx] = query.ColMeta{ Label: execute.DefaultValueColLabel, Type: typ, } for j, tag := range s.Tags { - cols[4+j] = execute.ColMeta{ + cols[4+j] = query.ColMeta{ Label: string(tag.Key), - Type: execute.TString, + Type: query.TString, } } return cols } -func partitionKeyForSeries(s *ReadResponse_SeriesFrame, readSpec *storage.ReadSpec) execute.PartitionKey { - cols := make([]execute.ColMeta, 0, len(s.Tags)) +func partitionKeyForSeries(s *ReadResponse_SeriesFrame, readSpec *storage.ReadSpec) query.PartitionKey { + cols := make([]query.ColMeta, 0, len(s.Tags)) values := make([]interface{}, 0, len(s.Tags)) if len(readSpec.GroupKeys) > 0 { for _, tag := range s.Tags { if !execute.ContainsStr(readSpec.GroupKeys, string(tag.Key)) { continue } - cols = append(cols, execute.ColMeta{ + cols = append(cols, query.ColMeta{ Label: string(tag.Key), - Type: execute.TString, + Type: query.TString, }) values = append(values, string(tag.Value)) } @@ -233,17 +234,17 @@ func partitionKeyForSeries(s *ReadResponse_SeriesFrame, readSpec *storage.ReadSp if !execute.ContainsStr(readSpec.GroupExcept, string(tag.Key)) { continue } - cols = append(cols, execute.ColMeta{ + cols = append(cols, query.ColMeta{ Label: string(tag.Key), - Type: execute.TString, + Type: query.TString, }) values = append(values, string(tag.Value)) } } else if !readSpec.MergeAll { for _, tag := range s.Tags { - cols = append(cols, execute.ColMeta{ + cols = append(cols, query.ColMeta{ Label: string(tag.Key), - Type: execute.TString, + Type: query.TString, }) values = append(values, string(tag.Value)) } @@ -256,8 +257,8 @@ func partitionKeyForSeries(s *ReadResponse_SeriesFrame, readSpec *storage.ReadSp // Since it can only be read once it is also a ValueIterator for itself. type block struct { bounds execute.Bounds - key execute.PartitionKey - cols []execute.ColMeta + key query.PartitionKey + cols []query.ColMeta // cache of the tags on the current series. // len(tags) == len(colMeta) @@ -289,8 +290,8 @@ type block struct { func newBlock( bounds execute.Bounds, - key execute.PartitionKey, - cols []execute.ColMeta, + key query.PartitionKey, + cols []query.ColMeta, ms *mergedStreams, readSpec *storage.ReadSpec, tags []Tag, @@ -320,16 +321,16 @@ func (b *block) wait() { <-b.done } -func (b *block) Key() execute.PartitionKey { +func (b *block) Key() query.PartitionKey { return b.key } -func (b *block) Cols() []execute.ColMeta { +func (b *block) Cols() []query.ColMeta { return b.cols } // onetime satisfies the OneTimeBlock interface since this block may only be read once. func (b *block) onetime() {} -func (b *block) Do(f func(execute.ColReader) error) error { +func (b *block) Do(f func(query.ColReader) error) error { defer close(b.done) for b.advance() { if err := f(b); err != nil { @@ -344,27 +345,27 @@ func (b *block) Len() int { } func (b *block) Bools(j int) []bool { - execute.CheckColType(b.cols[j], execute.TBool) + execute.CheckColType(b.cols[j], query.TBool) return b.colBufs[j].([]bool) } func (b *block) Ints(j int) []int64 { - execute.CheckColType(b.cols[j], execute.TInt) + execute.CheckColType(b.cols[j], query.TInt) return b.colBufs[j].([]int64) } func (b *block) UInts(j int) []uint64 { - execute.CheckColType(b.cols[j], execute.TUInt) + execute.CheckColType(b.cols[j], query.TUInt) return b.colBufs[j].([]uint64) } func (b *block) Floats(j int) []float64 { - execute.CheckColType(b.cols[j], execute.TFloat) + execute.CheckColType(b.cols[j], query.TFloat) return b.colBufs[j].([]float64) } func (b *block) Strings(j int) []string { - execute.CheckColType(b.cols[j], execute.TString) + execute.CheckColType(b.cols[j], query.TString) return b.colBufs[j].([]string) } func (b *block) Times(j int) []execute.Time { - execute.CheckColType(b.cols[j], execute.TTime) + execute.CheckColType(b.cols[j], query.TTime) return b.colBufs[j].([]execute.Time) } @@ -408,8 +409,8 @@ func (b *block) advance() bool { return true } case boolPointsType: - if b.cols[valueColIdx].Type != execute.TBool { - b.err = fmt.Errorf("value type changed from %s -> %s", b.cols[valueColIdx].Type, execute.TBool) + if b.cols[valueColIdx].Type != query.TBool { + b.err = fmt.Errorf("value type changed from %s -> %s", b.cols[valueColIdx].Type, query.TBool) // TODO: Add error handling // Type changed, return false @@ -440,8 +441,8 @@ func (b *block) advance() bool { b.appendBounds() return true case intPointsType: - if b.cols[valueColIdx].Type != execute.TInt { - b.err = fmt.Errorf("value type changed from %s -> %s", b.cols[valueColIdx].Type, execute.TInt) + if b.cols[valueColIdx].Type != query.TInt { + b.err = fmt.Errorf("value type changed from %s -> %s", b.cols[valueColIdx].Type, query.TInt) // TODO: Add error handling // Type changed, return false @@ -472,8 +473,8 @@ func (b *block) advance() bool { b.appendBounds() return true case uintPointsType: - if b.cols[valueColIdx].Type != execute.TUInt { - b.err = fmt.Errorf("value type changed from %s -> %s", b.cols[valueColIdx].Type, execute.TUInt) + if b.cols[valueColIdx].Type != query.TUInt { + b.err = fmt.Errorf("value type changed from %s -> %s", b.cols[valueColIdx].Type, query.TUInt) // TODO: Add error handling // Type changed, return false @@ -504,8 +505,8 @@ func (b *block) advance() bool { b.appendBounds() return true case floatPointsType: - if b.cols[valueColIdx].Type != execute.TFloat { - b.err = fmt.Errorf("value type changed from %s -> %s", b.cols[valueColIdx].Type, execute.TFloat) + if b.cols[valueColIdx].Type != query.TFloat { + b.err = fmt.Errorf("value type changed from %s -> %s", b.cols[valueColIdx].Type, query.TFloat) // TODO: Add error handling // Type changed, return false @@ -537,8 +538,8 @@ func (b *block) advance() bool { b.appendBounds() return true case stringPointsType: - if b.cols[valueColIdx].Type != execute.TString { - b.err = fmt.Errorf("value type changed from %s -> %s", b.cols[valueColIdx].Type, execute.TString) + if b.cols[valueColIdx].Type != query.TString { + b.err = fmt.Errorf("value type changed from %s -> %s", b.cols[valueColIdx].Type, query.TString) // TODO: Add error handling // Type changed, return false @@ -620,7 +621,7 @@ func (b *block) appendBounds() { type streamState struct { stream Storage_ReadClient rep ReadResponse - currentKey execute.PartitionKey + currentKey query.PartitionKey readSpec *storage.ReadSpec finished bool } @@ -652,7 +653,7 @@ func (s *streamState) more() bool { return true } -func (s *streamState) key() execute.PartitionKey { +func (s *streamState) key() query.PartitionKey { return s.currentKey } @@ -674,11 +675,11 @@ func (s *streamState) next() ReadResponse_Frame { type mergedStreams struct { streams []*streamState - currentKey execute.PartitionKey + currentKey query.PartitionKey i int } -func (s *mergedStreams) key() execute.PartitionKey { +func (s *mergedStreams) key() query.PartitionKey { if len(s.streams) == 1 { return s.streams[0].key() } @@ -725,7 +726,7 @@ func (s *mergedStreams) advance() bool { func (s *mergedStreams) determineNewKey() bool { minIdx := -1 - var minKey execute.PartitionKey + var minKey query.PartitionKey for i, stream := range s.streams { if !stream.more() { continue diff --git a/functions/storage/storage.go b/functions/storage/storage.go index e73165e81e..2afac1e396 100644 --- a/functions/storage/storage.go +++ b/functions/storage/storage.go @@ -4,8 +4,9 @@ import ( "context" "log" - "github.com/influxdata/platform/query/id" + "github.com/influxdata/platform/query" "github.com/influxdata/platform/query/execute" + "github.com/influxdata/platform/query/id" "github.com/influxdata/platform/query/semantic" opentracing "github.com/opentracing/opentracing-go" "github.com/pkg/errors" @@ -95,7 +96,7 @@ func (s *source) run(ctx context.Context) error { //TODO(nathanielc): Pass through context to actual network I/O. for blocks, mark, ok := s.next(ctx, trace); ok; blocks, mark, ok = s.next(ctx, trace) { - err := blocks.Do(func(b execute.Block) error { + err := blocks.Do(func(b query.Block) error { for _, t := range s.ts { if err := t.Process(s.id, b); err != nil { return err @@ -120,7 +121,7 @@ func (s *source) run(ctx context.Context) error { return nil } -func (s *source) next(ctx context.Context, trace map[string]string) (execute.BlockIterator, execute.Time, bool) { +func (s *source) next(ctx context.Context, trace map[string]string) (query.BlockIterator, execute.Time, bool) { start := s.currentTime - execute.Time(s.window.Period) stop := s.currentTime @@ -170,6 +171,6 @@ type ReadSpec struct { } type Reader interface { - Read(ctx context.Context, trace map[string]string, rs ReadSpec, start, stop execute.Time) (execute.BlockIterator, error) + Read(ctx context.Context, trace map[string]string, rs ReadSpec, start, stop execute.Time) (query.BlockIterator, error) Close() } diff --git a/functions/sum.go b/functions/sum.go index ce0203cbf4..0f215fd4c8 100644 --- a/functions/sum.go +++ b/functions/sum.go @@ -133,8 +133,8 @@ func (a *SumIntAgg) DoInt(vs []int64) { a.sum += v } } -func (a *SumIntAgg) Type() execute.DataType { - return execute.TInt +func (a *SumIntAgg) Type() query.DataType { + return query.TInt } func (a *SumIntAgg) ValueInt() int64 { return a.sum @@ -149,8 +149,8 @@ func (a *SumUIntAgg) DoUInt(vs []uint64) { a.sum += v } } -func (a *SumUIntAgg) Type() execute.DataType { - return execute.TUInt +func (a *SumUIntAgg) Type() query.DataType { + return query.TUInt } func (a *SumUIntAgg) ValueUInt() uint64 { return a.sum @@ -165,8 +165,8 @@ func (a *SumFloatAgg) DoFloat(vs []float64) { a.sum += v } } -func (a *SumFloatAgg) Type() execute.DataType { - return execute.TFloat +func (a *SumFloatAgg) Type() query.DataType { + return query.TFloat } func (a *SumFloatAgg) ValueFloat() float64 { return a.sum diff --git a/functions/unique.go b/functions/unique.go index f77808e221..1ba68a7b20 100644 --- a/functions/unique.go +++ b/functions/unique.go @@ -104,11 +104,11 @@ func NewUniqueTransformation(d execute.Dataset, cache execute.BlockBuilderCache, } } -func (t *uniqueTransformation) RetractBlock(id execute.DatasetID, key execute.PartitionKey) error { +func (t *uniqueTransformation) RetractBlock(id execute.DatasetID, key query.PartitionKey) error { return t.d.RetractBlock(key) } -func (t *uniqueTransformation) Process(id execute.DatasetID, b execute.Block) error { +func (t *uniqueTransformation) Process(id execute.DatasetID, b query.Block) error { builder, created := t.cache.BlockBuilder(b.Key()) if !created { return fmt.Errorf("unique found duplicate block with key: %v", b.Key()) @@ -130,56 +130,56 @@ func (t *uniqueTransformation) Process(id execute.DatasetID, b execute.Block) er timeUnique map[execute.Time]bool ) switch col.Type { - case execute.TBool: + case query.TBool: boolUnique = make(map[bool]bool) - case execute.TInt: + case query.TInt: intUnique = make(map[int64]bool) - case execute.TUInt: + case query.TUInt: uintUnique = make(map[uint64]bool) - case execute.TFloat: + case query.TFloat: floatUnique = make(map[float64]bool) - case execute.TString: + case query.TString: stringUnique = make(map[string]bool) - case execute.TTime: + case query.TTime: timeUnique = make(map[execute.Time]bool) } - return b.Do(func(cr execute.ColReader) error { + return b.Do(func(cr query.ColReader) error { l := cr.Len() for i := 0; i < l; i++ { // Check unique switch col.Type { - case execute.TBool: + case query.TBool: v := cr.Bools(colIdx)[i] if boolUnique[v] { continue } boolUnique[v] = true - case execute.TInt: + case query.TInt: v := cr.Ints(colIdx)[i] if intUnique[v] { continue } intUnique[v] = true - case execute.TUInt: + case query.TUInt: v := cr.UInts(colIdx)[i] if uintUnique[v] { continue } uintUnique[v] = true - case execute.TFloat: + case query.TFloat: v := cr.Floats(colIdx)[i] if floatUnique[v] { continue } floatUnique[v] = true - case execute.TString: + case query.TString: v := cr.Strings(colIdx)[i] if stringUnique[v] { continue } stringUnique[v] = true - case execute.TTime: + case query.TTime: v := cr.Times(colIdx)[i] if timeUnique[v] { continue diff --git a/functions/unique_test.go b/functions/unique_test.go index d4756f2554..1195f05308 100644 --- a/functions/unique_test.go +++ b/functions/unique_test.go @@ -3,10 +3,10 @@ package functions_test import ( "testing" - "github.com/influxdata/platform/query/functions" "github.com/influxdata/platform/query" "github.com/influxdata/platform/query/execute" "github.com/influxdata/platform/query/execute/executetest" + "github.com/influxdata/platform/query/functions" "github.com/influxdata/platform/query/querytest" ) @@ -38,7 +38,7 @@ func TestUnique_Process(t *testing.T) { testCases := []struct { name string spec *functions.UniqueProcedureSpec - data []execute.Block + data []query.Block want []*executetest.Block }{ { @@ -46,10 +46,10 @@ func TestUnique_Process(t *testing.T) { spec: &functions.UniqueProcedureSpec{ Column: "_value", }, - data: []execute.Block{&executetest.Block{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + data: []query.Block{&executetest.Block{ + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(1), 2.0}, @@ -59,9 +59,9 @@ func TestUnique_Process(t *testing.T) { }, }}, want: []*executetest.Block{{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(1), 2.0}, @@ -75,11 +75,11 @@ func TestUnique_Process(t *testing.T) { spec: &functions.UniqueProcedureSpec{ Column: "t1", }, - data: []execute.Block{&executetest.Block{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "t1", Type: execute.TString}, - {Label: "_value", Type: execute.TFloat}, + data: []query.Block{&executetest.Block{ + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "t1", Type: query.TString}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(1), "a", 2.0}, @@ -89,10 +89,10 @@ func TestUnique_Process(t *testing.T) { }, }}, want: []*executetest.Block{{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "t1", Type: execute.TString}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "t1", Type: query.TString}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(1), "a", 2.0}, @@ -106,11 +106,11 @@ func TestUnique_Process(t *testing.T) { spec: &functions.UniqueProcedureSpec{ Column: "_time", }, - data: []execute.Block{&executetest.Block{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "t1", Type: execute.TString}, - {Label: "_value", Type: execute.TFloat}, + data: []query.Block{&executetest.Block{ + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "t1", Type: query.TString}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(1), "a", 2.0}, @@ -120,10 +120,10 @@ func TestUnique_Process(t *testing.T) { }, }}, want: []*executetest.Block{{ - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "t1", Type: execute.TString}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "t1", Type: query.TString}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(1), "a", 2.0}, diff --git a/functions/window.go b/functions/window.go index 2f5ba8657b..b9bd24c859 100644 --- a/functions/window.go +++ b/functions/window.go @@ -227,7 +227,7 @@ func NewFixedWindowTransformation( } } -func (t *fixedWindowTransformation) RetractBlock(id execute.DatasetID, key execute.PartitionKey) (err error) { +func (t *fixedWindowTransformation) RetractBlock(id execute.DatasetID, key query.PartitionKey) (err error) { panic("not implemented") //tagKey := meta.Tags().Key() //t.cache.ForEachBuilder(func(bk execute.BlockKey, bld execute.BlockBuilder) { @@ -241,11 +241,11 @@ func (t *fixedWindowTransformation) RetractBlock(id execute.DatasetID, key execu //return } -func (t *fixedWindowTransformation) Process(id execute.DatasetID, b execute.Block) error { +func (t *fixedWindowTransformation) Process(id execute.DatasetID, b query.Block) error { timeIdx := execute.ColIdx(t.timeCol, b.Cols()) - newCols := make([]execute.ColMeta, 0, len(b.Cols())+2) - keyCols := make([]execute.ColMeta, 0, len(b.Cols())+2) + newCols := make([]query.ColMeta, 0, len(b.Cols())+2) + keyCols := make([]query.ColMeta, 0, len(b.Cols())+2) keyColMap := make([]int, 0, len(b.Cols())+2) startColIdx := -1 stopColIdx := -1 @@ -267,9 +267,9 @@ func (t *fixedWindowTransformation) Process(id execute.DatasetID, b execute.Bloc } if startColIdx == -1 { startColIdx = len(newCols) - c := execute.ColMeta{ + c := query.ColMeta{ Label: t.startColLabel, - Type: execute.TTime, + Type: query.TTime, } newCols = append(newCols, c) keyCols = append(keyCols, c) @@ -277,23 +277,23 @@ func (t *fixedWindowTransformation) Process(id execute.DatasetID, b execute.Bloc } if stopColIdx == -1 { stopColIdx = len(newCols) - c := execute.ColMeta{ + c := query.ColMeta{ Label: t.stopColLabel, - Type: execute.TTime, + Type: query.TTime, } newCols = append(newCols, c) keyCols = append(keyCols, c) keyColMap = append(keyColMap, stopColIdx) } - return b.Do(func(cr execute.ColReader) error { + return b.Do(func(cr query.ColReader) error { l := cr.Len() for i := 0; i < l; i++ { tm := cr.Times(timeIdx)[i] bounds := t.getWindowBounds(tm) for _, bnds := range bounds { // Update key - cols := make([]execute.ColMeta, len(keyCols)) + cols := make([]query.ColMeta, len(keyCols)) values := make([]interface{}, len(keyCols)) for j, c := range keyCols { cols[j] = c @@ -321,17 +321,17 @@ func (t *fixedWindowTransformation) Process(id execute.DatasetID, b execute.Bloc builder.AppendTime(stopColIdx, bnds.Stop) default: switch c.Type { - case execute.TBool: + case query.TBool: builder.AppendBool(j, cr.Bools(j)[i]) - case execute.TInt: + case query.TInt: builder.AppendInt(j, cr.Ints(j)[i]) - case execute.TUInt: + case query.TUInt: builder.AppendUInt(j, cr.UInts(j)[i]) - case execute.TFloat: + case query.TFloat: builder.AppendFloat(j, cr.Floats(j)[i]) - case execute.TString: + case query.TString: builder.AppendString(j, cr.Strings(j)[i]) - case execute.TTime: + case query.TTime: builder.AppendTime(j, cr.Times(j)[i]) default: execute.PanicUnknownType(c.Type) diff --git a/functions/window_test.go b/functions/window_test.go index 458a474b51..d99db55911 100644 --- a/functions/window_test.go +++ b/functions/window_test.go @@ -7,10 +7,10 @@ import ( "time" "github.com/google/go-cmp/cmp" - "github.com/influxdata/platform/query/functions" "github.com/influxdata/platform/query" "github.com/influxdata/platform/query/execute" "github.com/influxdata/platform/query/execute/executetest" + "github.com/influxdata/platform/query/functions" "github.com/influxdata/platform/query/querytest" ) @@ -97,7 +97,7 @@ func TestFixedWindow_PassThrough(t *testing.T) { func TestFixedWindow_Process(t *testing.T) { testCases := []struct { name string - valueCol execute.ColMeta + valueCol query.ColMeta start execute.Time every, period execute.Duration num int @@ -105,7 +105,7 @@ func TestFixedWindow_Process(t *testing.T) { }{ { name: "nonoverlapping_nonaligned", - valueCol: execute.ColMeta{Label: "_value", Type: execute.TFloat}, + valueCol: query.ColMeta{Label: "_value", Type: query.TFloat}, // Use a time that is *not* aligned with the every/period durations of the window start: execute.Time(time.Date(2017, 10, 10, 10, 10, 10, 10, time.UTC).UnixNano()), every: execute.Duration(time.Minute), @@ -115,11 +115,11 @@ func TestFixedWindow_Process(t *testing.T) { return []*executetest.Block{ { KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {start, start + execute.Time(time.Minute), start, 0.0}, @@ -132,11 +132,11 @@ func TestFixedWindow_Process(t *testing.T) { }, { KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {start + execute.Time(1*time.Minute), start + execute.Time(2*time.Minute), start + execute.Time(60*time.Second), 6.0}, @@ -149,11 +149,11 @@ func TestFixedWindow_Process(t *testing.T) { }, { KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {start + execute.Time(2*time.Minute), start + execute.Time(3*time.Minute), start + execute.Time(120*time.Second), 12.0}, @@ -166,7 +166,7 @@ func TestFixedWindow_Process(t *testing.T) { }, { name: "nonoverlapping_aligned", - valueCol: execute.ColMeta{Label: "_value", Type: execute.TFloat}, + valueCol: query.ColMeta{Label: "_value", Type: query.TFloat}, // Use a time that is aligned with the every/period durations of the window start: execute.Time(time.Date(2017, 10, 10, 10, 0, 0, 0, time.UTC).UnixNano()), every: execute.Duration(time.Minute), @@ -176,11 +176,11 @@ func TestFixedWindow_Process(t *testing.T) { return []*executetest.Block{ { KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {start, start + execute.Time(time.Minute), start, 0.0}, @@ -193,11 +193,11 @@ func TestFixedWindow_Process(t *testing.T) { }, { KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {start + execute.Time(1*time.Minute), start + execute.Time(2*time.Minute), start + execute.Time(60*time.Second), 6.0}, @@ -210,11 +210,11 @@ func TestFixedWindow_Process(t *testing.T) { }, { KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {start + execute.Time(2*time.Minute), start + execute.Time(3*time.Minute), start + execute.Time(120*time.Second), 12.0}, @@ -227,7 +227,7 @@ func TestFixedWindow_Process(t *testing.T) { }, { name: "overlapping_nonaligned", - valueCol: execute.ColMeta{Label: "_value", Type: execute.TFloat}, + valueCol: query.ColMeta{Label: "_value", Type: query.TFloat}, // Use a time that is *not* aligned with the every/period durations of the window start: execute.Time(time.Date(2017, 10, 10, 10, 10, 10, 10, time.UTC).UnixNano()), every: execute.Duration(time.Minute), @@ -237,11 +237,11 @@ func TestFixedWindow_Process(t *testing.T) { return []*executetest.Block{ { KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {start, start + execute.Time(time.Minute), start, 0.0}, @@ -254,11 +254,11 @@ func TestFixedWindow_Process(t *testing.T) { }, { KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {start, start + execute.Time(2*time.Minute), start, 0.0}, @@ -277,11 +277,11 @@ func TestFixedWindow_Process(t *testing.T) { }, { KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {start + execute.Time(1*time.Minute), start + execute.Time(3*time.Minute), start + execute.Time(60*time.Second), 6.0}, @@ -297,11 +297,11 @@ func TestFixedWindow_Process(t *testing.T) { }, { KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {start + execute.Time(2*time.Minute), start + execute.Time(4*time.Minute), start + execute.Time(120*time.Second), 12.0}, @@ -314,7 +314,7 @@ func TestFixedWindow_Process(t *testing.T) { }, { name: "overlapping_aligned", - valueCol: execute.ColMeta{Label: "_value", Type: execute.TFloat}, + valueCol: query.ColMeta{Label: "_value", Type: query.TFloat}, // Use a time that is aligned with the every/period durations of the window start: execute.Time(time.Date(2017, 10, 10, 10, 0, 0, 0, time.UTC).UnixNano()), every: execute.Duration(time.Minute), @@ -324,11 +324,11 @@ func TestFixedWindow_Process(t *testing.T) { return []*executetest.Block{ { KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {start, start + execute.Time(time.Minute), start, 0.0}, @@ -341,11 +341,11 @@ func TestFixedWindow_Process(t *testing.T) { }, { KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {start, start + execute.Time(2*time.Minute), start, 0.0}, @@ -364,11 +364,11 @@ func TestFixedWindow_Process(t *testing.T) { }, { KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {start + execute.Time(1*time.Minute), start + execute.Time(3*time.Minute), start + execute.Time(60*time.Second), 6.0}, @@ -384,11 +384,11 @@ func TestFixedWindow_Process(t *testing.T) { }, { KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {start + execute.Time(2*time.Minute), start + execute.Time(4*time.Minute), start + execute.Time(120*time.Second), 12.0}, @@ -401,7 +401,7 @@ func TestFixedWindow_Process(t *testing.T) { }, { name: "underlapping_nonaligned", - valueCol: execute.ColMeta{Label: "_value", Type: execute.TFloat}, + valueCol: query.ColMeta{Label: "_value", Type: query.TFloat}, // Use a time that is *not* aligned with the every/period durations of the window start: execute.Time(time.Date(2017, 10, 10, 10, 10, 10, 10, time.UTC).UnixNano()), every: execute.Duration(2 * time.Minute), @@ -411,11 +411,11 @@ func TestFixedWindow_Process(t *testing.T) { return []*executetest.Block{ { KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {start + 1*execute.Time(time.Minute), start + 2*execute.Time(time.Minute), start + execute.Time(60*time.Second), 6.0}, @@ -428,11 +428,11 @@ func TestFixedWindow_Process(t *testing.T) { }, { KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {start + execute.Time(3*time.Minute), start + execute.Time(4*time.Minute), start + execute.Time(180*time.Second), 18.0}, @@ -448,7 +448,7 @@ func TestFixedWindow_Process(t *testing.T) { }, { name: "underlapping_aligned", - valueCol: execute.ColMeta{Label: "_value", Type: execute.TFloat}, + valueCol: query.ColMeta{Label: "_value", Type: query.TFloat}, // Use a time that is aligned with the every/period durations of the window start: execute.Time(time.Date(2017, 10, 10, 10, 0, 0, 0, time.UTC).UnixNano()), every: execute.Duration(2 * time.Minute), @@ -458,11 +458,11 @@ func TestFixedWindow_Process(t *testing.T) { return []*executetest.Block{ { KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {start + 1*execute.Time(time.Minute), start + 2*execute.Time(time.Minute), start + execute.Time(60*time.Second), 6.0}, @@ -475,11 +475,11 @@ func TestFixedWindow_Process(t *testing.T) { }, { KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TFloat}, }, Data: [][]interface{}{ {start + execute.Time(3*time.Minute), start + execute.Time(4*time.Minute), start + execute.Time(180*time.Second), 18.0}, @@ -495,7 +495,7 @@ func TestFixedWindow_Process(t *testing.T) { }, { name: "nonoverlapping_aligned_int", - valueCol: execute.ColMeta{Label: "_value", Type: execute.TInt}, + valueCol: query.ColMeta{Label: "_value", Type: query.TInt}, // Use a time that is aligned with the every/period durations of the window start: execute.Time(time.Date(2017, 10, 10, 10, 0, 0, 0, time.UTC).UnixNano()), every: execute.Duration(time.Minute), @@ -505,11 +505,11 @@ func TestFixedWindow_Process(t *testing.T) { return []*executetest.Block{ { KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TInt}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TInt}, }, Data: [][]interface{}{ {start, start + execute.Time(time.Minute), start, int64(0.0)}, @@ -522,11 +522,11 @@ func TestFixedWindow_Process(t *testing.T) { }, { KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TInt}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TInt}, }, Data: [][]interface{}{ {start + execute.Time(1*time.Minute), start + execute.Time(2*time.Minute), start + execute.Time(60*time.Second), int64(6)}, @@ -539,11 +539,11 @@ func TestFixedWindow_Process(t *testing.T) { }, { KeyCols: []string{"_start", "_stop"}, - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, - {Label: "_value", Type: execute.TInt}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, + {Label: "_value", Type: query.TInt}, }, Data: [][]interface{}{ {start + execute.Time(2*time.Minute), start + execute.Time(3*time.Minute), start + execute.Time(120*time.Second), int64(12)}, @@ -584,10 +584,10 @@ func TestFixedWindow_Process(t *testing.T) { ) block0 := &executetest.Block{ - ColMeta: []execute.ColMeta{ - {Label: "_start", Type: execute.TTime}, - {Label: "_stop", Type: execute.TTime}, - {Label: "_time", Type: execute.TTime}, + ColMeta: []query.ColMeta{ + {Label: "_start", Type: query.TTime}, + {Label: "_stop", Type: query.TTime}, + {Label: "_time", Type: query.TTime}, tc.valueCol, }, } @@ -595,15 +595,15 @@ func TestFixedWindow_Process(t *testing.T) { for i := 0; i < tc.num; i++ { var v interface{} switch tc.valueCol.Type { - case execute.TBool: + case query.TBool: v = bool(i%2 == 0) - case execute.TInt: + case query.TInt: v = int64(i) - case execute.TUInt: + case query.TUInt: v = uint64(i) - case execute.TFloat: + case query.TFloat: v = float64(i) - case execute.TString: + case query.TString: v = strconv.Itoa(i) } block0.Data = append(block0.Data, []interface{}{ diff --git a/influxql/influxql_encoder_test.go b/influxql/influxql_encoder_test.go index 09b99157ed..4e4ba5809f 100644 --- a/influxql/influxql_encoder_test.go +++ b/influxql/influxql_encoder_test.go @@ -8,9 +8,9 @@ import ( "strings" "github.com/google/go-cmp/cmp" + "github.com/influxdata/platform/query" "github.com/influxdata/platform/query/execute" "github.com/influxdata/platform/query/execute/executetest" - "github.com/influxdata/platform" ) var epoch = time.Unix(0, 0) @@ -28,12 +28,12 @@ func TestMultiResultEncoder_Encode(t *testing.T) { blocks: map[string][]*executetest.Block{ "0": {{ KeyCols: []string{"_measurement", "_field", "host"}, - ColMeta: []execute.ColMeta{ - {Label: "_time", Type: execute.TTime}, - {Label: "_measurement", Type: execute.TString}, - {Label: "_field", Type: execute.TString}, - {Label: "host", Type: execute.TString}, - {Label: execute.DefaultValueColLabel, Type: execute.TFloat}, + ColMeta: []query.ColMeta{ + {Label: "_time", Type: query.TTime}, + {Label: "_measurement", Type: query.TString}, + {Label: "_field", Type: query.TString}, + {Label: "host", Type: query.TString}, + {Label: execute.DefaultValueColLabel, Type: query.TFloat}, }, Data: [][]interface{}{ {execute.Time(5), "cpu", "max", "localhost", 98.9}, @@ -48,13 +48,13 @@ func TestMultiResultEncoder_Encode(t *testing.T) { tc := tc t.Run(tc.name, func(t *testing.T) { - resultsmap := map[string]execute.Result{} + resultsmap := map[string]query.Result{} for k, v := range tc.blocks { resultsmap[k] = executetest.NewResult(v) } - results := platform.NewMapResultIterator(resultsmap) + results := query.NewMapResultIterator(resultsmap) var resp bytes.Buffer var influxQLEncoder MultiResultEncoder diff --git a/influxql/result.go b/influxql/result.go index 45c436e18c..b82bb2c17f 100644 --- a/influxql/result.go +++ b/influxql/result.go @@ -8,8 +8,8 @@ import ( "strconv" "time" + "github.com/influxdata/platform/query" "github.com/influxdata/platform/query/execute" - "github.com/influxdata/platform" ) // MultiResultEncoder encodes results as InfluxQL JSON format. @@ -24,7 +24,7 @@ type MultiResultEncoder struct{} // 3. If the _measurement name is present in the tags, it will be used as the row.Name for all rows. // Otherwise, we'll use the column value, which _must_ be present in that case. -func (e *MultiResultEncoder) Encode(w io.Writer, results platform.ResultIterator) error { +func (e *MultiResultEncoder) Encode(w io.Writer, results query.ResultIterator) error { resp := Response{} for results.More() { @@ -37,13 +37,13 @@ func (e *MultiResultEncoder) Encode(w io.Writer, results platform.ResultIterator } result := Result{StatementID: nameInt} - err = blocks.Do(func(b execute.Block) error { + err = blocks.Do(func(b query.Block) error { r := NewRow() fieldName := "" measurementVaries := -1 for j, c := range b.Key().Cols() { - if c.Type != execute.TString { + if c.Type != query.TString { return fmt.Errorf("partition column %q is not a string type", c.Label) } v := b.Key().Value(j).(string) @@ -76,10 +76,10 @@ func (e *MultiResultEncoder) Encode(w io.Writer, results platform.ResultIterator if timeIdx < 0 { return errors.New("table must have an _time column") } - if typ := b.Cols()[timeIdx].Type; typ != execute.TTime { + if typ := b.Cols()[timeIdx].Type; typ != query.TTime { return fmt.Errorf("column _time must be of type Time got %v", typ) } - err := b.Do(func(cr execute.ColReader) error { + err := b.Do(func(cr query.ColReader) error { ts := cr.Times(timeIdx) for i := range ts { var v []interface{} @@ -90,7 +90,7 @@ func (e *MultiResultEncoder) Encode(w io.Writer, results platform.ResultIterator } if j == measurementVaries { - if c.Type != execute.TString { + if c.Type != query.TString { return errors.New("unexpected type, _measurement is not a string") } r.Name = cr.Strings(j)[i] @@ -98,17 +98,17 @@ func (e *MultiResultEncoder) Encode(w io.Writer, results platform.ResultIterator } switch c.Type { - case execute.TFloat: + case query.TFloat: v = append(v, cr.Floats(j)[i]) - case execute.TInt: + case query.TInt: v = append(v, cr.Ints(j)[i]) - case execute.TString: + case query.TString: v = append(v, cr.Strings(j)[i]) - case execute.TUInt: + case query.TUInt: v = append(v, cr.UInts(j)[i]) - case execute.TBool: + case query.TBool: v = append(v, cr.Bools(j)[i]) - case execute.TTime: + case query.TTime: v = append(v, cr.Times(j)[i].Time().Format(time.RFC3339)) default: v = append(v, "unknown") diff --git a/query.go b/query.go index c766127457..59d508c52c 100644 --- a/query.go +++ b/query.go @@ -1,175 +1,178 @@ package query import ( - "errors" - "fmt" + "context" + "sort" + + "github.com/influxdata/platform" ) -// Spec specifies a query. -type Spec struct { - Operations []*Operation `json:"operations"` - Edges []Edge `json:"edges"` - Resources ResourceManagement `json:"resources"` +// QueryService represents a service for performing queries. +type QueryService interface { + // Query submits a query spec for execution returning a results iterator. + Query(ctx context.Context, orgID platform.ID, query *Spec) (ResultIterator, error) + // Query submits a query string for execution returning a results iterator. + QueryWithCompile(ctx context.Context, orgID platform.ID, query string) (ResultIterator, error) +} + +// ResultIterator allows iterating through all results +type ResultIterator interface { + // More indicates if there are more results. + // More must be called until it returns false in order to free all resources. + More() bool - sorted []*Operation - children map[OperationID][]*Operation - parents map[OperationID][]*Operation + // Next returns the next name and results. + // If More is false, Next panics. + Next() (string, Result) + + // Cancel discards the remaining results. + // If not all results are going to be read, Cancel must be called to free resources. + Cancel() + + // Err reports the first error encountered. + Err() error } -// Edge is a data flow relationship between a parent and a child -type Edge struct { - Parent OperationID `json:"parent"` - Child OperationID `json:"child"` +// AsyncQueryService represents a service for performing queries where the results are delivered asynchronously. +type AsyncQueryService interface { + // Query submits a query for execution returning immediately. + // The spec must not be modified while the query is still active. + // Done must be called on any returned Query objects. + Query(ctx context.Context, orgID platform.ID, query *Spec) (Query, error) + + // QueryWithCompile submits a query for execution returning immediately. + // The query string will be compiled before submitting for execution. + // Done must be called on returned Query objects. + QueryWithCompile(ctx context.Context, orgID platform.ID, query string) (Query, error) } -// Walk calls f on each operation exactly once. -// The function f will be called on an operation only after -// all of its parents have already been passed to f. -func (q *Spec) Walk(f func(o *Operation) error) error { - if len(q.sorted) == 0 { - if err := q.prepare(); err != nil { - return err - } +// Query represents an active query. +type Query interface { + // Spec returns the spec used to execute this query. + // Spec must not be modified. + Spec() *Spec + + // Ready returns a channel that will deliver the query results. + // Its possible that the channel is closed before any results arrive, + // in which case the query should be inspected for an error using Err(). + Ready() <-chan map[string]Result + + // Done must always be called to free resources. + Done() + + // Cancel will stop the query execution. + // Done must still be called to free resources. + Cancel() + + // Err reports any error the query may have encountered. + Err() error +} + +// QueryServiceBridge implements the QueryService interface while consuming the AsyncQueryService interface. +type QueryServiceBridge struct { + AsyncQueryService AsyncQueryService +} + +func (b QueryServiceBridge) Query(ctx context.Context, orgID platform.ID, spec *Spec) (ResultIterator, error) { + query, err := b.AsyncQueryService.Query(ctx, orgID, spec) + if err != nil { + return nil, err } - for _, o := range q.sorted { - err := f(o) - if err != nil { - return err - } + return newResultIterator(query), nil +} +func (b QueryServiceBridge) QueryWithCompile(ctx context.Context, orgID platform.ID, queryStr string) (ResultIterator, error) { + query, err := b.AsyncQueryService.QueryWithCompile(ctx, orgID, queryStr) + if err != nil { + return nil, err } - return nil + return newResultIterator(query), nil } -// Validate ensures the query is a valid DAG. -func (q *Spec) Validate() error { - return q.prepare() +// resultIterator implements a ResultIterator while consuming a Query +type resultIterator struct { + query Query + cancel chan struct{} + ready bool + results *MapResultIterator } -// Children returns a list of children for a given operation. -// If the query is invalid no children will be returned. -func (q *Spec) Children(id OperationID) []*Operation { - if q.children == nil { - err := q.prepare() - if err != nil { - return nil - } +func newResultIterator(q Query) *resultIterator { + return &resultIterator{ + query: q, + cancel: make(chan struct{}), } - return q.children[id] } -// Parents returns a list of parents for a given operation. -// If the query is invalid no parents will be returned. -func (q *Spec) Parents(id OperationID) []*Operation { - if q.parents == nil { - err := q.prepare() - if err != nil { - return nil +func (r *resultIterator) More() bool { + if !r.ready { + select { + case <-r.cancel: + goto DONE + case results, ok := <-r.query.Ready(): + if !ok { + goto DONE + } + r.ready = true + r.results = NewMapResultIterator(results) } } - return q.parents[id] + if r.results.More() { + return true + } + +DONE: + r.query.Done() + return false } -// prepare populates the internal datastructure needed to quickly navigate the query DAG. -// As a result the query DAG is validated. -func (q *Spec) prepare() error { - q.sorted = q.sorted[0:0] +func (r *resultIterator) Next() (string, Result) { + return r.results.Next() +} - parents, children, roots, err := q.determineParentsChildrenAndRoots() - if err != nil { - return err - } - if len(roots) == 0 { - return errors.New("query has no root nodes") +func (r *resultIterator) Cancel() { + select { + case <-r.cancel: + default: + close(r.cancel) } + r.query.Cancel() +} - q.parents = parents - q.children = children +func (r *resultIterator) Err() error { + return r.query.Err() +} - tMarks := make(map[OperationID]bool) - pMarks := make(map[OperationID]bool) +type MapResultIterator struct { + results map[string]Result + order []string +} - for _, r := range roots { - if err := q.visit(tMarks, pMarks, r); err != nil { - return err - } +func NewMapResultIterator(results map[string]Result) *MapResultIterator { + order := make([]string, 0, len(results)) + for k := range results { + order = append(order, k) } - //reverse q.sorted - for i, j := 0, len(q.sorted)-1; i < j; i, j = i+1, j-1 { - q.sorted[i], q.sorted[j] = q.sorted[j], q.sorted[i] + sort.Strings(order) + return &MapResultIterator{ + results: results, + order: order, } - return nil } -func (q *Spec) computeLookup() (map[OperationID]*Operation, error) { - lookup := make(map[OperationID]*Operation, len(q.Operations)) - for _, o := range q.Operations { - if _, ok := lookup[o.ID]; ok { - return nil, fmt.Errorf("found duplicate operation ID %q", o.ID) - } - lookup[o.ID] = o - } - return lookup, nil +func (r *MapResultIterator) More() bool { + return len(r.order) > 0 } -func (q *Spec) determineParentsChildrenAndRoots() (parents, children map[OperationID][]*Operation, roots []*Operation, _ error) { - lookup, err := q.computeLookup() - if err != nil { - return nil, nil, nil, err - } - children = make(map[OperationID][]*Operation, len(q.Operations)) - parents = make(map[OperationID][]*Operation, len(q.Operations)) - for _, e := range q.Edges { - // Build children map - c, ok := lookup[e.Child] - if !ok { - return nil, nil, nil, fmt.Errorf("edge references unknown child operation %q", e.Child) - } - children[e.Parent] = append(children[e.Parent], c) - - // Build parents map - p, ok := lookup[e.Parent] - if !ok { - return nil, nil, nil, fmt.Errorf("edge references unknown parent operation %q", e.Parent) - } - parents[e.Child] = append(parents[e.Child], p) - } - // Find roots, i.e operations with no parents. - for _, o := range q.Operations { - if len(parents[o.ID]) == 0 { - roots = append(roots, o) - } - } - return +func (r *MapResultIterator) Next() (string, Result) { + next := r.order[0] + r.order = r.order[1:] + return next, r.results[next] } -// Depth first search topological sorting of a DAG. -// https://en.wikipedia.org/wiki/Topological_sorting#Algorithms -func (q *Spec) visit(tMarks, pMarks map[OperationID]bool, o *Operation) error { - id := o.ID - if tMarks[id] { - return errors.New("found cycle in query") - } +func (r *MapResultIterator) Cancel() { - if !pMarks[id] { - tMarks[id] = true - for _, c := range q.children[id] { - if err := q.visit(tMarks, pMarks, c); err != nil { - return err - } - } - pMarks[id] = true - tMarks[id] = false - q.sorted = append(q.sorted, o) - } - return nil } -// Functions return the names of all functions used in the plan -func (q *Spec) Functions() ([]string, error) { - funcs := []string{} - err := q.Walk(func(o *Operation) error { - funcs = append(funcs, string(o.Spec.Kind())) - return nil - }) - return funcs, err +func (r *MapResultIterator) Err() error { + return nil } diff --git a/query_test/query_test.go b/query_test/query_test.go index 1bd491617b..d2645b1131 100644 --- a/query_test/query_test.go +++ b/query_test/query_test.go @@ -17,10 +17,10 @@ import ( "fmt" - "github.com/influxdata/platform/query/functions" - "github.com/influxdata/platform/query/id" "github.com/influxdata/platform/query" "github.com/influxdata/platform/query/control" + "github.com/influxdata/platform/query/functions" + "github.com/influxdata/platform/query/id" "strings" @@ -37,12 +37,12 @@ type wrapController struct { *control.Controller } -func (c wrapController) Query(ctx context.Context, orgID platform.ID, query *query.Spec) (platform.Query, error) { +func (c wrapController) Query(ctx context.Context, orgID platform.ID, query *query.Spec) (query.Query, error) { q, err := c.Controller.Query(ctx, id.ID(orgID), query) return q, err } -func (c wrapController) QueryWithCompile(ctx context.Context, orgID platform.ID, query string) (platform.Query, error) { +func (c wrapController) QueryWithCompile(ctx context.Context, orgID platform.ID, query string) (query.Query, error) { q, err := c.Controller.QueryWithCompile(ctx, id.ID(orgID), query) return q, err } @@ -55,7 +55,7 @@ func Test_QueryEndToEnd(t *testing.T) { c := control.New(config) - qs := platform.QueryServiceBridge{ + qs := query.QueryServiceBridge{ AsyncQueryService: wrapController{Controller: c}, } @@ -145,7 +145,7 @@ func ReplaceFromSpec(q *query.Spec, csvSrc string) { } } -func QueryTestCheckSpec(t *testing.T, qs platform.QueryServiceBridge, spec *query.Spec, input, want string) (bool, error) { +func QueryTestCheckSpec(t *testing.T, qs query.QueryServiceBridge, spec *query.Spec, input, want string) (bool, error) { t.Helper() ReplaceFromSpec(spec, input) id := platform.ID("max") diff --git a/repl/repl.go b/repl/repl.go index 1d67ec1148..8535c78615 100644 --- a/repl/repl.go +++ b/repl/repl.go @@ -15,13 +15,13 @@ import ( "path/filepath" prompt "github.com/c-bata/go-prompt" + "github.com/influxdata/platform/query" + "github.com/influxdata/platform/query/control" + "github.com/influxdata/platform/query/execute" "github.com/influxdata/platform/query/functions" "github.com/influxdata/platform/query/id" "github.com/influxdata/platform/query/interpreter" "github.com/influxdata/platform/query/parser" - "github.com/influxdata/platform/query" - "github.com/influxdata/platform/query/control" - "github.com/influxdata/platform/query/execute" "github.com/influxdata/platform/query/semantic" "github.com/influxdata/platform/query/values" "github.com/pkg/errors" @@ -224,7 +224,7 @@ func (r *REPL) doQuery(spec *query.Spec) error { r := results[name] blocks := r.Blocks() fmt.Println("Result:", name) - err := blocks.Do(func(b execute.Block) error { + err := blocks.Do(func(b query.Block) error { execute.NewFormatter(b, nil).WriteTo(os.Stdout) return nil }) diff --git a/result.go b/result.go index 7c41e0a890..0d2317ce1a 100644 --- a/result.go +++ b/result.go @@ -3,32 +3,134 @@ package query import ( "io" - "github.com/influxdata/platform/query/execute" - "github.com/influxdata/platform" + "github.com/influxdata/platform/query/values" ) +type Result interface { + // Blocks returns a BlockIterator for iterating through results + Blocks() BlockIterator +} + +type BlockIterator interface { + Do(f func(Block) error) error +} + +type Block interface { + Key() PartitionKey + + Cols() []ColMeta + + // Do calls f to process the data contained within the block. + // The function f will be called zero or more times. + Do(f func(ColReader) error) error + + // RefCount modifies the reference count on the block by n. + // When the RefCount goes to zero, the block is freed. + RefCount(n int) +} + +type ColMeta struct { + Label string + Type DataType +} + +type DataType int + +const ( + TInvalid DataType = iota + TBool + TInt + TUInt + TFloat + TString + TTime +) + +func (t DataType) String() string { + switch t { + case TInvalid: + return "invalid" + case TBool: + return "bool" + case TInt: + return "int" + case TUInt: + return "uint" + case TFloat: + return "float" + case TString: + return "string" + case TTime: + return "time" + default: + return "unknown" + } +} + +// ColReader allows access to reading slices of column data. +// All data the ColReader exposes is guaranteed to be in memory. +// Once a ColReader goes out of scope all slices are considered invalid. +type ColReader interface { + Key() PartitionKey + // Cols returns a list of column metadata. + Cols() []ColMeta + // Len returns the length of the slices. + // All slices will have the same length. + Len() int + Bools(j int) []bool + Ints(j int) []int64 + UInts(j int) []uint64 + Floats(j int) []float64 + Strings(j int) []string + Times(j int) []values.Time +} + +type PartitionKey interface { + Cols() []ColMeta + + HasCol(label string) bool + + ValueBool(j int) bool + ValueUInt(j int) uint64 + ValueInt(j int) int64 + ValueFloat(j int) float64 + ValueString(j int) string + ValueDuration(j int) values.Duration + ValueTime(j int) values.Time + Value(j int) interface{} + + // Intersect returns a new PartitionKey with only columns in the list of labels. + Intersect(labels []string) PartitionKey + // Diff returns the labels that exist in list of labels but not in the key's columns. + Diff(labels []string) []string + Hash() uint64 + Equal(o PartitionKey) bool + Less(o PartitionKey) bool + String() string +} + // ResultDecoder can decode a result from a reader. type ResultDecoder interface { // Decode decodes data from r into a result. - Decode(r io.Reader) (execute.Result, error) + Decode(r io.Reader) (Result, error) } // ResultEncoder can encode a result into a writer. type ResultEncoder interface { // Encode encodes data from the result into w. - Encode(w io.Writer, result execute.Result) error + Encode(w io.Writer, result Result) error } // MultiResultDecoder can decode multiple results from a reader. type MultiResultDecoder interface { // Decode decodes multiple results from r. - Decode(r io.Reader) (platform.ResultIterator, error) + Decode(r io.Reader) (ResultIterator, error) } // MultiResultEncoder can encode multiple results into a writer. type MultiResultEncoder interface { // Encode writes multiple results from r into w. - Encode(w io.Writer, results platform.ResultIterator) error + Encode(w io.Writer, results ResultIterator) error } // DelimitedMultiResultEncoder encodes multiple results using a trailing delimiter. @@ -43,7 +145,7 @@ type flusher interface { Flush() } -func (e *DelimitedMultiResultEncoder) Encode(w io.Writer, results platform.ResultIterator) error { +func (e *DelimitedMultiResultEncoder) Encode(w io.Writer, results ResultIterator) error { for results.More() { //TODO(nathanielc): Make the result name a property of a result. _, result := results.Next() diff --git a/spec.go b/spec.go new file mode 100644 index 0000000000..5fed512ce0 --- /dev/null +++ b/spec.go @@ -0,0 +1,176 @@ +package query + +import ( + "fmt" + + "github.com/pkg/errors" +) + +// Spec specifies a query. +type Spec struct { + Operations []*Operation `json:"operations"` + Edges []Edge `json:"edges"` + Resources ResourceManagement `json:"resources"` + + sorted []*Operation + children map[OperationID][]*Operation + parents map[OperationID][]*Operation +} + +// Edge is a data flow relationship between a parent and a child +type Edge struct { + Parent OperationID `json:"parent"` + Child OperationID `json:"child"` +} + +// Walk calls f on each operation exactly once. +// The function f will be called on an operation only after +// all of its parents have already been passed to f. +func (q *Spec) Walk(f func(o *Operation) error) error { + if len(q.sorted) == 0 { + if err := q.prepare(); err != nil { + return err + } + } + for _, o := range q.sorted { + err := f(o) + if err != nil { + return err + } + } + return nil +} + +// Validate ensures the query is a valid DAG. +func (q *Spec) Validate() error { + return q.prepare() +} + +// Children returns a list of children for a given operation. +// If the query is invalid no children will be returned. +func (q *Spec) Children(id OperationID) []*Operation { + if q.children == nil { + err := q.prepare() + if err != nil { + return nil + } + } + return q.children[id] +} + +// Parents returns a list of parents for a given operation. +// If the query is invalid no parents will be returned. +func (q *Spec) Parents(id OperationID) []*Operation { + if q.parents == nil { + err := q.prepare() + if err != nil { + return nil + } + } + return q.parents[id] +} + +// prepare populates the internal datastructure needed to quickly navigate the query DAG. +// As a result the query DAG is validated. +func (q *Spec) prepare() error { + q.sorted = q.sorted[0:0] + + parents, children, roots, err := q.determineParentsChildrenAndRoots() + if err != nil { + return err + } + if len(roots) == 0 { + return errors.New("query has no root nodes") + } + + q.parents = parents + q.children = children + + tMarks := make(map[OperationID]bool) + pMarks := make(map[OperationID]bool) + + for _, r := range roots { + if err := q.visit(tMarks, pMarks, r); err != nil { + return err + } + } + //reverse q.sorted + for i, j := 0, len(q.sorted)-1; i < j; i, j = i+1, j-1 { + q.sorted[i], q.sorted[j] = q.sorted[j], q.sorted[i] + } + return nil +} + +func (q *Spec) computeLookup() (map[OperationID]*Operation, error) { + lookup := make(map[OperationID]*Operation, len(q.Operations)) + for _, o := range q.Operations { + if _, ok := lookup[o.ID]; ok { + return nil, fmt.Errorf("found duplicate operation ID %q", o.ID) + } + lookup[o.ID] = o + } + return lookup, nil +} + +func (q *Spec) determineParentsChildrenAndRoots() (parents, children map[OperationID][]*Operation, roots []*Operation, _ error) { + lookup, err := q.computeLookup() + if err != nil { + return nil, nil, nil, err + } + children = make(map[OperationID][]*Operation, len(q.Operations)) + parents = make(map[OperationID][]*Operation, len(q.Operations)) + for _, e := range q.Edges { + // Build children map + c, ok := lookup[e.Child] + if !ok { + return nil, nil, nil, fmt.Errorf("edge references unknown child operation %q", e.Child) + } + children[e.Parent] = append(children[e.Parent], c) + + // Build parents map + p, ok := lookup[e.Parent] + if !ok { + return nil, nil, nil, fmt.Errorf("edge references unknown parent operation %q", e.Parent) + } + parents[e.Child] = append(parents[e.Child], p) + } + // Find roots, i.e operations with no parents. + for _, o := range q.Operations { + if len(parents[o.ID]) == 0 { + roots = append(roots, o) + } + } + return +} + +// Depth first search topological sorting of a DAG. +// https://en.wikipedia.org/wiki/Topological_sorting#Algorithms +func (q *Spec) visit(tMarks, pMarks map[OperationID]bool, o *Operation) error { + id := o.ID + if tMarks[id] { + return errors.New("found cycle in query") + } + + if !pMarks[id] { + tMarks[id] = true + for _, c := range q.children[id] { + if err := q.visit(tMarks, pMarks, c); err != nil { + return err + } + } + pMarks[id] = true + tMarks[id] = false + q.sorted = append(q.sorted, o) + } + return nil +} + +// Functions return the names of all functions used in the plan +func (q *Spec) Functions() ([]string, error) { + funcs := []string{} + err := q.Walk(func(o *Operation) error { + funcs = append(funcs, string(o.Spec.Kind())) + return nil + }) + return funcs, err +} diff --git a/transpiler.go b/transpiler.go index a94f24ff76..133a7293fd 100644 --- a/transpiler.go +++ b/transpiler.go @@ -3,17 +3,16 @@ package query import ( "context" - "github.com/influxdata/platform/query" "github.com/influxdata/platform" ) // Transpiler can convert a query from a source lanague into a query spec. type Transpiler interface { - Transpile(ctx context.Context, txt string) (*query.Spec, error) + Transpile(ctx context.Context, txt string) (*Spec, error) } // QueryWithTranspile executes a query by first transpiling the query. -func QueryWithTranspile(ctx context.Context, orgID platform.ID, q string, qs platform.QueryService, transpiler Transpiler) (platform.ResultIterator, error) { +func QueryWithTranspile(ctx context.Context, orgID platform.ID, q string, qs QueryService, transpiler Transpiler) (ResultIterator, error) { spec, err := transpiler.Transpile(ctx, q) if err != nil { return nil, err