Support subquery execution in the query language
This adds query syntax support for subqueries and adds support to the query engine to execute queries on subqueries. Subqueries act as a source for another query. It is the equivalent of writing the results of a query to a temporary database, executing a query on that temporary database, and then deleting the database (except this is all performed in-memory). The syntax is like this: SELECT sum(derivative) FROM (SELECT derivative(mean(value)) FROM cpu GROUP BY *) This will execute derivative and then sum the result of those derivatives. Another example: SELECT max(min) FROM (SELECT min(value) FROM cpu GROUP BY host) This would let you find the maximum minimum value of each host. There is complete freedom to mix subqueries with auxiliary fields. The only caveat is that the following two queries: SELECT mean(value) FROM cpu SELECT mean(value) FROM (SELECT value FROM cpu) Have different performance characteristics. The first will calculate `mean(value)` at the shard level and will be faster, especially when it comes to clustered setups. The second will process the mean at the top level and will not include that optimization.pull/7646/head
parent
153277c01d
commit
d7c8c7ca4f
|
@ -21,6 +21,7 @@ The stress tool `influx_stress` will be removed in a subsequent release. We reco
|
|||
- [#7709](https://github.com/influxdata/influxdb/pull/7709): Add clear command to cli.
|
||||
- [#7688](https://github.com/influxdata/influxdb/pull/7688): Adding ability to use parameters in queries in the v2 client using the `Parameters` map in the `Query` struct.
|
||||
- [#7323](https://github.com/influxdata/influxdb/pull/7323): Allow add items to array config via ENV
|
||||
- [#4619](https://github.com/influxdata/influxdb/issues/4619): Support subquery execution in the query language.
|
||||
|
||||
### Bugfixes
|
||||
|
||||
|
|
|
@ -179,9 +179,13 @@ func NewServer(c *Config, buildInfo *BuildInfo) (*Server, error) {
|
|||
// Initialize query executor.
|
||||
s.QueryExecutor = influxql.NewQueryExecutor()
|
||||
s.QueryExecutor.StatementExecutor = &coordinator.StatementExecutor{
|
||||
MetaClient: s.MetaClient,
|
||||
TaskManager: s.QueryExecutor.TaskManager,
|
||||
TSDBStore: coordinator.LocalTSDBStore{Store: s.TSDBStore},
|
||||
MetaClient: s.MetaClient,
|
||||
TaskManager: s.QueryExecutor.TaskManager,
|
||||
TSDBStore: coordinator.LocalTSDBStore{Store: s.TSDBStore},
|
||||
ShardMapper: &coordinator.LocalShardMapper{
|
||||
MetaClient: s.MetaClient,
|
||||
TSDBStore: coordinator.LocalTSDBStore{Store: s.TSDBStore},
|
||||
},
|
||||
Monitor: s.Monitor,
|
||||
PointsWriter: s.PointsWriter,
|
||||
MaxSelectPointN: c.Coordinator.MaxSelectPointN,
|
||||
|
|
|
@ -4549,6 +4549,201 @@ func TestServer_Query_GroupByTimeCutoffs(t *testing.T) {
|
|||
}
|
||||
}
|
||||
|
||||
func TestServer_Query_Subqueries(t *testing.T) {
|
||||
t.Parallel()
|
||||
s := OpenServer(NewConfig())
|
||||
defer s.Close()
|
||||
|
||||
if err := s.CreateDatabaseAndRetentionPolicy("db0", newRetentionPolicySpec("rp0", 1, 0), true); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
writes := []string{
|
||||
fmt.Sprintf(`cpu,host=server01 usage_user=70i,usage_system=30i %d`, mustParseTime(time.RFC3339Nano, "2000-01-01T00:00:00Z").UnixNano()),
|
||||
fmt.Sprintf(`cpu,host=server01 usage_user=45i,usage_system=55i %d`, mustParseTime(time.RFC3339Nano, "2000-01-01T00:00:10Z").UnixNano()),
|
||||
fmt.Sprintf(`cpu,host=server01 usage_user=23i,usage_system=77i %d`, mustParseTime(time.RFC3339Nano, "2000-01-01T00:00:20Z").UnixNano()),
|
||||
fmt.Sprintf(`cpu,host=server02 usage_user=11i,usage_system=89i %d`, mustParseTime(time.RFC3339Nano, "2000-01-01T00:00:00Z").UnixNano()),
|
||||
fmt.Sprintf(`cpu,host=server02 usage_user=28i,usage_system=72i %d`, mustParseTime(time.RFC3339Nano, "2000-01-01T00:00:10Z").UnixNano()),
|
||||
fmt.Sprintf(`cpu,host=server02 usage_user=12i,usage_system=53i %d`, mustParseTime(time.RFC3339Nano, "2000-01-01T00:00:20Z").UnixNano()),
|
||||
}
|
||||
test := NewTest("db0", "rp0")
|
||||
test.writes = Writes{
|
||||
&Write{data: strings.Join(writes, "\n")},
|
||||
}
|
||||
|
||||
test.addQueries([]*Query{
|
||||
&Query{
|
||||
params: url.Values{"db": []string{"db0"}},
|
||||
command: `SELECT mean FROM (SELECT mean(usage_user) FROM cpu) WHERE time >= '2000-01-01T00:00:00Z' AND time < '2000-01-01T00:00:30Z'`,
|
||||
exp: `{"results":[{"statement_id":0,"series":[{"name":"cpu","columns":["time","mean"],"values":[["2000-01-01T00:00:00Z",31.5]]}]}]}`,
|
||||
},
|
||||
&Query{
|
||||
params: url.Values{"db": []string{"db0"}},
|
||||
command: `SELECT value FROM (SELECT mean(usage_user) AS value FROM cpu) WHERE time >= '2000-01-01T00:00:00Z' AND time < '2000-01-01T00:00:30Z'`,
|
||||
exp: `{"results":[{"statement_id":0,"series":[{"name":"cpu","columns":["time","value"],"values":[["2000-01-01T00:00:00Z",31.5]]}]}]}`,
|
||||
},
|
||||
&Query{
|
||||
params: url.Values{"db": []string{"db0"}},
|
||||
command: `SELECT mean(usage) FROM (SELECT 100 - usage_user AS usage FROM cpu) WHERE time >= '2000-01-01T00:00:00Z' AND time < '2000-01-01T00:00:30Z'`,
|
||||
exp: `{"results":[{"statement_id":0,"series":[{"name":"cpu","columns":["time","mean"],"values":[["2000-01-01T00:00:00Z",68.5]]}]}]}`,
|
||||
},
|
||||
&Query{
|
||||
params: url.Values{"db": []string{"db0"}},
|
||||
command: `SELECT host FROM (SELECT min(usage_user), host FROM cpu) WHERE time >= '2000-01-01T00:00:00Z' AND time < '2000-01-01T00:00:30Z'`,
|
||||
exp: `{"results":[{"statement_id":0,"series":[{"name":"cpu","columns":["time","host"],"values":[["2000-01-01T00:00:00Z","server02"]]}]}]}`,
|
||||
},
|
||||
&Query{
|
||||
params: url.Values{"db": []string{"db0"}},
|
||||
command: `SELECT host FROM (SELECT min(usage_user) FROM cpu GROUP BY host) WHERE time >= '2000-01-01T00:00:00Z' AND time < '2000-01-01T00:00:30Z'`,
|
||||
exp: `{"results":[{"statement_id":0,"series":[{"name":"cpu","columns":["time","host"],"values":[["2000-01-01T00:00:00Z","server02"],["2000-01-01T00:00:20Z","server01"]]}]}]}`,
|
||||
},
|
||||
&Query{
|
||||
params: url.Values{"db": []string{"db0"}},
|
||||
command: `SELECT mean(min) FROM (SELECT min(usage_user) FROM cpu GROUP BY host) WHERE time >= '2000-01-01T00:00:00Z' AND time < '2000-01-01T00:00:30Z'`,
|
||||
exp: `{"results":[{"statement_id":0,"series":[{"name":"cpu","columns":["time","mean"],"values":[["2000-01-01T00:00:00Z",17]]}]}]}`,
|
||||
},
|
||||
&Query{
|
||||
params: url.Values{"db": []string{"db0"}},
|
||||
command: `SELECT max(min), host FROM (SELECT min(usage_user) FROM cpu GROUP BY host) WHERE time >= '2000-01-01T00:00:00Z' AND time < '2000-01-01T00:00:30Z'`,
|
||||
exp: `{"results":[{"statement_id":0,"series":[{"name":"cpu","columns":["time","max","host"],"values":[["2000-01-01T00:00:20Z",23,"server01"]]}]}]}`,
|
||||
},
|
||||
&Query{
|
||||
params: url.Values{"db": []string{"db0"}},
|
||||
command: `SELECT host FROM (SELECT mean(usage_user) FROM cpu GROUP BY host) WHERE time >= '2000-01-01T00:00:00Z' AND time < '2000-01-01T00:00:30Z'`,
|
||||
exp: `{"results":[{"statement_id":0}]}`,
|
||||
skip: true,
|
||||
},
|
||||
&Query{
|
||||
params: url.Values{"db": []string{"db0"}},
|
||||
command: `SELECT max(usage_system) FROM (SELECT min(usage_user), usage_system FROM cpu GROUP BY host) WHERE time >= '2000-01-01T00:00:00Z' AND time < '2000-01-01T00:00:30Z'`,
|
||||
exp: `{"results":[{"statement_id":0,"series":[{"name":"cpu","columns":["time","max"],"values":[["2000-01-01T00:00:00Z",89]]}]}]}`,
|
||||
},
|
||||
&Query{
|
||||
params: url.Values{"db": []string{"db0"}},
|
||||
command: `SELECT min(top), host FROM (SELECT top(usage_user, host, 2) FROM cpu) WHERE time >= '2000-01-01T00:00:00Z' AND time < '2000-01-01T00:00:30Z'`,
|
||||
exp: `{"results":[{"statement_id":0,"series":[{"name":"cpu","columns":["time","min","host"],"values":[["2000-01-01T00:00:10Z",28,"server02"]]}]}]}`,
|
||||
},
|
||||
&Query{
|
||||
params: url.Values{"db": []string{"db0"}},
|
||||
command: `SELECT min(top), host FROM (SELECT top(usage_user, 2), host FROM cpu) WHERE time >= '2000-01-01T00:00:00Z' AND time < '2000-01-01T00:00:30Z'`,
|
||||
exp: `{"results":[{"statement_id":0,"series":[{"name":"cpu","columns":["time","min","host"],"values":[["2000-01-01T00:00:10Z",45,"server01"]]}]}]}`,
|
||||
},
|
||||
&Query{
|
||||
params: url.Values{"db": []string{"db0"}},
|
||||
command: `SELECT count(host) FROM (SELECT top(usage_user, host, 2) FROM cpu) WHERE time >= '2000-01-01T00:00:00Z' AND time < '2000-01-01T00:00:30Z'`,
|
||||
exp: `{"results":[{"statement_id":0,"series":[{"name":"cpu","columns":["time","count"],"values":[["2000-01-01T00:00:00Z",2]]}]}]}`,
|
||||
},
|
||||
&Query{
|
||||
params: url.Values{"db": []string{"db0"}},
|
||||
command: `SELECT sum(derivative) FROM (SELECT derivative(usage_user) FROM cpu GROUP BY host) WHERE time >= '2000-01-01T00:00:00Z' AND time < '2000-01-01T00:00:30Z'`,
|
||||
exp: `{"results":[{"statement_id":0,"series":[{"name":"cpu","columns":["time","sum"],"values":[["2000-01-01T00:00:00Z",-4.6]]}]}]}`,
|
||||
},
|
||||
&Query{
|
||||
params: url.Values{"db": []string{"db0"}},
|
||||
command: `SELECT min(max) FROM (SELECT 100 - max(usage_user) FROM cpu GROUP BY host) WHERE time >= '2000-01-01T00:00:00Z' AND time < '2000-01-01T00:00:30Z'`,
|
||||
exp: `{"results":[{"statement_id":0,"series":[{"name":"cpu","columns":["time","min"],"values":[["2000-01-01T00:00:00Z",30]]}]}]}`,
|
||||
},
|
||||
&Query{
|
||||
params: url.Values{"db": []string{"db0"}},
|
||||
command: `SELECT min(usage_system) FROM (SELECT max(usage_user), 100 - usage_system FROM cpu GROUP BY host) WHERE time >= '2000-01-01T00:00:00Z' AND time < '2000-01-01T00:00:30Z'`,
|
||||
exp: `{"results":[{"statement_id":0,"series":[{"name":"cpu","columns":["time","min"],"values":[["2000-01-01T00:00:10Z",28]]}]}]}`,
|
||||
},
|
||||
&Query{
|
||||
params: url.Values{"db": []string{"db0"}},
|
||||
command: `SELECT min(value) FROM (SELECT max(usage_user), usage_user - usage_system AS value FROM cpu GROUP BY host) WHERE time >= '2000-01-01T00:00:00Z' AND time < '2000-01-01T00:00:30Z'`,
|
||||
exp: `{"results":[{"statement_id":0,"series":[{"name":"cpu","columns":["time","min"],"values":[["2000-01-01T00:00:10Z",-44]]}]}]}`,
|
||||
},
|
||||
}...)
|
||||
|
||||
for i, query := range test.queries {
|
||||
if i == 0 {
|
||||
if err := test.init(s); err != nil {
|
||||
t.Fatalf("test init failed: %s", err)
|
||||
}
|
||||
}
|
||||
if query.skip {
|
||||
t.Logf("SKIP:: %s", query.name)
|
||||
continue
|
||||
}
|
||||
if err := query.Execute(s); err != nil {
|
||||
t.Error(query.Error(err))
|
||||
} else if !query.success() {
|
||||
t.Error(query.failureMessage())
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestServer_Query_SubqueryWithGroupBy(t *testing.T) {
|
||||
t.Parallel()
|
||||
s := OpenServer(NewConfig())
|
||||
defer s.Close()
|
||||
|
||||
if err := s.CreateDatabaseAndRetentionPolicy("db0", newRetentionPolicySpec("rp0", 1, 0), true); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
writes := []string{
|
||||
fmt.Sprintf(`cpu,host=server01,region=uswest value=1i %d`, mustParseTime(time.RFC3339Nano, "2000-01-01T00:00:00Z").UnixNano()),
|
||||
fmt.Sprintf(`cpu,host=server01,region=uswest value=2i %d`, mustParseTime(time.RFC3339Nano, "2000-01-01T00:00:01Z").UnixNano()),
|
||||
fmt.Sprintf(`cpu,host=server01,region=uswest value=3i %d`, mustParseTime(time.RFC3339Nano, "2000-01-01T00:00:02Z").UnixNano()),
|
||||
fmt.Sprintf(`cpu,host=server01,region=uswest value=4i %d`, mustParseTime(time.RFC3339Nano, "2000-01-01T00:00:03Z").UnixNano()),
|
||||
fmt.Sprintf(`cpu,host=server02,region=uswest value=5i %d`, mustParseTime(time.RFC3339Nano, "2000-01-01T00:00:00Z").UnixNano()),
|
||||
fmt.Sprintf(`cpu,host=server02,region=uswest value=6i %d`, mustParseTime(time.RFC3339Nano, "2000-01-01T00:00:01Z").UnixNano()),
|
||||
fmt.Sprintf(`cpu,host=server02,region=uswest value=7i %d`, mustParseTime(time.RFC3339Nano, "2000-01-01T00:00:02Z").UnixNano()),
|
||||
fmt.Sprintf(`cpu,host=server02,region=uswest value=8i %d`, mustParseTime(time.RFC3339Nano, "2000-01-01T00:00:03Z").UnixNano()),
|
||||
fmt.Sprintf(`cpu,host=server01,region=useast value=9i %d`, mustParseTime(time.RFC3339Nano, "2000-01-01T00:00:00Z").UnixNano()),
|
||||
fmt.Sprintf(`cpu,host=server01,region=useast value=10i %d`, mustParseTime(time.RFC3339Nano, "2000-01-01T00:00:01Z").UnixNano()),
|
||||
fmt.Sprintf(`cpu,host=server01,region=useast value=11i %d`, mustParseTime(time.RFC3339Nano, "2000-01-01T00:00:02Z").UnixNano()),
|
||||
fmt.Sprintf(`cpu,host=server01,region=useast value=12i %d`, mustParseTime(time.RFC3339Nano, "2000-01-01T00:00:03Z").UnixNano()),
|
||||
fmt.Sprintf(`cpu,host=server02,region=useast value=13i %d`, mustParseTime(time.RFC3339Nano, "2000-01-01T00:00:00Z").UnixNano()),
|
||||
fmt.Sprintf(`cpu,host=server02,region=useast value=14i %d`, mustParseTime(time.RFC3339Nano, "2000-01-01T00:00:01Z").UnixNano()),
|
||||
fmt.Sprintf(`cpu,host=server02,region=useast value=15i %d`, mustParseTime(time.RFC3339Nano, "2000-01-01T00:00:02Z").UnixNano()),
|
||||
fmt.Sprintf(`cpu,host=server02,region=useast value=16i %d`, mustParseTime(time.RFC3339Nano, "2000-01-01T00:00:03Z").UnixNano()),
|
||||
}
|
||||
test := NewTest("db0", "rp0")
|
||||
test.writes = Writes{
|
||||
&Write{data: strings.Join(writes, "\n")},
|
||||
}
|
||||
|
||||
test.addQueries([]*Query{
|
||||
&Query{
|
||||
name: "group by time(2s) - time(2s), host",
|
||||
params: url.Values{"db": []string{"db0"}},
|
||||
command: `SELECT mean(mean) FROM (SELECT mean(value) FROM cpu GROUP BY time(2s), host) WHERE time >= '2000-01-01T00:00:00Z' AND time < '2000-01-01T00:00:04Z' GROUP BY time(2s)`,
|
||||
exp: `{"results":[{"statement_id":0,"series":[{"name":"cpu","columns":["time","mean"],"values":[["2000-01-01T00:00:00Z",7.5],["2000-01-01T00:00:02Z",9.5]]}]}]}`,
|
||||
},
|
||||
&Query{
|
||||
name: "group by time(4s), host - time(2s), host",
|
||||
params: url.Values{"db": []string{"db0"}},
|
||||
command: `SELECT mean(mean) FROM (SELECT mean(value) FROM cpu GROUP BY time(2s), host) WHERE time >= '2000-01-01T00:00:00Z' AND time < '2000-01-01T00:00:04Z' GROUP BY time(4s), host`,
|
||||
exp: `{"results":[{"statement_id":0,"series":[{"name":"cpu","tags":{"host":"server01"},"columns":["time","mean"],"values":[["2000-01-01T00:00:00Z",6.5]]},{"name":"cpu","tags":{"host":"server02"},"columns":["time","mean"],"values":[["2000-01-01T00:00:00Z",10.5]]}]}]}`,
|
||||
},
|
||||
&Query{
|
||||
name: "group by time(2s), host - time(2s), host, region",
|
||||
params: url.Values{"db": []string{"db0"}},
|
||||
command: `SELECT mean(mean) FROM (SELECT mean(value) FROM cpu GROUP BY time(2s), host, region) WHERE time >= '2000-01-01T00:00:00Z' AND time < '2000-01-01T00:00:04Z' GROUP BY time(2s), host`,
|
||||
exp: `{"results":[{"statement_id":0,"series":[{"name":"cpu","tags":{"host":"server01"},"columns":["time","mean"],"values":[["2000-01-01T00:00:00Z",5.5],["2000-01-01T00:00:02Z",7.5]]},{"name":"cpu","tags":{"host":"server02"},"columns":["time","mean"],"values":[["2000-01-01T00:00:00Z",9.5],["2000-01-01T00:00:02Z",11.5]]}]}]}`,
|
||||
},
|
||||
}...)
|
||||
|
||||
for i, query := range test.queries {
|
||||
if i == 0 {
|
||||
if err := test.init(s); err != nil {
|
||||
t.Fatalf("test init failed: %s", err)
|
||||
}
|
||||
}
|
||||
if query.skip {
|
||||
t.Logf("SKIP:: %s", query.name)
|
||||
continue
|
||||
}
|
||||
if err := query.Execute(s); err != nil {
|
||||
t.Error(query.Error(err))
|
||||
} else if !query.success() {
|
||||
t.Error(query.failureMessage())
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestServer_Write_Precision(t *testing.T) {
|
||||
t.Parallel()
|
||||
s := OpenServer(NewConfig())
|
||||
|
|
|
@ -26,7 +26,7 @@ type MetaClient interface {
|
|||
RetentionPolicy(database, name string) (rpi *meta.RetentionPolicyInfo, err error)
|
||||
SetAdminPrivilege(username string, admin bool) error
|
||||
SetPrivilege(username, database string, p influxql.Privilege) error
|
||||
ShardsByTimeRange(sources influxql.Sources, tmin, tmax time.Time) (a []meta.ShardInfo, err error)
|
||||
ShardGroupsByTimeRange(database, policy string, min, max time.Time) (a []meta.ShardGroupInfo, err error)
|
||||
UpdateRetentionPolicy(database, name string, rpu *meta.RetentionPolicyUpdate, makeDefault bool) error
|
||||
UpdateUser(name, password string) error
|
||||
UserPrivilege(username, database string) (*influxql.Privilege, error)
|
||||
|
|
|
@ -31,7 +31,7 @@ type MetaClient struct {
|
|||
RetentionPolicyFn func(database, name string) (rpi *meta.RetentionPolicyInfo, err error)
|
||||
SetAdminPrivilegeFn func(username string, admin bool) error
|
||||
SetPrivilegeFn func(username, database string, p influxql.Privilege) error
|
||||
ShardsByTimeRangeFn func(sources influxql.Sources, tmin, tmax time.Time) (a []meta.ShardInfo, err error)
|
||||
ShardGroupsByTimeRangeFn func(database, policy string, min, max time.Time) (a []meta.ShardGroupInfo, err error)
|
||||
UpdateRetentionPolicyFn func(database, name string, rpu *meta.RetentionPolicyUpdate, makeDefault bool) error
|
||||
UpdateUserFn func(name, password string) error
|
||||
UserPrivilegeFn func(username, database string) (*influxql.Privilege, error)
|
||||
|
@ -127,8 +127,8 @@ func (c *MetaClient) SetPrivilege(username, database string, p influxql.Privileg
|
|||
return c.SetPrivilegeFn(username, database, p)
|
||||
}
|
||||
|
||||
func (c *MetaClient) ShardsByTimeRange(sources influxql.Sources, tmin, tmax time.Time) (a []meta.ShardInfo, err error) {
|
||||
return c.ShardsByTimeRangeFn(sources, tmin, tmax)
|
||||
func (c *MetaClient) ShardGroupsByTimeRange(database, policy string, min, max time.Time) (a []meta.ShardGroupInfo, err error) {
|
||||
return c.ShardGroupsByTimeRangeFn(database, policy, min, max)
|
||||
}
|
||||
|
||||
func (c *MetaClient) UpdateRetentionPolicy(database, name string, rpu *meta.RetentionPolicyUpdate, makeDefault bool) error {
|
||||
|
|
|
@ -0,0 +1,181 @@
|
|||
package coordinator
|
||||
|
||||
import (
|
||||
"io"
|
||||
"time"
|
||||
|
||||
"github.com/influxdata/influxdb/influxql"
|
||||
"github.com/influxdata/influxdb/services/meta"
|
||||
"github.com/influxdata/influxdb/tsdb"
|
||||
)
|
||||
|
||||
// IteratorCreator is an interface that combines mapping fields and creating iterators.
|
||||
type IteratorCreator interface {
|
||||
influxql.IteratorCreator
|
||||
influxql.FieldMapper
|
||||
io.Closer
|
||||
}
|
||||
|
||||
// ShardMapper retrieves and maps shards into an IteratorCreator that can later be
|
||||
// used for executing queries.
|
||||
type ShardMapper interface {
|
||||
MapShards(sources influxql.Sources, opt *influxql.SelectOptions) (IteratorCreator, error)
|
||||
}
|
||||
|
||||
// LocalShardMapper implements a ShardMapper for local shards.
|
||||
type LocalShardMapper struct {
|
||||
MetaClient interface {
|
||||
ShardGroupsByTimeRange(database, policy string, min, max time.Time) (a []meta.ShardGroupInfo, err error)
|
||||
}
|
||||
|
||||
TSDBStore interface {
|
||||
ShardGroup(ids []uint64) tsdb.ShardGroup
|
||||
}
|
||||
}
|
||||
|
||||
// MapShards maps the sources to the appropriate shards into an IteratorCreator.
|
||||
func (e *LocalShardMapper) MapShards(sources influxql.Sources, opt *influxql.SelectOptions) (IteratorCreator, error) {
|
||||
a := &LocalShardMapping{
|
||||
ShardMap: make(map[Source]tsdb.ShardGroup),
|
||||
}
|
||||
|
||||
if err := e.mapShards(a, sources, opt); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return a, nil
|
||||
}
|
||||
|
||||
func (e *LocalShardMapper) mapShards(a *LocalShardMapping, sources influxql.Sources, opt *influxql.SelectOptions) error {
|
||||
for _, s := range sources {
|
||||
switch s := s.(type) {
|
||||
case *influxql.Measurement:
|
||||
source := Source{
|
||||
Database: s.Database,
|
||||
RetentionPolicy: s.RetentionPolicy,
|
||||
}
|
||||
|
||||
// Retrieve the list of shards for this database. This list of
|
||||
// shards is always the same regardless of which measurement we are
|
||||
// using.
|
||||
if _, ok := a.ShardMap[source]; !ok {
|
||||
groups, err := e.MetaClient.ShardGroupsByTimeRange(s.Database, s.RetentionPolicy, opt.MinTime, opt.MaxTime)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if len(groups) == 0 {
|
||||
a.ShardMap[source] = nil
|
||||
continue
|
||||
}
|
||||
|
||||
shardIDs := make([]uint64, 0, len(groups[0].Shards)*len(groups))
|
||||
for _, g := range groups {
|
||||
for _, si := range g.Shards {
|
||||
shardIDs = append(shardIDs, si.ID)
|
||||
}
|
||||
}
|
||||
a.ShardMap[source] = e.TSDBStore.ShardGroup(shardIDs)
|
||||
}
|
||||
case *influxql.SubQuery:
|
||||
if err := e.mapShards(a, s.Statement.Sources, opt); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// ShardMapper maps data sources to a list of shard information.
|
||||
type LocalShardMapping struct {
|
||||
ShardMap map[Source]tsdb.ShardGroup
|
||||
}
|
||||
|
||||
func (a *LocalShardMapping) FieldDimensions(m *influxql.Measurement) (fields map[string]influxql.DataType, dimensions map[string]struct{}, err error) {
|
||||
source := Source{
|
||||
Database: m.Database,
|
||||
RetentionPolicy: m.RetentionPolicy,
|
||||
}
|
||||
|
||||
sg := a.ShardMap[source]
|
||||
if sg == nil {
|
||||
return
|
||||
}
|
||||
|
||||
fields = make(map[string]influxql.DataType)
|
||||
dimensions = make(map[string]struct{})
|
||||
|
||||
var measurements []string
|
||||
if m.Regex != nil {
|
||||
measurements = sg.MeasurementsByRegex(m.Regex.Val)
|
||||
} else {
|
||||
measurements = []string{m.Name}
|
||||
}
|
||||
|
||||
f, d, err := sg.FieldDimensions(measurements)
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
for k, typ := range f {
|
||||
fields[k] = typ
|
||||
}
|
||||
for k := range d {
|
||||
dimensions[k] = struct{}{}
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
func (a *LocalShardMapping) MapType(m *influxql.Measurement, field string) influxql.DataType {
|
||||
source := Source{
|
||||
Database: m.Database,
|
||||
RetentionPolicy: m.RetentionPolicy,
|
||||
}
|
||||
|
||||
sg := a.ShardMap[source]
|
||||
if sg == nil {
|
||||
return influxql.Unknown
|
||||
}
|
||||
return sg.MapType(m.Name, field)
|
||||
}
|
||||
|
||||
func (a *LocalShardMapping) CreateIterator(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
source := Source{
|
||||
Database: m.Database,
|
||||
RetentionPolicy: m.RetentionPolicy,
|
||||
}
|
||||
|
||||
sg := a.ShardMap[source]
|
||||
if sg == nil {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
if m.Regex != nil {
|
||||
measurements := sg.MeasurementsByRegex(m.Regex.Val)
|
||||
inputs := make([]influxql.Iterator, 0, len(measurements))
|
||||
if err := func() error {
|
||||
for _, measurement := range measurements {
|
||||
input, err := sg.CreateIterator(measurement, opt)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
inputs = append(inputs, input)
|
||||
}
|
||||
return nil
|
||||
}(); err != nil {
|
||||
influxql.Iterators(inputs).Close()
|
||||
return nil, err
|
||||
}
|
||||
return influxql.Iterators(inputs).Merge(opt)
|
||||
}
|
||||
return sg.CreateIterator(m.Name, opt)
|
||||
}
|
||||
|
||||
// Close does nothing for a LocalShardMapping.
|
||||
func (a *LocalShardMapping) Close() error {
|
||||
return nil
|
||||
}
|
||||
|
||||
// Source contains the database and retention policy source for data.
|
||||
type Source struct {
|
||||
Database string
|
||||
RetentionPolicy string
|
||||
}
|
|
@ -0,0 +1,102 @@
|
|||
package coordinator_test
|
||||
|
||||
import (
|
||||
"reflect"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/influxdata/influxdb/coordinator"
|
||||
"github.com/influxdata/influxdb/influxql"
|
||||
"github.com/influxdata/influxdb/services/meta"
|
||||
"github.com/influxdata/influxdb/tsdb"
|
||||
)
|
||||
|
||||
func TestLocalShardMapper(t *testing.T) {
|
||||
var metaClient MetaClient
|
||||
metaClient.ShardGroupsByTimeRangeFn = func(database, policy string, min, max time.Time) ([]meta.ShardGroupInfo, error) {
|
||||
if database != "db0" {
|
||||
t.Errorf("unexpected database: %s", database)
|
||||
}
|
||||
if policy != "rp0" {
|
||||
t.Errorf("unexpected retention policy: %s", policy)
|
||||
}
|
||||
return []meta.ShardGroupInfo{
|
||||
{ID: 1, Shards: []meta.ShardInfo{
|
||||
{ID: 1, Owners: []meta.ShardOwner{{NodeID: 0}}},
|
||||
{ID: 2, Owners: []meta.ShardOwner{{NodeID: 0}}},
|
||||
}},
|
||||
{ID: 2, Shards: []meta.ShardInfo{
|
||||
{ID: 3, Owners: []meta.ShardOwner{{NodeID: 0}}},
|
||||
{ID: 4, Owners: []meta.ShardOwner{{NodeID: 0}}},
|
||||
}},
|
||||
}, nil
|
||||
}
|
||||
|
||||
var tsdbStore TSDBStore
|
||||
tsdbStore.ShardGroupFn = func(ids []uint64) tsdb.ShardGroup {
|
||||
if !reflect.DeepEqual(ids, []uint64{1, 2, 3, 4}) {
|
||||
t.Errorf("unexpected shard ids: %#v", ids)
|
||||
}
|
||||
|
||||
var sh MockShard
|
||||
sh.CreateIteratorFn = func(measurement string, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if measurement != "cpu" {
|
||||
t.Errorf("unexpected measurement: %s", measurement)
|
||||
}
|
||||
return &FloatIterator{}, nil
|
||||
}
|
||||
return &sh
|
||||
}
|
||||
|
||||
// Initialize the shard mapper.
|
||||
shardMapper := &coordinator.LocalShardMapper{
|
||||
MetaClient: &metaClient,
|
||||
TSDBStore: &tsdbStore,
|
||||
}
|
||||
|
||||
// Normal measurement.
|
||||
measurement := &influxql.Measurement{
|
||||
Database: "db0",
|
||||
RetentionPolicy: "rp0",
|
||||
Name: "cpu",
|
||||
}
|
||||
ic, err := shardMapper.MapShards([]influxql.Source{measurement}, &influxql.SelectOptions{})
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error: %s", err)
|
||||
}
|
||||
|
||||
// This should be a LocalShardMapping.
|
||||
m, ok := ic.(*coordinator.LocalShardMapping)
|
||||
if !ok {
|
||||
t.Fatalf("unexpected mapping type: %T", ic)
|
||||
} else if len(m.ShardMap) != 1 {
|
||||
t.Fatalf("unexpected number of shard mappings: %d", len(m.ShardMap))
|
||||
}
|
||||
|
||||
if _, err := ic.CreateIterator(measurement, influxql.IteratorOptions{}); err != nil {
|
||||
t.Fatalf("unexpected error: %s", err)
|
||||
}
|
||||
|
||||
// Subquery.
|
||||
subquery := &influxql.SubQuery{
|
||||
Statement: &influxql.SelectStatement{
|
||||
Sources: []influxql.Source{measurement},
|
||||
},
|
||||
}
|
||||
ic, err = shardMapper.MapShards([]influxql.Source{subquery}, &influxql.SelectOptions{})
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error: %s", err)
|
||||
}
|
||||
|
||||
// This should be a LocalShardMapping.
|
||||
m, ok = ic.(*coordinator.LocalShardMapping)
|
||||
if !ok {
|
||||
t.Fatalf("unexpected mapping type: %T", ic)
|
||||
} else if len(m.ShardMap) != 1 {
|
||||
t.Fatalf("unexpected number of shard mappings: %d", len(m.ShardMap))
|
||||
}
|
||||
|
||||
if _, err := ic.CreateIterator(measurement, influxql.IteratorOptions{}); err != nil {
|
||||
t.Fatalf("unexpected error: %s", err)
|
||||
}
|
||||
}
|
|
@ -35,6 +35,9 @@ type StatementExecutor struct {
|
|||
// TSDB storage for local node.
|
||||
TSDBStore TSDBStore
|
||||
|
||||
// ShardMapper for mapping shards when executing a SELECT statement.
|
||||
ShardMapper ShardMapper
|
||||
|
||||
// Holds monitoring data for SHOW STATS and SHOW DIAGNOSTICS.
|
||||
Monitor *monitor.Monitor
|
||||
|
||||
|
@ -495,11 +498,7 @@ func (e *StatementExecutor) createIterators(stmt *influxql.SelectStatement, ctx
|
|||
|
||||
// Replace instances of "now()" with the current time, and check the resultant times.
|
||||
nowValuer := influxql.NowValuer{Now: now}
|
||||
stmt.Condition = influxql.Reduce(stmt.Condition, &nowValuer)
|
||||
// Replace instances of "now()" with the current time in the dimensions.
|
||||
for _, d := range stmt.Dimensions {
|
||||
d.Expr = influxql.Reduce(d.Expr, &nowValuer)
|
||||
}
|
||||
stmt = stmt.Reduce(&nowValuer)
|
||||
|
||||
var err error
|
||||
opt.MinTime, opt.MaxTime, err = influxql.TimeRange(stmt.Condition)
|
||||
|
@ -508,21 +507,7 @@ func (e *StatementExecutor) createIterators(stmt *influxql.SelectStatement, ctx
|
|||
}
|
||||
|
||||
if opt.MaxTime.IsZero() {
|
||||
// In the case that we're executing a meta query where the user cannot
|
||||
// specify a time condition, then we expand the default max time
|
||||
// to the maximum possible value, to ensure that data where all points
|
||||
// are in the future are returned.
|
||||
if influxql.Sources(stmt.Sources).HasSystemSource() {
|
||||
opt.MaxTime = time.Unix(0, influxql.MaxTime).UTC()
|
||||
} else {
|
||||
if interval, err := stmt.GroupByInterval(); err != nil {
|
||||
return nil, stmt, err
|
||||
} else if interval > 0 {
|
||||
opt.MaxTime = now
|
||||
} else {
|
||||
opt.MaxTime = time.Unix(0, influxql.MaxTime).UTC()
|
||||
}
|
||||
}
|
||||
opt.MaxTime = time.Unix(0, influxql.MaxTime)
|
||||
}
|
||||
if opt.MinTime.IsZero() {
|
||||
opt.MinTime = time.Unix(0, influxql.MinTime).UTC()
|
||||
|
@ -534,23 +519,20 @@ func (e *StatementExecutor) createIterators(stmt *influxql.SelectStatement, ctx
|
|||
// Remove "time" from fields list.
|
||||
stmt.RewriteTimeFields()
|
||||
|
||||
// Rewrite time condition.
|
||||
if err := stmt.RewriteTimeCondition(now); err != nil {
|
||||
return nil, stmt, err
|
||||
}
|
||||
|
||||
// Rewrite any regex conditions that could make use of the index.
|
||||
stmt.RewriteRegexConditions()
|
||||
|
||||
// Create an iterator creator based on the shards in the cluster.
|
||||
ic, err := e.iteratorCreator(stmt, &opt)
|
||||
ic, err := e.ShardMapper.MapShards(stmt.Sources, &opt)
|
||||
if err != nil {
|
||||
return nil, stmt, err
|
||||
}
|
||||
|
||||
// Expand regex sources to their actual source names.
|
||||
if stmt.Sources.HasRegex() {
|
||||
sources, err := ic.ExpandSources(stmt.Sources)
|
||||
if err != nil {
|
||||
return nil, stmt, err
|
||||
}
|
||||
stmt.Sources = sources
|
||||
}
|
||||
defer ic.Close()
|
||||
|
||||
// Rewrite wildcards, if any exist.
|
||||
tmp, err := stmt.RewriteFields(ic)
|
||||
|
@ -591,16 +573,6 @@ func (e *StatementExecutor) createIterators(stmt *influxql.SelectStatement, ctx
|
|||
return itrs, stmt, nil
|
||||
}
|
||||
|
||||
// iteratorCreator returns a new instance of IteratorCreator based on stmt.
|
||||
func (e *StatementExecutor) iteratorCreator(stmt *influxql.SelectStatement, opt *influxql.SelectOptions) (influxql.IteratorCreator, error) {
|
||||
// Retrieve a list of shard IDs.
|
||||
shards, err := e.MetaClient.ShardsByTimeRange(stmt.Sources, opt.MinTime, opt.MaxTime)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return e.TSDBStore.IteratorCreator(shards, opt)
|
||||
}
|
||||
|
||||
func (e *StatementExecutor) executeShowContinuousQueriesStatement(stmt *influxql.ShowContinuousQueriesStatement) (models.Rows, error) {
|
||||
dis := e.MetaClient.Databases()
|
||||
|
||||
|
@ -1148,7 +1120,6 @@ type TSDBStore interface {
|
|||
DeleteRetentionPolicy(database, name string) error
|
||||
DeleteSeries(database string, sources []influxql.Source, condition influxql.Expr) error
|
||||
DeleteShard(id uint64) error
|
||||
IteratorCreator(shards []meta.ShardInfo, opt *influxql.SelectOptions) (influxql.IteratorCreator, error)
|
||||
|
||||
Measurements(database string, cond influxql.Expr) ([]string, error)
|
||||
TagValues(database string, cond influxql.Expr) ([]tsdb.TagValues, error)
|
||||
|
@ -1162,15 +1133,6 @@ type LocalTSDBStore struct {
|
|||
*tsdb.Store
|
||||
}
|
||||
|
||||
// IteratorCreator returns an influxql.IteratorCreator for the given shards, with the given select options.
|
||||
func (s LocalTSDBStore) IteratorCreator(shards []meta.ShardInfo, opt *influxql.SelectOptions) (influxql.IteratorCreator, error) {
|
||||
shardIDs := make([]uint64, len(shards))
|
||||
for i, sh := range shards {
|
||||
shardIDs[i] = sh.ID
|
||||
}
|
||||
return s.Store.IteratorCreator(shardIDs, opt)
|
||||
}
|
||||
|
||||
// ShardIteratorCreator is an interface for creating an IteratorCreator to access a specific shard.
|
||||
type ShardIteratorCreator interface {
|
||||
ShardIteratorCreator(id uint64) influxql.IteratorCreator
|
||||
|
|
|
@ -6,6 +6,7 @@ import (
|
|||
"io"
|
||||
"os"
|
||||
"reflect"
|
||||
"regexp"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
|
@ -32,28 +33,35 @@ func TestQueryExecutor_ExecuteQuery_SelectStatement(t *testing.T) {
|
|||
e := DefaultQueryExecutor()
|
||||
|
||||
// The meta client should return a single shard owned by the local node.
|
||||
e.MetaClient.ShardsByTimeRangeFn = func(sources influxql.Sources, tmin, tmax time.Time) (a []meta.ShardInfo, err error) {
|
||||
return []meta.ShardInfo{{ID: 100, Owners: []meta.ShardOwner{{NodeID: 0}}}}, nil
|
||||
e.MetaClient.ShardGroupsByTimeRangeFn = func(database, policy string, min, max time.Time) (a []meta.ShardGroupInfo, err error) {
|
||||
return []meta.ShardGroupInfo{
|
||||
{ID: 1, Shards: []meta.ShardInfo{
|
||||
{ID: 100, Owners: []meta.ShardOwner{{NodeID: 0}}},
|
||||
}},
|
||||
}, nil
|
||||
}
|
||||
|
||||
// The TSDB store should return an IteratorCreator for shard.
|
||||
// This IteratorCreator returns a single iterator with "value" in the aux fields.
|
||||
e.TSDBStore.ShardIteratorCreatorFn = func(id uint64) influxql.IteratorCreator {
|
||||
if id != 100 {
|
||||
t.Fatalf("unexpected shard id: %d", id)
|
||||
e.TSDBStore.ShardGroupFn = func(ids []uint64) tsdb.ShardGroup {
|
||||
if !reflect.DeepEqual(ids, []uint64{100}) {
|
||||
t.Fatalf("unexpected shard ids: %v", ids)
|
||||
}
|
||||
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
var sh MockShard
|
||||
sh.CreateIteratorFn = func(m string, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
return &FloatIterator{Points: []influxql.FloatPoint{
|
||||
{Name: "cpu", Time: int64(0 * time.Second), Aux: []interface{}{float64(100)}},
|
||||
{Name: "cpu", Time: int64(1 * time.Second), Aux: []interface{}{float64(200)}},
|
||||
}}, nil
|
||||
}
|
||||
ic.FieldDimensionsFn = func(sources influxql.Sources) (fields map[string]influxql.DataType, dimensions map[string]struct{}, err error) {
|
||||
sh.FieldDimensionsFn = func(measurements []string) (fields map[string]influxql.DataType, dimensions map[string]struct{}, err error) {
|
||||
if !reflect.DeepEqual(measurements, []string{"cpu"}) {
|
||||
t.Fatalf("unexpected source: %#v", measurements)
|
||||
}
|
||||
return map[string]influxql.DataType{"value": influxql.Float}, nil, nil
|
||||
}
|
||||
return &ic
|
||||
return &sh
|
||||
}
|
||||
|
||||
// Verify all results from the query.
|
||||
|
@ -80,22 +88,33 @@ func TestQueryExecutor_ExecuteQuery_MaxSelectBucketsN(t *testing.T) {
|
|||
e.StatementExecutor.MaxSelectBucketsN = 3
|
||||
|
||||
// The meta client should return a single shards on the local node.
|
||||
e.MetaClient.ShardsByTimeRangeFn = func(sources influxql.Sources, tmin, tmax time.Time) (a []meta.ShardInfo, err error) {
|
||||
return []meta.ShardInfo{
|
||||
{ID: 100, Owners: []meta.ShardOwner{{NodeID: 0}}},
|
||||
e.MetaClient.ShardGroupsByTimeRangeFn = func(database, policy string, min, max time.Time) (a []meta.ShardGroupInfo, err error) {
|
||||
return []meta.ShardGroupInfo{
|
||||
{ID: 1, Shards: []meta.ShardInfo{
|
||||
{ID: 100, Owners: []meta.ShardOwner{{NodeID: 0}}},
|
||||
}},
|
||||
}, nil
|
||||
}
|
||||
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
return &FloatIterator{
|
||||
Points: []influxql.FloatPoint{{Name: "cpu", Time: int64(0 * time.Second), Aux: []interface{}{float64(100)}}},
|
||||
}, nil
|
||||
e.TSDBStore.ShardGroupFn = func(ids []uint64) tsdb.ShardGroup {
|
||||
if !reflect.DeepEqual(ids, []uint64{100}) {
|
||||
t.Fatalf("unexpected shard ids: %v", ids)
|
||||
}
|
||||
|
||||
var sh MockShard
|
||||
sh.CreateIteratorFn = func(m string, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
return &FloatIterator{
|
||||
Points: []influxql.FloatPoint{{Name: "cpu", Time: int64(0 * time.Second), Aux: []interface{}{float64(100)}}},
|
||||
}, nil
|
||||
}
|
||||
sh.FieldDimensionsFn = func(measurements []string) (fields map[string]influxql.DataType, dimensions map[string]struct{}, err error) {
|
||||
if !reflect.DeepEqual(measurements, []string{"cpu"}) {
|
||||
t.Fatalf("unexpected source: %#v", measurements)
|
||||
}
|
||||
return map[string]influxql.DataType{"value": influxql.Float}, nil, nil
|
||||
}
|
||||
return &sh
|
||||
}
|
||||
ic.FieldDimensionsFn = func(sources influxql.Sources) (fields map[string]influxql.DataType, dimensions map[string]struct{}, err error) {
|
||||
return map[string]influxql.DataType{"value": influxql.Float}, nil, nil
|
||||
}
|
||||
e.TSDBStore.ShardIteratorCreatorFn = func(id uint64) influxql.IteratorCreator { return &ic }
|
||||
|
||||
// Verify all results from the query.
|
||||
if a := ReadAllResults(e.ExecuteQuery(`SELECT count(value) FROM cpu WHERE time >= '2000-01-01T00:00:05Z' AND time < '2000-01-01T00:00:35Z' GROUP BY time(10s)`, "db0", 0)); !reflect.DeepEqual(a, []*influxql.Result{
|
||||
|
@ -193,6 +212,10 @@ func NewQueryExecutor() *QueryExecutor {
|
|||
e.StatementExecutor = &coordinator.StatementExecutor{
|
||||
MetaClient: &e.MetaClient,
|
||||
TSDBStore: &e.TSDBStore,
|
||||
ShardMapper: &coordinator.LocalShardMapper{
|
||||
MetaClient: &e.MetaClient,
|
||||
TSDBStore: &e.TSDBStore,
|
||||
},
|
||||
}
|
||||
e.QueryExecutor.StatementExecutor = e.StatementExecutor
|
||||
|
||||
|
@ -237,7 +260,7 @@ type TSDBStore struct {
|
|||
DeleteShardFn func(id uint64) error
|
||||
DeleteSeriesFn func(database string, sources []influxql.Source, condition influxql.Expr) error
|
||||
DatabaseIndexFn func(name string) *tsdb.DatabaseIndex
|
||||
ShardIteratorCreatorFn func(id uint64) influxql.IteratorCreator
|
||||
ShardGroupFn func(ids []uint64) tsdb.ShardGroup
|
||||
}
|
||||
|
||||
func (s *TSDBStore) CreateShard(database, policy string, shardID uint64, enabled bool) error {
|
||||
|
@ -279,29 +302,8 @@ func (s *TSDBStore) DeleteSeries(database string, sources []influxql.Source, con
|
|||
return s.DeleteSeriesFn(database, sources, condition)
|
||||
}
|
||||
|
||||
func (s *TSDBStore) IteratorCreator(shards []meta.ShardInfo, opt *influxql.SelectOptions) (influxql.IteratorCreator, error) {
|
||||
// Generate iterators for each node.
|
||||
ics := make([]influxql.IteratorCreator, 0)
|
||||
if err := func() error {
|
||||
for _, shard := range shards {
|
||||
ic := s.ShardIteratorCreator(shard.ID)
|
||||
if ic == nil {
|
||||
continue
|
||||
}
|
||||
ics = append(ics, ic)
|
||||
}
|
||||
|
||||
return nil
|
||||
}(); err != nil {
|
||||
influxql.IteratorCreators(ics).Close()
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return influxql.IteratorCreators(ics), nil
|
||||
}
|
||||
|
||||
func (s *TSDBStore) ShardIteratorCreator(id uint64) influxql.IteratorCreator {
|
||||
return s.ShardIteratorCreatorFn(id)
|
||||
func (s *TSDBStore) ShardGroup(ids []uint64) tsdb.ShardGroup {
|
||||
return s.ShardGroupFn(ids)
|
||||
}
|
||||
|
||||
func (s *TSDBStore) DatabaseIndex(name string) *tsdb.DatabaseIndex {
|
||||
|
@ -316,6 +318,49 @@ func (s *TSDBStore) TagValues(database string, cond influxql.Expr) ([]tsdb.TagVa
|
|||
return nil, nil
|
||||
}
|
||||
|
||||
type MockShard struct {
|
||||
Measurements []string
|
||||
FieldDimensionsFn func(measurements []string) (fields map[string]influxql.DataType, dimensions map[string]struct{}, err error)
|
||||
CreateIteratorFn func(m string, opt influxql.IteratorOptions) (influxql.Iterator, error)
|
||||
ExpandSourcesFn func(sources influxql.Sources) (influxql.Sources, error)
|
||||
}
|
||||
|
||||
func (sh *MockShard) MeasurementsByRegex(re *regexp.Regexp) []string {
|
||||
names := make([]string, 0, len(sh.Measurements))
|
||||
for _, name := range sh.Measurements {
|
||||
if re.MatchString(name) {
|
||||
names = append(names, name)
|
||||
}
|
||||
}
|
||||
return names
|
||||
}
|
||||
|
||||
func (sh *MockShard) FieldDimensions(measurements []string) (fields map[string]influxql.DataType, dimensions map[string]struct{}, err error) {
|
||||
return sh.FieldDimensionsFn(measurements)
|
||||
}
|
||||
|
||||
func (sh *MockShard) MapType(measurement, field string) influxql.DataType {
|
||||
f, d, err := sh.FieldDimensions([]string{measurement})
|
||||
if err != nil {
|
||||
return influxql.Unknown
|
||||
}
|
||||
|
||||
if typ, ok := f[field]; ok {
|
||||
return typ
|
||||
} else if _, ok := d[field]; ok {
|
||||
return influxql.Tag
|
||||
}
|
||||
return influxql.Unknown
|
||||
}
|
||||
|
||||
func (sh *MockShard) CreateIterator(measurement string, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
return sh.CreateIteratorFn(measurement, opt)
|
||||
}
|
||||
|
||||
func (sh *MockShard) ExpandSources(sources influxql.Sources) (influxql.Sources, error) {
|
||||
return sh.ExpandSourcesFn(sources)
|
||||
}
|
||||
|
||||
// MustParseQuery parses s into a query. Panic on error.
|
||||
func MustParseQuery(s string) *influxql.Query {
|
||||
q, err := influxql.ParseQuery(s)
|
||||
|
@ -334,25 +379,6 @@ func ReadAllResults(c <-chan *influxql.Result) []*influxql.Result {
|
|||
return a
|
||||
}
|
||||
|
||||
// IteratorCreator is a mockable implementation of IteratorCreator.
|
||||
type IteratorCreator struct {
|
||||
CreateIteratorFn func(opt influxql.IteratorOptions) (influxql.Iterator, error)
|
||||
FieldDimensionsFn func(sources influxql.Sources) (fields map[string]influxql.DataType, dimensions map[string]struct{}, err error)
|
||||
ExpandSourcesFn func(sources influxql.Sources) (influxql.Sources, error)
|
||||
}
|
||||
|
||||
func (ic *IteratorCreator) CreateIterator(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
return ic.CreateIteratorFn(opt)
|
||||
}
|
||||
|
||||
func (ic *IteratorCreator) FieldDimensions(sources influxql.Sources) (fields map[string]influxql.DataType, dimensions map[string]struct{}, err error) {
|
||||
return ic.FieldDimensionsFn(sources)
|
||||
}
|
||||
|
||||
func (ic *IteratorCreator) ExpandSources(sources influxql.Sources) (influxql.Sources, error) {
|
||||
return ic.ExpandSourcesFn(sources)
|
||||
}
|
||||
|
||||
// FloatIterator is a represents an iterator that reads from a slice.
|
||||
type FloatIterator struct {
|
||||
Points []influxql.FloatPoint
|
||||
|
|
|
@ -931,7 +931,7 @@ points:
|
|||
|
||||
```
|
||||
type FloatIterator interface {
|
||||
Next() *FloatPoint
|
||||
Next() (*FloatPoint, error)
|
||||
}
|
||||
```
|
||||
|
||||
|
@ -939,7 +939,7 @@ These iterators are created through the `IteratorCreator` interface:
|
|||
|
||||
```
|
||||
type IteratorCreator interface {
|
||||
CreateIterator(opt *IteratorOptions) (Iterator, error)
|
||||
CreateIterator(m *Measurement, opt IteratorOptions) (Iterator, error)
|
||||
}
|
||||
```
|
||||
|
||||
|
@ -1046,3 +1046,89 @@ Some iterators are more complex or need to be implemented at a higher level.
|
|||
For example, the `DERIVATIVE()` needs to retrieve all points for a window first
|
||||
before performing the calculation. This iterator is created by the engine itself
|
||||
and is never requested to be created by the lower levels.
|
||||
|
||||
### Subqueries
|
||||
|
||||
Subqueries are built on top of iterators. Most of the work involved in
|
||||
supporting subqueries is in organizing how data is streamed to the
|
||||
iterators that will process the data.
|
||||
|
||||
The final ordering of the stream has to output all points from one
|
||||
series before moving to the next series and it also needs to ensure
|
||||
those points are printed in order. So there are two separate concepts we
|
||||
need to consider when creating an iterator: ordering and grouping.
|
||||
|
||||
When an inner query has a different grouping than the outermost query,
|
||||
we still need to group together related points into buckets, but we do
|
||||
not have to ensure that all points from one buckets are output before
|
||||
the points in another bucket. In fact, if we do that, we will be unable
|
||||
to perform the grouping for the outer query correctly. Instead, we group
|
||||
all points by the outermost query for an interval and then, within that
|
||||
interval, we group the points for the inner query. For example, here are
|
||||
series keys and times in seconds (fields are omitted since they don't
|
||||
matter in this example):
|
||||
|
||||
cpu,host=server01 0
|
||||
cpu,host=server01 10
|
||||
cpu,host=server01 20
|
||||
cpu,host=server01 30
|
||||
cpu,host=server02 0
|
||||
cpu,host=server02 10
|
||||
cpu,host=server02 20
|
||||
cpu,host=server02 30
|
||||
|
||||
With the following query:
|
||||
|
||||
SELECT mean(max) FROM (SELECT max(value) FROM cpu GROUP BY host, time(20s)) GROUP BY time(20s)
|
||||
|
||||
The final grouping keeps all of the points together which means we need
|
||||
to group `server01` with `server02`. That means we output the points
|
||||
from the underlying engine like this:
|
||||
|
||||
cpu,host=server01 0
|
||||
cpu,host=server01 10
|
||||
cpu,host=server02 0
|
||||
cpu,host=server02 10
|
||||
cpu,host=server01 20
|
||||
cpu,host=server01 30
|
||||
cpu,host=server02 20
|
||||
cpu,host=server02 30
|
||||
|
||||
Within each one of those time buckets, we calculate the `max()` value
|
||||
for each unique host so the output stream gets transformed to look like
|
||||
this:
|
||||
|
||||
cpu,host=server01 0
|
||||
cpu,host=server02 0
|
||||
cpu,host=server01 20
|
||||
cpu,host=server02 20
|
||||
|
||||
Then we can process the `mean()` on this stream of data instead and it
|
||||
will be output in the correct order. This is true of any order of
|
||||
grouping since grouping can only go from more specific to less specific.
|
||||
|
||||
When it comes to ordering, unordered data is faster to process, but we
|
||||
always need to produce ordered data. When processing a raw query with no
|
||||
aggregates, we need to ensure data coming from the engine is ordered so
|
||||
the output is ordered. When we have an aggregate, we know one point is
|
||||
being emitted for each interval and will always produce ordered output.
|
||||
So for aggregates, we can take unordered data as the input and get
|
||||
ordered output. Any ordered data as input will always result in ordered
|
||||
data so we just need to look at how an iterator processes unordered
|
||||
data.
|
||||
|
||||
| | raw query | selector (without group by time) | selector (with group by time) | aggregator |
|
||||
|-----------------|------------------|----------------------------------|-------------------------------|----------------|
|
||||
| ordered input | ordered output | ordered output | ordered output | ordered output |
|
||||
| unordered input | unordered output | unordered output | ordered output | ordered output |
|
||||
|
||||
Since we always need ordered output, we just need to work backwards and
|
||||
determine which pattern of input gives us ordered output. If both
|
||||
ordered and unordered input produce ordered output, we prefer unordered
|
||||
input since it is faster.
|
||||
|
||||
There are also certain aggregates that require ordered input like
|
||||
`median()` and `percentile()`. These functions will explicitly request
|
||||
ordered input. It is also important to realize that selectors that are
|
||||
grouped by time are the equivalent of an aggregator. It is only
|
||||
selectors without a group by time that are different.
|
||||
|
|
359
influxql/ast.go
359
influxql/ast.go
|
@ -169,6 +169,7 @@ func (*SortField) node() {}
|
|||
func (SortFields) node() {}
|
||||
func (Sources) node() {}
|
||||
func (*StringLiteral) node() {}
|
||||
func (*SubQuery) node() {}
|
||||
func (*Target) node() {}
|
||||
func (*TimeLiteral) node() {}
|
||||
func (*VarRef) node() {}
|
||||
|
@ -319,6 +320,7 @@ type Source interface {
|
|||
}
|
||||
|
||||
func (*Measurement) source() {}
|
||||
func (*SubQuery) source() {}
|
||||
|
||||
// Sources represents a list of sources.
|
||||
type Sources []Source
|
||||
|
@ -344,6 +346,9 @@ func (a Sources) Filter(database, retentionPolicy string) []Source {
|
|||
if s.Database == database && s.RetentionPolicy == retentionPolicy {
|
||||
sources = append(sources, s)
|
||||
}
|
||||
case *SubQuery:
|
||||
filteredSources := s.Statement.Sources.Filter(database, retentionPolicy)
|
||||
sources = append(sources, filteredSources...)
|
||||
}
|
||||
}
|
||||
return sources
|
||||
|
@ -390,6 +395,20 @@ func (a Sources) String() string {
|
|||
return buf.String()
|
||||
}
|
||||
|
||||
// Measurements returns all measurements including ones embedded in subqueries.
|
||||
func (a Sources) Measurements() []*Measurement {
|
||||
mms := make([]*Measurement, 0, len(a))
|
||||
for _, src := range a {
|
||||
switch src := src.(type) {
|
||||
case *Measurement:
|
||||
mms = append(mms, src)
|
||||
case *SubQuery:
|
||||
mms = append(mms, src.Statement.Sources.Measurements()...)
|
||||
}
|
||||
}
|
||||
return mms
|
||||
}
|
||||
|
||||
// MarshalBinary encodes a list of sources to a binary format.
|
||||
func (a Sources) MarshalBinary() ([]byte, error) {
|
||||
var pb internal.Measurements
|
||||
|
@ -989,6 +1008,21 @@ func (s *SelectStatement) IsSimpleDerivative() bool {
|
|||
return false
|
||||
}
|
||||
|
||||
// HasSelector returns true if there is exactly one selector.
|
||||
func (s *SelectStatement) HasSelector() bool {
|
||||
var selector *Call
|
||||
for _, f := range s.Fields {
|
||||
if call, ok := f.Expr.(*Call); ok {
|
||||
if selector != nil || !IsSelector(call) {
|
||||
// This is an aggregate call or there is already a selector.
|
||||
return false
|
||||
}
|
||||
selector = call
|
||||
}
|
||||
}
|
||||
return selector != nil
|
||||
}
|
||||
|
||||
// TimeAscending returns true if the time field is sorted in chronological order.
|
||||
func (s *SelectStatement) TimeAscending() bool {
|
||||
return len(s.SortFields) == 0 || s.SortFields[0].Ascending
|
||||
|
@ -1053,6 +1087,8 @@ func cloneSource(s Source) Source {
|
|||
m.Regex = &RegexLiteral{Val: regexp.MustCompile(s.Regex.Val.String())}
|
||||
}
|
||||
return m
|
||||
case *SubQuery:
|
||||
return &SubQuery{Statement: s.Statement.Clone()}
|
||||
default:
|
||||
panic("unreachable")
|
||||
}
|
||||
|
@ -1062,11 +1098,20 @@ func cloneSource(s Source) Source {
|
|||
// fields are replaced with the supplied fields, and any wildcard GROUP BY fields are replaced
|
||||
// with the supplied dimensions. Any fields with no type specifier are rewritten with the
|
||||
// appropriate type.
|
||||
func (s *SelectStatement) RewriteFields(ic IteratorCreator) (*SelectStatement, error) {
|
||||
// Retrieve a list of unique field and dimensions.
|
||||
fieldSet, dimensionSet, err := ic.FieldDimensions(s.Sources)
|
||||
if err != nil {
|
||||
return s, err
|
||||
func (s *SelectStatement) RewriteFields(m FieldMapper) (*SelectStatement, error) {
|
||||
// Clone the statement so we aren't rewriting the original.
|
||||
other := s.Clone()
|
||||
|
||||
// Iterate through the sources and rewrite any subqueries first.
|
||||
for _, src := range other.Sources {
|
||||
switch src := src.(type) {
|
||||
case *SubQuery:
|
||||
stmt, err := src.Statement.RewriteFields(m)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
src.Statement = stmt
|
||||
}
|
||||
}
|
||||
|
||||
// Rewrite all variable references in the fields with their types if one
|
||||
|
@ -1077,28 +1122,31 @@ func (s *SelectStatement) RewriteFields(ic IteratorCreator) (*SelectStatement, e
|
|||
return
|
||||
}
|
||||
|
||||
if typ, ok := fieldSet[ref.Val]; ok {
|
||||
ref.Type = typ
|
||||
} else if ref.Type != AnyField {
|
||||
if _, ok := dimensionSet[ref.Val]; ok {
|
||||
ref.Type = Tag
|
||||
}
|
||||
typ := EvalType(ref, other.Sources, m)
|
||||
if typ == Tag && ref.Type == AnyField {
|
||||
return
|
||||
}
|
||||
ref.Type = typ
|
||||
}
|
||||
WalkFunc(s.Fields, rewrite)
|
||||
WalkFunc(s.Condition, rewrite)
|
||||
WalkFunc(other.Fields, rewrite)
|
||||
WalkFunc(other.Condition, rewrite)
|
||||
|
||||
// Ignore if there are no wildcards.
|
||||
hasFieldWildcard := s.HasFieldWildcard()
|
||||
hasDimensionWildcard := s.HasDimensionWildcard()
|
||||
hasFieldWildcard := other.HasFieldWildcard()
|
||||
hasDimensionWildcard := other.HasDimensionWildcard()
|
||||
if !hasFieldWildcard && !hasDimensionWildcard {
|
||||
return s, nil
|
||||
return other, nil
|
||||
}
|
||||
|
||||
fieldSet, dimensionSet, err := FieldDimensions(other.Sources, m)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// If there are no dimension wildcards then merge dimensions to fields.
|
||||
if !hasDimensionWildcard {
|
||||
// Remove the dimensions present in the group by so they don't get added as fields.
|
||||
for _, d := range s.Dimensions {
|
||||
for _, d := range other.Dimensions {
|
||||
switch expr := d.Expr.(type) {
|
||||
case *VarRef:
|
||||
if _, ok := dimensionSet[expr.Val]; ok {
|
||||
|
@ -1125,13 +1173,11 @@ func (s *SelectStatement) RewriteFields(ic IteratorCreator) (*SelectStatement, e
|
|||
}
|
||||
dimensions := stringSetSlice(dimensionSet)
|
||||
|
||||
other := s.Clone()
|
||||
|
||||
// Rewrite all wildcard query fields
|
||||
if hasFieldWildcard {
|
||||
// Allocate a slice assuming there is exactly one wildcard for efficiency.
|
||||
rwFields := make(Fields, 0, len(s.Fields)+len(fields)-1)
|
||||
for _, f := range s.Fields {
|
||||
rwFields := make(Fields, 0, len(other.Fields)+len(fields)-1)
|
||||
for _, f := range other.Fields {
|
||||
switch expr := f.Expr.(type) {
|
||||
case *Wildcard:
|
||||
for _, ref := range fields {
|
||||
|
@ -1174,7 +1220,7 @@ func (s *SelectStatement) RewriteFields(ic IteratorCreator) (*SelectStatement, e
|
|||
switch expr := call.Args[0].(type) {
|
||||
case *Wildcard:
|
||||
if expr.Type == TAG {
|
||||
return s, fmt.Errorf("unable to use tag wildcard in %s()", call.Name)
|
||||
return nil, fmt.Errorf("unable to use tag wildcard in %s()", call.Name)
|
||||
}
|
||||
case *RegexLiteral:
|
||||
re = expr.Val
|
||||
|
@ -1226,8 +1272,8 @@ func (s *SelectStatement) RewriteFields(ic IteratorCreator) (*SelectStatement, e
|
|||
// Rewrite all wildcard GROUP BY fields
|
||||
if hasDimensionWildcard {
|
||||
// Allocate a slice assuming there is exactly one wildcard for efficiency.
|
||||
rwDimensions := make(Dimensions, 0, len(s.Dimensions)+len(dimensions)-1)
|
||||
for _, d := range s.Dimensions {
|
||||
rwDimensions := make(Dimensions, 0, len(other.Dimensions)+len(dimensions)-1)
|
||||
for _, d := range other.Dimensions {
|
||||
switch expr := d.Expr.(type) {
|
||||
case *Wildcard:
|
||||
for _, name := range dimensions {
|
||||
|
@ -1369,6 +1415,41 @@ func (s *SelectStatement) RewriteTimeFields() {
|
|||
}
|
||||
}
|
||||
|
||||
// RewriteTimeCondition adds time constraints to aggregate queries.
|
||||
func (s *SelectStatement) RewriteTimeCondition(now time.Time) error {
|
||||
interval, err := s.GroupByInterval()
|
||||
if err != nil {
|
||||
return err
|
||||
} else if interval > 0 && s.Condition != nil {
|
||||
_, tmax, err := TimeRange(s.Condition)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if tmax.IsZero() {
|
||||
s.Condition = &BinaryExpr{
|
||||
Op: AND,
|
||||
LHS: s.Condition,
|
||||
RHS: &BinaryExpr{
|
||||
Op: LTE,
|
||||
LHS: &VarRef{Val: "time"},
|
||||
RHS: &TimeLiteral{Val: now},
|
||||
},
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
for _, source := range s.Sources {
|
||||
switch source := source.(type) {
|
||||
case *SubQuery:
|
||||
if err := source.Statement.RewriteTimeCondition(now); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// ColumnNames will walk all fields and functions and return the appropriate field names for the select statement
|
||||
// while maintaining order of the field names.
|
||||
func (s *SelectStatement) ColumnNames() []string {
|
||||
|
@ -1439,6 +1520,45 @@ func (s *SelectStatement) ColumnNames() []string {
|
|||
return columnNames
|
||||
}
|
||||
|
||||
// FieldExprByName returns the expression that matches the field name and the
|
||||
// index where this was found. If the name matches one of the arguments to
|
||||
// "top" or "bottom", the variable reference inside of the function is returned
|
||||
// and the index is of the function call rather than the variable reference.
|
||||
// If no expression is found, -1 is returned for the index and the expression
|
||||
// will be nil.
|
||||
func (s *SelectStatement) FieldExprByName(name string) (int, Expr) {
|
||||
for i, f := range s.Fields {
|
||||
if f.Name() == name {
|
||||
return i, f.Expr
|
||||
} else if call, ok := f.Expr.(*Call); ok && (call.Name == "top" || call.Name == "bottom") && len(call.Args) > 2 {
|
||||
for _, arg := range call.Args[1 : len(call.Args)-1] {
|
||||
if arg, ok := arg.(*VarRef); ok && arg.Val == name {
|
||||
return i, arg
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return -1, nil
|
||||
}
|
||||
|
||||
// Reduce calls the Reduce function on the different components of the
|
||||
// SelectStatement to reduce the statement.
|
||||
func (s *SelectStatement) Reduce(valuer Valuer) *SelectStatement {
|
||||
stmt := s.Clone()
|
||||
stmt.Condition = Reduce(stmt.Condition, valuer)
|
||||
for _, d := range stmt.Dimensions {
|
||||
d.Expr = Reduce(d.Expr, valuer)
|
||||
}
|
||||
|
||||
for _, source := range stmt.Sources {
|
||||
switch source := source.(type) {
|
||||
case *SubQuery:
|
||||
source.Statement = source.Statement.Reduce(valuer)
|
||||
}
|
||||
}
|
||||
return stmt
|
||||
}
|
||||
|
||||
// HasTimeFieldSpecified will walk all fields and determine if the user explicitly asked for time.
|
||||
// This is needed to determine re-write behaviors for functions like TOP and BOTTOM.
|
||||
func (s *SelectStatement) HasTimeFieldSpecified() bool {
|
||||
|
@ -1923,8 +2043,41 @@ func (s *SelectStatement) validateAggregates(tr targetRequirement) error {
|
|||
|
||||
// If we have an aggregate function with a group by time without a where clause, it's an invalid statement
|
||||
if tr == targetNotRequired { // ignore create continuous query statements
|
||||
if !s.IsRawQuery && groupByDuration > 0 && !HasTimeExpr(s.Condition) {
|
||||
return fmt.Errorf("aggregate functions with GROUP BY time require a WHERE time clause")
|
||||
if err := s.validateGroupByInterval(); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// validateGroupByInterval ensures that any select statements that have a group
|
||||
// by interval either have a time expression limiting the time range or have a
|
||||
// parent query that does that.
|
||||
func (s *SelectStatement) validateGroupByInterval() error {
|
||||
// If we have a time expression, we and all subqueries are fine.
|
||||
if HasTimeExpr(s.Condition) {
|
||||
return nil
|
||||
}
|
||||
|
||||
// Check if this is not a raw query and if the group by duration exists.
|
||||
// If these are true, then we have an error.
|
||||
interval, err := s.GroupByInterval()
|
||||
if err != nil {
|
||||
return err
|
||||
} else if !s.IsRawQuery && interval > 0 {
|
||||
return fmt.Errorf("aggregate functions with GROUP BY time require a WHERE time clause")
|
||||
}
|
||||
|
||||
// Validate the subqueries. If we have a time expression in this select
|
||||
// statement, we don't need to do this because parent time ranges propagate
|
||||
// to children. So we only execute this when there is no time condition in
|
||||
// the parent.
|
||||
for _, source := range s.Sources {
|
||||
switch source := source.(type) {
|
||||
case *SubQuery:
|
||||
if err := source.Statement.validateGroupByInterval(); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
}
|
||||
return nil
|
||||
|
@ -3277,6 +3430,16 @@ func decodeMeasurement(pb *internal.Measurement) (*Measurement, error) {
|
|||
return mm, nil
|
||||
}
|
||||
|
||||
// SubQuery is a source with a SelectStatement as the backing store.
|
||||
type SubQuery struct {
|
||||
Statement *SelectStatement
|
||||
}
|
||||
|
||||
// String returns a string representation of the subquery.
|
||||
func (s *SubQuery) String() string {
|
||||
return fmt.Sprintf("(%s)", s.Statement.String())
|
||||
}
|
||||
|
||||
// VarRef represents a reference to a variable.
|
||||
type VarRef struct {
|
||||
Val string
|
||||
|
@ -3914,6 +4077,9 @@ func Walk(v Visitor, node Node) {
|
|||
Walk(v, s)
|
||||
}
|
||||
|
||||
case *SubQuery:
|
||||
Walk(v, n.Statement)
|
||||
|
||||
case Statements:
|
||||
for _, s := range n {
|
||||
Walk(v, s)
|
||||
|
@ -3959,6 +4125,9 @@ func Rewrite(r Rewriter, node Node) Node {
|
|||
n.Sources = Rewrite(r, n.Sources).(Sources)
|
||||
n.Condition = Rewrite(r, n.Condition).(Expr)
|
||||
|
||||
case *SubQuery:
|
||||
n.Statement = Rewrite(r, n.Statement).(*SelectStatement)
|
||||
|
||||
case Fields:
|
||||
for i, f := range n {
|
||||
n[i] = Rewrite(r, f).(*Field)
|
||||
|
@ -4219,6 +4388,132 @@ func EvalBool(expr Expr, m map[string]interface{}) bool {
|
|||
return v
|
||||
}
|
||||
|
||||
// TypeMapper maps a data type to the measurement and field.
|
||||
type TypeMapper interface {
|
||||
MapType(measurement *Measurement, field string) DataType
|
||||
}
|
||||
|
||||
type nilTypeMapper struct{}
|
||||
|
||||
func (nilTypeMapper) MapType(*Measurement, string) DataType { return Unknown }
|
||||
|
||||
// EvalType evaluates the expression's type.
|
||||
func EvalType(expr Expr, sources Sources, typmap TypeMapper) DataType {
|
||||
if typmap == nil {
|
||||
typmap = nilTypeMapper{}
|
||||
}
|
||||
|
||||
switch expr := expr.(type) {
|
||||
case *VarRef:
|
||||
// If this variable already has an assigned type, just use that.
|
||||
if expr.Type != Unknown && expr.Type != AnyField {
|
||||
return expr.Type
|
||||
}
|
||||
|
||||
var typ DataType
|
||||
for _, src := range sources {
|
||||
switch src := src.(type) {
|
||||
case *Measurement:
|
||||
t := typmap.MapType(src, expr.Val)
|
||||
if typ == Unknown || t < typ {
|
||||
typ = t
|
||||
}
|
||||
case *SubQuery:
|
||||
_, e := src.Statement.FieldExprByName(expr.Val)
|
||||
if e != nil {
|
||||
t := EvalType(e, src.Statement.Sources, typmap)
|
||||
if typ == Unknown || t < typ {
|
||||
typ = t
|
||||
}
|
||||
}
|
||||
|
||||
if typ == Unknown {
|
||||
for _, d := range src.Statement.Dimensions {
|
||||
if d, ok := d.Expr.(*VarRef); ok && expr.Val == d.Val {
|
||||
typ = Tag
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return typ
|
||||
case *Call:
|
||||
switch expr.Name {
|
||||
case "mean", "median":
|
||||
return Float
|
||||
case "count":
|
||||
return Integer
|
||||
default:
|
||||
return EvalType(expr.Args[0], sources, typmap)
|
||||
}
|
||||
case *ParenExpr:
|
||||
return EvalType(expr, sources, typmap)
|
||||
case *NumberLiteral:
|
||||
return Float
|
||||
case *IntegerLiteral:
|
||||
return Integer
|
||||
case *StringLiteral:
|
||||
return String
|
||||
case *BooleanLiteral:
|
||||
return Boolean
|
||||
case *BinaryExpr:
|
||||
lhs := EvalType(expr.LHS, sources, typmap)
|
||||
rhs := EvalType(expr.RHS, sources, typmap)
|
||||
if lhs != Unknown && rhs != Unknown {
|
||||
if lhs < rhs {
|
||||
return lhs
|
||||
} else {
|
||||
return rhs
|
||||
}
|
||||
} else if lhs != Unknown {
|
||||
return lhs
|
||||
} else {
|
||||
return rhs
|
||||
}
|
||||
}
|
||||
return Unknown
|
||||
}
|
||||
|
||||
func FieldDimensions(sources Sources, m FieldMapper) (fields map[string]DataType, dimensions map[string]struct{}, err error) {
|
||||
fields = make(map[string]DataType)
|
||||
dimensions = make(map[string]struct{})
|
||||
|
||||
for _, src := range sources {
|
||||
switch src := src.(type) {
|
||||
case *Measurement:
|
||||
f, d, err := m.FieldDimensions(src)
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
|
||||
for k, typ := range f {
|
||||
if _, ok := fields[k]; typ != Unknown && (!ok || typ < fields[k]) {
|
||||
fields[k] = typ
|
||||
}
|
||||
}
|
||||
for k := range d {
|
||||
dimensions[k] = struct{}{}
|
||||
}
|
||||
case *SubQuery:
|
||||
for _, f := range src.Statement.Fields {
|
||||
k := f.Name()
|
||||
typ := EvalType(f.Expr, src.Statement.Sources, m)
|
||||
|
||||
if _, ok := fields[k]; typ != Unknown && (!ok || typ < fields[k]) {
|
||||
fields[k] = typ
|
||||
}
|
||||
}
|
||||
for _, d := range src.Statement.Dimensions {
|
||||
switch d := d.Expr.(type) {
|
||||
case *VarRef:
|
||||
dimensions[d.Val] = struct{}{}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
// Reduce evaluates expr using the available values in valuer.
|
||||
// References that don't exist in valuer are ignored.
|
||||
func Reduce(expr Expr, valuer Valuer) Expr {
|
||||
|
@ -4245,6 +4540,8 @@ func reduce(expr Expr, valuer Valuer) Expr {
|
|||
return reduceParenExpr(expr, valuer)
|
||||
case *VarRef:
|
||||
return reduceVarRef(expr, valuer)
|
||||
case *nilLiteral:
|
||||
return expr
|
||||
default:
|
||||
return CloneExpr(expr)
|
||||
}
|
||||
|
@ -4771,3 +5068,13 @@ func (v *containsVarRefVisitor) Visit(n Node) Visitor {
|
|||
}
|
||||
return v
|
||||
}
|
||||
|
||||
func IsSelector(expr Expr) bool {
|
||||
if call, ok := expr.(*Call); ok {
|
||||
switch call.Name {
|
||||
case "first", "last", "min", "max", "percentile", "sample", "top", "bottom":
|
||||
return true
|
||||
}
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
|
|
@ -407,6 +407,12 @@ func TestSelectStatement_RewriteFields(t *testing.T) {
|
|||
stmt: `SELECT mean(/1/) FROM cpu`,
|
||||
rewrite: `SELECT mean(value1::float) AS mean_value1 FROM cpu`,
|
||||
},
|
||||
|
||||
// Rewrite subquery
|
||||
{
|
||||
stmt: `SELECT * FROM (SELECT mean(value1) FROM cpu GROUP BY host) GROUP BY *`,
|
||||
rewrite: `SELECT mean::float FROM (SELECT mean(value1::float) FROM cpu GROUP BY host) GROUP BY host`,
|
||||
},
|
||||
}
|
||||
|
||||
for i, tt := range tests {
|
||||
|
@ -417,9 +423,8 @@ func TestSelectStatement_RewriteFields(t *testing.T) {
|
|||
}
|
||||
|
||||
var ic IteratorCreator
|
||||
ic.FieldDimensionsFn = func(sources influxql.Sources) (fields map[string]influxql.DataType, dimensions map[string]struct{}, err error) {
|
||||
source := sources[0].(*influxql.Measurement)
|
||||
switch source.Name {
|
||||
ic.FieldDimensionsFn = func(m *influxql.Measurement) (fields map[string]influxql.DataType, dimensions map[string]struct{}, err error) {
|
||||
switch m.Name {
|
||||
case "cpu":
|
||||
fields = map[string]influxql.DataType{
|
||||
"value1": influxql.Float,
|
||||
|
@ -1071,6 +1076,80 @@ func TestEval(t *testing.T) {
|
|||
}
|
||||
}
|
||||
|
||||
type EvalFixture map[string]map[string]influxql.DataType
|
||||
|
||||
func (e EvalFixture) MapType(measurement *influxql.Measurement, field string) influxql.DataType {
|
||||
m := e[measurement.Name]
|
||||
if m == nil {
|
||||
return influxql.Unknown
|
||||
}
|
||||
return m[field]
|
||||
}
|
||||
|
||||
func TestEvalType(t *testing.T) {
|
||||
for i, tt := range []struct {
|
||||
name string
|
||||
in string
|
||||
typ influxql.DataType
|
||||
data EvalFixture
|
||||
}{
|
||||
{
|
||||
name: `a single data type`,
|
||||
in: `min(value)`,
|
||||
typ: influxql.Integer,
|
||||
data: EvalFixture{
|
||||
"cpu": map[string]influxql.DataType{
|
||||
"value": influxql.Integer,
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
name: `multiple data types`,
|
||||
in: `min(value)`,
|
||||
typ: influxql.Integer,
|
||||
data: EvalFixture{
|
||||
"cpu": map[string]influxql.DataType{
|
||||
"value": influxql.Integer,
|
||||
},
|
||||
"mem": map[string]influxql.DataType{
|
||||
"value": influxql.String,
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
name: `count() with a float`,
|
||||
in: `count(value)`,
|
||||
typ: influxql.Integer,
|
||||
data: EvalFixture{
|
||||
"cpu": map[string]influxql.DataType{
|
||||
"value": influxql.Float,
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
name: `mean() with an integer`,
|
||||
in: `mean(value)`,
|
||||
typ: influxql.Float,
|
||||
data: EvalFixture{
|
||||
"cpu": map[string]influxql.DataType{
|
||||
"value": influxql.Integer,
|
||||
},
|
||||
},
|
||||
},
|
||||
} {
|
||||
sources := make([]influxql.Source, 0, len(tt.data))
|
||||
for src := range tt.data {
|
||||
sources = append(sources, &influxql.Measurement{Name: src})
|
||||
}
|
||||
|
||||
expr := influxql.MustParseExpr(tt.in)
|
||||
typ := influxql.EvalType(expr, sources, tt.data)
|
||||
if typ != tt.typ {
|
||||
t.Errorf("%d. %s: unexpected type:\n\nexp=%#v\n\ngot=%#v\n\n", i, tt.name, tt.typ, typ)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Ensure an expression can be reduced.
|
||||
func TestReduce(t *testing.T) {
|
||||
now := mustParseTime("2000-01-01T00:00:00Z")
|
||||
|
|
|
@ -67,25 +67,25 @@ func newCountIterator(input Iterator, opt IteratorOptions) (Iterator, error) {
|
|||
fn := NewFloatFuncIntegerReducer(FloatCountReduce, &IntegerPoint{Value: 0, Time: ZeroTime})
|
||||
return fn, fn
|
||||
}
|
||||
return &floatReduceIntegerIterator{input: newBufFloatIterator(input), opt: opt, create: createFn}, nil
|
||||
return newFloatReduceIntegerIterator(input, opt, createFn), nil
|
||||
case IntegerIterator:
|
||||
createFn := func() (IntegerPointAggregator, IntegerPointEmitter) {
|
||||
fn := NewIntegerFuncReducer(IntegerCountReduce, &IntegerPoint{Value: 0, Time: ZeroTime})
|
||||
return fn, fn
|
||||
}
|
||||
return &integerReduceIntegerIterator{input: newBufIntegerIterator(input), opt: opt, create: createFn}, nil
|
||||
return newIntegerReduceIntegerIterator(input, opt, createFn), nil
|
||||
case StringIterator:
|
||||
createFn := func() (StringPointAggregator, IntegerPointEmitter) {
|
||||
fn := NewStringFuncIntegerReducer(StringCountReduce, &IntegerPoint{Value: 0, Time: ZeroTime})
|
||||
return fn, fn
|
||||
}
|
||||
return &stringReduceIntegerIterator{input: newBufStringIterator(input), opt: opt, create: createFn}, nil
|
||||
return newStringReduceIntegerIterator(input, opt, createFn), nil
|
||||
case BooleanIterator:
|
||||
createFn := func() (BooleanPointAggregator, IntegerPointEmitter) {
|
||||
fn := NewBooleanFuncIntegerReducer(BooleanCountReduce, &IntegerPoint{Value: 0, Time: ZeroTime})
|
||||
return fn, fn
|
||||
}
|
||||
return &booleanReduceIntegerIterator{input: newBufBooleanIterator(input), opt: opt, create: createFn}, nil
|
||||
return newBooleanReduceIntegerIterator(input, opt, createFn), nil
|
||||
default:
|
||||
return nil, fmt.Errorf("unsupported count iterator type: %T", input)
|
||||
}
|
||||
|
@ -131,19 +131,19 @@ func newMinIterator(input Iterator, opt IteratorOptions) (Iterator, error) {
|
|||
fn := NewFloatFuncReducer(FloatMinReduce, nil)
|
||||
return fn, fn
|
||||
}
|
||||
return &floatReduceFloatIterator{input: newBufFloatIterator(input), opt: opt, create: createFn}, nil
|
||||
return newFloatReduceFloatIterator(input, opt, createFn), nil
|
||||
case IntegerIterator:
|
||||
createFn := func() (IntegerPointAggregator, IntegerPointEmitter) {
|
||||
fn := NewIntegerFuncReducer(IntegerMinReduce, nil)
|
||||
return fn, fn
|
||||
}
|
||||
return &integerReduceIntegerIterator{input: newBufIntegerIterator(input), opt: opt, create: createFn}, nil
|
||||
return newIntegerReduceIntegerIterator(input, opt, createFn), nil
|
||||
case BooleanIterator:
|
||||
createFn := func() (BooleanPointAggregator, BooleanPointEmitter) {
|
||||
fn := NewBooleanFuncReducer(BooleanMinReduce, nil)
|
||||
return fn, fn
|
||||
}
|
||||
return &booleanReduceBooleanIterator{input: newBufBooleanIterator(input), opt: opt, create: createFn}, nil
|
||||
return newBooleanReduceBooleanIterator(input, opt, createFn), nil
|
||||
default:
|
||||
return nil, fmt.Errorf("unsupported min iterator type: %T", input)
|
||||
}
|
||||
|
@ -152,7 +152,7 @@ func newMinIterator(input Iterator, opt IteratorOptions) (Iterator, error) {
|
|||
// FloatMinReduce returns the minimum value between prev & curr.
|
||||
func FloatMinReduce(prev, curr *FloatPoint) (int64, float64, []interface{}) {
|
||||
if prev == nil || curr.Value < prev.Value || (curr.Value == prev.Value && curr.Time < prev.Time) {
|
||||
return curr.Time, curr.Value, curr.Aux
|
||||
return curr.Time, curr.Value, cloneAux(curr.Aux)
|
||||
}
|
||||
return prev.Time, prev.Value, prev.Aux
|
||||
}
|
||||
|
@ -160,7 +160,7 @@ func FloatMinReduce(prev, curr *FloatPoint) (int64, float64, []interface{}) {
|
|||
// IntegerMinReduce returns the minimum value between prev & curr.
|
||||
func IntegerMinReduce(prev, curr *IntegerPoint) (int64, int64, []interface{}) {
|
||||
if prev == nil || curr.Value < prev.Value || (curr.Value == prev.Value && curr.Time < prev.Time) {
|
||||
return curr.Time, curr.Value, curr.Aux
|
||||
return curr.Time, curr.Value, cloneAux(curr.Aux)
|
||||
}
|
||||
return prev.Time, prev.Value, prev.Aux
|
||||
}
|
||||
|
@ -168,7 +168,7 @@ func IntegerMinReduce(prev, curr *IntegerPoint) (int64, int64, []interface{}) {
|
|||
// BooleanMinReduce returns the minimum value between prev & curr.
|
||||
func BooleanMinReduce(prev, curr *BooleanPoint) (int64, bool, []interface{}) {
|
||||
if prev == nil || (curr.Value != prev.Value && !curr.Value) || (curr.Value == prev.Value && curr.Time < prev.Time) {
|
||||
return curr.Time, curr.Value, curr.Aux
|
||||
return curr.Time, curr.Value, cloneAux(curr.Aux)
|
||||
}
|
||||
return prev.Time, prev.Value, prev.Aux
|
||||
}
|
||||
|
@ -181,19 +181,19 @@ func newMaxIterator(input Iterator, opt IteratorOptions) (Iterator, error) {
|
|||
fn := NewFloatFuncReducer(FloatMaxReduce, nil)
|
||||
return fn, fn
|
||||
}
|
||||
return &floatReduceFloatIterator{input: newBufFloatIterator(input), opt: opt, create: createFn}, nil
|
||||
return newFloatReduceFloatIterator(input, opt, createFn), nil
|
||||
case IntegerIterator:
|
||||
createFn := func() (IntegerPointAggregator, IntegerPointEmitter) {
|
||||
fn := NewIntegerFuncReducer(IntegerMaxReduce, nil)
|
||||
return fn, fn
|
||||
}
|
||||
return &integerReduceIntegerIterator{input: newBufIntegerIterator(input), opt: opt, create: createFn}, nil
|
||||
return newIntegerReduceIntegerIterator(input, opt, createFn), nil
|
||||
case BooleanIterator:
|
||||
createFn := func() (BooleanPointAggregator, BooleanPointEmitter) {
|
||||
fn := NewBooleanFuncReducer(BooleanMaxReduce, nil)
|
||||
return fn, fn
|
||||
}
|
||||
return &booleanReduceBooleanIterator{input: newBufBooleanIterator(input), opt: opt, create: createFn}, nil
|
||||
return newBooleanReduceBooleanIterator(input, opt, createFn), nil
|
||||
default:
|
||||
return nil, fmt.Errorf("unsupported max iterator type: %T", input)
|
||||
}
|
||||
|
@ -202,7 +202,7 @@ func newMaxIterator(input Iterator, opt IteratorOptions) (Iterator, error) {
|
|||
// FloatMaxReduce returns the maximum value between prev & curr.
|
||||
func FloatMaxReduce(prev, curr *FloatPoint) (int64, float64, []interface{}) {
|
||||
if prev == nil || curr.Value > prev.Value || (curr.Value == prev.Value && curr.Time < prev.Time) {
|
||||
return curr.Time, curr.Value, curr.Aux
|
||||
return curr.Time, curr.Value, cloneAux(curr.Aux)
|
||||
}
|
||||
return prev.Time, prev.Value, prev.Aux
|
||||
}
|
||||
|
@ -210,7 +210,7 @@ func FloatMaxReduce(prev, curr *FloatPoint) (int64, float64, []interface{}) {
|
|||
// IntegerMaxReduce returns the maximum value between prev & curr.
|
||||
func IntegerMaxReduce(prev, curr *IntegerPoint) (int64, int64, []interface{}) {
|
||||
if prev == nil || curr.Value > prev.Value || (curr.Value == prev.Value && curr.Time < prev.Time) {
|
||||
return curr.Time, curr.Value, curr.Aux
|
||||
return curr.Time, curr.Value, cloneAux(curr.Aux)
|
||||
}
|
||||
return prev.Time, prev.Value, prev.Aux
|
||||
}
|
||||
|
@ -218,7 +218,7 @@ func IntegerMaxReduce(prev, curr *IntegerPoint) (int64, int64, []interface{}) {
|
|||
// BooleanMaxReduce returns the minimum value between prev & curr.
|
||||
func BooleanMaxReduce(prev, curr *BooleanPoint) (int64, bool, []interface{}) {
|
||||
if prev == nil || (curr.Value != prev.Value && curr.Value) || (curr.Value == prev.Value && curr.Time < prev.Time) {
|
||||
return curr.Time, curr.Value, curr.Aux
|
||||
return curr.Time, curr.Value, cloneAux(curr.Aux)
|
||||
}
|
||||
return prev.Time, prev.Value, prev.Aux
|
||||
}
|
||||
|
@ -231,13 +231,13 @@ func newSumIterator(input Iterator, opt IteratorOptions) (Iterator, error) {
|
|||
fn := NewFloatFuncReducer(FloatSumReduce, &FloatPoint{Value: 0, Time: ZeroTime})
|
||||
return fn, fn
|
||||
}
|
||||
return &floatReduceFloatIterator{input: newBufFloatIterator(input), opt: opt, create: createFn}, nil
|
||||
return newFloatReduceFloatIterator(input, opt, createFn), nil
|
||||
case IntegerIterator:
|
||||
createFn := func() (IntegerPointAggregator, IntegerPointEmitter) {
|
||||
fn := NewIntegerFuncReducer(IntegerSumReduce, &IntegerPoint{Value: 0, Time: ZeroTime})
|
||||
return fn, fn
|
||||
}
|
||||
return &integerReduceIntegerIterator{input: newBufIntegerIterator(input), opt: opt, create: createFn}, nil
|
||||
return newIntegerReduceIntegerIterator(input, opt, createFn), nil
|
||||
default:
|
||||
return nil, fmt.Errorf("unsupported sum iterator type: %T", input)
|
||||
}
|
||||
|
@ -267,25 +267,25 @@ func newFirstIterator(input Iterator, opt IteratorOptions) (Iterator, error) {
|
|||
fn := NewFloatFuncReducer(FloatFirstReduce, nil)
|
||||
return fn, fn
|
||||
}
|
||||
return &floatReduceFloatIterator{input: newBufFloatIterator(input), opt: opt, create: createFn}, nil
|
||||
return newFloatReduceFloatIterator(input, opt, createFn), nil
|
||||
case IntegerIterator:
|
||||
createFn := func() (IntegerPointAggregator, IntegerPointEmitter) {
|
||||
fn := NewIntegerFuncReducer(IntegerFirstReduce, nil)
|
||||
return fn, fn
|
||||
}
|
||||
return &integerReduceIntegerIterator{input: newBufIntegerIterator(input), opt: opt, create: createFn}, nil
|
||||
return newIntegerReduceIntegerIterator(input, opt, createFn), nil
|
||||
case StringIterator:
|
||||
createFn := func() (StringPointAggregator, StringPointEmitter) {
|
||||
fn := NewStringFuncReducer(StringFirstReduce, nil)
|
||||
return fn, fn
|
||||
}
|
||||
return &stringReduceStringIterator{input: newBufStringIterator(input), opt: opt, create: createFn}, nil
|
||||
return newStringReduceStringIterator(input, opt, createFn), nil
|
||||
case BooleanIterator:
|
||||
createFn := func() (BooleanPointAggregator, BooleanPointEmitter) {
|
||||
fn := NewBooleanFuncReducer(BooleanFirstReduce, nil)
|
||||
return fn, fn
|
||||
}
|
||||
return &booleanReduceBooleanIterator{input: newBufBooleanIterator(input), opt: opt, create: createFn}, nil
|
||||
return newBooleanReduceBooleanIterator(input, opt, createFn), nil
|
||||
default:
|
||||
return nil, fmt.Errorf("unsupported first iterator type: %T", input)
|
||||
}
|
||||
|
@ -294,7 +294,7 @@ func newFirstIterator(input Iterator, opt IteratorOptions) (Iterator, error) {
|
|||
// FloatFirstReduce returns the first point sorted by time.
|
||||
func FloatFirstReduce(prev, curr *FloatPoint) (int64, float64, []interface{}) {
|
||||
if prev == nil || curr.Time < prev.Time || (curr.Time == prev.Time && curr.Value > prev.Value) {
|
||||
return curr.Time, curr.Value, curr.Aux
|
||||
return curr.Time, curr.Value, cloneAux(curr.Aux)
|
||||
}
|
||||
return prev.Time, prev.Value, prev.Aux
|
||||
}
|
||||
|
@ -302,7 +302,7 @@ func FloatFirstReduce(prev, curr *FloatPoint) (int64, float64, []interface{}) {
|
|||
// IntegerFirstReduce returns the first point sorted by time.
|
||||
func IntegerFirstReduce(prev, curr *IntegerPoint) (int64, int64, []interface{}) {
|
||||
if prev == nil || curr.Time < prev.Time || (curr.Time == prev.Time && curr.Value > prev.Value) {
|
||||
return curr.Time, curr.Value, curr.Aux
|
||||
return curr.Time, curr.Value, cloneAux(curr.Aux)
|
||||
}
|
||||
return prev.Time, prev.Value, prev.Aux
|
||||
}
|
||||
|
@ -310,7 +310,7 @@ func IntegerFirstReduce(prev, curr *IntegerPoint) (int64, int64, []interface{})
|
|||
// StringFirstReduce returns the first point sorted by time.
|
||||
func StringFirstReduce(prev, curr *StringPoint) (int64, string, []interface{}) {
|
||||
if prev == nil || curr.Time < prev.Time || (curr.Time == prev.Time && curr.Value > prev.Value) {
|
||||
return curr.Time, curr.Value, curr.Aux
|
||||
return curr.Time, curr.Value, cloneAux(curr.Aux)
|
||||
}
|
||||
return prev.Time, prev.Value, prev.Aux
|
||||
}
|
||||
|
@ -318,7 +318,7 @@ func StringFirstReduce(prev, curr *StringPoint) (int64, string, []interface{}) {
|
|||
// BooleanFirstReduce returns the first point sorted by time.
|
||||
func BooleanFirstReduce(prev, curr *BooleanPoint) (int64, bool, []interface{}) {
|
||||
if prev == nil || curr.Time < prev.Time || (curr.Time == prev.Time && !curr.Value && prev.Value) {
|
||||
return curr.Time, curr.Value, curr.Aux
|
||||
return curr.Time, curr.Value, cloneAux(curr.Aux)
|
||||
}
|
||||
return prev.Time, prev.Value, prev.Aux
|
||||
}
|
||||
|
@ -331,25 +331,25 @@ func newLastIterator(input Iterator, opt IteratorOptions) (Iterator, error) {
|
|||
fn := NewFloatFuncReducer(FloatLastReduce, nil)
|
||||
return fn, fn
|
||||
}
|
||||
return &floatReduceFloatIterator{input: newBufFloatIterator(input), opt: opt, create: createFn}, nil
|
||||
return newFloatReduceFloatIterator(input, opt, createFn), nil
|
||||
case IntegerIterator:
|
||||
createFn := func() (IntegerPointAggregator, IntegerPointEmitter) {
|
||||
fn := NewIntegerFuncReducer(IntegerLastReduce, nil)
|
||||
return fn, fn
|
||||
}
|
||||
return &integerReduceIntegerIterator{input: newBufIntegerIterator(input), opt: opt, create: createFn}, nil
|
||||
return newIntegerReduceIntegerIterator(input, opt, createFn), nil
|
||||
case StringIterator:
|
||||
createFn := func() (StringPointAggregator, StringPointEmitter) {
|
||||
fn := NewStringFuncReducer(StringLastReduce, nil)
|
||||
return fn, fn
|
||||
}
|
||||
return &stringReduceStringIterator{input: newBufStringIterator(input), opt: opt, create: createFn}, nil
|
||||
return newStringReduceStringIterator(input, opt, createFn), nil
|
||||
case BooleanIterator:
|
||||
createFn := func() (BooleanPointAggregator, BooleanPointEmitter) {
|
||||
fn := NewBooleanFuncReducer(BooleanLastReduce, nil)
|
||||
return fn, fn
|
||||
}
|
||||
return &booleanReduceBooleanIterator{input: newBufBooleanIterator(input), opt: opt, create: createFn}, nil
|
||||
return newBooleanReduceBooleanIterator(input, opt, createFn), nil
|
||||
default:
|
||||
return nil, fmt.Errorf("unsupported last iterator type: %T", input)
|
||||
}
|
||||
|
@ -358,7 +358,7 @@ func newLastIterator(input Iterator, opt IteratorOptions) (Iterator, error) {
|
|||
// FloatLastReduce returns the last point sorted by time.
|
||||
func FloatLastReduce(prev, curr *FloatPoint) (int64, float64, []interface{}) {
|
||||
if prev == nil || curr.Time > prev.Time || (curr.Time == prev.Time && curr.Value > prev.Value) {
|
||||
return curr.Time, curr.Value, curr.Aux
|
||||
return curr.Time, curr.Value, cloneAux(curr.Aux)
|
||||
}
|
||||
return prev.Time, prev.Value, prev.Aux
|
||||
}
|
||||
|
@ -366,7 +366,7 @@ func FloatLastReduce(prev, curr *FloatPoint) (int64, float64, []interface{}) {
|
|||
// IntegerLastReduce returns the last point sorted by time.
|
||||
func IntegerLastReduce(prev, curr *IntegerPoint) (int64, int64, []interface{}) {
|
||||
if prev == nil || curr.Time > prev.Time || (curr.Time == prev.Time && curr.Value > prev.Value) {
|
||||
return curr.Time, curr.Value, curr.Aux
|
||||
return curr.Time, curr.Value, cloneAux(curr.Aux)
|
||||
}
|
||||
return prev.Time, prev.Value, prev.Aux
|
||||
}
|
||||
|
@ -374,7 +374,7 @@ func IntegerLastReduce(prev, curr *IntegerPoint) (int64, int64, []interface{}) {
|
|||
// StringLastReduce returns the first point sorted by time.
|
||||
func StringLastReduce(prev, curr *StringPoint) (int64, string, []interface{}) {
|
||||
if prev == nil || curr.Time > prev.Time || (curr.Time == prev.Time && curr.Value > prev.Value) {
|
||||
return curr.Time, curr.Value, curr.Aux
|
||||
return curr.Time, curr.Value, cloneAux(curr.Aux)
|
||||
}
|
||||
return prev.Time, prev.Value, prev.Aux
|
||||
}
|
||||
|
@ -382,7 +382,7 @@ func StringLastReduce(prev, curr *StringPoint) (int64, string, []interface{}) {
|
|||
// BooleanLastReduce returns the first point sorted by time.
|
||||
func BooleanLastReduce(prev, curr *BooleanPoint) (int64, bool, []interface{}) {
|
||||
if prev == nil || curr.Time > prev.Time || (curr.Time == prev.Time && curr.Value && !prev.Value) {
|
||||
return curr.Time, curr.Value, curr.Aux
|
||||
return curr.Time, curr.Value, cloneAux(curr.Aux)
|
||||
}
|
||||
return prev.Time, prev.Value, prev.Aux
|
||||
}
|
||||
|
@ -395,25 +395,25 @@ func NewDistinctIterator(input Iterator, opt IteratorOptions) (Iterator, error)
|
|||
fn := NewFloatDistinctReducer()
|
||||
return fn, fn
|
||||
}
|
||||
return &floatReduceFloatIterator{input: newBufFloatIterator(input), opt: opt, create: createFn}, nil
|
||||
return newFloatReduceFloatIterator(input, opt, createFn), nil
|
||||
case IntegerIterator:
|
||||
createFn := func() (IntegerPointAggregator, IntegerPointEmitter) {
|
||||
fn := NewIntegerDistinctReducer()
|
||||
return fn, fn
|
||||
}
|
||||
return &integerReduceIntegerIterator{input: newBufIntegerIterator(input), opt: opt, create: createFn}, nil
|
||||
return newIntegerReduceIntegerIterator(input, opt, createFn), nil
|
||||
case StringIterator:
|
||||
createFn := func() (StringPointAggregator, StringPointEmitter) {
|
||||
fn := NewStringDistinctReducer()
|
||||
return fn, fn
|
||||
}
|
||||
return &stringReduceStringIterator{input: newBufStringIterator(input), opt: opt, create: createFn}, nil
|
||||
return newStringReduceStringIterator(input, opt, createFn), nil
|
||||
case BooleanIterator:
|
||||
createFn := func() (BooleanPointAggregator, BooleanPointEmitter) {
|
||||
fn := NewBooleanDistinctReducer()
|
||||
return fn, fn
|
||||
}
|
||||
return &booleanReduceBooleanIterator{input: newBufBooleanIterator(input), opt: opt, create: createFn}, nil
|
||||
return newBooleanReduceBooleanIterator(input, opt, createFn), nil
|
||||
default:
|
||||
return nil, fmt.Errorf("unsupported distinct iterator type: %T", input)
|
||||
}
|
||||
|
@ -427,13 +427,13 @@ func newMeanIterator(input Iterator, opt IteratorOptions) (Iterator, error) {
|
|||
fn := NewFloatMeanReducer()
|
||||
return fn, fn
|
||||
}
|
||||
return &floatReduceFloatIterator{input: newBufFloatIterator(input), opt: opt, create: createFn}, nil
|
||||
return newFloatReduceFloatIterator(input, opt, createFn), nil
|
||||
case IntegerIterator:
|
||||
createFn := func() (IntegerPointAggregator, FloatPointEmitter) {
|
||||
fn := NewIntegerMeanReducer()
|
||||
return fn, fn
|
||||
}
|
||||
return &integerReduceFloatIterator{input: newBufIntegerIterator(input), opt: opt, create: createFn}, nil
|
||||
return newIntegerReduceFloatIterator(input, opt, createFn), nil
|
||||
default:
|
||||
return nil, fmt.Errorf("unsupported mean iterator type: %T", input)
|
||||
}
|
||||
|
@ -452,13 +452,13 @@ func newMedianIterator(input Iterator, opt IteratorOptions) (Iterator, error) {
|
|||
fn := NewFloatSliceFuncReducer(FloatMedianReduceSlice)
|
||||
return fn, fn
|
||||
}
|
||||
return &floatReduceFloatIterator{input: newBufFloatIterator(input), opt: opt, create: createFn}, nil
|
||||
return newFloatReduceFloatIterator(input, opt, createFn), nil
|
||||
case IntegerIterator:
|
||||
createFn := func() (IntegerPointAggregator, FloatPointEmitter) {
|
||||
fn := NewIntegerSliceFuncFloatReducer(IntegerMedianReduceSlice)
|
||||
return fn, fn
|
||||
}
|
||||
return &integerReduceFloatIterator{input: newBufIntegerIterator(input), opt: opt, create: createFn}, nil
|
||||
return newIntegerReduceFloatIterator(input, opt, createFn), nil
|
||||
default:
|
||||
return nil, fmt.Errorf("unsupported median iterator type: %T", input)
|
||||
}
|
||||
|
@ -508,26 +508,25 @@ func NewModeIterator(input Iterator, opt IteratorOptions) (Iterator, error) {
|
|||
fn := NewFloatSliceFuncReducer(FloatModeReduceSlice)
|
||||
return fn, fn
|
||||
}
|
||||
return &floatReduceFloatIterator{input: newBufFloatIterator(input), opt: opt, create: createFn}, nil
|
||||
return newFloatReduceFloatIterator(input, opt, createFn), nil
|
||||
case IntegerIterator:
|
||||
createFn := func() (IntegerPointAggregator, IntegerPointEmitter) {
|
||||
fn := NewIntegerSliceFuncReducer(IntegerModeReduceSlice)
|
||||
return fn, fn
|
||||
}
|
||||
return &integerReduceIntegerIterator{input: newBufIntegerIterator(input), opt: opt, create: createFn}, nil
|
||||
return newIntegerReduceIntegerIterator(input, opt, createFn), nil
|
||||
case StringIterator:
|
||||
createFn := func() (StringPointAggregator, StringPointEmitter) {
|
||||
fn := NewStringSliceFuncReducer(StringModeReduceSlice)
|
||||
return fn, fn
|
||||
}
|
||||
return &stringReduceStringIterator{input: newBufStringIterator(input), opt: opt, create: createFn}, nil
|
||||
return newStringReduceStringIterator(input, opt, createFn), nil
|
||||
case BooleanIterator:
|
||||
createFn := func() (BooleanPointAggregator, BooleanPointEmitter) {
|
||||
fn := NewBooleanSliceFuncReducer(BooleanModeReduceSlice)
|
||||
return fn, fn
|
||||
}
|
||||
return &booleanReduceBooleanIterator{input: newBufBooleanIterator(input), opt: opt, create: createFn}, nil
|
||||
|
||||
return newBooleanReduceBooleanIterator(input, opt, createFn), nil
|
||||
default:
|
||||
return nil, fmt.Errorf("unsupported median iterator type: %T", input)
|
||||
}
|
||||
|
@ -668,13 +667,13 @@ func newStddevIterator(input Iterator, opt IteratorOptions) (Iterator, error) {
|
|||
fn := NewFloatSliceFuncReducer(FloatStddevReduceSlice)
|
||||
return fn, fn
|
||||
}
|
||||
return &floatReduceFloatIterator{input: newBufFloatIterator(input), opt: opt, create: createFn}, nil
|
||||
return newFloatReduceFloatIterator(input, opt, createFn), nil
|
||||
case IntegerIterator:
|
||||
createFn := func() (IntegerPointAggregator, FloatPointEmitter) {
|
||||
fn := NewIntegerSliceFuncFloatReducer(IntegerStddevReduceSlice)
|
||||
return fn, fn
|
||||
}
|
||||
return &integerReduceFloatIterator{input: newBufIntegerIterator(input), opt: opt, create: createFn}, nil
|
||||
return newIntegerReduceFloatIterator(input, opt, createFn), nil
|
||||
default:
|
||||
return nil, fmt.Errorf("unsupported stddev iterator type: %T", input)
|
||||
}
|
||||
|
@ -746,13 +745,13 @@ func newSpreadIterator(input Iterator, opt IteratorOptions) (Iterator, error) {
|
|||
fn := NewFloatSliceFuncReducer(FloatSpreadReduceSlice)
|
||||
return fn, fn
|
||||
}
|
||||
return &floatReduceFloatIterator{input: newBufFloatIterator(input), opt: opt, create: createFn}, nil
|
||||
return newFloatReduceFloatIterator(input, opt, createFn), nil
|
||||
case IntegerIterator:
|
||||
createFn := func() (IntegerPointAggregator, IntegerPointEmitter) {
|
||||
fn := NewIntegerSliceFuncReducer(IntegerSpreadReduceSlice)
|
||||
return fn, fn
|
||||
}
|
||||
return &integerReduceIntegerIterator{input: newBufIntegerIterator(input), opt: opt, create: createFn}, nil
|
||||
return newIntegerReduceIntegerIterator(input, opt, createFn), nil
|
||||
default:
|
||||
return nil, fmt.Errorf("unsupported spread iterator type: %T", input)
|
||||
}
|
||||
|
@ -792,14 +791,14 @@ func newTopIterator(input Iterator, opt IteratorOptions, n *IntegerLiteral, tags
|
|||
fn := NewFloatSliceFuncReducer(aggregateFn)
|
||||
return fn, fn
|
||||
}
|
||||
return &floatReduceFloatIterator{input: newBufFloatIterator(input), opt: opt, create: createFn}, nil
|
||||
return newFloatReduceFloatIterator(input, opt, createFn), nil
|
||||
case IntegerIterator:
|
||||
aggregateFn := NewIntegerTopReduceSliceFunc(int(n.Val), tags, opt.Interval)
|
||||
createFn := func() (IntegerPointAggregator, IntegerPointEmitter) {
|
||||
fn := NewIntegerSliceFuncReducer(aggregateFn)
|
||||
return fn, fn
|
||||
}
|
||||
return &integerReduceIntegerIterator{input: newBufIntegerIterator(input), opt: opt, create: createFn}, nil
|
||||
return newIntegerReduceIntegerIterator(input, opt, createFn), nil
|
||||
default:
|
||||
return nil, fmt.Errorf("unsupported top iterator type: %T", input)
|
||||
}
|
||||
|
@ -905,14 +904,14 @@ func newBottomIterator(input Iterator, opt IteratorOptions, n *IntegerLiteral, t
|
|||
fn := NewFloatSliceFuncReducer(aggregateFn)
|
||||
return fn, fn
|
||||
}
|
||||
return &floatReduceFloatIterator{input: newBufFloatIterator(input), opt: opt, create: createFn}, nil
|
||||
return newFloatReduceFloatIterator(input, opt, createFn), nil
|
||||
case IntegerIterator:
|
||||
aggregateFn := NewIntegerBottomReduceSliceFunc(int(n.Val), tags, opt.Interval)
|
||||
createFn := func() (IntegerPointAggregator, IntegerPointEmitter) {
|
||||
fn := NewIntegerSliceFuncReducer(aggregateFn)
|
||||
return fn, fn
|
||||
}
|
||||
return &integerReduceIntegerIterator{input: newBufIntegerIterator(input), opt: opt, create: createFn}, nil
|
||||
return newIntegerReduceIntegerIterator(input, opt, createFn), nil
|
||||
default:
|
||||
return nil, fmt.Errorf("unsupported bottom iterator type: %T", input)
|
||||
}
|
||||
|
@ -1079,14 +1078,14 @@ func newPercentileIterator(input Iterator, opt IteratorOptions, percentile float
|
|||
fn := NewFloatSliceFuncReducer(floatPercentileReduceSlice)
|
||||
return fn, fn
|
||||
}
|
||||
return &floatReduceFloatIterator{input: newBufFloatIterator(input), opt: opt, create: createFn}, nil
|
||||
return newFloatReduceFloatIterator(input, opt, createFn), nil
|
||||
case IntegerIterator:
|
||||
integerPercentileReduceSlice := NewIntegerPercentileReduceSliceFunc(percentile)
|
||||
createFn := func() (IntegerPointAggregator, IntegerPointEmitter) {
|
||||
fn := NewIntegerSliceFuncReducer(integerPercentileReduceSlice)
|
||||
return fn, fn
|
||||
}
|
||||
return &integerReduceIntegerIterator{input: newBufIntegerIterator(input), opt: opt, create: createFn}, nil
|
||||
return newIntegerReduceIntegerIterator(input, opt, createFn), nil
|
||||
default:
|
||||
return nil, fmt.Errorf("unsupported percentile iterator type: %T", input)
|
||||
}
|
||||
|
@ -1103,7 +1102,7 @@ func NewFloatPercentileReduceSliceFunc(percentile float64) FloatReduceSliceFunc
|
|||
}
|
||||
|
||||
sort.Sort(floatPointsByValue(a))
|
||||
return []FloatPoint{{Time: a[i].Time, Value: a[i].Value, Aux: a[i].Aux}}
|
||||
return []FloatPoint{{Time: a[i].Time, Value: a[i].Value, Aux: cloneAux(a[i].Aux)}}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1118,7 +1117,7 @@ func NewIntegerPercentileReduceSliceFunc(percentile float64) IntegerReduceSliceF
|
|||
}
|
||||
|
||||
sort.Sort(integerPointsByValue(a))
|
||||
return []IntegerPoint{{Time: a[i].Time, Value: a[i].Value, Aux: a[i].Aux}}
|
||||
return []IntegerPoint{{Time: a[i].Time, Value: a[i].Value, Aux: cloneAux(a[i].Aux)}}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1242,13 +1241,13 @@ func newHoltWintersIterator(input Iterator, opt IteratorOptions, h, m int, inclu
|
|||
fn := NewFloatHoltWintersReducer(h, m, includeFitData, interval)
|
||||
return fn, fn
|
||||
}
|
||||
return &floatReduceFloatIterator{input: newBufFloatIterator(input), opt: opt, create: createFn}, nil
|
||||
return newFloatReduceFloatIterator(input, opt, createFn), nil
|
||||
case IntegerIterator:
|
||||
createFn := func() (IntegerPointAggregator, FloatPointEmitter) {
|
||||
fn := NewFloatHoltWintersReducer(h, m, includeFitData, interval)
|
||||
return fn, fn
|
||||
}
|
||||
return &integerReduceFloatIterator{input: newBufIntegerIterator(input), opt: opt, create: createFn}, nil
|
||||
return newIntegerReduceFloatIterator(input, opt, createFn), nil
|
||||
default:
|
||||
return nil, fmt.Errorf("unsupported elapsed iterator type: %T", input)
|
||||
}
|
||||
|
@ -1267,25 +1266,25 @@ func newSampleIterator(input Iterator, opt IteratorOptions, size int) (Iterator,
|
|||
fn := NewFloatSampleReducer(size)
|
||||
return fn, fn
|
||||
}
|
||||
return &floatReduceFloatIterator{input: newBufFloatIterator(input), opt: opt, create: createFn}, nil
|
||||
return newFloatReduceFloatIterator(input, opt, createFn), nil
|
||||
case IntegerIterator:
|
||||
createFn := func() (IntegerPointAggregator, IntegerPointEmitter) {
|
||||
fn := NewIntegerSampleReducer(size)
|
||||
return fn, fn
|
||||
}
|
||||
return &integerReduceIntegerIterator{input: newBufIntegerIterator(input), opt: opt, create: createFn}, nil
|
||||
return newIntegerReduceIntegerIterator(input, opt, createFn), nil
|
||||
case StringIterator:
|
||||
createFn := func() (StringPointAggregator, StringPointEmitter) {
|
||||
fn := NewStringSampleReducer(size)
|
||||
return fn, fn
|
||||
}
|
||||
return &stringReduceStringIterator{input: newBufStringIterator(input), opt: opt, create: createFn}, nil
|
||||
return newStringReduceStringIterator(input, opt, createFn), nil
|
||||
case BooleanIterator:
|
||||
createFn := func() (BooleanPointAggregator, BooleanPointEmitter) {
|
||||
fn := NewBooleanSampleReducer(size)
|
||||
return fn, fn
|
||||
}
|
||||
return &booleanReduceBooleanIterator{input: newBufBooleanIterator(input), opt: opt, create: createFn}, nil
|
||||
return newBooleanReduceBooleanIterator(input, opt, createFn), nil
|
||||
default:
|
||||
return nil, fmt.Errorf("unsupported elapsed iterator type: %T", input)
|
||||
}
|
||||
|
|
|
@ -148,7 +148,6 @@ func (e *Emitter) createRow(name string, tags Tags, values []interface{}) {
|
|||
// readAt returns the next slice of values from the iterators at time/name/tags.
|
||||
// Returns nil values once the iterators are exhausted.
|
||||
func (e *Emitter) readAt(t int64, name string, tags Tags) []interface{} {
|
||||
// If time is included then move colums over by one.
|
||||
offset := 1
|
||||
if e.OmitTime {
|
||||
offset = 0
|
||||
|
@ -158,29 +157,31 @@ func (e *Emitter) readAt(t int64, name string, tags Tags) []interface{} {
|
|||
if !e.OmitTime {
|
||||
values[0] = time.Unix(0, t).UTC()
|
||||
}
|
||||
e.readInto(t, name, tags, values[offset:])
|
||||
return values
|
||||
}
|
||||
|
||||
func (e *Emitter) readInto(t int64, name string, tags Tags, values []interface{}) {
|
||||
for i, p := range e.buf {
|
||||
// Skip if buffer is empty.
|
||||
if p == nil {
|
||||
values[i+offset] = nil
|
||||
values[i] = nil
|
||||
continue
|
||||
}
|
||||
|
||||
// Skip point if it doesn't match time/name/tags.
|
||||
pTags := p.tags()
|
||||
if p.time() != t || p.name() != name || !pTags.Equals(&tags) {
|
||||
values[i+offset] = nil
|
||||
values[i] = nil
|
||||
continue
|
||||
}
|
||||
|
||||
// Read point value.
|
||||
values[i+offset] = p.value()
|
||||
values[i] = p.value()
|
||||
|
||||
// Clear buffer.
|
||||
e.buf[i] = nil
|
||||
}
|
||||
|
||||
return values
|
||||
}
|
||||
|
||||
// readIterator reads the next point from itr.
|
||||
|
|
|
@ -381,7 +381,7 @@ func (r *FloatElapsedReducer) Emit() []IntegerPoint {
|
|||
return nil
|
||||
}
|
||||
|
||||
// FloatSampleReduces implements a reservoir sampling to calculate a random subset of points
|
||||
// FloatSampleReducer implements a reservoir sampling to calculate a random subset of points
|
||||
type FloatSampleReducer struct {
|
||||
count int // how many points we've iterated over
|
||||
rng *rand.Rand // random number generator for each reducer
|
||||
|
@ -402,7 +402,7 @@ func (r *FloatSampleReducer) AggregateFloat(p *FloatPoint) {
|
|||
r.count++
|
||||
// Fill the reservoir with the first n points
|
||||
if r.count-1 < len(r.points) {
|
||||
r.points[r.count-1] = *p
|
||||
p.CopyTo(&r.points[r.count-1])
|
||||
return
|
||||
}
|
||||
|
||||
|
@ -411,7 +411,7 @@ func (r *FloatSampleReducer) AggregateFloat(p *FloatPoint) {
|
|||
// replace the point at that index rnd with p.
|
||||
rnd := r.rng.Intn(r.count)
|
||||
if rnd < len(r.points) {
|
||||
r.points[rnd] = *p
|
||||
p.CopyTo(&r.points[rnd])
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -795,7 +795,7 @@ func (r *IntegerElapsedReducer) Emit() []IntegerPoint {
|
|||
return nil
|
||||
}
|
||||
|
||||
// IntegerSampleReduces implements a reservoir sampling to calculate a random subset of points
|
||||
// IntegerSampleReducer implements a reservoir sampling to calculate a random subset of points
|
||||
type IntegerSampleReducer struct {
|
||||
count int // how many points we've iterated over
|
||||
rng *rand.Rand // random number generator for each reducer
|
||||
|
@ -816,7 +816,7 @@ func (r *IntegerSampleReducer) AggregateInteger(p *IntegerPoint) {
|
|||
r.count++
|
||||
// Fill the reservoir with the first n points
|
||||
if r.count-1 < len(r.points) {
|
||||
r.points[r.count-1] = *p
|
||||
p.CopyTo(&r.points[r.count-1])
|
||||
return
|
||||
}
|
||||
|
||||
|
@ -825,7 +825,7 @@ func (r *IntegerSampleReducer) AggregateInteger(p *IntegerPoint) {
|
|||
// replace the point at that index rnd with p.
|
||||
rnd := r.rng.Intn(r.count)
|
||||
if rnd < len(r.points) {
|
||||
r.points[rnd] = *p
|
||||
p.CopyTo(&r.points[rnd])
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1209,7 +1209,7 @@ func (r *StringElapsedReducer) Emit() []IntegerPoint {
|
|||
return nil
|
||||
}
|
||||
|
||||
// StringSampleReduces implements a reservoir sampling to calculate a random subset of points
|
||||
// StringSampleReducer implements a reservoir sampling to calculate a random subset of points
|
||||
type StringSampleReducer struct {
|
||||
count int // how many points we've iterated over
|
||||
rng *rand.Rand // random number generator for each reducer
|
||||
|
@ -1230,7 +1230,7 @@ func (r *StringSampleReducer) AggregateString(p *StringPoint) {
|
|||
r.count++
|
||||
// Fill the reservoir with the first n points
|
||||
if r.count-1 < len(r.points) {
|
||||
r.points[r.count-1] = *p
|
||||
p.CopyTo(&r.points[r.count-1])
|
||||
return
|
||||
}
|
||||
|
||||
|
@ -1239,7 +1239,7 @@ func (r *StringSampleReducer) AggregateString(p *StringPoint) {
|
|||
// replace the point at that index rnd with p.
|
||||
rnd := r.rng.Intn(r.count)
|
||||
if rnd < len(r.points) {
|
||||
r.points[rnd] = *p
|
||||
p.CopyTo(&r.points[rnd])
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1623,7 +1623,7 @@ func (r *BooleanElapsedReducer) Emit() []IntegerPoint {
|
|||
return nil
|
||||
}
|
||||
|
||||
// BooleanSampleReduces implements a reservoir sampling to calculate a random subset of points
|
||||
// BooleanSampleReducer implements a reservoir sampling to calculate a random subset of points
|
||||
type BooleanSampleReducer struct {
|
||||
count int // how many points we've iterated over
|
||||
rng *rand.Rand // random number generator for each reducer
|
||||
|
@ -1644,7 +1644,7 @@ func (r *BooleanSampleReducer) AggregateBoolean(p *BooleanPoint) {
|
|||
r.count++
|
||||
// Fill the reservoir with the first n points
|
||||
if r.count-1 < len(r.points) {
|
||||
r.points[r.count-1] = *p
|
||||
p.CopyTo(&r.points[r.count-1])
|
||||
return
|
||||
}
|
||||
|
||||
|
@ -1653,7 +1653,7 @@ func (r *BooleanSampleReducer) AggregateBoolean(p *BooleanPoint) {
|
|||
// replace the point at that index rnd with p.
|
||||
rnd := r.rng.Intn(r.count)
|
||||
if rnd < len(r.points) {
|
||||
r.points[rnd] = *p
|
||||
p.CopyTo(&r.points[rnd])
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -170,7 +170,7 @@ func (r *{{$k.Name}}ElapsedReducer) Emit() []IntegerPoint {
|
|||
return nil
|
||||
}
|
||||
|
||||
// {{$k.Name}}SampleReduces implements a reservoir sampling to calculate a random subset of points
|
||||
// {{$k.Name}}SampleReducer implements a reservoir sampling to calculate a random subset of points
|
||||
type {{$k.Name}}SampleReducer struct {
|
||||
count int // how many points we've iterated over
|
||||
rng *rand.Rand // random number generator for each reducer
|
||||
|
@ -191,7 +191,7 @@ func (r *{{$k.Name}}SampleReducer) Aggregate{{$k.Name}}(p *{{$k.Name}}Point) {
|
|||
r.count++
|
||||
// Fill the reservoir with the first n points
|
||||
if r.count-1 < len(r.points) {
|
||||
r.points[r.count-1] = *p
|
||||
p.CopyTo(&r.points[r.count-1])
|
||||
return
|
||||
}
|
||||
|
||||
|
@ -200,7 +200,7 @@ func (r *{{$k.Name}}SampleReducer) Aggregate{{$k.Name}}(p *{{$k.Name}}Point) {
|
|||
// replace the point at that index rnd with p.
|
||||
rnd := r.rng.Intn(r.count)
|
||||
if rnd < len(r.points) {
|
||||
r.points[rnd] = *p
|
||||
p.CopyTo(&r.points[rnd])
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -188,6 +188,7 @@ type IteratorOptions struct {
|
|||
Sources []*Measurement `protobuf:"bytes,3,rep,name=Sources" json:"Sources,omitempty"`
|
||||
Interval *Interval `protobuf:"bytes,4,opt,name=Interval" json:"Interval,omitempty"`
|
||||
Dimensions []string `protobuf:"bytes,5,rep,name=Dimensions" json:"Dimensions,omitempty"`
|
||||
GroupBy []string `protobuf:"bytes,19,rep,name=GroupBy" json:"GroupBy,omitempty"`
|
||||
Fill *int32 `protobuf:"varint,6,opt,name=Fill" json:"Fill,omitempty"`
|
||||
FillValue *float64 `protobuf:"fixed64,7,opt,name=FillValue" json:"FillValue,omitempty"`
|
||||
Condition *string `protobuf:"bytes,8,opt,name=Condition" json:"Condition,omitempty"`
|
||||
|
@ -199,6 +200,8 @@ type IteratorOptions struct {
|
|||
SLimit *int64 `protobuf:"varint,14,opt,name=SLimit" json:"SLimit,omitempty"`
|
||||
SOffset *int64 `protobuf:"varint,15,opt,name=SOffset" json:"SOffset,omitempty"`
|
||||
Dedupe *bool `protobuf:"varint,16,opt,name=Dedupe" json:"Dedupe,omitempty"`
|
||||
MaxSeriesN *int64 `protobuf:"varint,18,opt,name=MaxSeriesN" json:"MaxSeriesN,omitempty"`
|
||||
Ordered *bool `protobuf:"varint,20,opt,name=Ordered" json:"Ordered,omitempty"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
}
|
||||
|
||||
|
@ -249,6 +252,13 @@ func (m *IteratorOptions) GetDimensions() []string {
|
|||
return nil
|
||||
}
|
||||
|
||||
func (m *IteratorOptions) GetGroupBy() []string {
|
||||
if m != nil {
|
||||
return m.GroupBy
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *IteratorOptions) GetFill() int32 {
|
||||
if m != nil && m.Fill != nil {
|
||||
return *m.Fill
|
||||
|
@ -326,6 +336,20 @@ func (m *IteratorOptions) GetDedupe() bool {
|
|||
return false
|
||||
}
|
||||
|
||||
func (m *IteratorOptions) GetMaxSeriesN() int64 {
|
||||
if m != nil && m.MaxSeriesN != nil {
|
||||
return *m.MaxSeriesN
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (m *IteratorOptions) GetOrdered() bool {
|
||||
if m != nil && m.Ordered != nil {
|
||||
return *m.Ordered
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
type Measurements struct {
|
||||
Items []*Measurement `protobuf:"bytes,1,rep,name=Items" json:"Items,omitempty"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
|
@ -481,48 +505,51 @@ func init() {
|
|||
func init() { proto.RegisterFile("internal/internal.proto", fileDescriptorInternal) }
|
||||
|
||||
var fileDescriptorInternal = []byte{
|
||||
// 685 bytes of a gzipped FileDescriptorProto
|
||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0x84, 0x54, 0xd1, 0x6a, 0xdc, 0x3a,
|
||||
0x10, 0xc5, 0xf6, 0x7a, 0x63, 0x6b, 0xb3, 0x37, 0xb9, 0x22, 0xf7, 0x46, 0x94, 0xd2, 0x1a, 0x3f,
|
||||
0x19, 0x4a, 0x37, 0x90, 0xd7, 0x42, 0x61, 0xdb, 0x24, 0xb0, 0xd0, 0x6e, 0x82, 0x1c, 0xf2, 0xae,
|
||||
0x66, 0x67, 0x8d, 0xc0, 0x2b, 0x6f, 0x65, 0xb9, 0x6c, 0xde, 0xfa, 0x1b, 0xfd, 0x86, 0x7e, 0x4c,
|
||||
0x7f, 0xa9, 0x68, 0x64, 0xaf, 0x9d, 0x14, 0x9a, 0x27, 0xcf, 0x39, 0x33, 0x92, 0x7c, 0x66, 0x8e,
|
||||
0x44, 0x4e, 0xa5, 0x32, 0xa0, 0x95, 0x28, 0xcf, 0xba, 0x60, 0xb6, 0xd5, 0x95, 0xa9, 0x68, 0x24,
|
||||
0xd5, 0xba, 0x6c, 0x76, 0x5f, 0xcb, 0xf4, 0x97, 0x4f, 0xc2, 0x9b, 0x4a, 0x2a, 0x43, 0x29, 0x19,
|
||||
0x2d, 0xc5, 0x06, 0x98, 0x97, 0xf8, 0x59, 0xcc, 0x31, 0xb6, 0xdc, 0xad, 0x28, 0x6a, 0xe6, 0x3b,
|
||||
0xce, 0xc6, 0xc8, 0xc9, 0x0d, 0xb0, 0x20, 0xf1, 0xb3, 0x80, 0x63, 0x4c, 0x8f, 0x49, 0xb0, 0x94,
|
||||
0x25, 0x1b, 0x25, 0x7e, 0x16, 0x71, 0x1b, 0xd2, 0xd7, 0x24, 0x98, 0x37, 0x3b, 0x16, 0x26, 0x41,
|
||||
0x36, 0x39, 0x9f, 0xce, 0xba, 0xf3, 0x66, 0xf3, 0x66, 0xc7, 0x6d, 0x86, 0xbe, 0x22, 0x64, 0x5e,
|
||||
0x14, 0x1a, 0x0a, 0x61, 0x60, 0xc5, 0xc6, 0x89, 0x97, 0x4d, 0xf9, 0x80, 0xb1, 0xf9, 0xab, 0xb2,
|
||||
0x12, 0xe6, 0x4e, 0x94, 0x0d, 0xb0, 0x83, 0xc4, 0xcb, 0x3c, 0x3e, 0x60, 0x68, 0x4a, 0x0e, 0x17,
|
||||
0xca, 0x40, 0x01, 0xda, 0x55, 0x44, 0x89, 0x97, 0x05, 0xfc, 0x11, 0x47, 0x13, 0x32, 0xc9, 0x8d,
|
||||
0x96, 0xaa, 0x70, 0x25, 0x71, 0xe2, 0x65, 0x31, 0x1f, 0x52, 0x76, 0x97, 0x0f, 0x55, 0x55, 0x82,
|
||||
0x50, 0xae, 0x84, 0x24, 0x5e, 0x16, 0xf1, 0x47, 0x1c, 0x7d, 0x4b, 0xc2, 0xdc, 0x08, 0x53, 0xb3,
|
||||
0x49, 0xe2, 0x65, 0x93, 0xf3, 0xd3, 0x5e, 0xcc, 0xc2, 0x80, 0x16, 0xa6, 0xd2, 0x98, 0xe6, 0xae,
|
||||
0x2a, 0xfd, 0xe9, 0xa1, 0x74, 0xfa, 0x82, 0x44, 0x17, 0xc2, 0x88, 0xdb, 0x87, 0xad, 0xeb, 0x69,
|
||||
0xc8, 0xf7, 0xf8, 0x89, 0x38, 0xff, 0x59, 0x71, 0xc1, 0xf3, 0xe2, 0x46, 0xcf, 0x8b, 0x0b, 0xff,
|
||||
0x14, 0x97, 0x7e, 0x1f, 0x91, 0xa3, 0x4e, 0xc6, 0xf5, 0xd6, 0xc8, 0x4a, 0xe1, 0x84, 0x2f, 0x77,
|
||||
0x5b, 0xcd, 0x3c, 0xdc, 0x12, 0x63, 0x3b, 0x61, 0x3b, 0x4f, 0x3f, 0x09, 0xb2, 0xd8, 0x0d, 0x30,
|
||||
0x23, 0xe3, 0x2b, 0x09, 0xe5, 0xaa, 0x66, 0xff, 0xe2, 0x90, 0x8f, 0xfb, 0xbe, 0xdc, 0x09, 0xcd,
|
||||
0x61, 0xcd, 0xdb, 0x3c, 0x3d, 0x23, 0x07, 0x79, 0xd5, 0xe8, 0x7b, 0xa8, 0x59, 0x80, 0xa5, 0xff,
|
||||
0xf5, 0xa5, 0x9f, 0x41, 0xd4, 0x8d, 0x86, 0x0d, 0x28, 0xc3, 0xbb, 0x2a, 0x3a, 0x23, 0x91, 0x95,
|
||||
0xaa, 0xbf, 0x89, 0x12, 0x75, 0x4d, 0xce, 0xe9, 0xa0, 0xe9, 0x6d, 0x86, 0xef, 0x6b, 0x6c, 0x3b,
|
||||
0x2f, 0xe4, 0x06, 0x54, 0x6d, 0x7f, 0x1f, 0x3d, 0x17, 0xf3, 0x01, 0x63, 0x05, 0x5d, 0xc9, 0xb2,
|
||||
0x44, 0x97, 0x85, 0x1c, 0x63, 0xfa, 0x92, 0xc4, 0xf6, 0x3b, 0xb4, 0x57, 0x4f, 0xd8, 0xec, 0xc7,
|
||||
0x4a, 0xad, 0xa4, 0x6d, 0x08, 0x5a, 0x2b, 0xe6, 0x3d, 0x61, 0xb3, 0xb9, 0x11, 0xda, 0xe0, 0x3d,
|
||||
0x88, 0x71, 0x36, 0x3d, 0x41, 0x19, 0x39, 0xb8, 0x54, 0x2b, 0xcc, 0x11, 0xcc, 0x75, 0xd0, 0xae,
|
||||
0x9b, 0xd7, 0xf7, 0xa0, 0x56, 0x52, 0x15, 0xe8, 0xa6, 0x88, 0xf7, 0x04, 0x3d, 0x21, 0xe1, 0x27,
|
||||
0xb9, 0x91, 0x86, 0x1d, 0xe2, 0x2a, 0x07, 0xe8, 0xff, 0x64, 0x7c, 0xbd, 0x5e, 0xd7, 0x60, 0xd8,
|
||||
0x14, 0xe9, 0x16, 0x59, 0x3e, 0x77, 0xe5, 0xff, 0x38, 0xde, 0x21, 0x7b, 0x7a, 0xde, 0x2e, 0x38,
|
||||
0x72, 0xa7, 0xe7, 0xfd, 0x8a, 0x0b, 0x58, 0x35, 0x5b, 0x60, 0xc7, 0x78, 0x74, 0x8b, 0xd2, 0x77,
|
||||
0xe4, 0x70, 0x30, 0x85, 0x9a, 0xbe, 0x21, 0xe1, 0xc2, 0xc0, 0xa6, 0x66, 0xde, 0xdf, 0x86, 0xe5,
|
||||
0x6a, 0xd2, 0x1f, 0x1e, 0x99, 0x0c, 0xe8, 0xce, 0xf5, 0x5f, 0x44, 0x0d, 0xad, 0x7f, 0xf6, 0x98,
|
||||
0x66, 0xe4, 0x88, 0x83, 0x01, 0x65, 0x7b, 0x78, 0x53, 0x95, 0xf2, 0xfe, 0x01, 0xad, 0x1f, 0xf3,
|
||||
0xa7, 0xf4, 0xfe, 0x2d, 0x0a, 0x9c, 0x03, 0xf1, 0x2d, 0x3a, 0x21, 0x21, 0x87, 0x02, 0x76, 0xad,
|
||||
0xd3, 0x1d, 0xb0, 0xe7, 0x2d, 0xea, 0x5b, 0xa1, 0x0b, 0x30, 0xad, 0xbf, 0xf7, 0x38, 0x7d, 0xdf,
|
||||
0xdb, 0x08, 0xff, 0xab, 0xd1, 0x02, 0xe7, 0xe9, 0x61, 0x5f, 0xf6, 0x78, 0xd0, 0x62, 0x7f, 0xd8,
|
||||
0xe2, 0x74, 0x4e, 0xa6, 0x8f, 0x6e, 0x38, 0xf6, 0x16, 0xb4, 0x84, 0x7a, 0xd9, 0xee, 0xd1, 0x41,
|
||||
0xbb, 0x05, 0xbe, 0xa2, 0xcb, 0x6e, 0x0b, 0x87, 0xd2, 0x19, 0x19, 0xbb, 0xcb, 0x60, 0x2f, 0xd0,
|
||||
0x9d, 0x28, 0xdb, 0xd7, 0xd5, 0x86, 0xf8, 0x90, 0xda, 0xc7, 0xc1, 0x77, 0xae, 0xb4, 0xf1, 0xef,
|
||||
0x00, 0x00, 0x00, 0xff, 0xff, 0x8c, 0x01, 0x5d, 0x11, 0xb2, 0x05, 0x00, 0x00,
|
||||
// 726 bytes of a gzipped FileDescriptorProto
|
||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0x84, 0x54, 0x51, 0x6b, 0xe3, 0x46,
|
||||
0x10, 0x46, 0x92, 0xe5, 0x58, 0xeb, 0xb8, 0x49, 0xb7, 0x69, 0xb2, 0x94, 0xd2, 0x0a, 0x3d, 0x09,
|
||||
0x4a, 0x1d, 0xc8, 0x6b, 0xa1, 0xe0, 0x34, 0x49, 0x31, 0x5c, 0x9c, 0xb0, 0x0a, 0x79, 0xdf, 0x8b,
|
||||
0xc6, 0x62, 0x41, 0x5e, 0xf9, 0x56, 0xab, 0xc3, 0xf9, 0x29, 0xf7, 0x1b, 0xee, 0xc7, 0xdc, 0xcb,
|
||||
0xfd, 0xa0, 0x63, 0x67, 0x25, 0x4b, 0xc9, 0xc1, 0xe5, 0x49, 0xf3, 0x7d, 0x33, 0xbb, 0xab, 0x99,
|
||||
0xf9, 0x66, 0xc8, 0x99, 0x54, 0x06, 0xb4, 0x12, 0xe5, 0x79, 0x67, 0xcc, 0xb7, 0xba, 0x32, 0x15,
|
||||
0x9d, 0x48, 0xb5, 0x2e, 0x9b, 0xdd, 0x87, 0x32, 0xf9, 0xe2, 0x93, 0xf0, 0xbe, 0x92, 0xca, 0x50,
|
||||
0x4a, 0x46, 0x2b, 0xb1, 0x01, 0xe6, 0xc5, 0x7e, 0x1a, 0x71, 0xb4, 0x2d, 0xf7, 0x20, 0x8a, 0x9a,
|
||||
0xf9, 0x8e, 0xb3, 0x36, 0x72, 0x72, 0x03, 0x2c, 0x88, 0xfd, 0x34, 0xe0, 0x68, 0xd3, 0x63, 0x12,
|
||||
0xac, 0x64, 0xc9, 0x46, 0xb1, 0x9f, 0x4e, 0xb8, 0x35, 0xe9, 0x9f, 0x24, 0x58, 0x34, 0x3b, 0x16,
|
||||
0xc6, 0x41, 0x3a, 0xbd, 0x98, 0xcd, 0xbb, 0xf7, 0xe6, 0x8b, 0x66, 0xc7, 0xad, 0x87, 0xfe, 0x41,
|
||||
0xc8, 0xa2, 0x28, 0x34, 0x14, 0xc2, 0x40, 0xce, 0xc6, 0xb1, 0x97, 0xce, 0xf8, 0x80, 0xb1, 0xfe,
|
||||
0x9b, 0xb2, 0x12, 0xe6, 0x51, 0x94, 0x0d, 0xb0, 0x83, 0xd8, 0x4b, 0x3d, 0x3e, 0x60, 0x68, 0x42,
|
||||
0x0e, 0x97, 0xca, 0x40, 0x01, 0xda, 0x45, 0x4c, 0x62, 0x2f, 0x0d, 0xf8, 0x0b, 0x8e, 0xc6, 0x64,
|
||||
0x9a, 0x19, 0x2d, 0x55, 0xe1, 0x42, 0xa2, 0xd8, 0x4b, 0x23, 0x3e, 0xa4, 0xec, 0x2d, 0x97, 0x55,
|
||||
0x55, 0x82, 0x50, 0x2e, 0x84, 0xc4, 0x5e, 0x3a, 0xe1, 0x2f, 0x38, 0xfa, 0x37, 0x09, 0x33, 0x23,
|
||||
0x4c, 0xcd, 0xa6, 0xb1, 0x97, 0x4e, 0x2f, 0xce, 0xfa, 0x64, 0x96, 0x06, 0xb4, 0x30, 0x95, 0x46,
|
||||
0x37, 0x77, 0x51, 0xc9, 0x67, 0x0f, 0x53, 0xa7, 0xbf, 0x91, 0xc9, 0x95, 0x30, 0xe2, 0xe1, 0x79,
|
||||
0xeb, 0x6a, 0x1a, 0xf2, 0x3d, 0x7e, 0x95, 0x9c, 0xff, 0x66, 0x72, 0xc1, 0xdb, 0xc9, 0x8d, 0xde,
|
||||
0x4e, 0x2e, 0xfc, 0x3e, 0xb9, 0xe4, 0xeb, 0x88, 0x1c, 0x75, 0x69, 0xdc, 0x6d, 0x8d, 0xac, 0x14,
|
||||
0x76, 0xf8, 0x7a, 0xb7, 0xd5, 0xcc, 0xc3, 0x2b, 0xd1, 0xb6, 0x1d, 0xb6, 0xfd, 0xf4, 0xe3, 0x20,
|
||||
0x8d, 0x5c, 0x03, 0x53, 0x32, 0xbe, 0x91, 0x50, 0xe6, 0x35, 0xfb, 0x19, 0x9b, 0x7c, 0xdc, 0xd7,
|
||||
0xe5, 0x51, 0x68, 0x0e, 0x6b, 0xde, 0xfa, 0xe9, 0x39, 0x39, 0xc8, 0xaa, 0x46, 0x3f, 0x41, 0xcd,
|
||||
0x02, 0x0c, 0xfd, 0xb5, 0x0f, 0xbd, 0x05, 0x51, 0x37, 0x1a, 0x36, 0xa0, 0x0c, 0xef, 0xa2, 0xe8,
|
||||
0x9c, 0x4c, 0x6c, 0xaa, 0xfa, 0xa3, 0x28, 0x31, 0xaf, 0xe9, 0x05, 0x1d, 0x14, 0xbd, 0xf5, 0xf0,
|
||||
0x7d, 0x8c, 0x2d, 0xe7, 0x95, 0xdc, 0x80, 0xaa, 0xed, 0xef, 0xa3, 0xe6, 0x22, 0x3e, 0x60, 0x28,
|
||||
0x23, 0x07, 0xff, 0xeb, 0xaa, 0xd9, 0x5e, 0x3e, 0xb3, 0x5f, 0xd0, 0xd9, 0x41, 0x9b, 0xea, 0x8d,
|
||||
0x2c, 0x4b, 0xd4, 0x5f, 0xc8, 0xd1, 0xa6, 0xbf, 0x93, 0xc8, 0x7e, 0x87, 0xc2, 0xeb, 0x09, 0xeb,
|
||||
0xfd, 0xaf, 0x52, 0xb9, 0xb4, 0xa5, 0x42, 0xd1, 0x45, 0xbc, 0x27, 0xac, 0x37, 0x33, 0x42, 0x1b,
|
||||
0x9c, 0x90, 0x08, 0xbb, 0xd6, 0x13, 0xf6, 0x3f, 0xae, 0x55, 0x8e, 0x3e, 0x82, 0xbe, 0x0e, 0xda,
|
||||
0x73, 0x8b, 0xfa, 0x09, 0x54, 0x2e, 0x55, 0x81, 0x3a, 0x9b, 0xf0, 0x9e, 0xa0, 0x27, 0x24, 0x7c,
|
||||
0x27, 0x37, 0xd2, 0xb0, 0x43, 0x3c, 0xe5, 0x00, 0x3d, 0x25, 0xe3, 0xbb, 0xf5, 0xba, 0x06, 0xc3,
|
||||
0x66, 0x48, 0xb7, 0xc8, 0xf2, 0x99, 0x0b, 0xff, 0xc9, 0xf1, 0x0e, 0xd9, 0xd7, 0xb3, 0xf6, 0xc0,
|
||||
0x91, 0x7b, 0x3d, 0xeb, 0x4f, 0x5c, 0x41, 0xde, 0x6c, 0x81, 0x1d, 0xe3, 0xd3, 0x2d, 0xb2, 0x75,
|
||||
0xbd, 0x15, 0xbb, 0x0c, 0xb4, 0x84, 0x7a, 0xc5, 0x28, 0x1e, 0x1a, 0x30, 0xf6, 0xc6, 0x3b, 0x9d,
|
||||
0x83, 0x86, 0x9c, 0x9d, 0xe0, 0xc1, 0x0e, 0x26, 0xff, 0x90, 0xc3, 0x41, 0x67, 0x6b, 0xfa, 0x17,
|
||||
0x09, 0x97, 0x06, 0x36, 0x35, 0xf3, 0x7e, 0x24, 0x00, 0x17, 0x93, 0x7c, 0xf2, 0xc8, 0x74, 0x40,
|
||||
0x77, 0x93, 0xf4, 0x5e, 0xd4, 0xd0, 0x6a, 0x72, 0x8f, 0x69, 0x4a, 0x8e, 0x38, 0x18, 0x50, 0xb6,
|
||||
0xfa, 0xf7, 0x55, 0x29, 0x9f, 0x9e, 0x71, 0x9c, 0x22, 0xfe, 0x9a, 0xde, 0xef, 0xb7, 0xc0, 0xa9,
|
||||
0x1a, 0xf7, 0xdb, 0x09, 0x09, 0x39, 0x14, 0xb0, 0x6b, 0xa7, 0xc7, 0x01, 0xfb, 0xde, 0xb2, 0x7e,
|
||||
0x10, 0xba, 0x00, 0xd3, 0xce, 0xcc, 0x1e, 0x27, 0xff, 0xf6, 0xd2, 0xc4, 0xff, 0x6a, 0xb4, 0x40,
|
||||
0x25, 0x78, 0x58, 0x9c, 0x3d, 0x1e, 0x34, 0xc7, 0x1f, 0x36, 0x27, 0x59, 0x90, 0xd9, 0x8b, 0xad,
|
||||
0x81, 0x5d, 0x69, 0x0b, 0xec, 0xb5, 0x5d, 0x69, 0xab, 0x7b, 0x4a, 0xc6, 0xb8, 0x99, 0x57, 0xdd,
|
||||
0x15, 0x0e, 0x25, 0x73, 0x32, 0x76, 0x03, 0x66, 0x87, 0xf2, 0x51, 0x94, 0xed, 0xc6, 0xb6, 0x26,
|
||||
0x2e, 0x67, 0xbb, 0x70, 0x7c, 0xa7, 0x67, 0x6b, 0x7f, 0x0b, 0x00, 0x00, 0xff, 0xff, 0xd1, 0x2a,
|
||||
0x83, 0x14, 0x06, 0x06, 0x00, 0x00,
|
||||
}
|
||||
|
|
|
@ -1,3 +1,4 @@
|
|||
syntax = "proto2";
|
||||
package influxql;
|
||||
|
||||
message Point {
|
||||
|
@ -31,6 +32,7 @@ message IteratorOptions {
|
|||
repeated Measurement Sources = 3;
|
||||
optional Interval Interval = 4;
|
||||
repeated string Dimensions = 5;
|
||||
repeated string GroupBy = 19;
|
||||
optional int32 Fill = 6;
|
||||
optional double FillValue = 7;
|
||||
optional string Condition = 8;
|
||||
|
@ -42,6 +44,8 @@ message IteratorOptions {
|
|||
optional int64 SLimit = 14;
|
||||
optional int64 SOffset = 15;
|
||||
optional bool Dedupe = 16;
|
||||
optional int64 MaxSeriesN = 18;
|
||||
optional bool Ordered = 20;
|
||||
}
|
||||
|
||||
message Measurements {
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -2,7 +2,6 @@ package influxql
|
|||
|
||||
import (
|
||||
"container/heap"
|
||||
"errors"
|
||||
"encoding/binary"
|
||||
"fmt"
|
||||
"io"
|
||||
|
@ -131,7 +130,7 @@ func new{{$k.Name}}MergeIterator(inputs []{{$k.Name}}Iterator, opt IteratorOptio
|
|||
inputs: inputs,
|
||||
heap: &{{$k.name}}MergeHeap{
|
||||
items: make([]*{{$k.name}}MergeHeapItem, 0, len(inputs)),
|
||||
opt: opt,
|
||||
opt: opt,
|
||||
},
|
||||
}
|
||||
|
||||
|
@ -202,7 +201,8 @@ func (itr *{{$k.name}}MergeIterator) Next() (*{{$k.Name}}Point, error) {
|
|||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
itr.window.name, itr.window.tags = p.Name, p.Tags.ID()
|
||||
tags := p.Tags.Subset(itr.heap.opt.Dimensions)
|
||||
itr.window.name, itr.window.tags = p.Name, tags.ID()
|
||||
itr.window.startTime, itr.window.endTime = itr.heap.opt.Window(p.Time)
|
||||
return p, nil
|
||||
}
|
||||
|
@ -223,7 +223,7 @@ func (itr *{{$k.name}}MergeIterator) Next() (*{{$k.Name}}Point, error) {
|
|||
inWindow := true
|
||||
if window := itr.window; window.name != p.Name {
|
||||
inWindow = false
|
||||
} else if window.tags != p.Tags.ID() {
|
||||
} else if tags := p.Tags.Subset(itr.heap.opt.Dimensions); window.tags != tags.ID() {
|
||||
inWindow = false
|
||||
} else if opt := itr.heap.opt; opt.Ascending && p.Time >= window.endTime {
|
||||
inWindow = false
|
||||
|
@ -250,9 +250,9 @@ type {{$k.name}}MergeHeap struct {
|
|||
items []*{{$k.name}}MergeHeapItem
|
||||
}
|
||||
|
||||
func (h {{$k.name}}MergeHeap) Len() int { return len(h.items) }
|
||||
func (h {{$k.name}}MergeHeap) Swap(i, j int) { h.items[i], h.items[j] = h.items[j], h.items[i] }
|
||||
func (h {{$k.name}}MergeHeap) Less(i, j int) bool {
|
||||
func (h *{{$k.name}}MergeHeap) Len() int { return len(h.items) }
|
||||
func (h *{{$k.name}}MergeHeap) Swap(i, j int) { h.items[i], h.items[j] = h.items[j], h.items[i] }
|
||||
func (h *{{$k.name}}MergeHeap) Less(i, j int) bool {
|
||||
x, err := h.items[i].itr.peek()
|
||||
if err != nil {
|
||||
return true
|
||||
|
@ -265,14 +265,14 @@ func (h {{$k.name}}MergeHeap) Less(i, j int) bool {
|
|||
if h.opt.Ascending {
|
||||
if x.Name != y.Name {
|
||||
return x.Name < y.Name
|
||||
} else if x.Tags.ID() != y.Tags.ID() {
|
||||
return x.Tags.ID() < y.Tags.ID()
|
||||
} else if xTags, yTags := x.Tags.Subset(h.opt.Dimensions), y.Tags.Subset(h.opt.Dimensions); xTags.ID() != yTags.ID() {
|
||||
return xTags.ID() < yTags.ID()
|
||||
}
|
||||
} else {
|
||||
if x.Name != y.Name {
|
||||
return x.Name > y.Name
|
||||
} else if x.Tags.ID() != y.Tags.ID() {
|
||||
return x.Tags.ID() > y.Tags.ID()
|
||||
} else if xTags, yTags := x.Tags.Subset(h.opt.Dimensions), y.Tags.Subset(h.opt.Dimensions); xTags.ID() != yTags.ID() {
|
||||
return xTags.ID() > yTags.ID()
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -305,8 +305,7 @@ type {{$k.name}}MergeHeapItem struct {
|
|||
// {{$k.name}}SortedMergeIterator is an iterator that sorts and merges multiple iterators into one.
|
||||
type {{$k.name}}SortedMergeIterator struct {
|
||||
inputs []{{$k.Name}}Iterator
|
||||
opt IteratorOptions
|
||||
heap {{$k.name}}SortedMergeHeap
|
||||
heap *{{$k.name}}SortedMergeHeap
|
||||
init bool
|
||||
}
|
||||
|
||||
|
@ -314,14 +313,16 @@ type {{$k.name}}SortedMergeIterator struct {
|
|||
func new{{$k.Name}}SortedMergeIterator(inputs []{{$k.Name}}Iterator, opt IteratorOptions) Iterator {
|
||||
itr := &{{$k.name}}SortedMergeIterator{
|
||||
inputs: inputs,
|
||||
heap: make({{$k.name}}SortedMergeHeap, 0, len(inputs)),
|
||||
opt: opt,
|
||||
heap: &{{$k.name}}SortedMergeHeap{
|
||||
items: make([]*{{$k.name}}SortedMergeHeapItem, 0, len(inputs)),
|
||||
opt: opt,
|
||||
},
|
||||
}
|
||||
|
||||
// Initialize heap items.
|
||||
for _, input := range inputs {
|
||||
// Append to the heap.
|
||||
itr.heap = append(itr.heap, &{{$k.name}}SortedMergeHeapItem{itr: input, ascending: opt.Ascending})
|
||||
itr.heap.items = append(itr.heap.items, &{{$k.name}}SortedMergeHeapItem{itr: input})
|
||||
}
|
||||
|
||||
return itr
|
||||
|
@ -352,8 +353,8 @@ func (itr *{{$k.name}}SortedMergeIterator) Next() (*{{$k.Name}}Point, error) { r
|
|||
func (itr *{{$k.name}}SortedMergeIterator) pop() (*{{$k.Name}}Point, error) {
|
||||
// Initialize the heap. See the MergeIterator to see why this has to be done lazily.
|
||||
if !itr.init {
|
||||
items := itr.heap
|
||||
itr.heap = make([]*{{$k.name}}SortedMergeHeapItem, 0, len(items))
|
||||
items := itr.heap.items
|
||||
itr.heap.items = make([]*{{$k.name}}SortedMergeHeapItem, 0, len(items))
|
||||
for _, item := range items {
|
||||
var err error
|
||||
if item.point, err = item.itr.Next(); err != nil {
|
||||
|
@ -361,18 +362,18 @@ func (itr *{{$k.name}}SortedMergeIterator) pop() (*{{$k.Name}}Point, error) {
|
|||
} else if item.point == nil {
|
||||
continue
|
||||
}
|
||||
itr.heap = append(itr.heap, item)
|
||||
itr.heap.items = append(itr.heap.items, item)
|
||||
}
|
||||
heap.Init(&itr.heap)
|
||||
heap.Init(itr.heap)
|
||||
itr.init = true
|
||||
}
|
||||
|
||||
if len(itr.heap) == 0 {
|
||||
if len(itr.heap.items) == 0 {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// Read the next item from the heap.
|
||||
item := heap.Pop(&itr.heap).(*{{$k.name}}SortedMergeHeapItem)
|
||||
item := heap.Pop(itr.heap).(*{{$k.name}}SortedMergeHeapItem)
|
||||
if item.err != nil {
|
||||
return nil, item.err
|
||||
} else if item.point == nil {
|
||||
|
@ -384,46 +385,49 @@ func (itr *{{$k.name}}SortedMergeIterator) pop() (*{{$k.Name}}Point, error) {
|
|||
|
||||
// Read the next item from the cursor. Push back to heap if one exists.
|
||||
if item.point, item.err = item.itr.Next(); item.point != nil {
|
||||
heap.Push(&itr.heap, item)
|
||||
heap.Push(itr.heap, item)
|
||||
}
|
||||
|
||||
return p, nil
|
||||
}
|
||||
|
||||
// {{$k.name}}SortedMergeHeap represents a heap of {{$k.name}}SortedMergeHeapItems.
|
||||
type {{$k.name}}SortedMergeHeap []*{{$k.name}}SortedMergeHeapItem
|
||||
type {{$k.name}}SortedMergeHeap struct {
|
||||
opt IteratorOptions
|
||||
items []*{{$k.name}}SortedMergeHeapItem
|
||||
}
|
||||
|
||||
func (h {{$k.name}}SortedMergeHeap) Len() int { return len(h) }
|
||||
func (h {{$k.name}}SortedMergeHeap) Swap(i, j int) { h[i], h[j] = h[j], h[i] }
|
||||
func (h {{$k.name}}SortedMergeHeap) Less(i, j int) bool {
|
||||
x, y := h[i].point, h[j].point
|
||||
func (h *{{$k.name}}SortedMergeHeap) Len() int { return len(h.items) }
|
||||
func (h *{{$k.name}}SortedMergeHeap) Swap(i, j int) { h.items[i], h.items[j] = h.items[j], h.items[i] }
|
||||
func (h *{{$k.name}}SortedMergeHeap) Less(i, j int) bool {
|
||||
x, y := h.items[i].point, h.items[j].point
|
||||
|
||||
if h[i].ascending {
|
||||
if h.opt.Ascending {
|
||||
if x.Name != y.Name {
|
||||
return x.Name < y.Name
|
||||
} else if !x.Tags.Equals(&y.Tags) {
|
||||
return x.Tags.ID() < y.Tags.ID()
|
||||
} else if xTags, yTags := x.Tags.Subset(h.opt.Dimensions), y.Tags.Subset(h.opt.Dimensions); !xTags.Equals(&yTags) {
|
||||
return xTags.ID() < yTags.ID()
|
||||
}
|
||||
return x.Time < y.Time
|
||||
}
|
||||
|
||||
if x.Name != y.Name {
|
||||
return x.Name > y.Name
|
||||
} else if !x.Tags.Equals(&y.Tags) {
|
||||
return x.Tags.ID() > y.Tags.ID()
|
||||
} else if xTags, yTags := x.Tags.Subset(h.opt.Dimensions), y.Tags.Subset(h.opt.Dimensions); !xTags.Equals(&yTags) {
|
||||
return xTags.ID() > yTags.ID()
|
||||
}
|
||||
return x.Time > y.Time
|
||||
}
|
||||
|
||||
func (h *{{$k.name}}SortedMergeHeap) Push(x interface{}) {
|
||||
*h = append(*h, x.(*{{$k.name}}SortedMergeHeapItem))
|
||||
h.items = append(h.items, x.(*{{$k.name}}SortedMergeHeapItem))
|
||||
}
|
||||
|
||||
func (h *{{$k.name}}SortedMergeHeap) Pop() interface{} {
|
||||
old := *h
|
||||
old := h.items
|
||||
n := len(old)
|
||||
item := old[n-1]
|
||||
*h = old[0 : n-1]
|
||||
h.items = old[0 : n-1]
|
||||
return item
|
||||
}
|
||||
|
||||
|
@ -431,7 +435,6 @@ type {{$k.name}}SortedMergeHeapItem struct {
|
|||
point *{{$k.Name}}Point
|
||||
err error
|
||||
itr {{$k.Name}}Iterator
|
||||
ascending bool
|
||||
}
|
||||
|
||||
// {{$k.name}}ParallelIterator represents an iterator that pulls data in a separate goroutine.
|
||||
|
@ -548,10 +551,6 @@ func (itr *{{$k.name}}LimitIterator) Next() (*{{$k.Name}}Point, error) {
|
|||
|
||||
// Read next point if we're beyond the limit.
|
||||
if itr.opt.Limit > 0 && (itr.n-itr.opt.Offset) > itr.opt.Limit {
|
||||
// If there's no interval, no groups, and a single source then simply exit.
|
||||
if itr.opt.Interval.IsZero() && len(itr.opt.Dimensions) == 0 && len(itr.opt.Sources) == 1 {
|
||||
return nil, nil
|
||||
}
|
||||
continue
|
||||
}
|
||||
|
||||
|
@ -841,7 +840,7 @@ type aux{{$k.Name}}Point struct {
|
|||
type {{$k.name}}AuxIterator struct {
|
||||
input *buf{{$k.Name}}Iterator
|
||||
output chan aux{{$k.Name}}Point
|
||||
fields auxIteratorFields
|
||||
fields *auxIteratorFields
|
||||
background bool
|
||||
}
|
||||
|
||||
|
@ -868,28 +867,6 @@ func (itr *{{$k.name}}AuxIterator) Next() (*{{$k.Name}}Point, error) {
|
|||
}
|
||||
func (itr *{{$k.name}}AuxIterator) Iterator(name string, typ DataType) Iterator { return itr.fields.iterator(name, typ) }
|
||||
|
||||
func (itr *{{$k.name}}AuxIterator) CreateIterator(opt IteratorOptions) (Iterator, error) {
|
||||
expr := opt.Expr
|
||||
if expr == nil {
|
||||
panic("unable to create an iterator with no expression from an aux iterator")
|
||||
}
|
||||
|
||||
switch expr := expr.(type) {
|
||||
case *VarRef:
|
||||
return itr.Iterator(expr.Val, expr.Type), nil
|
||||
default:
|
||||
panic(fmt.Sprintf("invalid expression type for an aux iterator: %T", expr))
|
||||
}
|
||||
}
|
||||
|
||||
func (itr *{{$k.name}}AuxIterator) FieldDimensions(sources Sources) (fields map[string]DataType, dimensions map[string]struct{}, err error) {
|
||||
return nil, nil, errors.New("not implemented")
|
||||
}
|
||||
|
||||
func (itr *{{$k.name}}AuxIterator) ExpandSources(sources Sources) (Sources, error) {
|
||||
return nil, errors.New("not implemented")
|
||||
}
|
||||
|
||||
func (itr *{{.name}}AuxIterator) stream() {
|
||||
for {
|
||||
// Read next point.
|
||||
|
@ -1016,10 +993,20 @@ func (itr *{{$k.name}}ChanIterator) Next() (*{{$k.Name}}Point, error) {
|
|||
type {{$k.name}}Reduce{{$v.Name}}Iterator struct {
|
||||
input *buf{{$k.Name}}Iterator
|
||||
create func() ({{$k.Name}}PointAggregator, {{$v.Name}}PointEmitter)
|
||||
dims []string
|
||||
opt IteratorOptions
|
||||
points []{{$v.Name}}Point
|
||||
}
|
||||
|
||||
func new{{$k.Name}}Reduce{{$v.Name}}Iterator(input {{$k.Name}}Iterator, opt IteratorOptions, createFn func() ({{$k.Name}}PointAggregator, {{$v.Name}}PointEmitter)) *{{$k.name}}Reduce{{$v.Name}}Iterator {
|
||||
return &{{$k.name}}Reduce{{$v.Name}}Iterator{
|
||||
input: newBuf{{$k.Name}}Iterator(input),
|
||||
create: createFn,
|
||||
dims: opt.GetDimensions(),
|
||||
opt: opt,
|
||||
}
|
||||
}
|
||||
|
||||
// Stats returns stats from the input iterator.
|
||||
func (itr *{{$k.name}}Reduce{{$v.Name}}Iterator) Stats() IteratorStats { return itr.input.Stats() }
|
||||
|
||||
|
@ -1055,11 +1042,20 @@ type {{$k.name}}Reduce{{$v.Name}}Point struct {
|
|||
// The previous value for the dimension is passed to fn.
|
||||
func (itr *{{$k.name}}Reduce{{$v.Name}}Iterator) reduce() ([]{{$v.Name}}Point, error) {
|
||||
// Calculate next window.
|
||||
t, err := itr.input.peekTime()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
var startTime, endTime int64
|
||||
for {
|
||||
p, err := itr.input.Next()
|
||||
if err != nil || p == nil {
|
||||
return nil, err
|
||||
} else if p.Nil {
|
||||
continue
|
||||
}
|
||||
|
||||
// Unread the point so it can be processed.
|
||||
itr.input.unread(p)
|
||||
startTime, endTime = itr.opt.Window(p.Time)
|
||||
break
|
||||
}
|
||||
startTime, endTime := itr.opt.Window(t)
|
||||
|
||||
// Create points by tags.
|
||||
m := make(map[string]*{{$k.name}}Reduce{{$v.Name}}Point)
|
||||
|
@ -1073,7 +1069,7 @@ func (itr *{{$k.name}}Reduce{{$v.Name}}Iterator) reduce() ([]{{$v.Name}}Point, e
|
|||
} else if curr.Nil {
|
||||
continue
|
||||
}
|
||||
tags := curr.Tags.Subset(itr.opt.Dimensions)
|
||||
tags := curr.Tags.Subset(itr.dims)
|
||||
|
||||
id := curr.Name
|
||||
if len(tags.m) > 0 {
|
||||
|
@ -1084,7 +1080,7 @@ func (itr *{{$k.name}}Reduce{{$v.Name}}Iterator) reduce() ([]{{$v.Name}}Point, e
|
|||
rp := m[id]
|
||||
if rp == nil {
|
||||
aggregator, emitter := itr.create()
|
||||
rp = &{{$k.name}}Reduce{{.Name}}Point{
|
||||
rp = &{{$k.name}}Reduce{{$v.Name}}Point{
|
||||
Name: curr.Name,
|
||||
Tags: tags,
|
||||
Aggregator: aggregator,
|
||||
|
@ -1104,6 +1100,8 @@ func (itr *{{$k.name}}Reduce{{$v.Name}}Iterator) reduce() ([]{{$v.Name}}Point, e
|
|||
sort.Sort(reverseStringSlice(keys))
|
||||
}
|
||||
|
||||
// Assume the points are already sorted until proven otherwise.
|
||||
sortedByTime := true
|
||||
a := make([]{{$v.Name}}Point, 0, len(m))
|
||||
for _, k := range keys {
|
||||
rp := m[k]
|
||||
|
@ -1114,11 +1112,18 @@ func (itr *{{$k.name}}Reduce{{$v.Name}}Iterator) reduce() ([]{{$v.Name}}Point, e
|
|||
// Set the points time to the interval time if the reducer didn't provide one.
|
||||
if points[i].Time == ZeroTime {
|
||||
points[i].Time = startTime
|
||||
} else {
|
||||
sortedByTime = false
|
||||
}
|
||||
a = append(a, points[i])
|
||||
}
|
||||
}
|
||||
|
||||
// Points may be out of order. Perform a stable sort by time if requested.
|
||||
if !sortedByTime && itr.opt.Ordered {
|
||||
sort.Stable(sort.Reverse({{$v.name}}PointsByTime(a)))
|
||||
}
|
||||
|
||||
return a, nil
|
||||
}
|
||||
|
||||
|
@ -1126,6 +1131,7 @@ func (itr *{{$k.name}}Reduce{{$v.Name}}Iterator) reduce() ([]{{$v.Name}}Point, e
|
|||
type {{$k.name}}Stream{{$v.Name}}Iterator struct {
|
||||
input *buf{{$k.Name}}Iterator
|
||||
create func() ({{$k.Name}}PointAggregator, {{$v.Name}}PointEmitter)
|
||||
dims []string
|
||||
opt IteratorOptions
|
||||
m map[string]*{{$k.name}}Reduce{{$v.Name}}Point
|
||||
points []{{$v.Name}}Point
|
||||
|
@ -1136,6 +1142,7 @@ func new{{$k.Name}}Stream{{$v.Name}}Iterator(input {{$k.Name}}Iterator, createFn
|
|||
return &{{$k.name}}Stream{{$v.Name}}Iterator{
|
||||
input: newBuf{{$k.Name}}Iterator(input),
|
||||
create: createFn,
|
||||
dims: opt.GetDimensions(),
|
||||
opt: opt,
|
||||
m: make(map[string]*{{$k.name}}Reduce{{$v.Name}}Point),
|
||||
}
|
||||
|
@ -1175,7 +1182,7 @@ func (itr *{{$k.name}}Stream{{$v.Name}}Iterator) reduce() ([]{{$v.Name}}Point, e
|
|||
} else if curr.Nil {
|
||||
continue
|
||||
}
|
||||
tags := curr.Tags.Subset(itr.opt.Dimensions)
|
||||
tags := curr.Tags.Subset(itr.dims)
|
||||
|
||||
id := curr.Name
|
||||
if len(tags.m) > 0 {
|
||||
|
|
|
@ -4,7 +4,6 @@ import (
|
|||
"errors"
|
||||
"fmt"
|
||||
"io"
|
||||
"sort"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
|
@ -132,18 +131,23 @@ func (a Iterators) Merge(opt IteratorOptions) (Iterator, error) {
|
|||
return nil, nil
|
||||
}
|
||||
|
||||
if opt.Expr != nil {
|
||||
if expr, ok := opt.Expr.(*Call); ok && expr.Name == "count" {
|
||||
opt.Expr = &Call{
|
||||
Name: "sum",
|
||||
Args: expr.Args,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if opt.InterruptCh != nil {
|
||||
itr = NewInterruptIterator(itr, opt.InterruptCh)
|
||||
}
|
||||
|
||||
call, ok := opt.Expr.(*Call)
|
||||
if !ok {
|
||||
// This is not a call expression so do not use a call iterator.
|
||||
return itr, nil
|
||||
}
|
||||
|
||||
// When merging the count() function, use sum() to sum the counted points.
|
||||
if call.Name == "count" {
|
||||
opt.Expr = &Call{
|
||||
Name: "sum",
|
||||
Args: call.Args,
|
||||
}
|
||||
}
|
||||
return NewCallIterator(itr, opt)
|
||||
}
|
||||
|
||||
|
@ -220,6 +224,8 @@ func NewSortedMergeIterator(inputs []Iterator, opt IteratorOptions) Iterator {
|
|||
inputs = Iterators(inputs).filterNonNil()
|
||||
if len(inputs) == 0 {
|
||||
return nil
|
||||
} else if len(inputs) == 1 {
|
||||
return inputs[0]
|
||||
}
|
||||
|
||||
switch inputs := Iterators(inputs).cast().(type) {
|
||||
|
@ -363,7 +369,6 @@ func NewCloseInterruptIterator(input Iterator, closing <-chan struct{}) Iterator
|
|||
// AuxIterator represents an iterator that can split off separate auxiliary iterators.
|
||||
type AuxIterator interface {
|
||||
Iterator
|
||||
IteratorCreator
|
||||
|
||||
// Auxilary iterator
|
||||
Iterator(name string, typ DataType) Iterator
|
||||
|
@ -415,26 +420,32 @@ func (f *auxIteratorField) close() {
|
|||
}
|
||||
}
|
||||
|
||||
type auxIteratorFields []*auxIteratorField
|
||||
type auxIteratorFields struct {
|
||||
fields []*auxIteratorField
|
||||
dimensions []string
|
||||
}
|
||||
|
||||
// newAuxIteratorFields returns a new instance of auxIteratorFields from a list of field names.
|
||||
func newAuxIteratorFields(opt IteratorOptions) auxIteratorFields {
|
||||
fields := make(auxIteratorFields, len(opt.Aux))
|
||||
func newAuxIteratorFields(opt IteratorOptions) *auxIteratorFields {
|
||||
fields := make([]*auxIteratorField, len(opt.Aux))
|
||||
for i, ref := range opt.Aux {
|
||||
fields[i] = &auxIteratorField{name: ref.Val, typ: ref.Type, opt: opt}
|
||||
}
|
||||
return fields
|
||||
return &auxIteratorFields{
|
||||
fields: fields,
|
||||
dimensions: opt.GetDimensions(),
|
||||
}
|
||||
}
|
||||
|
||||
func (a auxIteratorFields) close() {
|
||||
for _, f := range a {
|
||||
func (a *auxIteratorFields) close() {
|
||||
for _, f := range a.fields {
|
||||
f.close()
|
||||
}
|
||||
}
|
||||
|
||||
// iterator creates a new iterator for a named auxilary field.
|
||||
func (a auxIteratorFields) iterator(name string, typ DataType) Iterator {
|
||||
for _, f := range a {
|
||||
func (a *auxIteratorFields) iterator(name string, typ DataType) Iterator {
|
||||
for _, f := range a.fields {
|
||||
// Skip field if it's name doesn't match.
|
||||
// Exit if no points were received by the iterator.
|
||||
if f.name != name || (typ != Unknown && f.typ != typ) {
|
||||
|
@ -468,13 +479,13 @@ func (a auxIteratorFields) iterator(name string, typ DataType) Iterator {
|
|||
}
|
||||
|
||||
// send sends a point to all field iterators.
|
||||
func (a auxIteratorFields) send(p Point) (ok bool) {
|
||||
func (a *auxIteratorFields) send(p Point) (ok bool) {
|
||||
values := p.aux()
|
||||
for i, f := range a {
|
||||
for i, f := range a.fields {
|
||||
v := values[i]
|
||||
|
||||
tags := p.tags()
|
||||
tags = tags.Subset(f.opt.Dimensions)
|
||||
tags = tags.Subset(a.dimensions)
|
||||
|
||||
// Send new point for each aux iterator.
|
||||
// Primitive pointers represent nil values.
|
||||
|
@ -496,8 +507,8 @@ func (a auxIteratorFields) send(p Point) (ok bool) {
|
|||
return ok
|
||||
}
|
||||
|
||||
func (a auxIteratorFields) sendError(err error) {
|
||||
for _, f := range a {
|
||||
func (a *auxIteratorFields) sendError(err error) {
|
||||
for _, f := range a.fields {
|
||||
for _, itr := range f.itrs {
|
||||
switch itr := itr.(type) {
|
||||
case *floatChanIterator:
|
||||
|
@ -591,112 +602,14 @@ func NewReaderIterator(r io.Reader, typ DataType, stats IteratorStats) Iterator
|
|||
// IteratorCreator is an interface to create Iterators.
|
||||
type IteratorCreator interface {
|
||||
// Creates a simple iterator for use in an InfluxQL query.
|
||||
CreateIterator(opt IteratorOptions) (Iterator, error)
|
||||
|
||||
// Returns the unique fields and dimensions across a list of sources.
|
||||
FieldDimensions(sources Sources) (fields map[string]DataType, dimensions map[string]struct{}, err error)
|
||||
|
||||
// Expands regex sources to all matching sources.
|
||||
ExpandSources(sources Sources) (Sources, error)
|
||||
CreateIterator(source *Measurement, opt IteratorOptions) (Iterator, error)
|
||||
}
|
||||
|
||||
// IteratorCreators represents a list of iterator creators.
|
||||
type IteratorCreators []IteratorCreator
|
||||
// FieldMapper returns the data type for the field inside of the measurement.
|
||||
type FieldMapper interface {
|
||||
FieldDimensions(m *Measurement) (fields map[string]DataType, dimensions map[string]struct{}, err error)
|
||||
|
||||
// Close closes all iterator creators that implement io.Closer.
|
||||
func (a IteratorCreators) Close() error {
|
||||
for _, ic := range a {
|
||||
if ic, ok := ic.(io.Closer); ok {
|
||||
ic.Close()
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// CreateIterator returns a single combined iterator from multiple iterator creators.
|
||||
func (a IteratorCreators) CreateIterator(opt IteratorOptions) (Iterator, error) {
|
||||
// Create iterators for each shard.
|
||||
// Ensure that they are closed if an error occurs.
|
||||
itrs := make([]Iterator, 0, len(a))
|
||||
if err := func() error {
|
||||
for _, ic := range a {
|
||||
itr, err := ic.CreateIterator(opt)
|
||||
if err != nil {
|
||||
return err
|
||||
} else if itr == nil {
|
||||
continue
|
||||
}
|
||||
itrs = append(itrs, itr)
|
||||
}
|
||||
return nil
|
||||
}(); err != nil {
|
||||
Iterators(itrs).Close()
|
||||
return nil, err
|
||||
}
|
||||
|
||||
if len(itrs) == 0 {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
return Iterators(itrs).Merge(opt)
|
||||
}
|
||||
|
||||
// FieldDimensions returns unique fields and dimensions from multiple iterator creators.
|
||||
func (a IteratorCreators) FieldDimensions(sources Sources) (fields map[string]DataType, dimensions map[string]struct{}, err error) {
|
||||
fields = make(map[string]DataType)
|
||||
dimensions = make(map[string]struct{})
|
||||
|
||||
for _, ic := range a {
|
||||
f, d, err := ic.FieldDimensions(sources)
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
for k, typ := range f {
|
||||
if _, ok := fields[k]; typ != Unknown && (!ok || typ < fields[k]) {
|
||||
fields[k] = typ
|
||||
}
|
||||
}
|
||||
for k := range d {
|
||||
dimensions[k] = struct{}{}
|
||||
}
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
// ExpandSources expands sources across all iterator creators and returns a unique result.
|
||||
func (a IteratorCreators) ExpandSources(sources Sources) (Sources, error) {
|
||||
m := make(map[string]Source)
|
||||
|
||||
for _, ic := range a {
|
||||
expanded, err := ic.ExpandSources(sources)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
for _, src := range expanded {
|
||||
switch src := src.(type) {
|
||||
case *Measurement:
|
||||
m[src.String()] = src
|
||||
default:
|
||||
return nil, fmt.Errorf("IteratorCreators.ExpandSources: unsupported source type: %T", src)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Convert set to sorted slice.
|
||||
names := make([]string, 0, len(m))
|
||||
for name := range m {
|
||||
names = append(names, name)
|
||||
}
|
||||
sort.Strings(names)
|
||||
|
||||
// Convert set to a list of Sources.
|
||||
sorted := make(Sources, 0, len(m))
|
||||
for _, name := range names {
|
||||
sorted = append(sorted, m[name])
|
||||
}
|
||||
|
||||
return sorted, nil
|
||||
TypeMapper
|
||||
}
|
||||
|
||||
// IteratorOptions is an object passed to CreateIterator to specify creation options.
|
||||
|
@ -708,12 +621,14 @@ type IteratorOptions struct {
|
|||
// Auxilary tags or values to also retrieve for the point.
|
||||
Aux []VarRef
|
||||
|
||||
// Data sources from which to retrieve data.
|
||||
// Data sources from which to receive data. This is only used for encoding
|
||||
// measurements over RPC and is no longer used in the open source version.
|
||||
Sources []Source
|
||||
|
||||
// Group by interval and tags.
|
||||
Interval Interval
|
||||
Dimensions []string
|
||||
Dimensions []string // The final dimensions of the query (stays the same even in subqueries).
|
||||
GroupBy map[string]struct{} // Dimensions to group points by in intermediate iterators.
|
||||
|
||||
// Fill options.
|
||||
Fill FillOption
|
||||
|
@ -738,6 +653,12 @@ type IteratorOptions struct {
|
|||
// Removes duplicate rows from raw queries.
|
||||
Dedupe bool
|
||||
|
||||
// Determines if this is a query for raw data or an aggregate/selector.
|
||||
Ordered bool
|
||||
|
||||
// Limits on the creation of iterators.
|
||||
MaxSeriesN int
|
||||
|
||||
// If this channel is set and is closed, the iterator should try to exit
|
||||
// and close as soon as possible.
|
||||
InterruptCh <-chan struct{}
|
||||
|
@ -786,14 +707,18 @@ func newIteratorOptionsStmt(stmt *SelectStatement, sopt *SelectOptions) (opt Ite
|
|||
}
|
||||
opt.Interval.Duration = interval
|
||||
|
||||
// Determine if the input for this select call must be ordered.
|
||||
opt.Ordered = stmt.IsRawQuery
|
||||
|
||||
// Determine dimensions.
|
||||
opt.GroupBy = make(map[string]struct{}, len(opt.Dimensions))
|
||||
for _, d := range stmt.Dimensions {
|
||||
if d, ok := d.Expr.(*VarRef); ok {
|
||||
opt.Dimensions = append(opt.Dimensions, d.Val)
|
||||
opt.GroupBy[d.Val] = struct{}{}
|
||||
}
|
||||
}
|
||||
|
||||
opt.Sources = stmt.Sources
|
||||
opt.Condition = stmt.Condition
|
||||
opt.Ascending = stmt.TimeAscending()
|
||||
opt.Dedupe = stmt.Dedupe
|
||||
|
@ -808,14 +733,52 @@ func newIteratorOptionsStmt(stmt *SelectStatement, sopt *SelectOptions) (opt Ite
|
|||
opt.Limit, opt.Offset = stmt.Limit, stmt.Offset
|
||||
opt.SLimit, opt.SOffset = stmt.SLimit, stmt.SOffset
|
||||
if sopt != nil {
|
||||
opt.MaxSeriesN = sopt.MaxSeriesN
|
||||
opt.InterruptCh = sopt.InterruptCh
|
||||
}
|
||||
|
||||
return opt, nil
|
||||
}
|
||||
|
||||
func newIteratorOptionsSubstatement(stmt *SelectStatement, opt IteratorOptions) (IteratorOptions, error) {
|
||||
subOpt, err := newIteratorOptionsStmt(stmt, nil)
|
||||
if err != nil {
|
||||
return IteratorOptions{}, err
|
||||
}
|
||||
|
||||
if subOpt.StartTime < opt.StartTime {
|
||||
subOpt.StartTime = opt.StartTime
|
||||
}
|
||||
if subOpt.EndTime > opt.EndTime {
|
||||
subOpt.EndTime = opt.EndTime
|
||||
}
|
||||
subOpt.Dimensions = opt.Dimensions
|
||||
subOpt.InterruptCh = opt.InterruptCh
|
||||
|
||||
// Propagate the SLIMIT and SOFFSET from the outer query.
|
||||
subOpt.SLimit += opt.SLimit
|
||||
subOpt.SOffset += opt.SOffset
|
||||
|
||||
// If the inner query uses a null fill option, switch it to none so we
|
||||
// don't hit an unnecessary penalty from the fill iterator. Null values
|
||||
// will end up getting stripped by an outer query anyway so there's no
|
||||
// point in having them here. We still need all other types of fill
|
||||
// iterators because they can affect the result of the outer query.
|
||||
if subOpt.Fill == NullFill {
|
||||
subOpt.Fill = NoFill
|
||||
}
|
||||
|
||||
// Determine if the input to this iterator needs to be ordered so it outputs
|
||||
// the correct order to the outer query.
|
||||
interval, err := stmt.GroupByInterval()
|
||||
if err != nil {
|
||||
return IteratorOptions{}, err
|
||||
}
|
||||
subOpt.Ordered = opt.Ordered && (interval == 0 && stmt.HasSelector())
|
||||
return subOpt, nil
|
||||
}
|
||||
|
||||
// MergeSorted returns true if the options require a sorted merge.
|
||||
// This is only needed when the expression is a variable reference or there is no expr.
|
||||
func (opt IteratorOptions) MergeSorted() bool {
|
||||
if opt.Expr == nil {
|
||||
return true
|
||||
|
@ -882,6 +845,18 @@ func (opt IteratorOptions) ElapsedInterval() Interval {
|
|||
return Interval{Duration: time.Nanosecond}
|
||||
}
|
||||
|
||||
// GetDimensions retrieves the dimensions for this query.
|
||||
func (opt IteratorOptions) GetDimensions() []string {
|
||||
if len(opt.GroupBy) > 0 {
|
||||
dimensions := make([]string, 0, len(opt.GroupBy))
|
||||
for dim := range opt.GroupBy {
|
||||
dimensions = append(dimensions, dim)
|
||||
}
|
||||
return dimensions
|
||||
}
|
||||
return opt.Dimensions
|
||||
}
|
||||
|
||||
// MarshalBinary encodes opt into a binary format.
|
||||
func (opt *IteratorOptions) MarshalBinary() ([]byte, error) {
|
||||
return proto.Marshal(encodeIteratorOptions(opt))
|
||||
|
@ -916,6 +891,8 @@ func encodeIteratorOptions(opt *IteratorOptions) *internal.IteratorOptions {
|
|||
SLimit: proto.Int64(int64(opt.SLimit)),
|
||||
SOffset: proto.Int64(int64(opt.SOffset)),
|
||||
Dedupe: proto.Bool(opt.Dedupe),
|
||||
MaxSeriesN: proto.Int64(int64(opt.MaxSeriesN)),
|
||||
Ordered: proto.Bool(opt.Ordered),
|
||||
}
|
||||
|
||||
// Set expression, if set.
|
||||
|
@ -931,13 +908,24 @@ func encodeIteratorOptions(opt *IteratorOptions) *internal.IteratorOptions {
|
|||
pb.Aux[i] = ref.Val
|
||||
}
|
||||
|
||||
// Convert and encode sources to measurements.
|
||||
sources := make([]*internal.Measurement, len(opt.Sources))
|
||||
for i, source := range opt.Sources {
|
||||
mm := source.(*Measurement)
|
||||
sources[i] = encodeMeasurement(mm)
|
||||
// Encode group by dimensions from a map.
|
||||
if pb.GroupBy != nil {
|
||||
dimensions := make([]string, 0, len(opt.GroupBy))
|
||||
for dim := range opt.GroupBy {
|
||||
dimensions = append(dimensions, dim)
|
||||
}
|
||||
pb.GroupBy = dimensions
|
||||
}
|
||||
|
||||
// Convert and encode sources to measurements.
|
||||
if opt.Sources != nil {
|
||||
sources := make([]*internal.Measurement, len(opt.Sources))
|
||||
for i, source := range opt.Sources {
|
||||
mm := source.(*Measurement)
|
||||
sources[i] = encodeMeasurement(mm)
|
||||
}
|
||||
pb.Sources = sources
|
||||
}
|
||||
pb.Sources = sources
|
||||
|
||||
// Fill value can only be a number. Set it if available.
|
||||
if v, ok := opt.FillValue.(float64); ok {
|
||||
|
@ -966,6 +954,8 @@ func decodeIteratorOptions(pb *internal.IteratorOptions) (*IteratorOptions, erro
|
|||
SLimit: int(pb.GetSLimit()),
|
||||
SOffset: int(pb.GetSOffset()),
|
||||
Dedupe: pb.GetDedupe(),
|
||||
MaxSeriesN: int(pb.GetMaxSeriesN()),
|
||||
Ordered: pb.GetOrdered(),
|
||||
}
|
||||
|
||||
// Set expression, if set.
|
||||
|
@ -990,16 +980,27 @@ func decodeIteratorOptions(pb *internal.IteratorOptions) (*IteratorOptions, erro
|
|||
}
|
||||
}
|
||||
|
||||
// Convert and dencode sources to measurements.
|
||||
sources := make([]Source, len(pb.GetSources()))
|
||||
for i, source := range pb.GetSources() {
|
||||
mm, err := decodeMeasurement(source)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
// Convert and decode sources to measurements.
|
||||
if pb.Sources != nil {
|
||||
sources := make([]Source, len(pb.GetSources()))
|
||||
for i, source := range pb.GetSources() {
|
||||
mm, err := decodeMeasurement(source)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
sources[i] = mm
|
||||
}
|
||||
sources[i] = mm
|
||||
opt.Sources = sources
|
||||
}
|
||||
|
||||
// Convert group by dimensions to a map.
|
||||
if pb.GroupBy != nil {
|
||||
dimensions := make(map[string]struct{}, len(pb.GroupBy))
|
||||
for _, dim := range pb.GetGroupBy() {
|
||||
dimensions[dim] = struct{}{}
|
||||
}
|
||||
opt.GroupBy = dimensions
|
||||
}
|
||||
opt.Sources = sources
|
||||
|
||||
// Set condition, if set.
|
||||
if pb.Condition != nil {
|
||||
|
@ -1041,6 +1042,22 @@ func (v *selectInfo) Visit(n Node) Visitor {
|
|||
return v
|
||||
}
|
||||
|
||||
// FindSelector returns a selector from the selectInfo. This will only
|
||||
// return a selector if the Call is a selector and it's the only function
|
||||
// in the selectInfo.
|
||||
func (v *selectInfo) FindSelector() *Call {
|
||||
if len(v.calls) != 1 {
|
||||
return nil
|
||||
}
|
||||
|
||||
for s := range v.calls {
|
||||
if IsSelector(s) {
|
||||
return s
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// Interval represents a repeating interval for a query.
|
||||
type Interval struct {
|
||||
Duration time.Duration
|
||||
|
|
|
@ -0,0 +1,49 @@
|
|||
package influxql
|
||||
|
||||
type iteratorMapper struct {
|
||||
e *Emitter
|
||||
buf []interface{}
|
||||
fields []int // which iterator to use for an aux field
|
||||
auxFields []interface{}
|
||||
}
|
||||
|
||||
func NewIteratorMapper(itrs []Iterator, fields []int, opt IteratorOptions) Iterator {
|
||||
e := NewEmitter(itrs, opt.Ascending, 0)
|
||||
e.OmitTime = true
|
||||
return &iteratorMapper{
|
||||
e: e,
|
||||
buf: make([]interface{}, len(itrs)),
|
||||
fields: fields,
|
||||
auxFields: make([]interface{}, len(fields)),
|
||||
}
|
||||
}
|
||||
|
||||
func (itr *iteratorMapper) Next() (*FloatPoint, error) {
|
||||
t, name, tags, err := itr.e.loadBuf()
|
||||
if err != nil || t == ZeroTime {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
itr.e.readInto(t, name, tags, itr.buf)
|
||||
for i, f := range itr.fields {
|
||||
itr.auxFields[i] = itr.buf[f]
|
||||
}
|
||||
return &FloatPoint{
|
||||
Name: name,
|
||||
Tags: tags,
|
||||
Time: t,
|
||||
Aux: itr.auxFields,
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (itr *iteratorMapper) Stats() IteratorStats {
|
||||
stats := IteratorStats{}
|
||||
for _, itr := range itr.e.itrs {
|
||||
stats.Add(itr.Stats())
|
||||
}
|
||||
return stats
|
||||
}
|
||||
|
||||
func (itr *iteratorMapper) Close() error {
|
||||
return itr.e.Close()
|
||||
}
|
|
@ -0,0 +1,58 @@
|
|||
package influxql_test
|
||||
|
||||
import (
|
||||
"testing"
|
||||
|
||||
"github.com/davecgh/go-spew/spew"
|
||||
"github.com/influxdata/influxdb/influxql"
|
||||
"github.com/influxdata/influxdb/pkg/deep"
|
||||
)
|
||||
|
||||
func TestIteratorMapper(t *testing.T) {
|
||||
val1itr := &FloatIterator{Points: []influxql.FloatPoint{
|
||||
{Name: "cpu", Tags: ParseTags("host=A"), Time: 0, Value: 1},
|
||||
{Name: "cpu", Tags: ParseTags("host=A"), Time: 5, Value: 3},
|
||||
{Name: "cpu", Tags: ParseTags("host=B"), Time: 2, Value: 2},
|
||||
{Name: "cpu", Tags: ParseTags("host=B"), Time: 8, Value: 8},
|
||||
}}
|
||||
|
||||
val2itr := &StringIterator{Points: []influxql.StringPoint{
|
||||
{Name: "cpu", Tags: ParseTags("host=A"), Time: 0, Value: "a"},
|
||||
{Name: "cpu", Tags: ParseTags("host=A"), Time: 5, Value: "c"},
|
||||
{Name: "cpu", Tags: ParseTags("host=B"), Time: 2, Value: "b"},
|
||||
{Name: "cpu", Tags: ParseTags("host=B"), Time: 8, Value: "h"},
|
||||
}}
|
||||
inputs := []influxql.Iterator{val1itr, val2itr}
|
||||
|
||||
opt := influxql.IteratorOptions{
|
||||
Ascending: true,
|
||||
Aux: []influxql.VarRef{
|
||||
{Val: "val1", Type: influxql.Float},
|
||||
{Val: "val2", Type: influxql.String},
|
||||
},
|
||||
}
|
||||
itr := influxql.NewIteratorMapper(inputs, []int{0, 1}, opt)
|
||||
if a, err := Iterators([]influxql.Iterator{itr}).ReadAll(); err != nil {
|
||||
t.Fatalf("unexpected error: %s", err)
|
||||
} else if !deep.Equal(a, [][]influxql.Point{
|
||||
{&influxql.FloatPoint{Name: "cpu", Tags: ParseTags("host=A"), Time: 0, Aux: []interface{}{float64(1), "a"}}},
|
||||
{&influxql.FloatPoint{Name: "cpu", Tags: ParseTags("host=A"), Time: 5, Aux: []interface{}{float64(3), "c"}}},
|
||||
{&influxql.FloatPoint{Name: "cpu", Tags: ParseTags("host=B"), Time: 2, Aux: []interface{}{float64(2), "b"}}},
|
||||
{&influxql.FloatPoint{Name: "cpu", Tags: ParseTags("host=B"), Time: 8, Aux: []interface{}{float64(8), "h"}}},
|
||||
}) {
|
||||
t.Errorf("unexpected points: %s", spew.Sdump(a))
|
||||
}
|
||||
|
||||
for i, input := range inputs {
|
||||
switch input := input.(type) {
|
||||
case *FloatIterator:
|
||||
if !input.Closed {
|
||||
t.Errorf("iterator %d not closed", i)
|
||||
}
|
||||
case *StringIterator:
|
||||
if !input.Closed {
|
||||
t.Errorf("iterator %d not closed", i)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -6,7 +6,6 @@ import (
|
|||
"math"
|
||||
"math/rand"
|
||||
"reflect"
|
||||
"regexp"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
|
@ -39,7 +38,8 @@ func TestMergeIterator_Float(t *testing.T) {
|
|||
Interval: influxql.Interval{
|
||||
Duration: 10 * time.Nanosecond,
|
||||
},
|
||||
Ascending: true,
|
||||
Dimensions: []string{"host"},
|
||||
Ascending: true,
|
||||
})
|
||||
if a, err := Iterators([]influxql.Iterator{itr}).ReadAll(); err != nil {
|
||||
t.Fatalf("unexpected error: %s", err)
|
||||
|
@ -86,7 +86,8 @@ func TestMergeIterator_Integer(t *testing.T) {
|
|||
Interval: influxql.Interval{
|
||||
Duration: 10 * time.Nanosecond,
|
||||
},
|
||||
Ascending: true,
|
||||
Dimensions: []string{"host"},
|
||||
Ascending: true,
|
||||
})
|
||||
|
||||
if a, err := Iterators([]influxql.Iterator{itr}).ReadAll(); err != nil {
|
||||
|
@ -134,7 +135,8 @@ func TestMergeIterator_String(t *testing.T) {
|
|||
Interval: influxql.Interval{
|
||||
Duration: 10 * time.Nanosecond,
|
||||
},
|
||||
Ascending: true,
|
||||
Dimensions: []string{"host"},
|
||||
Ascending: true,
|
||||
})
|
||||
|
||||
if a, err := Iterators([]influxql.Iterator{itr}).ReadAll(); err != nil {
|
||||
|
@ -182,7 +184,8 @@ func TestMergeIterator_Boolean(t *testing.T) {
|
|||
Interval: influxql.Interval{
|
||||
Duration: 10 * time.Nanosecond,
|
||||
},
|
||||
Ascending: true,
|
||||
Dimensions: []string{"host"},
|
||||
Ascending: true,
|
||||
})
|
||||
|
||||
if a, err := Iterators([]influxql.Iterator{itr}).ReadAll(); err != nil {
|
||||
|
@ -236,7 +239,8 @@ func TestMergeIterator_Cast_Float(t *testing.T) {
|
|||
Interval: influxql.Interval{
|
||||
Duration: 10 * time.Nanosecond,
|
||||
},
|
||||
Ascending: true,
|
||||
Dimensions: []string{"host"},
|
||||
Ascending: true,
|
||||
})
|
||||
if a, err := Iterators([]influxql.Iterator{itr}).ReadAll(); err != nil {
|
||||
t.Fatalf("unexpected error: %s", err)
|
||||
|
@ -290,7 +294,8 @@ func TestSortedMergeIterator_Float(t *testing.T) {
|
|||
Interval: influxql.Interval{
|
||||
Duration: 10 * time.Nanosecond,
|
||||
},
|
||||
Ascending: true,
|
||||
Dimensions: []string{"host"},
|
||||
Ascending: true,
|
||||
})
|
||||
if a, err := Iterators([]influxql.Iterator{itr}).ReadAll(); err != nil {
|
||||
t.Fatalf("unexpected error: %s", err)
|
||||
|
@ -337,7 +342,8 @@ func TestSortedMergeIterator_Integer(t *testing.T) {
|
|||
Interval: influxql.Interval{
|
||||
Duration: 10 * time.Nanosecond,
|
||||
},
|
||||
Ascending: true,
|
||||
Dimensions: []string{"host"},
|
||||
Ascending: true,
|
||||
})
|
||||
if a, err := Iterators([]influxql.Iterator{itr}).ReadAll(); err != nil {
|
||||
t.Fatalf("unexpected error: %s", err)
|
||||
|
@ -384,7 +390,8 @@ func TestSortedMergeIterator_String(t *testing.T) {
|
|||
Interval: influxql.Interval{
|
||||
Duration: 10 * time.Nanosecond,
|
||||
},
|
||||
Ascending: true,
|
||||
Dimensions: []string{"host"},
|
||||
Ascending: true,
|
||||
})
|
||||
if a, err := Iterators([]influxql.Iterator{itr}).ReadAll(); err != nil {
|
||||
t.Fatalf("unexpected error: %s", err)
|
||||
|
@ -431,7 +438,8 @@ func TestSortedMergeIterator_Boolean(t *testing.T) {
|
|||
Interval: influxql.Interval{
|
||||
Duration: 10 * time.Nanosecond,
|
||||
},
|
||||
Ascending: true,
|
||||
Dimensions: []string{"host"},
|
||||
Ascending: true,
|
||||
})
|
||||
if a, err := Iterators([]influxql.Iterator{itr}).ReadAll(); err != nil {
|
||||
t.Fatalf("unexpected error: %s", err)
|
||||
|
@ -484,7 +492,8 @@ func TestSortedMergeIterator_Cast_Float(t *testing.T) {
|
|||
Interval: influxql.Interval{
|
||||
Duration: 10 * time.Nanosecond,
|
||||
},
|
||||
Ascending: true,
|
||||
Dimensions: []string{"host"},
|
||||
Ascending: true,
|
||||
})
|
||||
if a, err := Iterators([]influxql.Iterator{itr}).ReadAll(); err != nil {
|
||||
t.Fatalf("unexpected error: %s", err)
|
||||
|
@ -832,26 +841,6 @@ func TestIteratorOptions_SeekTime_Descending(t *testing.T) {
|
|||
}
|
||||
}
|
||||
|
||||
func TestIteratorOptions_MergeSorted(t *testing.T) {
|
||||
opt := influxql.IteratorOptions{}
|
||||
sorted := opt.MergeSorted()
|
||||
if !sorted {
|
||||
t.Error("expected no expression to be sorted, got unsorted")
|
||||
}
|
||||
|
||||
opt.Expr = &influxql.VarRef{}
|
||||
sorted = opt.MergeSorted()
|
||||
if !sorted {
|
||||
t.Error("expected expression with varref to be sorted, got unsorted")
|
||||
}
|
||||
|
||||
opt.Expr = &influxql.Call{}
|
||||
sorted = opt.MergeSorted()
|
||||
if sorted {
|
||||
t.Error("expected expression without varref to be unsorted, got sorted")
|
||||
}
|
||||
}
|
||||
|
||||
func TestIteratorOptions_DerivativeInterval_Default(t *testing.T) {
|
||||
opt := influxql.IteratorOptions{}
|
||||
expected := influxql.Interval{Duration: time.Second}
|
||||
|
@ -945,9 +934,6 @@ func TestIteratorOptions_MarshalBinary(t *testing.T) {
|
|||
opt := &influxql.IteratorOptions{
|
||||
Expr: MustParseExpr("count(value)"),
|
||||
Aux: []influxql.VarRef{{Val: "a"}, {Val: "b"}, {Val: "c"}},
|
||||
Sources: []influxql.Source{
|
||||
&influxql.Measurement{Database: "db0", RetentionPolicy: "rp0", Name: "mm0"},
|
||||
},
|
||||
Interval: influxql.Interval{
|
||||
Duration: 1 * time.Hour,
|
||||
Offset: 20 * time.Minute,
|
||||
|
@ -981,29 +967,6 @@ func TestIteratorOptions_MarshalBinary(t *testing.T) {
|
|||
}
|
||||
}
|
||||
|
||||
// Ensure iterator options with a regex measurement can be marshaled.
|
||||
func TestIteratorOptions_MarshalBinary_Measurement_Regex(t *testing.T) {
|
||||
opt := &influxql.IteratorOptions{
|
||||
Sources: []influxql.Source{
|
||||
&influxql.Measurement{Database: "db1", RetentionPolicy: "rp2", Regex: &influxql.RegexLiteral{Val: regexp.MustCompile(`series.+`)}},
|
||||
},
|
||||
}
|
||||
|
||||
// Marshal to binary.
|
||||
buf, err := opt.MarshalBinary()
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
// Unmarshal back to an object.
|
||||
var other influxql.IteratorOptions
|
||||
if err := other.UnmarshalBinary(buf); err != nil {
|
||||
t.Fatal(err)
|
||||
} else if v := other.Sources[0].(*influxql.Measurement).Regex.Val.String(); v != `series.+` {
|
||||
t.Fatalf("unexpected measurement regex: %s", v)
|
||||
}
|
||||
}
|
||||
|
||||
// Ensure iterator can be encoded and decoded over a byte stream.
|
||||
func TestIterator_EncodeDecode(t *testing.T) {
|
||||
var buf bytes.Buffer
|
||||
|
@ -1056,21 +1019,31 @@ func TestIterator_EncodeDecode(t *testing.T) {
|
|||
|
||||
// IteratorCreator is a mockable implementation of SelectStatementExecutor.IteratorCreator.
|
||||
type IteratorCreator struct {
|
||||
CreateIteratorFn func(opt influxql.IteratorOptions) (influxql.Iterator, error)
|
||||
FieldDimensionsFn func(sources influxql.Sources) (fields map[string]influxql.DataType, dimensions map[string]struct{}, err error)
|
||||
ExpandSourcesFn func(sources influxql.Sources) (influxql.Sources, error)
|
||||
CreateIteratorFn func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error)
|
||||
FieldDimensionsFn func(m *influxql.Measurement) (fields map[string]influxql.DataType, dimensions map[string]struct{}, err error)
|
||||
}
|
||||
|
||||
func (ic *IteratorCreator) CreateIterator(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
return ic.CreateIteratorFn(opt)
|
||||
func (ic *IteratorCreator) CreateIterator(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
return ic.CreateIteratorFn(m, opt)
|
||||
}
|
||||
|
||||
func (ic *IteratorCreator) FieldDimensions(sources influxql.Sources) (fields map[string]influxql.DataType, dimensions map[string]struct{}, err error) {
|
||||
return ic.FieldDimensionsFn(sources)
|
||||
func (ic *IteratorCreator) FieldDimensions(m *influxql.Measurement) (fields map[string]influxql.DataType, dimensions map[string]struct{}, err error) {
|
||||
return ic.FieldDimensionsFn(m)
|
||||
}
|
||||
|
||||
func (ic *IteratorCreator) ExpandSources(sources influxql.Sources) (influxql.Sources, error) {
|
||||
return ic.ExpandSourcesFn(sources)
|
||||
func (ic *IteratorCreator) MapType(m *influxql.Measurement, field string) influxql.DataType {
|
||||
f, d, err := ic.FieldDimensions(m)
|
||||
if err != nil {
|
||||
return influxql.Unknown
|
||||
}
|
||||
|
||||
if typ, ok := f[field]; ok {
|
||||
return typ
|
||||
}
|
||||
if _, ok := d[field]; ok {
|
||||
return influxql.Tag
|
||||
}
|
||||
return influxql.Unknown
|
||||
}
|
||||
|
||||
// Test implementation of influxql.FloatIterator
|
||||
|
|
|
@ -928,7 +928,7 @@ func (p *Parser) parseSelectStatement(tr targetRequirement) (*SelectStatement, e
|
|||
if tok, pos, lit := p.scanIgnoreWhitespace(); tok != FROM {
|
||||
return nil, newParseError(tokstr(tok, lit), []string{"FROM"}, pos)
|
||||
}
|
||||
if stmt.Sources, err = p.parseSources(); err != nil {
|
||||
if stmt.Sources, err = p.parseSources(true); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
|
@ -993,6 +993,7 @@ type targetRequirement int
|
|||
const (
|
||||
targetRequired targetRequirement = iota
|
||||
targetNotRequired
|
||||
targetSubquery
|
||||
)
|
||||
|
||||
// parseTarget parses a string and returns a Target.
|
||||
|
@ -1049,7 +1050,7 @@ func (p *Parser) parseDeleteStatement() (Statement, error) {
|
|||
|
||||
if tok == FROM {
|
||||
// Parse source.
|
||||
if stmt.Sources, err = p.parseSources(); err != nil {
|
||||
if stmt.Sources, err = p.parseSources(false); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
|
@ -1107,7 +1108,7 @@ func (p *Parser) parseShowSeriesStatement() (*ShowSeriesStatement, error) {
|
|||
|
||||
// Parse optional FROM.
|
||||
if tok, _, _ := p.scanIgnoreWhitespace(); tok == FROM {
|
||||
if stmt.Sources, err = p.parseSources(); err != nil {
|
||||
if stmt.Sources, err = p.parseSources(false); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
} else {
|
||||
|
@ -1166,7 +1167,7 @@ func (p *Parser) parseShowMeasurementsStatement() (*ShowMeasurementsStatement, e
|
|||
switch tok {
|
||||
case EQ, EQREGEX:
|
||||
// Parse required source (measurement name or regex).
|
||||
if stmt.Source, err = p.parseSource(); err != nil {
|
||||
if stmt.Source, err = p.parseSource(false); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
default:
|
||||
|
@ -1245,7 +1246,7 @@ func (p *Parser) parseShowTagKeysStatement() (*ShowTagKeysStatement, error) {
|
|||
|
||||
// Parse optional source.
|
||||
if tok, _, _ := p.scanIgnoreWhitespace(); tok == FROM {
|
||||
if stmt.Sources, err = p.parseSources(); err != nil {
|
||||
if stmt.Sources, err = p.parseSources(false); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
} else {
|
||||
|
@ -1304,7 +1305,7 @@ func (p *Parser) parseShowTagValuesStatement() (*ShowTagValuesStatement, error)
|
|||
|
||||
// Parse optional source.
|
||||
if tok, _, _ := p.scanIgnoreWhitespace(); tok == FROM {
|
||||
if stmt.Sources, err = p.parseSources(); err != nil {
|
||||
if stmt.Sources, err = p.parseSources(false); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
} else {
|
||||
|
@ -1420,7 +1421,7 @@ func (p *Parser) parseShowFieldKeysStatement() (*ShowFieldKeysStatement, error)
|
|||
|
||||
// Parse optional source.
|
||||
if tok, _, _ := p.scanIgnoreWhitespace(); tok == FROM {
|
||||
if stmt.Sources, err = p.parseSources(); err != nil {
|
||||
if stmt.Sources, err = p.parseSources(false); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
} else {
|
||||
|
@ -1470,7 +1471,7 @@ func (p *Parser) parseDropSeriesStatement() (*DropSeriesStatement, error) {
|
|||
|
||||
if tok == FROM {
|
||||
// Parse source.
|
||||
if stmt.Sources, err = p.parseSources(); err != nil {
|
||||
if stmt.Sources, err = p.parseSources(false); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
|
@ -2050,11 +2051,11 @@ func (p *Parser) parseAlias() (string, error) {
|
|||
}
|
||||
|
||||
// parseSources parses a comma delimited list of sources.
|
||||
func (p *Parser) parseSources() (Sources, error) {
|
||||
func (p *Parser) parseSources(subqueries bool) (Sources, error) {
|
||||
var sources Sources
|
||||
|
||||
for {
|
||||
s, err := p.parseSource()
|
||||
s, err := p.parseSource(subqueries)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -2079,7 +2080,7 @@ func (p *Parser) peekRune() rune {
|
|||
return r
|
||||
}
|
||||
|
||||
func (p *Parser) parseSource() (Source, error) {
|
||||
func (p *Parser) parseSource(subqueries bool) (Source, error) {
|
||||
m := &Measurement{}
|
||||
|
||||
// Attempt to parse a regex.
|
||||
|
@ -2092,6 +2093,28 @@ func (p *Parser) parseSource() (Source, error) {
|
|||
return m, nil
|
||||
}
|
||||
|
||||
// If there is no regular expression, this might be a subquery.
|
||||
// Parse the subquery if we are in a query that allows them as a source.
|
||||
if m.Regex == nil && subqueries {
|
||||
if tok, _, _ := p.scanIgnoreWhitespace(); tok == LPAREN {
|
||||
if err := p.parseTokens([]Token{SELECT}); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
stmt, err := p.parseSelectStatement(targetSubquery)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
if err := p.parseTokens([]Token{RPAREN}); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return &SubQuery{Statement: stmt}, nil
|
||||
} else {
|
||||
p.unscan()
|
||||
}
|
||||
}
|
||||
|
||||
// Didn't find a regex so parse segmented identifiers.
|
||||
idents, err := p.parseSegmentedIdents()
|
||||
if err != nil {
|
||||
|
|
|
@ -1103,6 +1103,151 @@ func TestParser_ParseStatement(t *testing.T) {
|
|||
},
|
||||
},
|
||||
|
||||
// SELECT statement with a subquery
|
||||
{
|
||||
s: `SELECT sum(derivative) FROM (SELECT derivative(value) FROM cpu GROUP BY host) WHERE time >= now() - 1d GROUP BY time(1h)`,
|
||||
stmt: &influxql.SelectStatement{
|
||||
Fields: []*influxql.Field{{
|
||||
Expr: &influxql.Call{
|
||||
Name: "sum",
|
||||
Args: []influxql.Expr{
|
||||
&influxql.VarRef{Val: "derivative"},
|
||||
}},
|
||||
}},
|
||||
Dimensions: []*influxql.Dimension{{
|
||||
Expr: &influxql.Call{
|
||||
Name: "time",
|
||||
Args: []influxql.Expr{
|
||||
&influxql.DurationLiteral{Val: time.Hour},
|
||||
},
|
||||
},
|
||||
}},
|
||||
Sources: []influxql.Source{
|
||||
&influxql.SubQuery{
|
||||
Statement: &influxql.SelectStatement{
|
||||
Fields: []*influxql.Field{{
|
||||
Expr: &influxql.Call{
|
||||
Name: "derivative",
|
||||
Args: []influxql.Expr{
|
||||
&influxql.VarRef{Val: "value"},
|
||||
},
|
||||
},
|
||||
}},
|
||||
Dimensions: []*influxql.Dimension{{
|
||||
Expr: &influxql.VarRef{Val: "host"},
|
||||
}},
|
||||
Sources: []influxql.Source{
|
||||
&influxql.Measurement{Name: "cpu"},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
Condition: &influxql.BinaryExpr{
|
||||
Op: influxql.GTE,
|
||||
LHS: &influxql.VarRef{Val: "time"},
|
||||
RHS: &influxql.BinaryExpr{
|
||||
Op: influxql.SUB,
|
||||
LHS: &influxql.Call{Name: "now"},
|
||||
RHS: &influxql.DurationLiteral{Val: 24 * time.Hour},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
|
||||
{
|
||||
s: `SELECT sum(mean) FROM (SELECT mean(value) FROM cpu GROUP BY time(1h)) WHERE time >= now() - 1d`,
|
||||
stmt: &influxql.SelectStatement{
|
||||
Fields: []*influxql.Field{{
|
||||
Expr: &influxql.Call{
|
||||
Name: "sum",
|
||||
Args: []influxql.Expr{
|
||||
&influxql.VarRef{Val: "mean"},
|
||||
}},
|
||||
}},
|
||||
Sources: []influxql.Source{
|
||||
&influxql.SubQuery{
|
||||
Statement: &influxql.SelectStatement{
|
||||
Fields: []*influxql.Field{{
|
||||
Expr: &influxql.Call{
|
||||
Name: "mean",
|
||||
Args: []influxql.Expr{
|
||||
&influxql.VarRef{Val: "value"},
|
||||
},
|
||||
},
|
||||
}},
|
||||
Dimensions: []*influxql.Dimension{{
|
||||
Expr: &influxql.Call{
|
||||
Name: "time",
|
||||
Args: []influxql.Expr{
|
||||
&influxql.DurationLiteral{Val: time.Hour},
|
||||
},
|
||||
},
|
||||
}},
|
||||
Sources: []influxql.Source{
|
||||
&influxql.Measurement{Name: "cpu"},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
Condition: &influxql.BinaryExpr{
|
||||
Op: influxql.GTE,
|
||||
LHS: &influxql.VarRef{Val: "time"},
|
||||
RHS: &influxql.BinaryExpr{
|
||||
Op: influxql.SUB,
|
||||
LHS: &influxql.Call{Name: "now"},
|
||||
RHS: &influxql.DurationLiteral{Val: 24 * time.Hour},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
|
||||
{
|
||||
s: `SELECT sum(mean) FROM (SELECT mean(value) FROM cpu WHERE time >= now() - 1d GROUP BY time(1h))`,
|
||||
stmt: &influxql.SelectStatement{
|
||||
Fields: []*influxql.Field{{
|
||||
Expr: &influxql.Call{
|
||||
Name: "sum",
|
||||
Args: []influxql.Expr{
|
||||
&influxql.VarRef{Val: "mean"},
|
||||
}},
|
||||
}},
|
||||
Sources: []influxql.Source{
|
||||
&influxql.SubQuery{
|
||||
Statement: &influxql.SelectStatement{
|
||||
Fields: []*influxql.Field{{
|
||||
Expr: &influxql.Call{
|
||||
Name: "mean",
|
||||
Args: []influxql.Expr{
|
||||
&influxql.VarRef{Val: "value"},
|
||||
},
|
||||
},
|
||||
}},
|
||||
Dimensions: []*influxql.Dimension{{
|
||||
Expr: &influxql.Call{
|
||||
Name: "time",
|
||||
Args: []influxql.Expr{
|
||||
&influxql.DurationLiteral{Val: time.Hour},
|
||||
},
|
||||
},
|
||||
}},
|
||||
Condition: &influxql.BinaryExpr{
|
||||
Op: influxql.GTE,
|
||||
LHS: &influxql.VarRef{Val: "time"},
|
||||
RHS: &influxql.BinaryExpr{
|
||||
Op: influxql.SUB,
|
||||
LHS: &influxql.Call{Name: "now"},
|
||||
RHS: &influxql.DurationLiteral{Val: 24 * time.Hour},
|
||||
},
|
||||
},
|
||||
Sources: []influxql.Source{
|
||||
&influxql.Measurement{Name: "cpu"},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
|
||||
// See issues https://github.com/influxdata/influxdb/issues/1647
|
||||
// and https://github.com/influxdata/influxdb/issues/4404
|
||||
// DELETE statement
|
||||
|
@ -2342,6 +2487,7 @@ func TestParser_ParseStatement(t *testing.T) {
|
|||
{s: `SELECT (count(foo + sum(bar))) FROM cpu`, err: `expected field argument in count()`},
|
||||
{s: `SELECT sum(value) + count(foo + sum(bar)) FROM cpu`, err: `binary expressions cannot mix aggregates and raw fields`},
|
||||
{s: `SELECT mean(value) FROM cpu FILL + value`, err: `fill must be a function call`},
|
||||
{s: `SELECT sum(mean) FROM (SELECT mean(value) FROM cpu GROUP BY time(1h))`, err: `aggregate functions with GROUP BY time require a WHERE time clause`},
|
||||
// See issues https://github.com/influxdata/influxdb/issues/1647
|
||||
// and https://github.com/influxdata/influxdb/issues/4404
|
||||
//{s: `DELETE`, err: `found EOF, expected FROM at line 1, char 8`},
|
||||
|
@ -2520,6 +2666,12 @@ func TestParser_ParseStatement(t *testing.T) {
|
|||
// We are memoizing a field so for testing we need to...
|
||||
if s, ok := tt.stmt.(*influxql.SelectStatement); ok {
|
||||
s.GroupByInterval()
|
||||
for _, source := range s.Sources {
|
||||
switch source := source.(type) {
|
||||
case *influxql.SubQuery:
|
||||
source.Statement.GroupByInterval()
|
||||
}
|
||||
}
|
||||
} else if st, ok := stmt.(*influxql.CreateContinuousQueryStatement); ok { // if it's a CQ, there is a non-exported field that gets memoized during parsing that needs to be set
|
||||
if st != nil && st.Source != nil {
|
||||
tt.stmt.(*influxql.CreateContinuousQueryStatement).Source.GroupByInterval()
|
||||
|
|
|
@ -58,6 +58,15 @@ func (v *FloatPoint) Clone() *FloatPoint {
|
|||
return &other
|
||||
}
|
||||
|
||||
// CopyTo makes a deep copy into the point.
|
||||
func (v *FloatPoint) CopyTo(other *FloatPoint) {
|
||||
*other = *v
|
||||
if v.Aux != nil {
|
||||
other.Aux = make([]interface{}, len(v.Aux))
|
||||
copy(other.Aux, v.Aux)
|
||||
}
|
||||
}
|
||||
|
||||
func encodeFloatPoint(p *FloatPoint) *internal.Point {
|
||||
return &internal.Point{
|
||||
Name: proto.String(p.Name),
|
||||
|
@ -260,6 +269,15 @@ func (v *IntegerPoint) Clone() *IntegerPoint {
|
|||
return &other
|
||||
}
|
||||
|
||||
// CopyTo makes a deep copy into the point.
|
||||
func (v *IntegerPoint) CopyTo(other *IntegerPoint) {
|
||||
*other = *v
|
||||
if v.Aux != nil {
|
||||
other.Aux = make([]interface{}, len(v.Aux))
|
||||
copy(other.Aux, v.Aux)
|
||||
}
|
||||
}
|
||||
|
||||
func encodeIntegerPoint(p *IntegerPoint) *internal.Point {
|
||||
return &internal.Point{
|
||||
Name: proto.String(p.Name),
|
||||
|
@ -462,6 +480,15 @@ func (v *StringPoint) Clone() *StringPoint {
|
|||
return &other
|
||||
}
|
||||
|
||||
// CopyTo makes a deep copy into the point.
|
||||
func (v *StringPoint) CopyTo(other *StringPoint) {
|
||||
*other = *v
|
||||
if v.Aux != nil {
|
||||
other.Aux = make([]interface{}, len(v.Aux))
|
||||
copy(other.Aux, v.Aux)
|
||||
}
|
||||
}
|
||||
|
||||
func encodeStringPoint(p *StringPoint) *internal.Point {
|
||||
return &internal.Point{
|
||||
Name: proto.String(p.Name),
|
||||
|
@ -664,6 +691,15 @@ func (v *BooleanPoint) Clone() *BooleanPoint {
|
|||
return &other
|
||||
}
|
||||
|
||||
// CopyTo makes a deep copy into the point.
|
||||
func (v *BooleanPoint) CopyTo(other *BooleanPoint) {
|
||||
*other = *v
|
||||
if v.Aux != nil {
|
||||
other.Aux = make([]interface{}, len(v.Aux))
|
||||
copy(other.Aux, v.Aux)
|
||||
}
|
||||
}
|
||||
|
||||
func encodeBooleanPoint(p *BooleanPoint) *internal.Point {
|
||||
return &internal.Point{
|
||||
Name: proto.String(p.Name),
|
||||
|
|
|
@ -54,6 +54,15 @@ func (v *{{.Name}}Point) Clone() *{{.Name}}Point {
|
|||
return &other
|
||||
}
|
||||
|
||||
// CopyTo makes a deep copy into the point.
|
||||
func (v *{{.Name}}Point) CopyTo(other *{{.Name}}Point) {
|
||||
*other = *v
|
||||
if v.Aux != nil {
|
||||
other.Aux = make([]interface{}, len(v.Aux))
|
||||
copy(other.Aux, v.Aux)
|
||||
}
|
||||
}
|
||||
|
||||
func encode{{.Name}}Point(p *{{.Name}}Point) *internal.Point {
|
||||
return &internal.Point{
|
||||
Name: proto.String(p.Name),
|
||||
|
|
|
@ -283,6 +283,15 @@ func decodeAux(pb []*internal.Aux) []interface{} {
|
|||
return aux
|
||||
}
|
||||
|
||||
func cloneAux(src []interface{}) []interface{} {
|
||||
if src == nil {
|
||||
return src
|
||||
}
|
||||
dest := make([]interface{}, len(src))
|
||||
copy(dest, src)
|
||||
return dest
|
||||
}
|
||||
|
||||
// PointDecoder decodes generic points from a reader.
|
||||
type PointDecoder struct {
|
||||
r io.Reader
|
||||
|
|
1027
influxql/select.go
1027
influxql/select.go
File diff suppressed because it is too large
Load Diff
|
@ -17,7 +17,10 @@ const Second = int64(time.Second)
|
|||
// Ensure a SELECT min() query can be executed.
|
||||
func TestSelect_Min(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
if !reflect.DeepEqual(opt.Expr, MustParseExpr(`min(value)`)) {
|
||||
t.Fatalf("unexpected expr: %s", spew.Sdump(opt.Expr))
|
||||
}
|
||||
|
@ -51,7 +54,10 @@ func TestSelect_Min(t *testing.T) {
|
|||
// Ensure a SELECT distinct() query can be executed.
|
||||
func TestSelect_Distinct_Float(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &FloatIterator{Points: []influxql.FloatPoint{
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=A"), Time: 0 * Second, Value: 20},
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=A"), Time: 1 * Second, Value: 19},
|
||||
|
@ -82,7 +88,10 @@ func TestSelect_Distinct_Float(t *testing.T) {
|
|||
// Ensure a SELECT distinct() query can be executed.
|
||||
func TestSelect_Distinct_Integer(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &IntegerIterator{Points: []influxql.IntegerPoint{
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=A"), Time: 0 * Second, Value: 20},
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=A"), Time: 1 * Second, Value: 19},
|
||||
|
@ -113,7 +122,10 @@ func TestSelect_Distinct_Integer(t *testing.T) {
|
|||
// Ensure a SELECT distinct() query can be executed.
|
||||
func TestSelect_Distinct_String(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &StringIterator{Points: []influxql.StringPoint{
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=A"), Time: 0 * Second, Value: "a"},
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=A"), Time: 1 * Second, Value: "b"},
|
||||
|
@ -144,7 +156,10 @@ func TestSelect_Distinct_String(t *testing.T) {
|
|||
// Ensure a SELECT distinct() query can be executed.
|
||||
func TestSelect_Distinct_Boolean(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &BooleanIterator{Points: []influxql.BooleanPoint{
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=A"), Time: 0 * Second, Value: true},
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=A"), Time: 1 * Second, Value: false},
|
||||
|
@ -176,7 +191,10 @@ func TestSelect_Distinct_Boolean(t *testing.T) {
|
|||
// Ensure a SELECT mean() query can be executed.
|
||||
func TestSelect_Mean_Float(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return influxql.NewCallIterator(&FloatIterator{Points: []influxql.FloatPoint{
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=A"), Time: 0 * Second, Value: 20},
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=B"), Time: 5 * Second, Value: 10},
|
||||
|
@ -213,7 +231,10 @@ func TestSelect_Mean_Float(t *testing.T) {
|
|||
// Ensure a SELECT mean() query can be executed.
|
||||
func TestSelect_Mean_Integer(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return influxql.NewCallIterator(&IntegerIterator{Points: []influxql.IntegerPoint{
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=A"), Time: 0 * Second, Value: 20},
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=B"), Time: 5 * Second, Value: 10},
|
||||
|
@ -250,7 +271,10 @@ func TestSelect_Mean_Integer(t *testing.T) {
|
|||
// Ensure a SELECT mean() query cannot be executed on strings.
|
||||
func TestSelect_Mean_String(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return influxql.NewCallIterator(&StringIterator{}, opt)
|
||||
}
|
||||
|
||||
|
@ -268,7 +292,10 @@ func TestSelect_Mean_String(t *testing.T) {
|
|||
// Ensure a SELECT mean() query cannot be executed on booleans.
|
||||
func TestSelect_Mean_Boolean(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return influxql.NewCallIterator(&BooleanIterator{}, opt)
|
||||
}
|
||||
|
||||
|
@ -286,7 +313,10 @@ func TestSelect_Mean_Boolean(t *testing.T) {
|
|||
// Ensure a SELECT median() query can be executed.
|
||||
func TestSelect_Median_Float(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &FloatIterator{Points: []influxql.FloatPoint{
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=A"), Time: 0 * Second, Value: 20},
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=B"), Time: 5 * Second, Value: 10},
|
||||
|
@ -323,7 +353,10 @@ func TestSelect_Median_Float(t *testing.T) {
|
|||
// Ensure a SELECT median() query can be executed.
|
||||
func TestSelect_Median_Integer(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &IntegerIterator{Points: []influxql.IntegerPoint{
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=A"), Time: 0 * Second, Value: 20},
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=B"), Time: 5 * Second, Value: 10},
|
||||
|
@ -360,7 +393,10 @@ func TestSelect_Median_Integer(t *testing.T) {
|
|||
// Ensure a SELECT median() query cannot be executed on strings.
|
||||
func TestSelect_Median_String(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &StringIterator{}, nil
|
||||
}
|
||||
|
||||
|
@ -378,7 +414,10 @@ func TestSelect_Median_String(t *testing.T) {
|
|||
// Ensure a SELECT median() query cannot be executed on booleans.
|
||||
func TestSelect_Median_Boolean(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &BooleanIterator{}, nil
|
||||
}
|
||||
|
||||
|
@ -396,7 +435,10 @@ func TestSelect_Median_Boolean(t *testing.T) {
|
|||
// Ensure a SELECT mode() query can be executed.
|
||||
func TestSelect_Mode_Float(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &FloatIterator{Points: []influxql.FloatPoint{
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=A"), Time: 0 * Second, Value: 10},
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=B"), Time: 5 * Second, Value: 10},
|
||||
|
@ -433,7 +475,10 @@ func TestSelect_Mode_Float(t *testing.T) {
|
|||
// Ensure a SELECT mode() query can be executed.
|
||||
func TestSelect_Mode_Integer(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &IntegerIterator{Points: []influxql.IntegerPoint{
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=A"), Time: 0 * Second, Value: 10},
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=B"), Time: 5 * Second, Value: 10},
|
||||
|
@ -470,7 +515,10 @@ func TestSelect_Mode_Integer(t *testing.T) {
|
|||
// Ensure a SELECT mode() query cannot be executed on strings.
|
||||
func TestSelect_Mode_String(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &StringIterator{Points: []influxql.StringPoint{
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=A"), Time: 0 * Second, Value: "a"},
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=A"), Time: 1 * Second, Value: "a"},
|
||||
|
@ -503,7 +551,10 @@ func TestSelect_Mode_String(t *testing.T) {
|
|||
// Ensure a SELECT mode() query cannot be executed on booleans.
|
||||
func TestSelect_Mode_Boolean(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &BooleanIterator{Points: []influxql.BooleanPoint{
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=A"), Time: 0 * Second, Value: true},
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=A"), Time: 1 * Second, Value: false},
|
||||
|
@ -535,7 +586,10 @@ func TestSelect_Mode_Boolean(t *testing.T) {
|
|||
// Ensure a SELECT top() query can be executed.
|
||||
func TestSelect_Top_NoTags_Float(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &FloatIterator{Points: []influxql.FloatPoint{
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=A"), Time: 0 * Second, Value: 20},
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=B"), Time: 5 * Second, Value: 10},
|
||||
|
@ -573,7 +627,10 @@ func TestSelect_Top_NoTags_Float(t *testing.T) {
|
|||
// Ensure a SELECT top() query can be executed.
|
||||
func TestSelect_Top_NoTags_Integer(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &IntegerIterator{Points: []influxql.IntegerPoint{
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=A"), Time: 0 * Second, Value: 20},
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=B"), Time: 5 * Second, Value: 10},
|
||||
|
@ -611,7 +668,10 @@ func TestSelect_Top_NoTags_Integer(t *testing.T) {
|
|||
// Ensure a SELECT top() query can be executed with tags.
|
||||
func TestSelect_Top_Tags_Float(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &FloatIterator{Points: []influxql.FloatPoint{
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=A"), Time: 0 * Second, Value: 20, Aux: []interface{}{"A"}},
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=B"), Time: 5 * Second, Value: 10, Aux: []interface{}{"B"}},
|
||||
|
@ -659,7 +719,10 @@ func TestSelect_Top_Tags_Float(t *testing.T) {
|
|||
// Ensure a SELECT top() query can be executed with tags.
|
||||
func TestSelect_Top_Tags_Integer(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &IntegerIterator{Points: []influxql.IntegerPoint{
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=A"), Time: 0 * Second, Value: 20, Aux: []interface{}{"A"}},
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=B"), Time: 5 * Second, Value: 10, Aux: []interface{}{"B"}},
|
||||
|
@ -707,7 +770,10 @@ func TestSelect_Top_Tags_Integer(t *testing.T) {
|
|||
// Ensure a SELECT top() query can be executed with tags and group by.
|
||||
func TestSelect_Top_GroupByTags_Float(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &FloatIterator{Points: []influxql.FloatPoint{
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=A"), Time: 0 * Second, Value: 20, Aux: []interface{}{"A"}},
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=B"), Time: 5 * Second, Value: 10, Aux: []interface{}{"B"}},
|
||||
|
@ -751,7 +817,10 @@ func TestSelect_Top_GroupByTags_Float(t *testing.T) {
|
|||
// Ensure a SELECT top() query can be executed with tags and group by.
|
||||
func TestSelect_Top_GroupByTags_Integer(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &IntegerIterator{Points: []influxql.IntegerPoint{
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=A"), Time: 0 * Second, Value: 20, Aux: []interface{}{"A"}},
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=B"), Time: 5 * Second, Value: 10, Aux: []interface{}{"B"}},
|
||||
|
@ -795,7 +864,10 @@ func TestSelect_Top_GroupByTags_Integer(t *testing.T) {
|
|||
// Ensure a SELECT bottom() query can be executed.
|
||||
func TestSelect_Bottom_NoTags_Float(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &FloatIterator{Points: []influxql.FloatPoint{
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=A"), Time: 0 * Second, Value: 20},
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=B"), Time: 5 * Second, Value: 10},
|
||||
|
@ -833,7 +905,10 @@ func TestSelect_Bottom_NoTags_Float(t *testing.T) {
|
|||
// Ensure a SELECT bottom() query can be executed.
|
||||
func TestSelect_Bottom_NoTags_Integer(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &IntegerIterator{Points: []influxql.IntegerPoint{
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=A"), Time: 0 * Second, Value: 20},
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=B"), Time: 5 * Second, Value: 10},
|
||||
|
@ -871,7 +946,10 @@ func TestSelect_Bottom_NoTags_Integer(t *testing.T) {
|
|||
// Ensure a SELECT bottom() query can be executed with tags.
|
||||
func TestSelect_Bottom_Tags_Float(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &FloatIterator{Points: []influxql.FloatPoint{
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=A"), Time: 0 * Second, Value: 20, Aux: []interface{}{"A"}},
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=B"), Time: 5 * Second, Value: 10, Aux: []interface{}{"B"}},
|
||||
|
@ -919,7 +997,10 @@ func TestSelect_Bottom_Tags_Float(t *testing.T) {
|
|||
// Ensure a SELECT bottom() query can be executed with tags.
|
||||
func TestSelect_Bottom_Tags_Integer(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &IntegerIterator{Points: []influxql.IntegerPoint{
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=A"), Time: 0 * Second, Value: 20, Aux: []interface{}{"A"}},
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=B"), Time: 5 * Second, Value: 10, Aux: []interface{}{"B"}},
|
||||
|
@ -967,7 +1048,10 @@ func TestSelect_Bottom_Tags_Integer(t *testing.T) {
|
|||
// Ensure a SELECT bottom() query can be executed with tags and group by.
|
||||
func TestSelect_Bottom_GroupByTags_Float(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &FloatIterator{Points: []influxql.FloatPoint{
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=A"), Time: 0 * Second, Value: 20, Aux: []interface{}{"A"}},
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=B"), Time: 5 * Second, Value: 10, Aux: []interface{}{"B"}},
|
||||
|
@ -1011,7 +1095,10 @@ func TestSelect_Bottom_GroupByTags_Float(t *testing.T) {
|
|||
// Ensure a SELECT bottom() query can be executed with tags and group by.
|
||||
func TestSelect_Bottom_GroupByTags_Integer(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &IntegerIterator{Points: []influxql.IntegerPoint{
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=A"), Time: 0 * Second, Value: 20, Aux: []interface{}{"A"}},
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=B"), Time: 5 * Second, Value: 10, Aux: []interface{}{"B"}},
|
||||
|
@ -1055,7 +1142,10 @@ func TestSelect_Bottom_GroupByTags_Integer(t *testing.T) {
|
|||
// Ensure a SELECT query with a fill(null) statement can be executed.
|
||||
func TestSelect_Fill_Null_Float(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return influxql.NewCallIterator(&FloatIterator{Points: []influxql.FloatPoint{
|
||||
{Name: "cpu", Tags: ParseTags("host=A"), Time: 12 * Second, Value: 2},
|
||||
}}, opt)
|
||||
|
@ -1082,7 +1172,10 @@ func TestSelect_Fill_Null_Float(t *testing.T) {
|
|||
// Ensure a SELECT query with a fill(<number>) statement can be executed.
|
||||
func TestSelect_Fill_Number_Float(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return influxql.NewCallIterator(&FloatIterator{Points: []influxql.FloatPoint{
|
||||
{Name: "cpu", Tags: ParseTags("host=A"), Time: 12 * Second, Value: 2},
|
||||
}}, opt)
|
||||
|
@ -1109,7 +1202,10 @@ func TestSelect_Fill_Number_Float(t *testing.T) {
|
|||
// Ensure a SELECT query with a fill(previous) statement can be executed.
|
||||
func TestSelect_Fill_Previous_Float(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return influxql.NewCallIterator(&FloatIterator{Points: []influxql.FloatPoint{
|
||||
{Name: "cpu", Tags: ParseTags("host=A"), Time: 12 * Second, Value: 2},
|
||||
}}, opt)
|
||||
|
@ -1136,7 +1232,10 @@ func TestSelect_Fill_Previous_Float(t *testing.T) {
|
|||
// Ensure a SELECT query with a fill(linear) statement can be executed.
|
||||
func TestSelect_Fill_Linear_Float_One(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return influxql.NewCallIterator(&FloatIterator{Points: []influxql.FloatPoint{
|
||||
{Name: "cpu", Tags: ParseTags("host=A"), Time: 12 * Second, Value: 2},
|
||||
{Name: "cpu", Tags: ParseTags("host=A"), Time: 32 * Second, Value: 4},
|
||||
|
@ -1163,7 +1262,10 @@ func TestSelect_Fill_Linear_Float_One(t *testing.T) {
|
|||
|
||||
func TestSelect_Fill_Linear_Float_Many(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return influxql.NewCallIterator(&FloatIterator{Points: []influxql.FloatPoint{
|
||||
{Name: "cpu", Tags: ParseTags("host=A"), Time: 12 * Second, Value: 2},
|
||||
{Name: "cpu", Tags: ParseTags("host=A"), Time: 62 * Second, Value: 7},
|
||||
|
@ -1192,7 +1294,10 @@ func TestSelect_Fill_Linear_Float_Many(t *testing.T) {
|
|||
// Ensure a SELECT query with a fill(linear) statement can be executed for integers.
|
||||
func TestSelect_Fill_Linear_Integer_One(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return influxql.NewCallIterator(&IntegerIterator{Points: []influxql.IntegerPoint{
|
||||
{Name: "cpu", Tags: ParseTags("host=A"), Time: 12 * Second, Value: 1},
|
||||
{Name: "cpu", Tags: ParseTags("host=A"), Time: 32 * Second, Value: 4},
|
||||
|
@ -1219,7 +1324,10 @@ func TestSelect_Fill_Linear_Integer_One(t *testing.T) {
|
|||
|
||||
func TestSelect_Fill_Linear_Integer_Many(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return influxql.NewCallIterator(&IntegerIterator{Points: []influxql.IntegerPoint{
|
||||
{Name: "cpu", Tags: ParseTags("host=A"), Time: 12 * Second, Value: 1},
|
||||
{Name: "cpu", Tags: ParseTags("host=A"), Time: 72 * Second, Value: 10},
|
||||
|
@ -1249,7 +1357,10 @@ func TestSelect_Fill_Linear_Integer_Many(t *testing.T) {
|
|||
// Ensure a SELECT stddev() query can be executed.
|
||||
func TestSelect_Stddev_Float(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &FloatIterator{Points: []influxql.FloatPoint{
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=A"), Time: 0 * Second, Value: 20},
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=B"), Time: 5 * Second, Value: 10},
|
||||
|
@ -1286,7 +1397,10 @@ func TestSelect_Stddev_Float(t *testing.T) {
|
|||
// Ensure a SELECT stddev() query can be executed.
|
||||
func TestSelect_Stddev_Integer(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &IntegerIterator{Points: []influxql.IntegerPoint{
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=A"), Time: 0 * Second, Value: 20},
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=B"), Time: 5 * Second, Value: 10},
|
||||
|
@ -1323,7 +1437,10 @@ func TestSelect_Stddev_Integer(t *testing.T) {
|
|||
// Ensure a SELECT spread() query can be executed.
|
||||
func TestSelect_Spread_Float(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &FloatIterator{Points: []influxql.FloatPoint{
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=A"), Time: 0 * Second, Value: 20},
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=B"), Time: 5 * Second, Value: 10},
|
||||
|
@ -1360,7 +1477,10 @@ func TestSelect_Spread_Float(t *testing.T) {
|
|||
// Ensure a SELECT spread() query can be executed.
|
||||
func TestSelect_Spread_Integer(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &IntegerIterator{Points: []influxql.IntegerPoint{
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=A"), Time: 0 * Second, Value: 20},
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=B"), Time: 5 * Second, Value: 10},
|
||||
|
@ -1397,7 +1517,10 @@ func TestSelect_Spread_Integer(t *testing.T) {
|
|||
// Ensure a SELECT percentile() query can be executed.
|
||||
func TestSelect_Percentile_Float(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &FloatIterator{Points: []influxql.FloatPoint{
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=A"), Time: 0 * Second, Value: 20},
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=B"), Time: 5 * Second, Value: 10},
|
||||
|
@ -1439,7 +1562,10 @@ func TestSelect_Percentile_Float(t *testing.T) {
|
|||
// Ensure a SELECT percentile() query can be executed.
|
||||
func TestSelect_Percentile_Integer(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &IntegerIterator{Points: []influxql.IntegerPoint{
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=A"), Time: 0 * Second, Value: 20},
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=B"), Time: 5 * Second, Value: 10},
|
||||
|
@ -1481,7 +1607,10 @@ func TestSelect_Percentile_Integer(t *testing.T) {
|
|||
// Ensure a SELECT sample() query can be executed.
|
||||
func TestSelect_Sample_Float(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &FloatIterator{Points: []influxql.FloatPoint{
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=A"), Time: 0 * Second, Value: 20},
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=A"), Time: 5 * Second, Value: 10},
|
||||
|
@ -1509,7 +1638,10 @@ func TestSelect_Sample_Float(t *testing.T) {
|
|||
// Ensure a SELECT sample() query can be executed.
|
||||
func TestSelect_Sample_Integer(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &IntegerIterator{Points: []influxql.IntegerPoint{
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=A"), Time: 0 * Second, Value: 20},
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=A"), Time: 5 * Second, Value: 10},
|
||||
|
@ -1537,7 +1669,10 @@ func TestSelect_Sample_Integer(t *testing.T) {
|
|||
// Ensure a SELECT sample() query can be executed.
|
||||
func TestSelect_Sample_Boolean(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &BooleanIterator{Points: []influxql.BooleanPoint{
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=A"), Time: 0 * Second, Value: true},
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=A"), Time: 5 * Second, Value: false},
|
||||
|
@ -1565,7 +1700,10 @@ func TestSelect_Sample_Boolean(t *testing.T) {
|
|||
// Ensure a SELECT sample() query can be executed.
|
||||
func TestSelect_Sample_String(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &StringIterator{Points: []influxql.StringPoint{
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=A"), Time: 0 * Second, Value: "a"},
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=A"), Time: 5 * Second, Value: "b"},
|
||||
|
@ -1594,7 +1732,10 @@ func TestSelect_Sample_String(t *testing.T) {
|
|||
func TestSelect_Raw(t *testing.T) {
|
||||
// Mock two iterators -- one for each value in the query.
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
if !reflect.DeepEqual(opt.Aux, []influxql.VarRef{{Val: "v1", Type: influxql.Float}, {Val: "v2", Type: influxql.Float}}) {
|
||||
t.Fatalf("unexpected options: %s", spew.Sdump(opt.Expr))
|
||||
|
||||
|
@ -1633,7 +1774,10 @@ func TestSelect_Raw(t *testing.T) {
|
|||
// Ensure a SELECT binary expr queries can be executed as floats.
|
||||
func TestSelect_BinaryExpr_Float(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
makeAuxFields := func(value float64) []interface{} {
|
||||
aux := make([]interface{}, len(opt.Aux))
|
||||
for i := range aux {
|
||||
|
@ -1647,7 +1791,10 @@ func TestSelect_BinaryExpr_Float(t *testing.T) {
|
|||
{Name: "cpu", Time: 9 * Second, Value: 19, Aux: makeAuxFields(19)},
|
||||
}}, nil
|
||||
}
|
||||
ic.FieldDimensionsFn = func(sources influxql.Sources) (map[string]influxql.DataType, map[string]struct{}, error) {
|
||||
ic.FieldDimensionsFn = func(m *influxql.Measurement) (map[string]influxql.DataType, map[string]struct{}, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return map[string]influxql.DataType{"value": influxql.Float}, nil, nil
|
||||
}
|
||||
|
||||
|
@ -1856,7 +2003,10 @@ func TestSelect_BinaryExpr_Float(t *testing.T) {
|
|||
// Ensure a SELECT binary expr queries can be executed as integers.
|
||||
func TestSelect_BinaryExpr_Integer(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
makeAuxFields := func(value int64) []interface{} {
|
||||
aux := make([]interface{}, len(opt.Aux))
|
||||
for i := range aux {
|
||||
|
@ -1870,7 +2020,10 @@ func TestSelect_BinaryExpr_Integer(t *testing.T) {
|
|||
{Name: "cpu", Time: 9 * Second, Value: 19, Aux: makeAuxFields(19)},
|
||||
}}, nil
|
||||
}
|
||||
ic.FieldDimensionsFn = func(sources influxql.Sources) (map[string]influxql.DataType, map[string]struct{}, error) {
|
||||
ic.FieldDimensionsFn = func(m *influxql.Measurement) (map[string]influxql.DataType, map[string]struct{}, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return map[string]influxql.DataType{"value": influxql.Integer}, nil, nil
|
||||
}
|
||||
|
||||
|
@ -2079,14 +2232,20 @@ func TestSelect_BinaryExpr_Integer(t *testing.T) {
|
|||
// Ensure a SELECT binary expr queries can be executed on mixed iterators.
|
||||
func TestSelect_BinaryExpr_Mixed(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &IntegerIterator{Points: []influxql.IntegerPoint{
|
||||
{Name: "cpu", Time: 0 * Second, Value: 20, Aux: []interface{}{float64(20), int64(10)}},
|
||||
{Name: "cpu", Time: 5 * Second, Value: 10, Aux: []interface{}{float64(10), int64(15)}},
|
||||
{Name: "cpu", Time: 9 * Second, Value: 19, Aux: []interface{}{float64(19), int64(5)}},
|
||||
}}, nil
|
||||
}
|
||||
ic.FieldDimensionsFn = func(sources influxql.Sources) (map[string]influxql.DataType, map[string]struct{}, error) {
|
||||
ic.FieldDimensionsFn = func(m *influxql.Measurement) (map[string]influxql.DataType, map[string]struct{}, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return map[string]influxql.DataType{
|
||||
"total": influxql.Float,
|
||||
"value": influxql.Integer,
|
||||
|
@ -2156,14 +2315,20 @@ func TestSelect_BinaryExpr_Mixed(t *testing.T) {
|
|||
// but not the other.
|
||||
func TestSelect_BinaryExpr_NilValues(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &FloatIterator{Points: []influxql.FloatPoint{
|
||||
{Name: "cpu", Time: 0 * Second, Value: 20, Aux: []interface{}{float64(20), nil}},
|
||||
{Name: "cpu", Time: 5 * Second, Value: 10, Aux: []interface{}{float64(10), float64(15)}},
|
||||
{Name: "cpu", Time: 9 * Second, Value: 19, Aux: []interface{}{nil, float64(5)}},
|
||||
}}, nil
|
||||
}
|
||||
ic.FieldDimensionsFn = func(sources influxql.Sources) (map[string]influxql.DataType, map[string]struct{}, error) {
|
||||
ic.FieldDimensionsFn = func(m *influxql.Measurement) (map[string]influxql.DataType, map[string]struct{}, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return map[string]influxql.DataType{
|
||||
"total": influxql.Float,
|
||||
"value": influxql.Float,
|
||||
|
@ -2231,7 +2396,10 @@ func TestSelect_BinaryExpr_NilValues(t *testing.T) {
|
|||
// Ensure a SELECT (...) query can be executed.
|
||||
func TestSelect_ParenExpr(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
if !reflect.DeepEqual(opt.Expr, MustParseExpr(`min(value)`)) {
|
||||
t.Fatalf("unexpected expr: %s", spew.Sdump(opt.Expr))
|
||||
}
|
||||
|
@ -2261,7 +2429,10 @@ func TestSelect_ParenExpr(t *testing.T) {
|
|||
t.Fatalf("unexpected points: %s", spew.Sdump(a))
|
||||
}
|
||||
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &FloatIterator{Points: []influxql.FloatPoint{
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=A"), Time: 0 * Second, Value: 20},
|
||||
{Name: "cpu", Tags: ParseTags("region=west,host=A"), Time: 1 * Second, Value: 19},
|
||||
|
@ -2291,7 +2462,10 @@ func TestSelect_ParenExpr(t *testing.T) {
|
|||
|
||||
func TestSelect_Derivative_Float(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &FloatIterator{Points: []influxql.FloatPoint{
|
||||
{Name: "cpu", Time: 0 * Second, Value: 20},
|
||||
{Name: "cpu", Time: 4 * Second, Value: 10},
|
||||
|
@ -2317,7 +2491,10 @@ func TestSelect_Derivative_Float(t *testing.T) {
|
|||
|
||||
func TestSelect_Derivative_Integer(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &IntegerIterator{Points: []influxql.IntegerPoint{
|
||||
{Name: "cpu", Time: 0 * Second, Value: 20},
|
||||
{Name: "cpu", Time: 4 * Second, Value: 10},
|
||||
|
@ -2343,7 +2520,10 @@ func TestSelect_Derivative_Integer(t *testing.T) {
|
|||
|
||||
func TestSelect_Derivative_Desc_Float(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &FloatIterator{Points: []influxql.FloatPoint{
|
||||
{Name: "cpu", Time: 12 * Second, Value: 3},
|
||||
{Name: "cpu", Time: 8 * Second, Value: 19},
|
||||
|
@ -2369,7 +2549,10 @@ func TestSelect_Derivative_Desc_Float(t *testing.T) {
|
|||
|
||||
func TestSelect_Derivative_Desc_Integer(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &IntegerIterator{Points: []influxql.IntegerPoint{
|
||||
{Name: "cpu", Time: 12 * Second, Value: 3},
|
||||
{Name: "cpu", Time: 8 * Second, Value: 19},
|
||||
|
@ -2395,7 +2578,10 @@ func TestSelect_Derivative_Desc_Integer(t *testing.T) {
|
|||
|
||||
func TestSelect_Derivative_Duplicate_Float(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &FloatIterator{Points: []influxql.FloatPoint{
|
||||
{Name: "cpu", Time: 0 * Second, Value: 20},
|
||||
{Name: "cpu", Time: 0 * Second, Value: 19},
|
||||
|
@ -2419,7 +2605,10 @@ func TestSelect_Derivative_Duplicate_Float(t *testing.T) {
|
|||
|
||||
func TestSelect_Derivative_Duplicate_Integer(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &IntegerIterator{Points: []influxql.IntegerPoint{
|
||||
{Name: "cpu", Time: 0 * Second, Value: 20},
|
||||
{Name: "cpu", Time: 0 * Second, Value: 19},
|
||||
|
@ -2443,7 +2632,10 @@ func TestSelect_Derivative_Duplicate_Integer(t *testing.T) {
|
|||
|
||||
func TestSelect_Difference_Float(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &FloatIterator{Points: []influxql.FloatPoint{
|
||||
{Name: "cpu", Time: 0 * Second, Value: 20},
|
||||
{Name: "cpu", Time: 4 * Second, Value: 10},
|
||||
|
@ -2469,7 +2661,10 @@ func TestSelect_Difference_Float(t *testing.T) {
|
|||
|
||||
func TestSelect_Difference_Integer(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &IntegerIterator{Points: []influxql.IntegerPoint{
|
||||
{Name: "cpu", Time: 0 * Second, Value: 20},
|
||||
{Name: "cpu", Time: 4 * Second, Value: 10},
|
||||
|
@ -2495,7 +2690,10 @@ func TestSelect_Difference_Integer(t *testing.T) {
|
|||
|
||||
func TestSelect_Difference_Duplicate_Float(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &FloatIterator{Points: []influxql.FloatPoint{
|
||||
{Name: "cpu", Time: 0 * Second, Value: 20},
|
||||
{Name: "cpu", Time: 0 * Second, Value: 19},
|
||||
|
@ -2519,7 +2717,10 @@ func TestSelect_Difference_Duplicate_Float(t *testing.T) {
|
|||
|
||||
func TestSelect_Difference_Duplicate_Integer(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &IntegerIterator{Points: []influxql.IntegerPoint{
|
||||
{Name: "cpu", Time: 0 * Second, Value: 20},
|
||||
{Name: "cpu", Time: 0 * Second, Value: 19},
|
||||
|
@ -2543,7 +2744,10 @@ func TestSelect_Difference_Duplicate_Integer(t *testing.T) {
|
|||
|
||||
func TestSelect_Elapsed_Float(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &FloatIterator{Points: []influxql.FloatPoint{
|
||||
{Name: "cpu", Time: 0 * Second, Value: 20},
|
||||
{Name: "cpu", Time: 4 * Second, Value: 10},
|
||||
|
@ -2569,7 +2773,10 @@ func TestSelect_Elapsed_Float(t *testing.T) {
|
|||
|
||||
func TestSelect_Elapsed_Integer(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &IntegerIterator{Points: []influxql.IntegerPoint{
|
||||
{Name: "cpu", Time: 0 * Second, Value: 20},
|
||||
{Name: "cpu", Time: 4 * Second, Value: 10},
|
||||
|
@ -2595,7 +2802,10 @@ func TestSelect_Elapsed_Integer(t *testing.T) {
|
|||
|
||||
func TestSelect_Elapsed_String(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &StringIterator{Points: []influxql.StringPoint{
|
||||
{Name: "cpu", Time: 0 * Second, Value: "a"},
|
||||
{Name: "cpu", Time: 4 * Second, Value: "b"},
|
||||
|
@ -2621,7 +2831,10 @@ func TestSelect_Elapsed_String(t *testing.T) {
|
|||
|
||||
func TestSelect_Elapsed_Boolean(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &BooleanIterator{Points: []influxql.BooleanPoint{
|
||||
{Name: "cpu", Time: 0 * Second, Value: true},
|
||||
{Name: "cpu", Time: 4 * Second, Value: false},
|
||||
|
@ -2647,7 +2860,10 @@ func TestSelect_Elapsed_Boolean(t *testing.T) {
|
|||
|
||||
func TestSelect_MovingAverage_Float(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &FloatIterator{Points: []influxql.FloatPoint{
|
||||
{Name: "cpu", Time: 0 * Second, Value: 20},
|
||||
{Name: "cpu", Time: 4 * Second, Value: 10},
|
||||
|
@ -2673,7 +2889,10 @@ func TestSelect_MovingAverage_Float(t *testing.T) {
|
|||
|
||||
func TestSelect_MovingAverage_Integer(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &IntegerIterator{Points: []influxql.IntegerPoint{
|
||||
{Name: "cpu", Time: 0 * Second, Value: 20},
|
||||
{Name: "cpu", Time: 4 * Second, Value: 10},
|
||||
|
@ -2699,7 +2918,10 @@ func TestSelect_MovingAverage_Integer(t *testing.T) {
|
|||
|
||||
func TestSelect_CumulativeSum_Float(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &FloatIterator{Points: []influxql.FloatPoint{
|
||||
{Name: "cpu", Time: 0 * Second, Value: 20},
|
||||
{Name: "cpu", Time: 4 * Second, Value: 10},
|
||||
|
@ -2726,7 +2948,10 @@ func TestSelect_CumulativeSum_Float(t *testing.T) {
|
|||
|
||||
func TestSelect_CumulativeSum_Integer(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &IntegerIterator{Points: []influxql.IntegerPoint{
|
||||
{Name: "cpu", Time: 0 * Second, Value: 20},
|
||||
{Name: "cpu", Time: 4 * Second, Value: 10},
|
||||
|
@ -2753,7 +2978,10 @@ func TestSelect_CumulativeSum_Integer(t *testing.T) {
|
|||
|
||||
func TestSelect_CumulativeSum_Duplicate_Float(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &FloatIterator{Points: []influxql.FloatPoint{
|
||||
{Name: "cpu", Time: 0 * Second, Value: 20},
|
||||
{Name: "cpu", Time: 0 * Second, Value: 19},
|
||||
|
@ -2780,7 +3008,10 @@ func TestSelect_CumulativeSum_Duplicate_Float(t *testing.T) {
|
|||
|
||||
func TestSelect_CumulativeSum_Duplicate_Integer(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &IntegerIterator{Points: []influxql.IntegerPoint{
|
||||
{Name: "cpu", Time: 0 * Second, Value: 20},
|
||||
{Name: "cpu", Time: 0 * Second, Value: 19},
|
||||
|
@ -2807,7 +3038,10 @@ func TestSelect_CumulativeSum_Duplicate_Integer(t *testing.T) {
|
|||
|
||||
func TestSelect_HoltWinters_GroupBy_Agg(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return influxql.NewCallIterator(&FloatIterator{Points: []influxql.FloatPoint{
|
||||
{Name: "cpu", Time: 10 * Second, Value: 4},
|
||||
{Name: "cpu", Time: 11 * Second, Value: 6},
|
||||
|
@ -2842,7 +3076,10 @@ func TestSelect_HoltWinters_GroupBy_Agg(t *testing.T) {
|
|||
|
||||
func TestSelect_UnsupportedCall(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &FloatIterator{}, nil
|
||||
}
|
||||
|
||||
|
@ -2854,7 +3091,10 @@ func TestSelect_UnsupportedCall(t *testing.T) {
|
|||
|
||||
func TestSelect_InvalidQueries(t *testing.T) {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if m.Name != "cpu" {
|
||||
t.Fatalf("unexpected source: %s", m.Name)
|
||||
}
|
||||
return &FloatIterator{}, nil
|
||||
}
|
||||
|
||||
|
@ -2907,7 +3147,7 @@ func benchmarkSelect(b *testing.B, stmt *influxql.SelectStatement, ic influxql.I
|
|||
// NewRawBenchmarkIteratorCreator returns a new mock iterator creator with generated fields.
|
||||
func NewRawBenchmarkIteratorCreator(pointN int) *IteratorCreator {
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if opt.Expr != nil {
|
||||
panic("unexpected expression")
|
||||
}
|
||||
|
@ -2939,7 +3179,7 @@ func benchmarkSelectDedupe(b *testing.B, seriesN, pointsPerSeries int) {
|
|||
stmt.Dedupe = true
|
||||
|
||||
var ic IteratorCreator
|
||||
ic.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
ic.CreateIteratorFn = func(m *influxql.Measurement, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if opt.Expr != nil {
|
||||
panic("unexpected expression")
|
||||
}
|
||||
|
|
|
@ -34,16 +34,16 @@ type MetaClientMock struct {
|
|||
|
||||
RetentionPolicyFn func(database, name string) (rpi *meta.RetentionPolicyInfo, err error)
|
||||
|
||||
SetAdminPrivilegeFn func(username string, admin bool) error
|
||||
SetDataFn func(*meta.Data) error
|
||||
SetPrivilegeFn func(username, database string, p influxql.Privilege) error
|
||||
ShardsByTimeRangeFn func(sources influxql.Sources, tmin, tmax time.Time) (a []meta.ShardInfo, err error)
|
||||
ShardOwnerFn func(shardID uint64) (database, policy string, sgi *meta.ShardGroupInfo)
|
||||
UpdateRetentionPolicyFn func(database, name string, rpu *meta.RetentionPolicyUpdate, makeDefault bool) error
|
||||
UpdateUserFn func(name, password string) error
|
||||
UserPrivilegeFn func(username, database string) (*influxql.Privilege, error)
|
||||
UserPrivilegesFn func(username string) (map[string]influxql.Privilege, error)
|
||||
UsersFn func() []meta.UserInfo
|
||||
SetAdminPrivilegeFn func(username string, admin bool) error
|
||||
SetDataFn func(*meta.Data) error
|
||||
SetPrivilegeFn func(username, database string, p influxql.Privilege) error
|
||||
ShardGroupsByTimeRangeFn func(database, policy string, min, max time.Time) (a []meta.ShardGroupInfo, err error)
|
||||
ShardOwnerFn func(shardID uint64) (database, policy string, sgi *meta.ShardGroupInfo)
|
||||
UpdateRetentionPolicyFn func(database, name string, rpu *meta.RetentionPolicyUpdate, makeDefault bool) error
|
||||
UpdateUserFn func(name, password string) error
|
||||
UserPrivilegeFn func(username, database string) (*influxql.Privilege, error)
|
||||
UserPrivilegesFn func(username string) (map[string]influxql.Privilege, error)
|
||||
UsersFn func() []meta.UserInfo
|
||||
}
|
||||
|
||||
func (c *MetaClientMock) Close() error {
|
||||
|
@ -126,8 +126,8 @@ func (c *MetaClientMock) SetPrivilege(username, database string, p influxql.Priv
|
|||
return c.SetPrivilegeFn(username, database, p)
|
||||
}
|
||||
|
||||
func (c *MetaClientMock) ShardsByTimeRange(sources influxql.Sources, tmin, tmax time.Time) (a []meta.ShardInfo, err error) {
|
||||
return c.ShardsByTimeRangeFn(sources, tmin, tmax)
|
||||
func (c *MetaClientMock) ShardGroupsByTimeRange(database, policy string, min, max time.Time) (a []meta.ShardGroupInfo, err error) {
|
||||
return c.ShardGroupsByTimeRangeFn(database, policy, min, max)
|
||||
}
|
||||
|
||||
func (c *MetaClientMock) ShardOwner(shardID uint64) (database, policy string, sgi *meta.ShardGroupInfo) {
|
||||
|
|
|
@ -637,12 +637,7 @@ func (c *Client) ShardGroupsByTimeRange(database, policy string, min, max time.T
|
|||
// ShardsByTimeRange returns a slice of shards that may contain data in the time range.
|
||||
func (c *Client) ShardsByTimeRange(sources influxql.Sources, tmin, tmax time.Time) (a []ShardInfo, err error) {
|
||||
m := make(map[*ShardInfo]struct{})
|
||||
for _, src := range sources {
|
||||
mm, ok := src.(*influxql.Measurement)
|
||||
if !ok {
|
||||
return nil, fmt.Errorf("invalid source type: %#v", src)
|
||||
}
|
||||
|
||||
for _, mm := range sources.Measurements() {
|
||||
groups, err := c.ShardGroupsByTimeRange(mm.Database, mm.RetentionPolicy, tmin, tmax)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
|
|
|
@ -34,7 +34,7 @@ type Engine interface {
|
|||
Backup(w io.Writer, basePath string, since time.Time) error
|
||||
Restore(r io.Reader, basePath string) error
|
||||
|
||||
CreateIterator(opt influxql.IteratorOptions) (influxql.Iterator, error)
|
||||
CreateIterator(measurement string, opt influxql.IteratorOptions) (influxql.Iterator, error)
|
||||
WritePoints(points []models.Point) error
|
||||
ContainsSeries(keys []string) (map[string]bool, error)
|
||||
DeleteSeries(keys []string) error
|
||||
|
|
|
@ -1243,8 +1243,8 @@ func (e *Engine) KeyCursor(key string, t int64, ascending bool) *KeyCursor {
|
|||
return e.FileStore.KeyCursor(key, t, ascending)
|
||||
}
|
||||
|
||||
// CreateIterator returns an iterator based on opt.
|
||||
func (e *Engine) CreateIterator(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
// CreateIterator returns an iterator for the measurement based on opt.
|
||||
func (e *Engine) CreateIterator(measurement string, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if call, ok := opt.Expr.(*influxql.Call); ok {
|
||||
refOpt := opt
|
||||
refOpt.Expr = call.Args[0].(*influxql.VarRef)
|
||||
|
@ -1256,14 +1256,16 @@ func (e *Engine) CreateIterator(opt influxql.IteratorOptions) (influxql.Iterator
|
|||
aggregate = false
|
||||
refOpt.Limit = 1
|
||||
refOpt.Ascending = true
|
||||
refOpt.Ordered = true
|
||||
case "last":
|
||||
aggregate = false
|
||||
refOpt.Limit = 1
|
||||
refOpt.Ascending = false
|
||||
refOpt.Ordered = true
|
||||
}
|
||||
}
|
||||
|
||||
inputs, err := e.createVarRefIterator(refOpt, aggregate)
|
||||
inputs, err := e.createVarRefIterator(measurement, refOpt, aggregate)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
} else if len(inputs) == 0 {
|
||||
|
@ -1286,7 +1288,7 @@ func (e *Engine) CreateIterator(opt influxql.IteratorOptions) (influxql.Iterator
|
|||
return influxql.NewParallelMergeIterator(inputs, opt, runtime.GOMAXPROCS(0)), nil
|
||||
}
|
||||
|
||||
itrs, err := e.createVarRefIterator(opt, false)
|
||||
itrs, err := e.createVarRefIterator(measurement, opt, false)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -1301,35 +1303,43 @@ func (e *Engine) CreateIterator(opt influxql.IteratorOptions) (influxql.Iterator
|
|||
// createVarRefIterator creates an iterator for a variable reference.
|
||||
// The aggregate argument determines this is being created for an aggregate.
|
||||
// If this is an aggregate, the limit optimization is disabled temporarily. See #6661.
|
||||
func (e *Engine) createVarRefIterator(opt influxql.IteratorOptions, aggregate bool) ([]influxql.Iterator, error) {
|
||||
func (e *Engine) createVarRefIterator(measurement string, opt influxql.IteratorOptions, aggregate bool) ([]influxql.Iterator, error) {
|
||||
ref, _ := opt.Expr.(*influxql.VarRef)
|
||||
|
||||
var itrs []influxql.Iterator
|
||||
if err := func() error {
|
||||
mms := tsdb.Measurements(e.index.MeasurementsByName(influxql.Sources(opt.Sources).Names()))
|
||||
mm := e.index.Measurement(measurement)
|
||||
if mm == nil {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
for _, mm := range mms {
|
||||
// Determine tagsets for this measurement based on dimensions and filters.
|
||||
tagSets, err := mm.TagSets(e.id, opt.Dimensions, opt.Condition)
|
||||
// Determine tagsets for this measurement based on dimensions and filters.
|
||||
tagSets, err := mm.TagSets(e.id, opt.Dimensions, opt.Condition)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// Calculate tag sets and apply SLIMIT/SOFFSET.
|
||||
tagSets = influxql.LimitTagSets(tagSets, opt.SLimit, opt.SOffset)
|
||||
|
||||
itrs := make([]influxql.Iterator, 0, len(tagSets))
|
||||
if err := func() error {
|
||||
for _, t := range tagSets {
|
||||
inputs, err := e.createTagSetIterators(ref, mm, t, opt)
|
||||
if err != nil {
|
||||
return err
|
||||
} else if len(inputs) == 0 {
|
||||
continue
|
||||
}
|
||||
|
||||
itr, err := influxql.Iterators(inputs).Merge(opt)
|
||||
if err != nil {
|
||||
influxql.Iterators(inputs).Close()
|
||||
return err
|
||||
}
|
||||
|
||||
// Calculate tag sets and apply SLIMIT/SOFFSET.
|
||||
tagSets = influxql.LimitTagSets(tagSets, opt.SLimit, opt.SOffset)
|
||||
|
||||
for _, t := range tagSets {
|
||||
inputs, err := e.createTagSetIterators(ref, mm, t, opt)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if !aggregate && len(inputs) > 0 && (opt.Limit > 0 || opt.Offset > 0) {
|
||||
itrs = append(itrs, newLimitIterator(influxql.NewSortedMergeIterator(inputs, opt), opt))
|
||||
} else {
|
||||
itrs = append(itrs, inputs...)
|
||||
}
|
||||
if !aggregate && len(inputs) > 0 && (opt.Limit > 0 || opt.Offset > 0) {
|
||||
itr = newLimitIterator(itr, opt)
|
||||
}
|
||||
itrs = append(itrs, itr)
|
||||
}
|
||||
return nil
|
||||
}(); err != nil {
|
||||
|
@ -1526,7 +1536,8 @@ func (e *Engine) createVarRefSeriesIterator(ref *influxql.VarRef, mm *tsdb.Measu
|
|||
condNames := influxql.VarRefs(conditionFields).Strings()
|
||||
|
||||
// Limit tags to only the dimensions selected.
|
||||
tags = tags.Subset(opt.Dimensions)
|
||||
dimensions := opt.GetDimensions()
|
||||
tags = tags.Subset(dimensions)
|
||||
|
||||
// If it's only auxiliary fields then it doesn't matter what type of iterator we use.
|
||||
if ref == nil {
|
||||
|
|
|
@ -234,10 +234,9 @@ func TestEngine_CreateIterator_Cache_Ascending(t *testing.T) {
|
|||
t.Fatalf("failed to write points: %s", err.Error())
|
||||
}
|
||||
|
||||
itr, err := e.CreateIterator(influxql.IteratorOptions{
|
||||
itr, err := e.CreateIterator("cpu", influxql.IteratorOptions{
|
||||
Expr: influxql.MustParseExpr(`value`),
|
||||
Dimensions: []string{"host"},
|
||||
Sources: []influxql.Source{&influxql.Measurement{Name: "cpu"}},
|
||||
StartTime: influxql.MinTime,
|
||||
EndTime: influxql.MaxTime,
|
||||
Ascending: true,
|
||||
|
@ -289,10 +288,9 @@ func TestEngine_CreateIterator_Cache_Descending(t *testing.T) {
|
|||
t.Fatalf("failed to write points: %s", err.Error())
|
||||
}
|
||||
|
||||
itr, err := e.CreateIterator(influxql.IteratorOptions{
|
||||
itr, err := e.CreateIterator("cpu", influxql.IteratorOptions{
|
||||
Expr: influxql.MustParseExpr(`value`),
|
||||
Dimensions: []string{"host"},
|
||||
Sources: []influxql.Source{&influxql.Measurement{Name: "cpu"}},
|
||||
StartTime: influxql.MinTime,
|
||||
EndTime: influxql.MaxTime,
|
||||
Ascending: false,
|
||||
|
@ -345,10 +343,9 @@ func TestEngine_CreateIterator_TSM_Ascending(t *testing.T) {
|
|||
}
|
||||
e.MustWriteSnapshot()
|
||||
|
||||
itr, err := e.CreateIterator(influxql.IteratorOptions{
|
||||
itr, err := e.CreateIterator("cpu", influxql.IteratorOptions{
|
||||
Expr: influxql.MustParseExpr(`value`),
|
||||
Dimensions: []string{"host"},
|
||||
Sources: []influxql.Source{&influxql.Measurement{Name: "cpu"}},
|
||||
StartTime: influxql.MinTime,
|
||||
EndTime: influxql.MaxTime,
|
||||
Ascending: true,
|
||||
|
@ -401,10 +398,9 @@ func TestEngine_CreateIterator_TSM_Descending(t *testing.T) {
|
|||
}
|
||||
e.MustWriteSnapshot()
|
||||
|
||||
itr, err := e.CreateIterator(influxql.IteratorOptions{
|
||||
itr, err := e.CreateIterator("cpu", influxql.IteratorOptions{
|
||||
Expr: influxql.MustParseExpr(`value`),
|
||||
Dimensions: []string{"host"},
|
||||
Sources: []influxql.Source{&influxql.Measurement{Name: "cpu"}},
|
||||
StartTime: influxql.MinTime,
|
||||
EndTime: influxql.MaxTime,
|
||||
Ascending: false,
|
||||
|
@ -459,11 +455,10 @@ func TestEngine_CreateIterator_Aux(t *testing.T) {
|
|||
t.Fatalf("failed to write points: %s", err.Error())
|
||||
}
|
||||
|
||||
itr, err := e.CreateIterator(influxql.IteratorOptions{
|
||||
itr, err := e.CreateIterator("cpu", influxql.IteratorOptions{
|
||||
Expr: influxql.MustParseExpr(`value`),
|
||||
Aux: []influxql.VarRef{{Val: "F"}},
|
||||
Dimensions: []string{"host"},
|
||||
Sources: []influxql.Source{&influxql.Measurement{Name: "cpu"}},
|
||||
StartTime: influxql.MinTime,
|
||||
EndTime: influxql.MaxTime,
|
||||
Ascending: true,
|
||||
|
@ -525,11 +520,10 @@ func TestEngine_CreateIterator_Condition(t *testing.T) {
|
|||
t.Fatalf("failed to write points: %s", err.Error())
|
||||
}
|
||||
|
||||
itr, err := e.CreateIterator(influxql.IteratorOptions{
|
||||
itr, err := e.CreateIterator("cpu", influxql.IteratorOptions{
|
||||
Expr: influxql.MustParseExpr(`value`),
|
||||
Dimensions: []string{"host"},
|
||||
Condition: influxql.MustParseExpr(`X = 10 OR Y > 150`),
|
||||
Sources: []influxql.Source{&influxql.Measurement{Name: "cpu"}},
|
||||
StartTime: influxql.MinTime,
|
||||
EndTime: influxql.MaxTime,
|
||||
Ascending: true,
|
||||
|
@ -680,7 +674,6 @@ func BenchmarkEngine_CreateIterator_Count_1M(b *testing.B) {
|
|||
func benchmarkEngineCreateIteratorCount(b *testing.B, pointN int) {
|
||||
benchmarkIterator(b, influxql.IteratorOptions{
|
||||
Expr: influxql.MustParseExpr("count(value)"),
|
||||
Sources: []influxql.Source{&influxql.Measurement{Name: "cpu"}},
|
||||
Ascending: true,
|
||||
StartTime: influxql.MinTime,
|
||||
EndTime: influxql.MaxTime,
|
||||
|
@ -700,7 +693,6 @@ func BenchmarkEngine_CreateIterator_First_1M(b *testing.B) {
|
|||
func benchmarkEngineCreateIteratorFirst(b *testing.B, pointN int) {
|
||||
benchmarkIterator(b, influxql.IteratorOptions{
|
||||
Expr: influxql.MustParseExpr("first(value)"),
|
||||
Sources: []influxql.Source{&influxql.Measurement{Name: "cpu"}},
|
||||
Dimensions: []string{"host"},
|
||||
Ascending: true,
|
||||
StartTime: influxql.MinTime,
|
||||
|
@ -721,7 +713,6 @@ func BenchmarkEngine_CreateIterator_Last_1M(b *testing.B) {
|
|||
func benchmarkEngineCreateIteratorLast(b *testing.B, pointN int) {
|
||||
benchmarkIterator(b, influxql.IteratorOptions{
|
||||
Expr: influxql.MustParseExpr("last(value)"),
|
||||
Sources: []influxql.Source{&influxql.Measurement{Name: "cpu"}},
|
||||
Dimensions: []string{"host"},
|
||||
Ascending: true,
|
||||
StartTime: influxql.MinTime,
|
||||
|
@ -849,7 +840,6 @@ func benchmarkEngine_WritePoints_Parallel(b *testing.B, batchSize int) {
|
|||
func benchmarkEngineCreateIteratorLimit(b *testing.B, pointN int) {
|
||||
benchmarkIterator(b, influxql.IteratorOptions{
|
||||
Expr: influxql.MustParseExpr("value"),
|
||||
Sources: []influxql.Source{&influxql.Measurement{Name: "cpu"}},
|
||||
Dimensions: []string{"host"},
|
||||
Ascending: true,
|
||||
StartTime: influxql.MinTime,
|
||||
|
@ -864,7 +854,7 @@ func benchmarkIterator(b *testing.B, opt influxql.IteratorOptions, pointN int) {
|
|||
b.ReportAllocs()
|
||||
|
||||
for i := 0; i < b.N; i++ {
|
||||
itr, err := e.CreateIterator(opt)
|
||||
itr, err := e.CreateIterator("cpu", opt)
|
||||
if err != nil {
|
||||
b.Fatal(err)
|
||||
}
|
||||
|
|
303
tsdb/shard.go
303
tsdb/shard.go
|
@ -7,6 +7,7 @@ import (
|
|||
"io"
|
||||
"os"
|
||||
"path/filepath"
|
||||
"regexp"
|
||||
"sort"
|
||||
"strings"
|
||||
"sync"
|
||||
|
@ -669,27 +670,20 @@ func (s *Shard) WriteTo(w io.Writer) (int64, error) {
|
|||
}
|
||||
|
||||
// CreateIterator returns an iterator for the data in the shard.
|
||||
func (s *Shard) CreateIterator(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
func (s *Shard) CreateIterator(measurement string, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
if err := s.ready(); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
if influxql.Sources(opt.Sources).HasSystemSource() {
|
||||
return s.createSystemIterator(opt)
|
||||
if strings.HasPrefix(measurement, "_") {
|
||||
return s.createSystemIterator(measurement, opt)
|
||||
}
|
||||
opt.Sources = influxql.Sources(opt.Sources).Filter(s.database, s.retentionPolicy)
|
||||
return s.engine.CreateIterator(opt)
|
||||
return s.engine.CreateIterator(measurement, opt)
|
||||
}
|
||||
|
||||
// createSystemIterator returns an iterator for a system source.
|
||||
func (s *Shard) createSystemIterator(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
// Only support a single system source.
|
||||
if len(opt.Sources) > 1 {
|
||||
return nil, errors.New("cannot select from multiple system sources")
|
||||
}
|
||||
|
||||
m := opt.Sources[0].(*influxql.Measurement)
|
||||
switch m.Name {
|
||||
func (s *Shard) createSystemIterator(measurement string, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
switch measurement {
|
||||
case "_fieldKeys":
|
||||
return NewFieldKeysIterator(s, opt)
|
||||
case "_series":
|
||||
|
@ -697,71 +691,112 @@ func (s *Shard) createSystemIterator(opt influxql.IteratorOptions) (influxql.Ite
|
|||
case "_tagKeys":
|
||||
return NewTagKeysIterator(s, opt)
|
||||
default:
|
||||
return nil, fmt.Errorf("unknown system source: %s", m.Name)
|
||||
return nil, fmt.Errorf("unknown system source: %s", measurement)
|
||||
}
|
||||
}
|
||||
|
||||
// FieldDimensions returns unique sets of fields and dimensions across a list of sources.
|
||||
func (s *Shard) FieldDimensions(sources influxql.Sources) (fields map[string]influxql.DataType, dimensions map[string]struct{}, err error) {
|
||||
func (s *Shard) FieldDimensions(measurements []string) (fields map[string]influxql.DataType, dimensions map[string]struct{}, err error) {
|
||||
if err := s.ready(); err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
|
||||
if sources.HasSystemSource() {
|
||||
// Only support a single system source.
|
||||
if len(sources) > 1 {
|
||||
return nil, nil, errors.New("cannot select from multiple system sources")
|
||||
}
|
||||
|
||||
switch m := sources[0].(type) {
|
||||
case *influxql.Measurement:
|
||||
switch m.Name {
|
||||
case "_fieldKeys":
|
||||
return map[string]influxql.DataType{
|
||||
"fieldKey": influxql.String,
|
||||
"fieldType": influxql.String,
|
||||
}, nil, nil
|
||||
case "_series":
|
||||
return map[string]influxql.DataType{
|
||||
"key": influxql.String,
|
||||
}, nil, nil
|
||||
case "_tagKeys":
|
||||
return map[string]influxql.DataType{
|
||||
"tagKey": influxql.String,
|
||||
}, nil, nil
|
||||
}
|
||||
}
|
||||
return nil, nil, nil
|
||||
}
|
||||
|
||||
fields = make(map[string]influxql.DataType)
|
||||
dimensions = make(map[string]struct{})
|
||||
|
||||
for _, src := range sources {
|
||||
switch m := src.(type) {
|
||||
case *influxql.Measurement:
|
||||
// Retrieve measurement.
|
||||
mm := s.index.Measurement(m.Name)
|
||||
if mm == nil {
|
||||
continue
|
||||
for _, name := range measurements {
|
||||
// Handle system sources.
|
||||
if strings.HasPrefix(name, "_") {
|
||||
var keys []string
|
||||
switch name {
|
||||
case "_fieldKeys":
|
||||
keys = []string{"fieldKey", "fieldType"}
|
||||
case "_series":
|
||||
keys = []string{"key"}
|
||||
case "_tagKeys":
|
||||
keys = []string{"tagKey"}
|
||||
}
|
||||
|
||||
// Append fields and dimensions.
|
||||
mf := s.engine.MeasurementFields(m.Name)
|
||||
if mf != nil {
|
||||
for name, typ := range mf.FieldSet() {
|
||||
fields[name] = typ
|
||||
for _, k := range keys {
|
||||
if _, ok := fields[k]; !ok || influxql.String < fields[k] {
|
||||
fields[k] = influxql.String
|
||||
}
|
||||
}
|
||||
for _, key := range mm.TagKeys() {
|
||||
dimensions[key] = struct{}{}
|
||||
continue
|
||||
}
|
||||
|
||||
// Retrieve measurement.
|
||||
mm := s.index.Measurement(name)
|
||||
if mm == nil {
|
||||
continue
|
||||
}
|
||||
|
||||
// Append fields and dimensions.
|
||||
mf := s.engine.MeasurementFields(name)
|
||||
if mf != nil {
|
||||
for k, typ := range mf.FieldSet() {
|
||||
if _, ok := fields[k]; !ok || typ < fields[k] {
|
||||
fields[k] = typ
|
||||
}
|
||||
}
|
||||
}
|
||||
for _, key := range mm.TagKeys() {
|
||||
dimensions[key] = struct{}{}
|
||||
}
|
||||
}
|
||||
|
||||
return
|
||||
}
|
||||
|
||||
func (s *Shard) MeasurementsByRegex(re *regexp.Regexp) []string {
|
||||
mms := s.index.MeasurementsByRegex(re)
|
||||
names := make([]string, len(mms))
|
||||
for i, mm := range mms {
|
||||
names[i] = mm.Name
|
||||
}
|
||||
return names
|
||||
}
|
||||
|
||||
// MapType returns the data type for the field within the measurement.
|
||||
func (s *Shard) MapType(measurement, field string) influxql.DataType {
|
||||
// Process system measurements.
|
||||
if strings.HasPrefix(measurement, "_") {
|
||||
switch measurement {
|
||||
case "_fieldKeys":
|
||||
if field == "fieldKey" || field == "fieldType" {
|
||||
return influxql.String
|
||||
}
|
||||
case "_series":
|
||||
if field == "key" {
|
||||
return influxql.String
|
||||
}
|
||||
case "_tagKeys":
|
||||
if field == "tagKey" {
|
||||
return influxql.String
|
||||
}
|
||||
}
|
||||
return influxql.Unknown
|
||||
}
|
||||
|
||||
mm := s.index.Measurement(measurement)
|
||||
if mm == nil {
|
||||
return influxql.Unknown
|
||||
}
|
||||
|
||||
mf := s.engine.MeasurementFields(measurement)
|
||||
if mf != nil {
|
||||
f := mf.Field(field)
|
||||
if f != nil {
|
||||
return f.Type
|
||||
}
|
||||
}
|
||||
|
||||
if mm.HasTagKey(field) {
|
||||
return influxql.Tag
|
||||
}
|
||||
return influxql.Unknown
|
||||
}
|
||||
|
||||
// ExpandSources expands regex sources and removes duplicates.
|
||||
// NOTE: sources must be normalized (db and rp set) before calling this function.
|
||||
func (s *Shard) ExpandSources(sources influxql.Sources) (influxql.Sources, error) {
|
||||
|
@ -879,6 +914,14 @@ func (s *Shard) monitor() {
|
|||
}
|
||||
}
|
||||
|
||||
type ShardGroup interface {
|
||||
MeasurementsByRegex(re *regexp.Regexp) []string
|
||||
FieldDimensions(measurements []string) (fields map[string]influxql.DataType, dimensions map[string]struct{}, err error)
|
||||
MapType(measurement, field string) influxql.DataType
|
||||
CreateIterator(measurement string, opt influxql.IteratorOptions) (influxql.Iterator, error)
|
||||
ExpandSources(sources influxql.Sources) (influxql.Sources, error)
|
||||
}
|
||||
|
||||
// Shards represents a sortable list of shards.
|
||||
type Shards []*Shard
|
||||
|
||||
|
@ -891,6 +934,119 @@ func (a Shards) Less(i, j int) bool { return a[i].id < a[j].id }
|
|||
// Swap implements sort.Interface.
|
||||
func (a Shards) Swap(i, j int) { a[i], a[j] = a[j], a[i] }
|
||||
|
||||
func (a Shards) MeasurementsByRegex(re *regexp.Regexp) []string {
|
||||
m := make(map[string]struct{})
|
||||
for _, sh := range a {
|
||||
names := sh.MeasurementsByRegex(re)
|
||||
for _, name := range names {
|
||||
m[name] = struct{}{}
|
||||
}
|
||||
}
|
||||
|
||||
if len(m) == 0 {
|
||||
return nil
|
||||
}
|
||||
|
||||
names := make([]string, 0, len(m))
|
||||
for key := range m {
|
||||
names = append(names, key)
|
||||
}
|
||||
sort.Strings(names)
|
||||
return names
|
||||
}
|
||||
|
||||
func (a Shards) FieldDimensions(measurements []string) (fields map[string]influxql.DataType, dimensions map[string]struct{}, err error) {
|
||||
fields = make(map[string]influxql.DataType)
|
||||
dimensions = make(map[string]struct{})
|
||||
|
||||
for _, sh := range a {
|
||||
f, d, err := sh.FieldDimensions(measurements)
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
for k, typ := range f {
|
||||
if _, ok := fields[k]; typ != influxql.Unknown && (!ok || typ < fields[k]) {
|
||||
fields[k] = typ
|
||||
}
|
||||
}
|
||||
for k := range d {
|
||||
dimensions[k] = struct{}{}
|
||||
}
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
func (a Shards) MapType(measurement, field string) influxql.DataType {
|
||||
var typ influxql.DataType
|
||||
for _, sh := range a {
|
||||
t := sh.MapType(measurement, field)
|
||||
if typ == influxql.Unknown || (t != influxql.Unknown && t < typ) {
|
||||
typ = t
|
||||
}
|
||||
}
|
||||
return typ
|
||||
}
|
||||
|
||||
func (a Shards) CreateIterator(measurement string, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
itrs := make([]influxql.Iterator, 0, len(a))
|
||||
for _, sh := range a {
|
||||
itr, err := sh.CreateIterator(measurement, opt)
|
||||
if err != nil {
|
||||
influxql.Iterators(itrs).Close()
|
||||
return nil, err
|
||||
} else if itr == nil {
|
||||
continue
|
||||
}
|
||||
itrs = append(itrs, itr)
|
||||
|
||||
// Enforce series limit at creation time.
|
||||
if opt.MaxSeriesN > 0 {
|
||||
stats := itr.Stats()
|
||||
if stats.SeriesN > opt.MaxSeriesN {
|
||||
influxql.Iterators(itrs).Close()
|
||||
return nil, fmt.Errorf("max-select-series limit exceeded: (%d/%d)", stats.SeriesN, opt.MaxSeriesN)
|
||||
}
|
||||
}
|
||||
}
|
||||
return influxql.Iterators(itrs).Merge(opt)
|
||||
}
|
||||
|
||||
func (a Shards) ExpandSources(sources influxql.Sources) (influxql.Sources, error) {
|
||||
// Use a map as a set to prevent duplicates.
|
||||
set := map[string]influxql.Source{}
|
||||
|
||||
// Iterate through every shard and expand the sources.
|
||||
for _, sh := range a {
|
||||
expanded, err := sh.ExpandSources(sources)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
for _, src := range expanded {
|
||||
switch src := src.(type) {
|
||||
case *influxql.Measurement:
|
||||
set[src.String()] = src
|
||||
default:
|
||||
return nil, fmt.Errorf("Store.ExpandSources: unsupported source type: %T", src)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Convert set to sorted slice.
|
||||
names := make([]string, 0, len(set))
|
||||
for name := range set {
|
||||
names = append(names, name)
|
||||
}
|
||||
sort.Strings(names)
|
||||
|
||||
// Convert set to a list of Sources.
|
||||
sorted := make([]influxql.Source, 0, len(set))
|
||||
for _, name := range names {
|
||||
sorted = append(sorted, set[name])
|
||||
}
|
||||
return sorted, nil
|
||||
}
|
||||
|
||||
// MeasurementFields holds the fields of a measurement and their codec.
|
||||
type MeasurementFields struct {
|
||||
mu sync.RWMutex
|
||||
|
@ -1006,41 +1162,6 @@ type Field struct {
|
|||
Type influxql.DataType `json:"type,omitempty"`
|
||||
}
|
||||
|
||||
// shardIteratorCreator creates iterators for a local shard.
|
||||
// This simply wraps the shard so that Close() does not close the underlying shard.
|
||||
type shardIteratorCreator struct {
|
||||
sh *Shard
|
||||
maxSeriesN int
|
||||
}
|
||||
|
||||
func (ic *shardIteratorCreator) Close() error { return nil }
|
||||
|
||||
func (ic *shardIteratorCreator) CreateIterator(opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
itr, err := ic.sh.CreateIterator(opt)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
} else if itr == nil {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// Enforce series limit at creation time.
|
||||
if ic.maxSeriesN > 0 {
|
||||
stats := itr.Stats()
|
||||
if stats.SeriesN > ic.maxSeriesN {
|
||||
itr.Close()
|
||||
return nil, fmt.Errorf("max-select-series limit exceeded: (%d/%d)", stats.SeriesN, ic.maxSeriesN)
|
||||
}
|
||||
}
|
||||
|
||||
return itr, nil
|
||||
}
|
||||
func (ic *shardIteratorCreator) FieldDimensions(sources influxql.Sources) (fields map[string]influxql.DataType, dimensions map[string]struct{}, err error) {
|
||||
return ic.sh.FieldDimensions(sources)
|
||||
}
|
||||
func (ic *shardIteratorCreator) ExpandSources(sources influxql.Sources) (influxql.Sources, error) {
|
||||
return ic.sh.ExpandSources(sources)
|
||||
}
|
||||
|
||||
// NewFieldKeysIterator returns an iterator that can be iterated over to
|
||||
// retrieve field keys.
|
||||
func NewFieldKeysIterator(sh *Shard, opt influxql.IteratorOptions) (influxql.Iterator, error) {
|
||||
|
|
|
@ -485,7 +485,7 @@ func TestShard_CreateIterator_Ascending(t *testing.T) {
|
|||
|
||||
// Calling CreateIterator when the engine is not open will return
|
||||
// ErrEngineClosed.
|
||||
_, got := sh.CreateIterator(influxql.IteratorOptions{})
|
||||
_, got := sh.CreateIterator("cpu", influxql.IteratorOptions{})
|
||||
if exp := tsdb.ErrEngineClosed; got != exp {
|
||||
t.Fatalf("got %v, expected %v", got, exp)
|
||||
}
|
||||
|
@ -502,18 +502,13 @@ cpu,host=serverB,region=uswest value=25 0
|
|||
`)
|
||||
|
||||
// Create iterator.
|
||||
itr, err := sh.CreateIterator(influxql.IteratorOptions{
|
||||
itr, err := sh.CreateIterator("cpu", influxql.IteratorOptions{
|
||||
Expr: influxql.MustParseExpr(`value`),
|
||||
Aux: []influxql.VarRef{{Val: "val2"}},
|
||||
Dimensions: []string{"host"},
|
||||
Sources: []influxql.Source{&influxql.Measurement{
|
||||
Name: "cpu",
|
||||
Database: "db0",
|
||||
RetentionPolicy: "rp0",
|
||||
}},
|
||||
Ascending: true,
|
||||
StartTime: influxql.MinTime,
|
||||
EndTime: influxql.MaxTime,
|
||||
Ascending: true,
|
||||
StartTime: influxql.MinTime,
|
||||
EndTime: influxql.MaxTime,
|
||||
})
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
|
@ -565,7 +560,7 @@ func TestShard_CreateIterator_Descending(t *testing.T) {
|
|||
|
||||
// Calling CreateIterator when the engine is not open will return
|
||||
// ErrEngineClosed.
|
||||
_, got := sh.CreateIterator(influxql.IteratorOptions{})
|
||||
_, got := sh.CreateIterator("cpu", influxql.IteratorOptions{})
|
||||
if exp := tsdb.ErrEngineClosed; got != exp {
|
||||
t.Fatalf("got %v, expected %v", got, exp)
|
||||
}
|
||||
|
@ -582,18 +577,13 @@ cpu,host=serverB,region=uswest value=25 0
|
|||
`)
|
||||
|
||||
// Create iterator.
|
||||
itr, err := sh.CreateIterator(influxql.IteratorOptions{
|
||||
itr, err := sh.CreateIterator("cpu", influxql.IteratorOptions{
|
||||
Expr: influxql.MustParseExpr(`value`),
|
||||
Aux: []influxql.VarRef{{Val: "val2"}},
|
||||
Dimensions: []string{"host"},
|
||||
Sources: []influxql.Source{&influxql.Measurement{
|
||||
Name: "cpu",
|
||||
Database: "db0",
|
||||
RetentionPolicy: "rp0",
|
||||
}},
|
||||
Ascending: false,
|
||||
StartTime: influxql.MinTime,
|
||||
EndTime: influxql.MaxTime,
|
||||
Ascending: false,
|
||||
StartTime: influxql.MinTime,
|
||||
EndTime: influxql.MaxTime,
|
||||
})
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
|
@ -663,7 +653,7 @@ func TestShard_Disabled_WriteQuery(t *testing.T) {
|
|||
t.Fatalf(err.Error())
|
||||
}
|
||||
|
||||
_, got := sh.CreateIterator(influxql.IteratorOptions{})
|
||||
_, got := sh.CreateIterator("cpu", influxql.IteratorOptions{})
|
||||
if err == nil {
|
||||
t.Fatalf("expected shard disabled error")
|
||||
}
|
||||
|
@ -678,11 +668,70 @@ func TestShard_Disabled_WriteQuery(t *testing.T) {
|
|||
t.Fatalf("unexpected error: %v", err)
|
||||
}
|
||||
|
||||
if _, err = sh.CreateIterator(influxql.IteratorOptions{}); err != nil {
|
||||
if _, err = sh.CreateIterator("cpu", influxql.IteratorOptions{}); err != nil {
|
||||
t.Fatalf("unexpected error: %v", got)
|
||||
}
|
||||
}
|
||||
|
||||
func TestShard_FieldDimensions(t *testing.T) {
|
||||
sh := NewShard()
|
||||
|
||||
if err := sh.Open(); err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
defer sh.Close()
|
||||
|
||||
sh.MustWritePointsString(`
|
||||
cpu,host=serverA,region=uswest value=100 0
|
||||
cpu,host=serverA,region=uswest value=50,val2=5 10
|
||||
cpu,host=serverB,region=uswest value=25 0
|
||||
mem,host=serverA value=25i 0
|
||||
mem,host=serverB value=50i,val3=t 10
|
||||
`)
|
||||
|
||||
for i, tt := range []struct {
|
||||
sources []string
|
||||
f map[string]influxql.DataType
|
||||
d map[string]struct{}
|
||||
}{
|
||||
{
|
||||
sources: []string{"cpu"},
|
||||
f: map[string]influxql.DataType{
|
||||
"value": influxql.Float,
|
||||
"val2": influxql.Float,
|
||||
},
|
||||
d: map[string]struct{}{
|
||||
"host": struct{}{},
|
||||
"region": struct{}{},
|
||||
},
|
||||
},
|
||||
{
|
||||
sources: []string{"cpu", "mem"},
|
||||
f: map[string]influxql.DataType{
|
||||
"value": influxql.Float,
|
||||
"val2": influxql.Float,
|
||||
"val3": influxql.Boolean,
|
||||
},
|
||||
d: map[string]struct{}{
|
||||
"host": struct{}{},
|
||||
"region": struct{}{},
|
||||
},
|
||||
},
|
||||
} {
|
||||
f, d, err := sh.FieldDimensions(tt.sources)
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error: %v", err)
|
||||
}
|
||||
|
||||
if !reflect.DeepEqual(f, tt.f) {
|
||||
t.Errorf("%d. unexpected fields:\n\nexp=%#v\n\ngot=%#v\n\n", i, tt.f, f)
|
||||
}
|
||||
if !reflect.DeepEqual(d, tt.d) {
|
||||
t.Errorf("%d. unexpected dimensions:\n\nexp=%#v\n\ngot=%#v\n\n", i, tt.d, d)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func BenchmarkWritePoints_NewSeries_1K(b *testing.B) { benchmarkWritePoints(b, 38, 3, 3, 1) }
|
||||
func BenchmarkWritePoints_NewSeries_100K(b *testing.B) { benchmarkWritePoints(b, 32, 5, 5, 1) }
|
||||
func BenchmarkWritePoints_NewSeries_250K(b *testing.B) { benchmarkWritePoints(b, 80, 5, 5, 1) }
|
||||
|
|
|
@ -272,6 +272,11 @@ func (s *Store) Shards(ids []uint64) []*Shard {
|
|||
return a
|
||||
}
|
||||
|
||||
// ShardGroup returns a ShardGroup with a list of shards by id.
|
||||
func (s *Store) ShardGroup(ids []uint64) ShardGroup {
|
||||
return Shards(s.Shards(ids))
|
||||
}
|
||||
|
||||
// ShardN returns the number of shards in the store.
|
||||
func (s *Store) ShardN() int {
|
||||
s.mu.RLock()
|
||||
|
@ -379,18 +384,6 @@ func (s *Store) DeleteShard(shardID uint64) error {
|
|||
return nil
|
||||
}
|
||||
|
||||
// ShardIteratorCreator returns an iterator creator for a shard.
|
||||
func (s *Store) ShardIteratorCreator(id uint64, opt *influxql.SelectOptions) influxql.IteratorCreator {
|
||||
sh := s.Shard(id)
|
||||
if sh == nil {
|
||||
return nil
|
||||
}
|
||||
return &shardIteratorCreator{
|
||||
sh: sh,
|
||||
maxSeriesN: opt.MaxSeriesN,
|
||||
}
|
||||
}
|
||||
|
||||
// DeleteDatabase will close all shards associated with a database and remove the directory and files from disk.
|
||||
func (s *Store) DeleteDatabase(name string) error {
|
||||
s.mu.RLock()
|
||||
|
@ -769,41 +762,12 @@ func (s *Store) deleteSeries(database string, seriesKeys []string, min, max int6
|
|||
|
||||
// ExpandSources expands sources against all local shards.
|
||||
func (s *Store) ExpandSources(sources influxql.Sources) (influxql.Sources, error) {
|
||||
return s.IteratorCreators().ExpandSources(sources)
|
||||
}
|
||||
|
||||
// IteratorCreators returns a set of all local shards as iterator creators.
|
||||
func (s *Store) IteratorCreators() influxql.IteratorCreators {
|
||||
s.mu.RLock()
|
||||
defer s.mu.RUnlock()
|
||||
|
||||
a := make(influxql.IteratorCreators, 0, len(s.shards))
|
||||
for _, sh := range s.shards {
|
||||
a = append(a, sh)
|
||||
}
|
||||
return a
|
||||
}
|
||||
|
||||
// IteratorCreator returns an iterator creator for all shards in the given shard IDs.
|
||||
func (s *Store) IteratorCreator(shards []uint64, opt *influxql.SelectOptions) (influxql.IteratorCreator, error) {
|
||||
// Generate iterators for each node.
|
||||
ics := make([]influxql.IteratorCreator, 0)
|
||||
if err := func() error {
|
||||
for _, id := range shards {
|
||||
ic := s.ShardIteratorCreator(id, opt)
|
||||
if ic == nil {
|
||||
continue
|
||||
}
|
||||
ics = append(ics, ic)
|
||||
}
|
||||
|
||||
return nil
|
||||
}(); err != nil {
|
||||
influxql.IteratorCreators(ics).Close()
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return influxql.IteratorCreators(ics), nil
|
||||
shards := func() Shards {
|
||||
s.mu.RLock()
|
||||
defer s.mu.RUnlock()
|
||||
return Shards(s.shardsSlice())
|
||||
}()
|
||||
return shards.ExpandSources(sources)
|
||||
}
|
||||
|
||||
// WriteToShard writes a list of points to a shard identified by its ID.
|
||||
|
|
|
@ -239,25 +239,16 @@ func TestShards_CreateIterator(t *testing.T) {
|
|||
`cpu,host=serverC value=3 60`,
|
||||
)
|
||||
|
||||
// Retrieve shards and convert to iterator creators.
|
||||
shards := s.Shards([]uint64{0, 1})
|
||||
ics := make(influxql.IteratorCreators, len(shards))
|
||||
for i := range ics {
|
||||
ics[i] = shards[i]
|
||||
}
|
||||
// Retrieve shard group.
|
||||
shards := s.ShardGroup([]uint64{0, 1})
|
||||
|
||||
// Create iterator.
|
||||
itr, err := ics.CreateIterator(influxql.IteratorOptions{
|
||||
itr, err := shards.CreateIterator("cpu", influxql.IteratorOptions{
|
||||
Expr: influxql.MustParseExpr(`value`),
|
||||
Dimensions: []string{"host"},
|
||||
Sources: []influxql.Source{&influxql.Measurement{
|
||||
Name: "cpu",
|
||||
Database: "db0",
|
||||
RetentionPolicy: "rp0",
|
||||
}},
|
||||
Ascending: true,
|
||||
StartTime: influxql.MinTime,
|
||||
EndTime: influxql.MaxTime,
|
||||
Ascending: true,
|
||||
StartTime: influxql.MinTime,
|
||||
EndTime: influxql.MaxTime,
|
||||
})
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
|
@ -332,13 +323,8 @@ func TestStore_BackupRestoreShard(t *testing.T) {
|
|||
}
|
||||
|
||||
// Read data from
|
||||
itr, err := s1.Shard(100).CreateIterator(influxql.IteratorOptions{
|
||||
Expr: influxql.MustParseExpr(`value`),
|
||||
Sources: []influxql.Source{&influxql.Measurement{
|
||||
Name: "cpu",
|
||||
Database: "db0",
|
||||
RetentionPolicy: "rp0",
|
||||
}},
|
||||
itr, err := s1.Shard(100).CreateIterator("cpu", influxql.IteratorOptions{
|
||||
Expr: influxql.MustParseExpr(`value`),
|
||||
Ascending: true,
|
||||
StartTime: influxql.MinTime,
|
||||
EndTime: influxql.MaxTime,
|
||||
|
|
Loading…
Reference in New Issue