Add time range extraction and group by support to execution engine.

pull/1232/head
Ben Johnson 2014-12-15 08:34:32 -07:00
parent 8fb8e72c33
commit 38263e0fac
7 changed files with 1073 additions and 245 deletions

641
influxql/NOTES Normal file
View File

@ -0,0 +1,641 @@
select derivative(mean(value))
from cpu
group by time(5m)
select mean(value) from cpu group by time(5m)
select top(10, value) from cpu group by host where time > now() - 1h
this query uses this type of cycle
-------REMOTE HOST ------------- -----HOST THAT GOT QUERY ---
map -> reduce -> combine -> map -> reduce -> combine -> user
select mean(value) cpu group by time(5m), host where time > now() -4h
map -> reduce -> combine -> user
map -> reduce -> map -> reduce -> combine -> user
map -> reduce -> combine -> map -> reduce -> combine -> user
select value from
(
select mean(value) AS value FROM cpu GROUP BY time(5m)
)
[
{
name: cpu,
tags: {
host: servera,
},
columns: [time, mean],
values : [
[23423423, 88.8]
]
},
{
name: cpu,
tags: {
host: serverb,
}
}
]
================================================================================
// list series ->
/*
[
{
"name": "cpu",
"columns": ["id", "region", "host"],
"values": [
1, "uswest", "servera",
2, "uswest", "serverb"
]
},
{
""
}
]
list series where region = 'uswest'
list tags where name = 'cpu'
list tagKeys where name = 'cpu'
list series where name = 'cpu' and region = 'uswest'
select distinct(region) from cpu
list names
list tagKeys
list tagValeus where tagKey = 'region' and time > now() -1h
select a.value, b.value from a join b where a.user_id == 100
select a.value from a where a.user_id == 100
select b.value from b
3 1 2
select sum(a.value) + (sum(b.value) / min(b.value)) from a join b group by region
select suM(a.value) from a group by time(5m)
select sum(b.value) from b group by time(5m)
execute sum MR on series [23, 65, 88, 99, 101, 232]
map -> 1 tick per 5m
reduce -> combines ticks per 5m interval -> outputs
planner -> take reduce output per 5m interval from the two reducers
and combine with the join function, which is +
[1,/,2,+,3]
for v := s[0].Next(); v != nil; v = 2[0].Next() {
var result interface{}
for i := 1; i < len(s); i += 2 {
/ it's an operator
if i % 2 == 1 {
}
}
}
select count(distinct(host)) from cpu where time > now() - 5m
type mapper interface {
Map(iterator)
}
type floatCountMapper struct {}
func(m *floatCountMapper) Map(i Iterator) {
itr := i.(*floatIterator)
}
type Iterator interface {
itr()
}
type iterator struct {
cursor *bolt.Cursor
timeBucket time.Time
name string
seriesID uint32
tags map[string]string
fieldID uint8
where *WhereClause
}
func (i *intIterator) itr() {}
func (i *intIterator) Next() (k int64, v float64) {
// loop through bolt cursor applying where clause and yield next point
// if cursor is at end or time is out of range, yield nil
}
*/
field: ipaddress
select top(10, count, ipaddress) from hits group by time(5m), host
map -> 10 records, <key(time,host)>, <value(count,ipaddresses)>
reducer -> take in all map outputs for each 5m bucket
combine them, sort, take out the top 10
output -> 10 records, count, ipaddresses, time
==========
select top(10, count, host) from hits group by time(5m)
select host, value from cpu where time > now() - 1h
select last(value) from cpu group by time(auto), host fill(previous) where time > now() - 1h
select sum(value) from cpu group by host where time > now() - 1h
select sum(value) from cpu where time > now() - 1h
select * from a;
[
{
"name": "cpu",
"tags": {
"host": "servera"
},
"fields": [
"time",
"count",
"ipaddress"
]
"values": [
[t, v, "123.23.22.2"],
[t, v, "192.232.2.2"],
]
},
{
"name": "cpu",
"tags": {
"host": "serverb"
},
"values": [
[t, v],
[t + 1, v],
]
},
]
[t, v, "servera"]
[t, v, "serverb"]
[t+1, v, "servera"]
[t+1, v, "serverb"]
======
a INNER JOIN b
- planner always has "group by"
select count(errors.value) / count(requests.value) as error_rate
from errors join requests as "mysuperseries"
group by time(5m)
fill(previous)
where time > now() - 4h
select mean(value) as cpu_mean from cpu group by time(5m) where host = 'servera'
select count(value) from errors group by time(5m) fill(previous) where..
select count(value) from requests group by time(5m) fill(previ...
{
"name": "errors.requests",
"tags": {},
"fields": ["time", "errors.count", "requests.count"],
"values": [
[t, n, m]
]
}
a MERGE b
a - t
b - t
a - t + 1
b - t + 1
b - t + 2
a - t + 3
<cpu, host>
select value from cpu
select mean(value) from cpu group by time(5m)
select first(value) from cpu
=====
1. Group by time
2. Group by
3. Raw
======
SELECT sum(value) FROM myseries
host=servera
host=serverb
{"host":"servera", "value":100}
{"host":"serverb", "value":"hello!"}
series = <name, tags>
series = seriesID
seriesID -> name
name has_many seriesIDs
name has_many fields
field -> (type, id)
<seriesName,fieldID> -> (type, id)
<seriesID, time> -> fieldValues
field
type topMapper struct {
count int
}
func newTopMaper(count int) {
}
func (t *topCountMapper) Map(i Iterator) {
topValues := make(map[string]int)
for p := i.Next(); p != nil; p = i.Next() {
topValues[p.String()] += 1
}
for k, v := range topValues {
t.job.Emit(k, v)
}
}
type topCountReducer struct {
count int
}
func (r *topCountReducer) Reduce(i Iterator) {
realzTop10 := make(map[string]int)
for v := i.Next(); v != nil; v = i.Next() {
top10 := v.(map[string]int)
for k, n := range top10 {
realzTop10[k] += n
}
}
realyrealTop10 := make(map[string]int)
// do sorty magic on reazTop10 and set realyreal
r.job.Emit(realyrealTop10)
}
type Transformer interface {
Transform(interface{}) Series
}
type ReduceOutput struct {
values [][]interface{}
fieldIDs []
}
// for topCountReducer ReduceOutput would look like
// values = [t, c, "some strign"]
// fieldIDs = [0, 0, 3]
SELECT val1, val2 FROM abc
select mean(value) from cpu where region='uswest' group by time(5m), host
2000 series
200 series to each machine
================================================================================
type Mapper interface {
Map(Iterator)
}
type countMapper struct {}
// Iterator is the entire series if not an aggregate query
// or iterator is the entire time bucket if an aggregate query
func (m *sumMapper) Map(i Iterator) {
var sum int
for p := i.Next(); p != nil; p = i.Next() {
sum += p.Float()
}
m.Emitter.Emit(k, sum)
}
type Point interface {
String(name)
Int(name)
}
type cursorIterator struct {
Cursor *bolt.Cursor
FieldID uint8
Value []byte
}
func (i cursorIterator) Next() Point {
_, i.Value = i.Cursor.Next()
return byteSlicePoint(i.Value)
}
type byteSlicePoint []byte
func (p byteSlicePoint) String() string {
// unmarshal from byte slice.
}
/*
{
"name": "foo",
"fields": {
"value": 23.2,
"user_id": 23
},
"tags": {
}
}
*/
CNT ID0 VALUEVALUEVALUEVALUEVALUEVALUEVALUEVALU
0001 0000 0000 0000 0000 0000 0000 0000 0000 0000
CNT ID0 ID1 ID2 FLOATFLOA STRINGSTR STRINGSTR
0002 0001 0002 0003 0000 0000 0000 0000 0000 0000
// SELECT count() FROM cpu GROUP BY host
// SELECT mean(value) from cpu where region = 'uswest'
// SELECT derivative(value) from redis_key_count GROUP BY time(5m)
// SELECT host, mean(value)
// FROM cpu
// GROUP BY host
// HAVING top(20, mean)
// WHERE time > now() - 1h
// AND region = 'uswest'
// SELECT ipaddress, count(ipaddress)
// FROM hits
// GROUP BY ipaddress
// HAVING top(10, count)
// WHERE time > now() - 1h
series := meta.DistinctTagValues("cpu", "host")
tye Series struct {
name string
fields map[uint8]string
}
type SeriesData struct {
ID
tags map[string]string
}
<id, time, value>
mrJobs := make([]*MRJob, 0, len(series))
for _, s := range series {
j := NewMRJob(s)
mrJobs = append(mrJobs, j)
j.Execute()
}
for _, j := range mrJobs {
// pull in results
// construct series object with same tags as series
}
================================================================================
type Mapper interface {
Map(Iterator)
}
type countMapper struct {}
// Iterator is the entire series if not an aggregate query
// or iterator is the entire time bucket if an aggregate query
func (m *sumMapper) Map(i Iterator) {
var sum int
for p := i.Next(); p != nil; p = i.Next() {
sum += p.Float()
}
m.Emitter.Emit(k, sum)
}
type Point interface {
String(name)
Int(name)
}
type cursorIterator struct {
Cursor *bolt.Cursor
FieldID uint8
Value []byte
}
func (i cursorIterator) Next() Point {
_, i.Value = i.Cursor.Next()
return byteSlicePoint(i.Value)
}
type byteSlicePoint []byte
func (p byteSlicePoint) String() string {
// unmarshal from byte slice.
}
/*
{
"name": "foo",
"fields": {
"value": 23.2,
"user_id": 23
},
"tags": {
}
}
*/
CNT ID0 VALUEVALUEVALUEVALUEVALUEVALUEVALUEVALU
0001 0000 0000 0000 0000 0000 0000 0000 0000 0000
CNT ID0 ID1 ID2 FLOATFLOA STRINGSTR STRINGSTR
0002 0001 0002 0003 0000 0000 0000 0000 0000 0000
// SELECT count() FROM cpu GROUP BY host
// SELECT mean(value) from cpu where region = 'uswest'
// SELECT derivative(value) from redis_key_count GROUP BY time(5m)
// SELECT host, mean(value)
// FROM cpu
// GROUP BY host
// HAVING top(20, mean)
// WHERE time > now() - 1h
// AND region = 'uswest'
// SELECT ipaddress, count(ipaddress)
// FROM hits
// GROUP BY ipaddress
// HAVING top(10, count)
// WHERE time > now() - 1h
series := meta.DistinctTagValues("cpu", "host")
mrJobs := make([]*MRJob, 0, len(series))
for _, s := range series {
j := NewMRJob(s)
mrJobs = append(mrJobs, j)
j.Execute()
}
for _, j := range mrJobs {
// pull in results
// construct series object with same tags as series
}
================================================================================
type Iterator interface {
Next() (interface{}, bool)
}
type iteratorCounter struct {
iterator Iterator
}
func (iteratorCounter) Next() {
}
SELECT max(a.value), min(a.value), max(b.value)
FROM a, b
WHERE a.host = 'influxdb.org'
grouper {
[]Iterator
}
SELECT max(a.value) FROM a WHERE a.host = 'influxdb.org' --> 1 value
SELECT min(a.value) FROM a WHERE a.host = 'influxdb.org' --> 1 value
SELECT max(b.value) FROM b --> 1 value
SELECT max(a.value) FROM a GROUP BY time WHERE a.host = 'influxdb.org' --> key,value
timeGrouper {
[]Iterator
}
type maxMapper struct {
}
IntervalIterator {
}
maxMapper.Map(Iterator)
- GROUP BY time
- GROUP BY time, <tag>
- GROUP BY <tag>
COUNT(field)
MIN(field)
MAX(field)
MEAN(field)
MODE(field)
MEDIAN(field)
COUNT(DISTINCT field)
PERCENTILE(field, N)
HISTOGRAM(field [, bucketSize])
DERIVATIVE(field)
SUM(field)
STDDEV(field)
FIRST(field)
LAST(field)
DIFFERENCE(field)
TOP(field, N)
BOTTOM(field, N) <----- multivalue
================================================================================

View File

@ -538,7 +538,7 @@ type TimeLiteral struct {
// String returns a string representation of the literal.
func (l *TimeLiteral) String() string {
return `"` + l.Val.UTC().Format("2006-01-02 15:04:05.999999") + `"`
return `"` + l.Val.UTC().Format(TimeFormat) + `"`
}
// DurationLiteral represents a duration literal.
@ -796,6 +796,72 @@ type Visitor interface {
Visit(Node) Visitor
}
// TimeRange returns the minimum and maximum times specified by an expression.
// Returns zero times for if there is no bound.
func TimeRange(expr Expr) (min, max time.Time) {
WalkFunc(expr, func(n Node) {
if n, ok := n.(*BinaryExpr); ok {
// Extract literal expression & operator on LHS.
// Check for "time" on the left-hand side first.
// Otherwise check for for the right-hand side and flip the operator.
value, op := timeExprValue(n.LHS, n.RHS), n.Op
if value.IsZero() {
if value = timeExprValue(n.RHS, n.LHS); value.IsZero() {
return
} else if op == LT {
op = GT
} else if op == LTE {
op = GTE
} else if op == GT {
op = LT
} else if op == GTE {
op = LTE
}
}
// Update the min/max depending on the operator.
// The GT & LT update the value by +/- 1µs not make them "not equal".
switch op {
case GT:
if min.IsZero() || value.After(min) {
min = value.Add(time.Microsecond)
}
case GTE:
if min.IsZero() || value.After(min) {
min = value
}
case LT:
if max.IsZero() || value.Before(max) {
max = value.Add(-time.Microsecond)
}
case LTE:
if max.IsZero() || value.Before(max) {
max = value
}
case EQ:
if min.IsZero() || value.After(min) {
min = value
}
if max.IsZero() || value.Before(max) {
max = value
}
}
}
})
return
}
// timeExprValue returns the time literal value of a "time == <TimeLiteral>" expression.
// Returns zero time if the expression is not a time expression.
func timeExprValue(ref Expr, lit Expr) time.Time {
if ref, ok := ref.(*VarRef); ok && strings.ToLower(ref.Val) == "time" {
if lit, ok := lit.(*TimeLiteral); ok {
return lit.Val
}
}
return time.Time{}
}
// Walk traverses a node hierarchy in depth-first order.
func Walk(v Visitor, node Node) {
if v = v.Visit(node); v == nil {

View File

@ -160,3 +160,49 @@ func TestFold_WithoutNow(t *testing.T) {
t.Fatalf("unexpected expr: %s", s)
}
}
// Ensure the time range of an expression can be extracted.
func TestTimeRange(t *testing.T) {
for i, tt := range []struct {
expr string
min, max string
}{
// LHS VarRef
{expr: `time > "2000-01-01 00:00:00"`, min: `2000-01-01 00:00:00.000001`, max: `0001-01-01 00:00:00`},
{expr: `time >= "2000-01-01 00:00:00"`, min: `2000-01-01 00:00:00`, max: `0001-01-01 00:00:00`},
{expr: `time < "2000-01-01 00:00:00"`, min: `0001-01-01 00:00:00`, max: `1999-12-31 23:59:59.999999`},
{expr: `time <= "2000-01-01 00:00:00"`, min: `0001-01-01 00:00:00`, max: `2000-01-01 00:00:00`},
// RHS VarRef
{expr: `"2000-01-01 00:00:00" > time`, min: `0001-01-01 00:00:00`, max: `1999-12-31 23:59:59.999999`},
{expr: `"2000-01-01 00:00:00" >= time`, min: `0001-01-01 00:00:00`, max: `2000-01-01 00:00:00`},
{expr: `"2000-01-01 00:00:00" < time`, min: `2000-01-01 00:00:00.000001`, max: `0001-01-01 00:00:00`},
{expr: `"2000-01-01 00:00:00" <= time`, min: `2000-01-01 00:00:00`, max: `0001-01-01 00:00:00`},
// Equality
{expr: `time = "2000-01-01 00:00:00"`, min: `2000-01-01 00:00:00`, max: `2000-01-01 00:00:00`},
// Multiple time expressions.
{expr: `time >= "2000-01-01 00:00:00" AND time < "2000-01-02 00:00:00"`, min: `2000-01-01 00:00:00`, max: `2000-01-01 23:59:59.999999`},
// Non-comparative expressions.
{expr: `time`, min: `0001-01-01 00:00:00`, max: `0001-01-01 00:00:00`},
{expr: `time + 2`, min: `0001-01-01 00:00:00`, max: `0001-01-01 00:00:00`},
{expr: `time - "2000-01-01 00:00:00"`, min: `0001-01-01 00:00:00`, max: `0001-01-01 00:00:00`},
{expr: `time AND "2000-01-01 00:00:00"`, min: `0001-01-01 00:00:00`, max: `0001-01-01 00:00:00`},
} {
// Extract time range.
expr := MustParseExpr(tt.expr)
min, max := influxql.TimeRange(expr)
// Compare with expected min/max.
if min := min.Format(influxql.TimeFormat); tt.min != min {
t.Errorf("%d. %s: unexpected min:\n\nexp=%s\n\ngot=%s\n\n", i, tt.expr, tt.min, min)
continue
}
if max := max.Format(influxql.TimeFormat); tt.max != max {
t.Errorf("%d. %s: unexpected max:\n\nexp=%s\n\ngot=%s\n\n", i, tt.expr, tt.max, max)
continue
}
}
}

View File

@ -1,109 +1,13 @@
package influxql
import (
"errors"
"fmt"
"strings"
"time"
)
// Plan generates an executable plan for a SELECT statement.
// list series ->
/*
[
{
"name": "cpu",
"columns": ["id", "region", "host"],
"values": [
1, "uswest", "servera",
2, "uswest", "serverb"
]
},
{
""
}
]
list series where region = 'uswest'
list tags where name = 'cpu'
list tagKeys where name = 'cpu'
list series where name = 'cpu' and region = 'uswest'
select distinct(region) from cpu
list names
list tagKeys
list tagValeus where tagKey = 'region' and time > now() -1h
select a.value, b.value from a join b where a.user_id == 100
select a.value from a where a.user_id == 100
select b.value from b
3 1 2
select sum(a.value) + (sum(b.value) / min(b.value)) from a join b group by region
select suM(a.value) from a group by time(5m)
select sum(b.value) from b group by time(5m)
execute sum MR on series [23, 65, 88, 99, 101, 232]
map -> 1 tick per 5m
reduce -> combines ticks per 5m interval -> outputs
planner -> take reduce output per 5m interval from the two reducers
and combine with the join function, which is +
[1,/,2,+,3]
for v := s[0].Next(); v != nil; v = 2[0].Next() {
var result interface{}
for i := 1; i < len(s); i += 2 {
/ it's an operator
if i % 2 == 1 {
}
}
}
select count(distinct(host)) from cpu where time > now() - 5m
type mapper interface {
Map(iterator)
}
type floatCountMapper struct {}
func(m *floatCountMapper) Map(i Iterator) {
itr := i.(*floatIterator)
}
type Iterator interface {
itr()
}
type iterator struct {
cursor *bolt.Cursor
timeBucket time.Time
name string
seriesID uint32
tags map[string]string
fieldID uint8
where *WhereClause
}
func (i *intIterator) itr() {}
func (i *intIterator) Next() (k int64, v float64) {
// loop through bolt cursor applying where clause and yield next point
// if cursor is at end or time is out of range, yield nil
}
*/
// DB represents an interface to the underlying storage.
type DB interface {
// Returns a list of series data ids matching a name and tags.
MatchSeries(name string, tags map[string]string) []uint32
@ -112,12 +16,15 @@ type DB interface {
Field(name, field string) (fieldID uint8, typ DataType)
// Returns an iterator given a series data id, field id, & field data type.
CreateIterator(id uint32, fieldID uint8, typ DataType) Iterator
CreateIterator(id uint32, fieldID uint8, typ DataType, min, max time.Time, duration time.Duration) Iterator
}
// Planner represents an object for creating execution plans.
type Planner struct {
DB DB
// The underlying storage that holds series and field meta data.
DB DB
// Returns the current time. Defaults to time.Now().
Now func() time.Time
}
@ -137,6 +44,27 @@ func (p *Planner) Plan(stmt *SelectStatement) (*Executor, error) {
processors: make([]processor, len(stmt.Fields)),
}
// Fold conditional.
now := p.Now()
stmt.Condition = Fold(stmt.Condition, &now)
// Extract the time range.
min, max := TimeRange(stmt.Condition)
if max.IsZero() {
max = now
}
if max.Before(min) {
return nil, fmt.Errorf("invalid time range: %s - %s", min.Format(TimeFormat), max.Format(TimeFormat))
}
e.min, e.max = min, max
// Determine group by interval.
interval, dimensions, err := p.normalizeDimensions(stmt.Dimensions)
if err != nil {
return nil, err
}
e.interval, e.dimensions = interval, dimensions
// Generate a processor for each field.
for i, f := range stmt.Fields {
p, err := p.planField(e, f)
@ -149,6 +77,32 @@ func (p *Planner) Plan(stmt *SelectStatement) (*Executor, error) {
return e, nil
}
// normalizeDimensions extacts the time interval, if specified.
// Returns all remaining dimensions.
func (p *Planner) normalizeDimensions(dimensions Dimensions) (time.Duration, Dimensions, error) {
// Ignore if there are no dimensions.
if len(dimensions) == 0 {
return 0, nil, nil
}
// If the first dimension is a "time(duration)" then extract the duration.
if call, ok := dimensions[0].Expr.(*Call); ok && strings.ToLower(call.Name) == "time" {
// Make sure there is exactly one argument.
if len(call.Args) != 1 {
return 0, nil, errors.New("time dimension expected one argument")
}
// Ensure the argument is a duration.
lit, ok := call.Args[0].(*DurationLiteral)
if !ok {
return 0, nil, errors.New("time dimension must have one duration argument")
}
return lit.Val, dimensions[1:], nil
}
return 0, dimensions, nil
}
// planField returns a processor for field.
func (p *Planner) planField(e *Executor, f *Field) (processor, error) {
return p.planExpr(e, f.Expr)
@ -217,16 +171,24 @@ func (p *Planner) planCall(e *Executor, c *Call) (processor, error) {
// Generate mappers for each id.
r.mappers = make([]*mapper, len(seriesIDs))
for i, seriesID := range seriesIDs {
r.mappers[i] = newMapper(e, seriesID, fieldID, typ)
m := newMapper(e, seriesID, fieldID, typ)
m.min, m.max = e.min.UnixNano(), e.max.UnixNano()
m.interval = int64(e.interval)
r.mappers[i] = m
}
// Set the appropriate reducer function.
switch strings.ToLower(c.Name) {
case "count":
r.fn = reduceCount
r.fn = reduceSum
for _, m := range r.mappers {
m.fn = mapCount
}
case "sum":
r.fn = reduceSum
for _, m := range r.mappers {
m.fn = mapSum
}
default:
return nil, fmt.Errorf("function not found: %q", c.Name)
}
@ -243,9 +205,12 @@ func (p *Planner) planBinaryExpr(e *Executor, expr *BinaryExpr) (processor, erro
// Executor represents the implementation of Executor.
// It executes all reducers and combines their result into a row.
type Executor struct {
db DB
stmt *SelectStatement
processors []processor
db DB // source database
stmt *SelectStatement // original statement
processors []processor // per-field processors
min, max time.Time // time range
interval time.Duration // group by duration
dimensions Dimensions // non-interval dimensions
}
// Execute begins execution of the query and returns a channel to receive rows.
@ -266,86 +231,56 @@ func (e *Executor) Execute() (<-chan *Row, error) {
func (e *Executor) execute(out chan *Row) {
// TODO: Support multi-value rows.
// Initialize row.
row := &Row{}
row.Name = e.processors[0].name()
// Create column names.
row.Columns = make([]string, len(e.stmt.Fields))
row.Columns = make([]string, 0)
if e.interval != 0 {
row.Columns = append(row.Columns, "time")
}
for i, f := range e.stmt.Fields {
name := f.Name()
if name == "" {
name = fmt.Sprintf("col%d", i)
}
row.Columns[i] = name
row.Columns = append(row.Columns, name)
}
// Combine values from each processor.
row.Values = [][]interface{}{make([]interface{}, 1, len(e.processors))}
for i, p := range e.processors {
// Retrieve data from the processor.
m, ok := <-p.C()
if !ok {
continue
loop:
for {
values := make([]interface{}, len(e.processors)+1)
for i, p := range e.processors {
// Retrieve data from the processor.
m, ok := <-p.C()
if !ok {
break loop
}
// Set values on returned row.
for k, v := range m {
values[0] = k / int64(time.Microsecond) // TODO: Set once per row.
values[i+1] = v
}
}
// Set values on returned row.
row.Name = p.name()
for _, v := range m {
row.Values[0][i] = v
// Remove timestamp if there is no group by interval.
if e.interval == 0 {
values = values[1:]
}
row.Values = append(row.Values, values)
}
// Send row to the channel.
if len(row.Values[0]) != 0 {
out <- row
}
out <- row
// Mark the end of the output channel.
close(out)
}
/*
select derivative(mean(value))
from cpu
group by time(5m)
select mean(value) from cpu group by time(5m)
select top(10, value) from cpu group by host where time > now() - 1h
this query uses this type of cycle
-------REMOTE HOST ------------- -----HOST THAT GOT QUERY ---
map -> reduce -> combine -> map -> reduce -> combine -> user
select mean(value) cpu group by time(5m), host where time > now() -4h
map -> reduce -> combine -> user
map -> reduce -> map -> reduce -> combine -> user
map -> reduce -> combine -> map -> reduce -> combine -> user
select value from
(
select mean(value) AS value FROM cpu GROUP BY time(5m)
)
[
{
name: cpu,
tags: {
host: servera,
},
columns: [time, mean],
values : [
[23423423, 88.8]
]
},
{
name: cpu,
tags: {
host: serverb,
}
}
]
*/
// mapper represents an object for processing iterators.
type mapper struct {
executor *Executor // parent executor
@ -353,6 +288,8 @@ type mapper struct {
fieldID uint8 // field id
typ DataType // field data type
itr Iterator // series iterator
min, max int64 // time range
interval int64 // group by interval
fn mapFunc // map function
c chan map[int64]interface{}
@ -373,7 +310,8 @@ func newMapper(e *Executor, seriesID uint32, fieldID uint8, typ DataType) *mappe
// start begins processing the iterator.
func (m *mapper) start() {
m.itr = m.executor.db.CreateIterator(m.seriesID, m.fieldID, m.typ)
m.itr = m.executor.db.CreateIterator(m.seriesID, m.fieldID, m.typ,
m.executor.min, m.executor.max, m.executor.interval)
go m.run()
}
@ -385,7 +323,9 @@ func (m *mapper) C() <-chan map[int64]interface{} { return m.c }
// run executes the map function against the iterator.
func (m *mapper) run() {
m.fn(m.itr, m)
for m.itr.NextIterval() {
m.fn(m.itr, m)
}
close(m.c)
}
@ -406,6 +346,15 @@ func mapCount(itr Iterator, m *mapper) {
m.emit(itr.Time(), float64(n))
}
// mapSum computes the summation of values in an iterator.
func mapSum(itr Iterator, m *mapper) {
n := float64(0)
for k, v := itr.Next(); k != 0; k, v = itr.Next() {
n += v.(float64)
}
m.emit(itr.Time(), n)
}
// reducer represents an object for processing mapper output.
// Implements processor.
type reducer struct {
@ -451,18 +400,24 @@ func (r *reducer) name() string { return r.stmt.Source.(*Series).Name }
// run runs the reducer loop to read mapper output and reduce it.
func (r *reducer) run() {
// Combine all data from the mappers.
data := make(map[int64][]interface{})
for _, m := range r.mappers {
kv := <-m.C()
for k, v := range kv {
data[k] = append(data[k], v)
loop:
for {
// Combine all data from the mappers.
data := make(map[int64][]interface{})
for _, m := range r.mappers {
kv, ok := <-m.C()
if !ok {
break loop
}
for k, v := range kv {
data[k] = append(data[k], v)
}
}
}
// Reduce each key.
for k, v := range data {
r.fn(k, v, r)
// Reduce each key.
for k, v := range data {
r.fn(k, v, r)
}
}
// Mark the channel as complete.
@ -477,8 +432,8 @@ func (r *reducer) emit(key int64, value interface{}) {
// reduceFunc represents a function used for reducing mapper output.
type reduceFunc func(int64, []interface{}, *reducer)
// reduceCount computes the number of values for each key.
func reduceCount(key int64, values []interface{}, r *reducer) {
// reduceSum computes the sum of values for each key.
func reduceSum(key int64, values []interface{}, r *reducer) {
var n float64
for _, v := range values {
n += v.(float64)
@ -542,15 +497,19 @@ func syncClose(done chan chan struct{}) {
}
// Iterator represents a forward-only iterator over a set of points.
// The iterator groups points together in interval sets.
type Iterator interface {
// Next returns the next value from the iterator.
Next() (key int64, value interface{})
// NextIterval moves to the next iterval. Returns true unless EOF.
NextIterval() bool
// Time returns start time of the current interval.
Time() int64
// Duration returns the group by duration.
Duration() time.Duration
// Interval returns the group by duration.
Interval() time.Duration
}
// Row represents a single row returned from the execution of a statement.

View File

@ -1,20 +1,22 @@
package influxql_test
import (
"bytes"
"encoding/json"
"fmt"
"os"
"reflect"
"sort"
"strings"
"testing"
"time"
"github.com/influxdb/influxdb/influxql"
)
// Ensure the planner can plan and execute a query.
func TestPlanner_Plan(t *testing.T) {
db := NewDB()
// Ensure the planner can plan and execute a simple count query.
func TestPlanner_Plan_Count(t *testing.T) {
db := NewDB("2000-01-01T12:00:00Z")
db.WriteSeries("cpu", map[string]string{}, "2000-01-01T00:00:00Z", map[string]interface{}{"value": float64(100)})
db.WriteSeries("cpu", map[string]string{}, "2000-01-01T00:00:10Z", map[string]interface{}{"value": float64(90)})
db.WriteSeries("cpu", map[string]string{}, "2000-01-01T00:00:20Z", map[string]interface{}{"value": float64(80)})
@ -22,51 +24,50 @@ func TestPlanner_Plan(t *testing.T) {
db.WriteSeries("cpu", map[string]string{}, "2000-01-01T00:00:40Z", map[string]interface{}{"value": float64(60)})
db.WriteSeries("cpu", map[string]string{}, "2000-01-01T00:00:50Z", map[string]interface{}{"value": float64(50)})
for i, tt := range []struct {
q string // querystring
rs []*influxql.Row // resultset
}{
// 0. Simple count
{
q: `SELECT count(value) FROM cpu`,
rs: []*influxql.Row{
{
Name: "cpu",
Tags: map[string]string{},
Columns: []string{"count"},
Values: [][]interface{}{
{6},
},
},
},
},
} {
// Plan statement.
var p = influxql.NewPlanner(db)
e, err := p.Plan(MustParseSelectStatement(tt.q))
if err != nil {
t.Errorf("%d. %s: plan error: %s", i, tt.q, err)
continue
}
// Expected resultset.
exp := minify(`[{"name":"cpu","columns":["count"],"values":[[6]]}]`)
// Execute plan.
ch, err := e.Execute()
if err != nil {
t.Errorf("%d. %s: execute error: %s", i, tt.q, err)
continue
}
// Execute and compare.
rs := db.MustPlanAndExecute(`SELECT count(value) FROM cpu`)
if act := minify(jsonify(rs)); exp != act {
t.Fatalf("unexpected resultset: %s", act)
}
}
// Collect resultset.
var rs []*influxql.Row
for row := range ch {
rs = append(rs, row)
}
// Ensure the planner can plan and execute a count query grouped by hour.
func TestPlanner_Plan_CountByHour(t *testing.T) {
db := NewDB("2000-01-01T12:00:00Z")
db.WriteSeries("cpu", map[string]string{}, "2000-01-01T09:00:00Z", map[string]interface{}{"value": float64(100)})
db.WriteSeries("cpu", map[string]string{}, "2000-01-01T09:00:00Z", map[string]interface{}{"value": float64(90)})
db.WriteSeries("cpu", map[string]string{}, "2000-01-01T09:30:00Z", map[string]interface{}{"value": float64(80)})
db.WriteSeries("cpu", map[string]string{}, "2000-01-01T11:00:00Z", map[string]interface{}{"value": float64(70)})
db.WriteSeries("cpu", map[string]string{}, "2000-01-01T11:00:00Z", map[string]interface{}{"value": float64(60)})
db.WriteSeries("cpu", map[string]string{}, "2000-01-01T11:30:00Z", map[string]interface{}{"value": float64(50)})
// Compare resultset.
if b0, b1 := mustMarshalJSON(tt.rs), mustMarshalJSON(rs); string(b0) != string(b1) {
t.Errorf("%d. resultset mismatch:\n\n%s\n\nexp=%s\n\ngot=%s\n\n", i, tt.q, b0, b1)
continue
}
// Expected resultset.
exp := minify(`[{
"name":"cpu",
"columns":["time","sum"],
"values":[
[946717200000000,190],
[946719000000000,80],
[946720800000000,0],
[946722600000000,0],
[946724400000000,130],
[946726200000000,50]
]
}]`)
// Query for data since 3 hours ago until now, grouped every 30 minutes.
rs := db.MustPlanAndExecute(`
SELECT sum(value)
FROM cpu
WHERE time >= now() - 3h
GROUP BY time(30m)`)
// Compare resultsets.
if act := jsonify(rs); exp != act {
t.Fatalf("unexpected resultset: %s", indent(act))
}
}
@ -74,16 +75,53 @@ func TestPlanner_Plan(t *testing.T) {
type DB struct {
measurements map[string]*Measurement
series map[uint32]*Series
Now time.Time
}
// NewDB returns a new instance of DB.
func NewDB() *DB {
// NewDB returns a new instance of DB at a given time.
func NewDB(now string) *DB {
return &DB{
measurements: make(map[string]*Measurement),
series: make(map[uint32]*Series),
Now: mustParseTime(now),
}
}
// PlanAndExecute plans, executes, and retrieves all rows.
func (db *DB) PlanAndExecute(querystring string) ([]*influxql.Row, error) {
// Plan statement.
p := influxql.NewPlanner(db)
p.Now = func() time.Time { return db.Now }
e, err := p.Plan(MustParseSelectStatement(querystring))
if err != nil {
return nil, err
}
// Execute plan.
ch, err := e.Execute()
if err != nil {
return nil, err
}
// Collect resultset.
var rs []*influxql.Row
for row := range ch {
rs = append(rs, row)
}
return rs, nil
}
// MustPlanAndExecute plans, executes, and retrieves all rows. Panic on error.
func (db *DB) MustPlanAndExecute(querystring string) []*influxql.Row {
rs, err := db.PlanAndExecute(querystring)
if err != nil {
panic(err.Error())
}
return rs
}
// WriteSeries writes a series
func (db *DB) WriteSeries(name string, tags map[string]string, timestamp string, values map[string]interface{}) {
// Find or create measurement & series.
@ -186,20 +224,29 @@ func (db *DB) Field(name, field string) (fieldID uint8, typ influxql.DataType) {
}
// CreateIterator returns a new iterator for a given field.
func (db *DB) CreateIterator(seriesID uint32, fieldID uint8, typ influxql.DataType) influxql.Iterator {
func (db *DB) CreateIterator(seriesID uint32, fieldID uint8, typ influxql.DataType, min, max time.Time, interval time.Duration) influxql.Iterator {
s := db.series[seriesID]
if s == nil {
panic(fmt.Sprintf("series not found: %d", seriesID))
}
// Return iterator.
return &iterator{
// Create iterator.
i := &iterator{
points: s.points,
fieldID: fieldID,
typ: typ,
time: time.Unix(0, 0).UTC(),
duration: 0,
imin: -1,
interval: int64(interval),
}
if !min.IsZero() {
i.min = min.UnixNano()
}
if !max.IsZero() {
i.max = max.UnixNano()
}
return i
}
// iterator represents an implementation of Iterator over a set of points.
@ -210,8 +257,39 @@ type iterator struct {
index int
points points
time time.Time // bucket start time
duration time.Duration // bucket duration
min, max int64 // time range
imin, imax int64 // interval time range
interval int64 // interval duration
}
// NextIterval moves the iterator to the next available interval.
// Returns true if another iterval is available.
func (i *iterator) NextIterval() bool {
// Initialize interval start time if not set.
// If there's no duration then there's only one interval.
// Otherwise increment it by the interval.
if i.imin == -1 {
i.imin = i.min
} else if i.interval == 0 {
return false
} else {
// Update interval start time if it's before iterator end time.
// Otherwise return false.
if imin := i.imin + i.interval; i.max == 0 || imin < i.max {
i.imin = imin
} else {
return false
}
}
// Interval end time should be the start time plus interval duration.
// If the end time is beyond the iterator end time then shorten it.
i.imax = i.imin + i.interval
if max := i.max; i.imax > max {
i.imax = max
}
return true
}
// Next returns the next point's timestamp and field value.
@ -229,6 +307,13 @@ func (i *iterator) Next() (timestamp int64, value interface{}) {
// Move cursor forward.
i.index++
// If timestamp is beyond bucket time range then move index back and exit.
timestamp := p.timestamp
if timestamp >= i.imax && i.imax != 0 {
i.index--
return 0, nil
}
// Return value if it is non-nil.
// Otherwise loop again and try the next point.
if v != nil {
@ -238,10 +323,10 @@ func (i *iterator) Next() (timestamp int64, value interface{}) {
}
// Time returns start time of the current interval.
func (i *iterator) Time() int64 { return i.time.UnixNano() }
func (i *iterator) Time() int64 { return i.imin }
// Duration returns the group by duration.
func (i *iterator) Duration() time.Duration { return i.duration }
// Interval returns the group by duration.
func (i *iterator) Interval() time.Duration { return time.Duration(i.interval) }
type Measurement struct {
name string
@ -319,3 +404,16 @@ func mustMarshalJSON(v interface{}) []byte {
}
return b
}
// jsonify encodes a value to JSON and returns as a string.
func jsonify(v interface{}) string { return string(mustMarshalJSON(v)) }
// ident indents a JSON string.
func indent(s string) string {
var buf bytes.Buffer
json.Indent(&buf, []byte(s), "", " ")
return buf.String()
}
// minify removes tabs and newlines.
func minify(s string) string { return strings.NewReplacer("\n", "", "\t", "").Replace(s) }

View File

@ -10,6 +10,9 @@ import (
"time"
)
// TimeFormat represents the format for time literals.
const TimeFormat = "2006-01-02 15:04:05.999999"
// Parser represents an InfluxQL parser.
type Parser struct {
s *bufScanner
@ -543,6 +546,14 @@ func (p *Parser) parseUnaryExpr() (Expr, error) {
return &VarRef{Val: lit}, nil
}
case STRING:
// If literal looks like a date time then parse it as a time literal.
if isTimeString(lit) {
t, err := time.Parse(TimeFormat, lit)
if err != nil {
return nil, &ParseError{Message: "unable to parse time", Pos: pos}
}
return &TimeLiteral{Val: t}, nil
}
return &StringLiteral{Val: lit}, nil
case NUMBER:
v, err := strconv.ParseFloat(lit, 64)
@ -718,6 +729,11 @@ func split(s string) (a []rune) {
return
}
// isTimeString returns true if the string looks like a time literal.
func isTimeString(s string) bool { return timeStringRegexp.MatchString(s) }
var timeStringRegexp = regexp.MustCompile(`^\d{4}-\d{2}-\d{2} \d{2}:\d{2}:\d{2}(?:\.\d+)?$`)
// ErrInvalidDuration is returned when parsing a malformatted duration.
var ErrInvalidDuration = errors.New("invalid duration")

View File

@ -212,14 +212,16 @@ func TestParser_ParseExpr(t *testing.T) {
expr influxql.Expr
err string
}{
// 0-4. Primitives
// Primitives
{s: `100`, expr: &influxql.NumberLiteral{Val: 100}},
{s: `"foo bar"`, expr: &influxql.StringLiteral{Val: "foo bar"}},
{s: `true`, expr: &influxql.BooleanLiteral{Val: true}},
{s: `false`, expr: &influxql.BooleanLiteral{Val: false}},
{s: `my_ident`, expr: &influxql.VarRef{Val: "my_ident"}},
{s: `"2000-01-01 00:00:00"`, expr: &influxql.TimeLiteral{Val: mustParseTime("2000-01-01T00:00:00Z")}},
{s: `"2000-01-32 00:00:00"`, err: `unable to parse time at line 1, char 1`},
// 5. Simple binary expression
// Simple binary expression
{
s: `1 + 2`,
expr: &influxql.BinaryExpr{
@ -229,7 +231,7 @@ func TestParser_ParseExpr(t *testing.T) {
},
},
// 6. Binary expression with LHS precedence
// Binary expression with LHS precedence
{
s: `1 * 2 + 3`,
expr: &influxql.BinaryExpr{
@ -243,7 +245,7 @@ func TestParser_ParseExpr(t *testing.T) {
},
},
// 7. Binary expression with RHS precedence
// Binary expression with RHS precedence
{
s: `1 + 2 * 3`,
expr: &influxql.BinaryExpr{
@ -257,7 +259,7 @@ func TestParser_ParseExpr(t *testing.T) {
},
},
// 8. Binary expression with LHS paren group.
// Binary expression with LHS paren group.
{
s: `(1 + 2) * 3`,
expr: &influxql.BinaryExpr{
@ -273,7 +275,7 @@ func TestParser_ParseExpr(t *testing.T) {
},
},
// 9. Complex binary expression.
// Complex binary expression.
{
s: `value + 3 < 30 AND 1 + 2 OR true`,
expr: &influxql.BinaryExpr{
@ -299,7 +301,7 @@ func TestParser_ParseExpr(t *testing.T) {
},
},
// 10. Function call (empty)
// Function call (empty)
{
s: `my_func()`,
expr: &influxql.Call{
@ -307,7 +309,7 @@ func TestParser_ParseExpr(t *testing.T) {
},
},
// 11. Function call (multi-arg)
// Function call (multi-arg)
{
s: `my_func(1, 2 + 3)`,
expr: &influxql.Call{