From e3b4b71c13db844686eb7092dec8acccf266681a Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Fri, 12 Feb 2016 15:10:02 -0700 Subject: [PATCH] refactor query executor This commit moves the `QueryExecutor` to the `cluster` package and provides an interface to it inside the `influxql` package. --- cluster/points_writer.go | 2 +- cluster/query_executor.go | 932 +++++++++++++ cmd/influxd/restore/restore.go | 4 +- cmd/influxd/run/server.go | 45 +- cmd/influxd/run/server_helpers_test.go | 2 +- influxql/query_executor.go | 26 + influxql/statement_rewriter.go | 113 ++ monitor/service.go | 16 +- monitor/service_test.go | 69 - monitor/statement_executor.go | 82 -- services/continuous_querier/service.go | 16 +- services/continuous_querier/service_test.go | 43 +- services/httpd/handler.go | 19 +- services/httpd/handler_test.go | 44 +- services/meta/client.go | 31 +- services/meta/query_authorizer.go | 101 ++ services/meta/service_test.go | 316 ++--- services/meta/statement_executor.go | 456 ------- .../meta/statement_executor_test.go.fixme | 1192 ----------------- services/meta/store.go | 8 +- tsdb/query_executor.go | 1086 --------------- tsdb/query_executor_test.go | 490 ------- tsdb/store.go | 357 ++++- 23 files changed, 1760 insertions(+), 3690 deletions(-) create mode 100644 cluster/query_executor.go create mode 100644 influxql/query_executor.go create mode 100644 influxql/statement_rewriter.go delete mode 100644 monitor/service_test.go delete mode 100644 monitor/statement_executor.go create mode 100644 services/meta/query_authorizer.go delete mode 100644 services/meta/statement_executor.go delete mode 100644 services/meta/statement_executor_test.go.fixme delete mode 100644 tsdb/query_executor.go delete mode 100644 tsdb/query_executor_test.go diff --git a/cluster/points_writer.go b/cluster/points_writer.go index 56c4c85c1c1..da21cb51e37 100644 --- a/cluster/points_writer.go +++ b/cluster/points_writer.go @@ -220,7 +220,7 @@ func (w *PointsWriter) MapShards(wp *WritePointsRequest) (*ShardMapping, error) // WritePointsInto is a copy of WritePoints that uses a tsdb structure instead of // a cluster structure for information. This is to avoid a circular dependency -func (w *PointsWriter) WritePointsInto(p *tsdb.IntoWriteRequest) error { +func (w *PointsWriter) WritePointsInto(p *IntoWriteRequest) error { req := WritePointsRequest{ Database: p.Database, RetentionPolicy: p.RetentionPolicy, diff --git a/cluster/query_executor.go b/cluster/query_executor.go new file mode 100644 index 00000000000..9d44dd8b241 --- /dev/null +++ b/cluster/query_executor.go @@ -0,0 +1,932 @@ +package cluster + +import ( + "bytes" + "errors" + "fmt" + "io" + "io/ioutil" + "log" + "sort" + "strconv" + "time" + + "github.com/influxdata/influxdb" + "github.com/influxdata/influxdb/influxql" + "github.com/influxdata/influxdb/models" + "github.com/influxdata/influxdb/monitor" + "github.com/influxdata/influxdb/services/meta" + "github.com/influxdata/influxdb/tsdb" +) + +// QueryExecutor +type QueryExecutor struct { + MetaClient *meta.Client + + // TSDB storage for local node. + TSDBStore *tsdb.Store + + // Holds monitoring data for SHOW STATS and SHOW DIAGNOSTICS. + Monitor *monitor.Monitor + + // Used for rewriting points back into system for SELECT INTO statements. + PointsWriter *PointsWriter + + // Output of all logging. + // Defaults to discarding all log output. + LogOutput io.Writer +} + +// NewQueryExecutor returns a new instance of QueryExecutor. +func NewQueryExecutor() *QueryExecutor { + return &QueryExecutor{ + LogOutput: ioutil.Discard, + } +} + +// ExecuteQuery executes each statement within a query. +func (e *QueryExecutor) ExecuteQuery(query *influxql.Query, database string, chunkSize int, closing chan struct{}) <-chan *influxql.Result { + results := make(chan *influxql.Result) + go e.executeQuery(query, database, chunkSize, closing, results) + return results +} + +func (e *QueryExecutor) executeQuery(query *influxql.Query, database string, chunkSize int, closing chan struct{}, results chan *influxql.Result) { + defer close(results) + logger := e.logger() + + var i int + for ; i < len(query.Statements); i++ { + stmt := query.Statements[i] + + // If a default database wasn't passed in by the caller, check the statement. + defaultDB := database + if defaultDB == "" { + if s, ok := stmt.(influxql.HasDefaultDatabase); ok { + defaultDB = s.DefaultDatabase() + } + } + + // Rewrite statements, if necessary. + // This can occur on meta read statements which convert to SELECT statements. + newStmt, err := influxql.RewriteStatement(stmt) + if err != nil { + results <- &influxql.Result{Err: err} + break + } + stmt = newStmt + + // Normalize each statement. + if err := e.normalizeStatement(stmt, defaultDB); err != nil { + results <- &influxql.Result{Err: err} + break + } + + // Log each normalized statement. + logger.Println(stmt.String()) + + // Select statements are handled separately so that they can be streamed. + if stmt, ok := stmt.(*influxql.SelectStatement); ok { + if err := e.executeSelectStatement(stmt, chunkSize, i, results, closing); err != nil { + results <- &influxql.Result{StatementID: i, Err: err} + break + } + continue + } + + var rows models.Rows + switch stmt := stmt.(type) { + case *influxql.AlterRetentionPolicyStatement: + err = e.executeAlterRetentionPolicyStatement(stmt) + case *influxql.CreateContinuousQueryStatement: + err = e.executeCreateContinuousQueryStatement(stmt) + case *influxql.CreateDatabaseStatement: + err = e.executeCreateDatabaseStatement(stmt) + case *influxql.CreateRetentionPolicyStatement: + err = e.executeCreateRetentionPolicyStatement(stmt) + case *influxql.CreateSubscriptionStatement: + err = e.executeCreateSubscriptionStatement(stmt) + case *influxql.CreateUserStatement: + err = e.executeCreateUserStatement(stmt) + case *influxql.DropContinuousQueryStatement: + err = e.executeDropContinuousQueryStatement(stmt) + case *influxql.DropDatabaseStatement: + err = e.executeDropDatabaseStatement(stmt) + case *influxql.DropMeasurementStatement: + err = e.executeDropMeasurementStatement(stmt, database) + case *influxql.DropSeriesStatement: + err = e.executeDropSeriesStatement(stmt, database) + case *influxql.DropRetentionPolicyStatement: + err = e.executeDropRetentionPolicyStatement(stmt) + case *influxql.DropServerStatement: + err = e.executeDropServerStatement(stmt) + case *influxql.DropSubscriptionStatement: + err = e.executeDropSubscriptionStatement(stmt) + case *influxql.DropUserStatement: + err = e.executeDropUserStatement(stmt) + case *influxql.GrantStatement: + err = e.executeGrantStatement(stmt) + case *influxql.GrantAdminStatement: + err = e.executeGrantAdminStatement(stmt) + case *influxql.RevokeStatement: + err = e.executeRevokeStatement(stmt) + case *influxql.RevokeAdminStatement: + err = e.executeRevokeAdminStatement(stmt) + case *influxql.ShowContinuousQueriesStatement: + rows, err = e.executeShowContinuousQueriesStatement(stmt) + case *influxql.ShowDatabasesStatement: + rows, err = e.executeShowDatabasesStatement(stmt) + case *influxql.ShowDiagnosticsStatement: + rows, err = e.executeShowDiagnosticsStatement(stmt) + case *influxql.ShowFieldKeysStatement: + rows, err = e.executeShowFieldKeysStatement(stmt, database) + case *influxql.ShowGrantsForUserStatement: + rows, err = e.executeShowGrantsForUserStatement(stmt) + case *influxql.ShowRetentionPoliciesStatement: + rows, err = e.executeShowRetentionPoliciesStatement(stmt) + case *influxql.ShowSeriesStatement: + rows, err = e.executeShowSeriesStatement(stmt, database) + case *influxql.ShowServersStatement: + rows, err = e.executeShowServersStatement(stmt) + case *influxql.ShowShardsStatement: + rows, err = e.executeShowShardsStatement(stmt) + case *influxql.ShowShardGroupsStatement: + rows, err = e.executeShowShardGroupsStatement(stmt) + case *influxql.ShowStatsStatement: + rows, err = e.executeShowStatsStatement(stmt) + case *influxql.ShowSubscriptionsStatement: + rows, err = e.executeShowSubscriptionsStatement(stmt) + case *influxql.ShowTagValuesStatement: + rows, err = e.executeShowTagValuesStatement(stmt, database) + case *influxql.ShowUsersStatement: + rows, err = e.executeShowUsersStatement(stmt) + case *influxql.SetPasswordUserStatement: + err = e.executeSetPasswordUserStatement(stmt) + default: + err = influxql.ErrInvalidQuery + } + + // Send results for each statement. + results <- &influxql.Result{ + StatementID: i, + Series: rows, + Err: err, + } + + // Stop of the first error. + if err != nil { + break + } + } + + // Send error results for any statements which were not executed. + for ; i < len(query.Statements)-1; i++ { + results <- &influxql.Result{ + StatementID: i, + Err: influxql.ErrNotExecuted, + } + } +} + +func (e *QueryExecutor) executeAlterRetentionPolicyStatement(stmt *influxql.AlterRetentionPolicyStatement) error { + rpu := &meta.RetentionPolicyUpdate{ + Duration: stmt.Duration, + ReplicaN: stmt.Replication, + } + + // Update the retention policy. + if err := e.MetaClient.UpdateRetentionPolicy(stmt.Database, stmt.Name, rpu); err != nil { + return err + } + + // If requested, set as default retention policy. + if stmt.Default { + if err := e.MetaClient.SetDefaultRetentionPolicy(stmt.Database, stmt.Name); err != nil { + return err + } + } + + return nil +} + +func (e *QueryExecutor) executeCreateContinuousQueryStatement(q *influxql.CreateContinuousQueryStatement) error { + return e.MetaClient.CreateContinuousQuery(q.Database, q.Name, q.String()) +} + +func (e *QueryExecutor) executeCreateDatabaseStatement(stmt *influxql.CreateDatabaseStatement) error { + if !stmt.RetentionPolicyCreate { + _, err := e.MetaClient.CreateDatabase(stmt.Name) + return err + } + + rpi := meta.NewRetentionPolicyInfo(stmt.RetentionPolicyName) + rpi.Duration = stmt.RetentionPolicyDuration + rpi.ReplicaN = stmt.RetentionPolicyReplication + _, err := e.MetaClient.CreateDatabaseWithRetentionPolicy(stmt.Name, rpi) + return err +} + +func (e *QueryExecutor) executeCreateRetentionPolicyStatement(stmt *influxql.CreateRetentionPolicyStatement) error { + rpi := meta.NewRetentionPolicyInfo(stmt.Name) + rpi.Duration = stmt.Duration + rpi.ReplicaN = stmt.Replication + + // Create new retention policy. + if _, err := e.MetaClient.CreateRetentionPolicy(stmt.Database, rpi); err != nil { + return err + } + + // If requested, set new policy as the default. + if stmt.Default { + if err := e.MetaClient.SetDefaultRetentionPolicy(stmt.Database, stmt.Name); err != nil { + return err + } + } + return nil +} + +func (e *QueryExecutor) executeCreateSubscriptionStatement(q *influxql.CreateSubscriptionStatement) error { + return e.MetaClient.CreateSubscription(q.Database, q.RetentionPolicy, q.Name, q.Mode, q.Destinations) +} + +func (e *QueryExecutor) executeCreateUserStatement(q *influxql.CreateUserStatement) error { + _, err := e.MetaClient.CreateUser(q.Name, q.Password, q.Admin) + return err +} + +func (e *QueryExecutor) executeDropContinuousQueryStatement(q *influxql.DropContinuousQueryStatement) error { + return e.MetaClient.DropContinuousQuery(q.Database, q.Name) +} + +func (e *QueryExecutor) executeDropDatabaseStatement(stmt *influxql.DropDatabaseStatement) error { + dbi, err := e.MetaClient.Database(stmt.Name) + if err != nil { + return err + } else if dbi == nil { + if stmt.IfExists { + return nil + } + return influxql.ErrDatabaseNotFound(stmt.Name) + } + + // Remove database from meta-store first so that in-flight writes can + // complete without error, but new ones will be rejected. + if err := e.MetaClient.DropDatabase(stmt.Name); err != nil { + return err + } + + // Retrieve a list of all shard ids. + var shardIDs []uint64 + for _, rp := range dbi.RetentionPolicies { + for _, sg := range rp.ShardGroups { + for _, s := range sg.Shards { + shardIDs = append(shardIDs, s.ID) + } + } + } + + // Remove the database from the local store + if err := e.TSDBStore.DeleteDatabase(stmt.Name, shardIDs); err != nil { + return err + } + + return nil +} + +func (e *QueryExecutor) executeDropMeasurementStatement(stmt *influxql.DropMeasurementStatement, database string) error { + return e.TSDBStore.DeleteMeasurement(database, stmt.Name) +} + +func (e *QueryExecutor) executeDropSeriesStatement(stmt *influxql.DropSeriesStatement, database string) error { + // Check for time in WHERE clause (not supported). + if influxql.HasTimeExpr(stmt.Condition) { + return errors.New("DROP SERIES doesn't support time in WHERE clause") + } + + if err := e.TSDBStore.DeleteSeries(database, stmt.Sources, stmt.Condition); err != nil { + return err + } + return nil +} + +func (e *QueryExecutor) executeDropServerStatement(q *influxql.DropServerStatement) error { + if q.Meta { + return e.MetaClient.DeleteMetaNode(q.NodeID) + } + return e.MetaClient.DeleteDataNode(q.NodeID) +} + +func (e *QueryExecutor) executeDropRetentionPolicyStatement(q *influxql.DropRetentionPolicyStatement) error { + return e.MetaClient.DropRetentionPolicy(q.Database, q.Name) +} + +func (e *QueryExecutor) executeDropSubscriptionStatement(q *influxql.DropSubscriptionStatement) error { + return e.MetaClient.DropSubscription(q.Database, q.RetentionPolicy, q.Name) +} + +func (e *QueryExecutor) executeDropUserStatement(q *influxql.DropUserStatement) error { + return e.MetaClient.DropUser(q.Name) +} + +func (e *QueryExecutor) executeGrantStatement(stmt *influxql.GrantStatement) error { + return e.MetaClient.SetPrivilege(stmt.User, stmt.On, stmt.Privilege) +} + +func (e *QueryExecutor) executeGrantAdminStatement(stmt *influxql.GrantAdminStatement) error { + return e.MetaClient.SetAdminPrivilege(stmt.User, true) +} + +func (e *QueryExecutor) executeRevokeStatement(stmt *influxql.RevokeStatement) error { + priv := influxql.NoPrivileges + + // Revoking all privileges means there's no need to look at existing user privileges. + if stmt.Privilege != influxql.AllPrivileges { + p, err := e.MetaClient.UserPrivilege(stmt.User, stmt.On) + if err != nil { + return err + } + // Bit clear (AND NOT) the user's privilege with the revoked privilege. + priv = *p &^ stmt.Privilege + } + + return e.MetaClient.SetPrivilege(stmt.User, stmt.On, priv) +} + +func (e *QueryExecutor) executeRevokeAdminStatement(stmt *influxql.RevokeAdminStatement) error { + return e.MetaClient.SetAdminPrivilege(stmt.User, false) +} + +func (e *QueryExecutor) executeSetPasswordUserStatement(q *influxql.SetPasswordUserStatement) error { + return e.MetaClient.UpdateUser(q.Name, q.Password) +} + +func (e *QueryExecutor) executeSelectStatement(stmt *influxql.SelectStatement, chunkSize, statementID int, results chan *influxql.Result, closing <-chan struct{}) error { + // It is important to "stamp" this time so that everywhere we evaluate `now()` in the statement is EXACTLY the same `now` + now := time.Now().UTC() + opt := influxql.SelectOptions{} + + // Replace instances of "now()" with the current time, and check the resultant times. + stmt.Condition = influxql.Reduce(stmt.Condition, &influxql.NowValuer{Now: now}) + opt.MinTime, opt.MaxTime = influxql.TimeRange(stmt.Condition) + if opt.MaxTime.IsZero() { + opt.MaxTime = now + } + if opt.MinTime.IsZero() { + opt.MinTime = time.Unix(0, 0) + } + + // Expand regex sources to their actual source names. + sources, err := e.TSDBStore.ExpandSources(stmt.Sources) + if err != nil { + return err + } + stmt.Sources = sources + + // Convert DISTINCT into a call. + stmt.RewriteDistinct() + + // Remove "time" from fields list. + stmt.RewriteTimeFields() + + // Filter only shards that contain date range. + shardIDs, err := e.MetaClient.ShardIDsByTimeRange(stmt.Sources, opt.MinTime, opt.MaxTime) + if err != nil { + return err + } + shards := e.TSDBStore.Shards(shardIDs) + + // Rewrite wildcards, if any exist. + tmp, err := stmt.RewriteWildcards(tsdb.Shards(shards)) + if err != nil { + return err + } + stmt = tmp + + // Create a set of iterators from a selection. + itrs, err := influxql.Select(stmt, tsdb.Shards(shards), &opt) + if err != nil { + return err + } + + // Generate a row emitter from the iterator set. + em := influxql.NewEmitter(itrs, stmt.TimeAscending()) + em.Columns = stmt.ColumnNames() + em.OmitTime = stmt.OmitTime + defer em.Close() + + // Emit rows to the results channel. + var writeN int64 + var emitted bool + for { + row := em.Emit() + if row == nil { + break + } + + result := &influxql.Result{ + StatementID: statementID, + Series: []*models.Row{row}, + } + + // Write points back into system for INTO statements. + if stmt.Target != nil { + if err := e.writeInto(stmt, row); err != nil { + return err + } + writeN += int64(len(row.Values)) + continue + } + + // Send results or exit if closing. + select { + case <-closing: + return nil + case results <- result: + } + + emitted = true + } + + // Emit write count if an INTO statement. + if stmt.Target != nil { + results <- &influxql.Result{ + StatementID: statementID, + Series: []*models.Row{{ + Name: "result", + Columns: []string{"time", "written"}, + Values: [][]interface{}{{time.Unix(0, 0).UTC(), writeN}}, + }}, + } + return nil + } + + // Always emit at least one result. + if !emitted { + results <- &influxql.Result{ + StatementID: statementID, + Series: make([]*models.Row, 0), + } + } + + return nil +} + +func (e *QueryExecutor) executeShowContinuousQueriesStatement(stmt *influxql.ShowContinuousQueriesStatement) (models.Rows, error) { + dis, err := e.MetaClient.Databases() + if err != nil { + return nil, err + } + + rows := []*models.Row{} + for _, di := range dis { + row := &models.Row{Columns: []string{"name", "query"}, Name: di.Name} + for _, cqi := range di.ContinuousQueries { + row.Values = append(row.Values, []interface{}{cqi.Name, cqi.Query}) + } + rows = append(rows, row) + } + return rows, nil +} + +func (e *QueryExecutor) executeShowDatabasesStatement(q *influxql.ShowDatabasesStatement) (models.Rows, error) { + dis, err := e.MetaClient.Databases() + if err != nil { + return nil, err + } + + row := &models.Row{Name: "databases", Columns: []string{"name"}} + for _, di := range dis { + row.Values = append(row.Values, []interface{}{di.Name}) + } + return []*models.Row{row}, nil +} + +func (e *QueryExecutor) executeShowDiagnosticsStatement(stmt *influxql.ShowDiagnosticsStatement) (models.Rows, error) { + diags, err := e.Monitor.Diagnostics() + if err != nil { + return nil, err + } + + // Get a sorted list of diagnostics keys. + sortedKeys := make([]string, 0, len(diags)) + for k := range diags { + sortedKeys = append(sortedKeys, k) + } + sort.Strings(sortedKeys) + + rows := make([]*models.Row, 0, len(diags)) + for _, k := range sortedKeys { + if stmt.Module != "" && k != stmt.Module { + continue + } + + row := &models.Row{Name: k} + + row.Columns = diags[k].Columns + row.Values = diags[k].Rows + rows = append(rows, row) + } + return rows, nil +} + +func (e *QueryExecutor) executeShowFieldKeysStatement(stmt *influxql.ShowFieldKeysStatement, database string) (models.Rows, error) { + // FIXME(benbjohnson): Rewrite to use new query engine. + return e.TSDBStore.ExecuteShowFieldKeysStatement(stmt, database) +} + +func (e *QueryExecutor) executeShowGrantsForUserStatement(q *influxql.ShowGrantsForUserStatement) (models.Rows, error) { + priv, err := e.MetaClient.UserPrivileges(q.Name) + if err != nil { + return nil, err + } + + row := &models.Row{Columns: []string{"database", "privilege"}} + for d, p := range priv { + row.Values = append(row.Values, []interface{}{d, p.String()}) + } + return []*models.Row{row}, nil +} + +func (e *QueryExecutor) executeShowRetentionPoliciesStatement(q *influxql.ShowRetentionPoliciesStatement) (models.Rows, error) { + di, err := e.MetaClient.Database(q.Database) + if err != nil { + return nil, err + } else if di == nil { + return nil, influxdb.ErrDatabaseNotFound(q.Database) + } + + row := &models.Row{Columns: []string{"name", "duration", "replicaN", "default"}} + for _, rpi := range di.RetentionPolicies { + row.Values = append(row.Values, []interface{}{rpi.Name, rpi.Duration.String(), rpi.ReplicaN, di.DefaultRetentionPolicy == rpi.Name}) + } + return []*models.Row{row}, nil +} + +func (e *QueryExecutor) executeShowSeriesStatement(stmt *influxql.ShowSeriesStatement, database string) (models.Rows, error) { + return e.TSDBStore.ExecuteShowSeriesStatement(stmt, database) +} + +func (e *QueryExecutor) executeShowServersStatement(q *influxql.ShowServersStatement) (models.Rows, error) { + nis, err := e.MetaClient.DataNodes() + if err != nil { + return nil, err + } + + dataNodes := &models.Row{Columns: []string{"id", "http_addr", "tcp_addr"}} + dataNodes.Name = "data_nodes" + for _, ni := range nis { + dataNodes.Values = append(dataNodes.Values, []interface{}{ni.ID, ni.Host, ni.TCPHost}) + } + + nis, err = e.MetaClient.MetaNodes() + if err != nil { + return nil, err + } + + metaNodes := &models.Row{Columns: []string{"id", "http_addr", "tcp_addr"}} + metaNodes.Name = "meta_nodes" + for _, ni := range nis { + metaNodes.Values = append(metaNodes.Values, []interface{}{ni.ID, ni.Host, ni.TCPHost}) + } + + return []*models.Row{dataNodes, metaNodes}, nil +} + +func (e *QueryExecutor) executeShowShardsStatement(stmt *influxql.ShowShardsStatement) (models.Rows, error) { + dis, err := e.MetaClient.Databases() + if err != nil { + return nil, err + } + + rows := []*models.Row{} + for _, di := range dis { + row := &models.Row{Columns: []string{"id", "database", "retention_policy", "shard_group", "start_time", "end_time", "expiry_time", "owners"}, Name: di.Name} + for _, rpi := range di.RetentionPolicies { + for _, sgi := range rpi.ShardGroups { + // Shards associated with deleted shard groups are effectively deleted. + // Don't list them. + if sgi.Deleted() { + continue + } + + for _, si := range sgi.Shards { + ownerIDs := make([]uint64, len(si.Owners)) + for i, owner := range si.Owners { + ownerIDs[i] = owner.NodeID + } + + row.Values = append(row.Values, []interface{}{ + si.ID, + di.Name, + rpi.Name, + sgi.ID, + sgi.StartTime.UTC().Format(time.RFC3339), + sgi.EndTime.UTC().Format(time.RFC3339), + sgi.EndTime.Add(rpi.Duration).UTC().Format(time.RFC3339), + joinUint64(ownerIDs), + }) + } + } + } + rows = append(rows, row) + } + return rows, nil +} + +func (e *QueryExecutor) executeShowShardGroupsStatement(stmt *influxql.ShowShardGroupsStatement) (models.Rows, error) { + dis, err := e.MetaClient.Databases() + if err != nil { + return nil, err + } + + row := &models.Row{Columns: []string{"id", "database", "retention_policy", "start_time", "end_time", "expiry_time"}, Name: "shard groups"} + for _, di := range dis { + for _, rpi := range di.RetentionPolicies { + for _, sgi := range rpi.ShardGroups { + // Shards associated with deleted shard groups are effectively deleted. + // Don't list them. + if sgi.Deleted() { + continue + } + + row.Values = append(row.Values, []interface{}{ + sgi.ID, + di.Name, + rpi.Name, + sgi.StartTime.UTC().Format(time.RFC3339), + sgi.EndTime.UTC().Format(time.RFC3339), + sgi.EndTime.Add(rpi.Duration).UTC().Format(time.RFC3339), + }) + } + } + } + + return []*models.Row{row}, nil +} + +func (e *QueryExecutor) executeShowStatsStatement(stmt *influxql.ShowStatsStatement) (models.Rows, error) { + stats, err := e.Monitor.Statistics(nil) + if err != nil { + return nil, err + } + + var rows []*models.Row + for _, stat := range stats { + if stmt.Module != "" && stat.Name != stmt.Module { + continue + } + row := &models.Row{Name: stat.Name, Tags: stat.Tags} + + values := make([]interface{}, 0, len(stat.Values)) + for _, k := range stat.ValueNames() { + row.Columns = append(row.Columns, k) + values = append(values, stat.Values[k]) + } + row.Values = [][]interface{}{values} + rows = append(rows, row) + } + return rows, nil +} + +func (e *QueryExecutor) executeShowSubscriptionsStatement(stmt *influxql.ShowSubscriptionsStatement) (models.Rows, error) { + dis, err := e.MetaClient.Databases() + if err != nil { + return nil, err + } + + rows := []*models.Row{} + for _, di := range dis { + row := &models.Row{Columns: []string{"retention_policy", "name", "mode", "destinations"}, Name: di.Name} + for _, rpi := range di.RetentionPolicies { + for _, si := range rpi.Subscriptions { + row.Values = append(row.Values, []interface{}{rpi.Name, si.Name, si.Mode, si.Destinations}) + } + } + if len(row.Values) > 0 { + rows = append(rows, row) + } + } + return rows, nil +} + +func (e *QueryExecutor) executeShowTagValuesStatement(stmt *influxql.ShowTagValuesStatement, database string) (models.Rows, error) { + return e.TSDBStore.ExecuteShowTagValuesStatement(stmt, database) +} + +func (e *QueryExecutor) executeShowUsersStatement(q *influxql.ShowUsersStatement) (models.Rows, error) { + row := &models.Row{Columns: []string{"user", "admin"}} + for _, ui := range e.MetaClient.Users() { + row.Values = append(row.Values, []interface{}{ui.Name, ui.Admin}) + } + return []*models.Row{row}, nil +} + +func (e *QueryExecutor) logger() *log.Logger { + return log.New(e.LogOutput, "[query] ", log.LstdFlags) +} + +func (e *QueryExecutor) writeInto(stmt *influxql.SelectStatement, row *models.Row) error { + if stmt.Target.Measurement.Database == "" { + return errNoDatabaseInTarget + } + + // It might seem a bit weird that this is where we do this, since we will have to + // convert rows back to points. The Executors (both aggregate and raw) are complex + // enough that changing them to write back to the DB is going to be clumsy + // + // it might seem weird to have the write be in the QueryExecutor, but the interweaving of + // limitedRowWriter and ExecuteAggregate/Raw makes it ridiculously hard to make sure that the + // results will be the same as when queried normally. + name := stmt.Target.Measurement.Name + if name == "" { + name = row.Name + } + + points, err := convertRowToPoints(name, row) + if err != nil { + return err + } + + if err := e.PointsWriter.WritePointsInto(&IntoWriteRequest{ + Database: stmt.Target.Measurement.Database, + RetentionPolicy: stmt.Target.Measurement.RetentionPolicy, + Points: points, + }); err != nil { + return err + } + + return nil +} + +var errNoDatabaseInTarget = errors.New("no database in target") + +// convertRowToPoints will convert a query result Row into Points that can be written back in. +func convertRowToPoints(measurementName string, row *models.Row) ([]models.Point, error) { + // figure out which parts of the result are the time and which are the fields + timeIndex := -1 + fieldIndexes := make(map[string]int) + for i, c := range row.Columns { + if c == "time" { + timeIndex = i + } else { + fieldIndexes[c] = i + } + } + + if timeIndex == -1 { + return nil, errors.New("error finding time index in result") + } + + points := make([]models.Point, 0, len(row.Values)) + for _, v := range row.Values { + vals := make(map[string]interface{}) + for fieldName, fieldIndex := range fieldIndexes { + val := v[fieldIndex] + if val != nil { + vals[fieldName] = v[fieldIndex] + } + } + + p, err := models.NewPoint(measurementName, row.Tags, vals, v[timeIndex].(time.Time)) + if err != nil { + // Drop points that can't be stored + continue + } + + points = append(points, p) + } + + return points, nil +} + +// normalizeStatement adds a default database and policy to the measurements in statement. +func (e *QueryExecutor) normalizeStatement(stmt influxql.Statement, defaultDatabase string) (err error) { + influxql.WalkFunc(stmt, func(node influxql.Node) { + if err != nil { + return + } + switch node := node.(type) { + case *influxql.Measurement: + e := e.normalizeMeasurement(node, defaultDatabase) + if e != nil { + err = e + return + } + } + }) + return +} + +func (e *QueryExecutor) normalizeMeasurement(m *influxql.Measurement, defaultDatabase string) error { + // Targets (measurements in an INTO clause) can have blank names, which means it will be + // the same as the measurement name it came from in the FROM clause. + if !m.IsTarget && m.Name == "" && m.Regex == nil { + return errors.New("invalid measurement") + } + + // Measurement does not have an explicit database? Insert default. + if m.Database == "" { + m.Database = defaultDatabase + } + + // The database must now be specified by this point. + if m.Database == "" { + return errors.New("database name required") + } + + // Find database. + di, err := e.MetaClient.Database(m.Database) + if err != nil { + return err + } else if di == nil { + return influxdb.ErrDatabaseNotFound(m.Database) + } + + // If no retention policy was specified, use the default. + if m.RetentionPolicy == "" { + if di.DefaultRetentionPolicy == "" { + return fmt.Errorf("default retention policy not set for: %s", di.Name) + } + m.RetentionPolicy = di.DefaultRetentionPolicy + } + + return nil +} + +// IntoWriteRequest is a partial copy of cluster.WriteRequest +type IntoWriteRequest struct { + Database string + RetentionPolicy string + Points []models.Point +} + +// joinUint64 returns a comma-delimited string of uint64 numbers. +func joinUint64(a []uint64) string { + var buf bytes.Buffer + for i, x := range a { + buf.WriteString(strconv.FormatUint(x, 10)) + if i < len(a)-1 { + buf.WriteRune(',') + } + } + return buf.String() +} + +// stringSet represents a set of strings. +type stringSet map[string]struct{} + +// newStringSet returns an empty stringSet. +func newStringSet() stringSet { + return make(map[string]struct{}) +} + +// add adds strings to the set. +func (s stringSet) add(ss ...string) { + for _, n := range ss { + s[n] = struct{}{} + } +} + +// contains returns whether the set contains the given string. +func (s stringSet) contains(ss string) bool { + _, ok := s[ss] + return ok +} + +// list returns the current elements in the set, in sorted order. +func (s stringSet) list() []string { + l := make([]string, 0, len(s)) + for k := range s { + l = append(l, k) + } + sort.Strings(l) + return l +} + +// union returns the union of this set and another. +func (s stringSet) union(o stringSet) stringSet { + ns := newStringSet() + for k := range s { + ns[k] = struct{}{} + } + for k := range o { + ns[k] = struct{}{} + } + return ns +} + +// intersect returns the intersection of this set and another. +func (s stringSet) intersect(o stringSet) stringSet { + shorter, longer := s, o + if len(longer) < len(shorter) { + shorter, longer = longer, shorter + } + + ns := newStringSet() + for k := range shorter { + if _, ok := longer[k]; ok { + ns[k] = struct{}{} + } + } + return ns +} diff --git a/cmd/influxd/restore/restore.go b/cmd/influxd/restore/restore.go index b4e21ecdfc3..764d1bd526f 100644 --- a/cmd/influxd/restore/restore.go +++ b/cmd/influxd/restore/restore.go @@ -214,7 +214,9 @@ func (cmd *Command) unpackMeta() error { default: } - client := meta.NewClient([]string{store.HTTPAddr()}, false) + client := meta.NewClient() + client.SetMetaServers([]string{store.HTTPAddr()}) + client.SetTLS(false) client.SetLogger(log.New(ioutil.Discard, "", 0)) if err := client.Open(); err != nil { return err diff --git a/cmd/influxd/run/server.go b/cmd/influxd/run/server.go index 00643a4e1a6..f1359b4932c 100644 --- a/cmd/influxd/run/server.go +++ b/cmd/influxd/run/server.go @@ -14,6 +14,7 @@ import ( "github.com/influxdata/influxdb" "github.com/influxdata/influxdb/cluster" + "github.com/influxdata/influxdb/models" "github.com/influxdata/influxdb/monitor" "github.com/influxdata/influxdb/services/admin" "github.com/influxdata/influxdb/services/collectd" @@ -62,7 +63,7 @@ type Server struct { MetaService *meta.Service TSDBStore *tsdb.Store - QueryExecutor *tsdb.QueryExecutor + QueryExecutor *cluster.QueryExecutor PointsWriter *cluster.PointsWriter ShardWriter *cluster.ShardWriter IteratorCreator *cluster.IteratorCreator @@ -176,7 +177,8 @@ func NewServer(c *Config, buildInfo *BuildInfo) (*Server, error) { BindAddress: bind, - Node: node, + Node: node, + MetaClient: meta.NewClient(), Monitor: monitor.New(c.Monitor), @@ -205,12 +207,6 @@ func NewServer(c *Config, buildInfo *BuildInfo) (*Server, error) { s.TSDBStore.EngineOptions.WALFlushInterval = time.Duration(c.Data.WALFlushInterval) s.TSDBStore.EngineOptions.WALPartitionFlushDelay = time.Duration(c.Data.WALPartitionFlushDelay) - // Initialize query executor. - s.QueryExecutor = tsdb.NewQueryExecutor() - s.QueryExecutor.Store = s.TSDBStore - s.QueryExecutor.MonitorStatementExecutor = &monitor.StatementExecutor{Monitor: s.Monitor} - s.QueryExecutor.QueryLogEnabled = c.Data.QueryLogEnabled - // Set the shard writer s.ShardWriter = cluster.NewShardWriter(time.Duration(c.Cluster.ShardWriterTimeout), c.Cluster.MaxRemoteWriteConnections) @@ -231,15 +227,22 @@ func NewServer(c *Config, buildInfo *BuildInfo) (*Server, error) { s.PointsWriter.Subscriber = s.Subscriber s.PointsWriter.Node = s.Node - // needed for executing INTO queries. - s.QueryExecutor.IntoWriter = s.PointsWriter + // Initialize query executor. + s.QueryExecutor = cluster.NewQueryExecutor() + s.QueryExecutor.MetaClient = s.MetaClient + s.QueryExecutor.TSDBStore = s.TSDBStore + s.QueryExecutor.Monitor = s.Monitor + s.QueryExecutor.PointsWriter = s.PointsWriter + if c.Data.QueryLogEnabled { + s.QueryExecutor.LogOutput = os.Stderr + } // Initialize the monitor s.Monitor.Version = s.buildInfo.Version s.Monitor.Commit = s.buildInfo.Commit s.Monitor.Branch = s.buildInfo.Branch s.Monitor.BuildTime = s.buildInfo.Time - s.Monitor.PointsWriter = s.PointsWriter + s.Monitor.PointsWriter = (*monitorPointsWriter)(s.PointsWriter) } return s, nil @@ -438,7 +441,6 @@ func (s *Server) Open() error { } s.Subscriber.MetaClient = s.MetaClient - s.QueryExecutor.MetaClient = s.MetaClient s.ShardWriter.MetaClient = s.MetaClient s.HintedHandoff.MetaClient = s.MetaClient s.Subscriber.MetaClient = s.MetaClient @@ -631,10 +633,12 @@ func (s *Server) initializeMetaClient() error { if s.MetaService == nil { return fmt.Errorf("server not set to join existing cluster must run also as a meta node") } - s.MetaClient = meta.NewClient([]string{s.MetaService.HTTPAddr()}, s.metaUseTLS) + s.MetaClient.SetMetaServers([]string{s.MetaService.HTTPAddr()}) + s.MetaClient.SetTLS(s.metaUseTLS) } else { // join this node to the cluster - s.MetaClient = meta.NewClient(s.joinPeers, s.metaUseTLS) + s.MetaClient.SetMetaServers(s.joinPeers) + s.MetaClient.SetTLS(s.metaUseTLS) } if err := s.MetaClient.Open(); err != nil { return err @@ -720,3 +724,16 @@ type tcpaddr struct{ host string } func (a *tcpaddr) Network() string { return "tcp" } func (a *tcpaddr) String() string { return a.host } + +// monitorPointsWriter is a wrapper around `cluster.PointsWriter` that helps +// to prevent a circular dependency between the `cluster` and `monitor` packages. +type monitorPointsWriter cluster.PointsWriter + +func (pw *monitorPointsWriter) WritePoints(database, retentionPolicy string, points models.Points) error { + return (*cluster.PointsWriter)(pw).WritePoints(&cluster.WritePointsRequest{ + Database: database, + RetentionPolicy: retentionPolicy, + ConsistencyLevel: cluster.ConsistencyLevelOne, + Points: points, + }) +} diff --git a/cmd/influxd/run/server_helpers_test.go b/cmd/influxd/run/server_helpers_test.go index b5eda187de5..a5bee471d2d 100644 --- a/cmd/influxd/run/server_helpers_test.go +++ b/cmd/influxd/run/server_helpers_test.go @@ -501,7 +501,7 @@ func configureLogging(s *Server) { s.TSDBStore.Logger = nullLogger s.HintedHandoff.SetLogger(nullLogger) s.Monitor.SetLogger(nullLogger) - s.QueryExecutor.SetLogger(nullLogger) + s.QueryExecutor.LogOutput = ioutil.Discard s.Subscriber.SetLogger(nullLogger) for _, service := range s.Services { if service, ok := service.(logSetter); ok { diff --git a/influxql/query_executor.go b/influxql/query_executor.go new file mode 100644 index 00000000000..197c4576c33 --- /dev/null +++ b/influxql/query_executor.go @@ -0,0 +1,26 @@ +package influxql + +import ( + "errors" + "fmt" +) + +// QueryExecutor executes every statement in an Query. +type QueryExecutor interface { + ExecuteQuery(query *Query, database string, chunkSize int, closing chan struct{}) <-chan *Result +} + +var ( + // ErrInvalidQuery is returned when executing an unknown query type. + ErrInvalidQuery = errors.New("invalid query") + + // ErrNotExecuted is returned when a statement is not executed in a query. + // This can occur when a previous statement in the same query has errored. + ErrNotExecuted = errors.New("not executed") +) + +// ErrDatabaseNotFound returns a database not found error for the given database name. +func ErrDatabaseNotFound(name string) error { return fmt.Errorf("database not found: %s", name) } + +// ErrMeasurementNotFound returns a measurement not found error for the given measurement name. +func ErrMeasurementNotFound(name string) error { return fmt.Errorf("measurement not found: %s", name) } diff --git a/influxql/statement_rewriter.go b/influxql/statement_rewriter.go new file mode 100644 index 00000000000..289f10892f6 --- /dev/null +++ b/influxql/statement_rewriter.go @@ -0,0 +1,113 @@ +package influxql + +import ( + "errors" +) + +// RewriteStatement rewrites stmt into a new statement, if applicable. +func RewriteStatement(stmt Statement) (Statement, error) { + switch stmt := stmt.(type) { + case *ShowMeasurementsStatement: + return rewriteShowMeasurementsStatement(stmt) + case *ShowTagKeysStatement: + return rewriteShowTagKeysStatement(stmt) + default: + return stmt, nil + } +} + +func rewriteShowMeasurementsStatement(stmt *ShowMeasurementsStatement) (Statement, error) { + // Check for time in WHERE clause (not supported). + if HasTimeExpr(stmt.Condition) { + return nil, errors.New("SHOW MEASUREMENTS doesn't support time in WHERE clause") + } + + condition := stmt.Condition + if source, ok := stmt.Source.(*Measurement); ok { + var expr Expr + if source.Regex != nil { + expr = &BinaryExpr{ + Op: EQREGEX, + LHS: &VarRef{Val: "name"}, + RHS: &RegexLiteral{Val: source.Regex.Val}, + } + } else if source.Name != "" { + expr = &BinaryExpr{ + Op: EQ, + LHS: &VarRef{Val: "name"}, + RHS: &StringLiteral{Val: source.Name}, + } + } + + // Set condition or "AND" together. + if condition == nil { + condition = expr + } else { + condition = &BinaryExpr{Op: AND, LHS: expr, RHS: condition} + } + } + + return &SelectStatement{ + Fields: Fields([]*Field{ + {Expr: &VarRef{Val: "name"}}, + }), + Sources: Sources([]Source{ + &Measurement{Name: "_measurements"}, + }), + Condition: condition, + Offset: stmt.Offset, + Limit: stmt.Limit, + SortFields: stmt.SortFields, + OmitTime: true, + Dedupe: true, + }, nil +} + +func rewriteShowTagKeysStatement(stmt *ShowTagKeysStatement) (Statement, error) { + // Check for time in WHERE clause (not supported). + if HasTimeExpr(stmt.Condition) { + return nil, errors.New("SHOW TAG KEYS doesn't support time in WHERE clause") + } + + condition := stmt.Condition + if len(stmt.Sources) > 0 { + if source, ok := stmt.Sources[0].(*Measurement); ok { + var expr Expr + if source.Regex != nil { + expr = &BinaryExpr{ + Op: EQREGEX, + LHS: &VarRef{Val: "name"}, + RHS: &RegexLiteral{Val: source.Regex.Val}, + } + } else if source.Name != "" { + expr = &BinaryExpr{ + Op: EQ, + LHS: &VarRef{Val: "name"}, + RHS: &StringLiteral{Val: source.Name}, + } + } + + // Set condition or "AND" together. + if condition == nil { + condition = expr + } else { + condition = &BinaryExpr{Op: AND, LHS: expr, RHS: condition} + } + } + } + + return &SelectStatement{ + Fields: []*Field{ + {Expr: &VarRef{Val: "tagKey"}}, + }, + Sources: []Source{ + &Measurement{Name: "_tagKeys"}, + }, + Condition: condition, + Offset: stmt.Offset, + Limit: stmt.Limit, + SortFields: stmt.SortFields, + OmitTime: true, + Dedupe: true, + }, nil +} diff --git a/monitor/service.go b/monitor/service.go index 2b8f017f788..d9a13afc7a6 100644 --- a/monitor/service.go +++ b/monitor/service.go @@ -12,7 +12,6 @@ import ( "time" "github.com/influxdata/influxdb" - "github.com/influxdata/influxdb/cluster" "github.com/influxdata/influxdb/models" "github.com/influxdata/influxdb/monitor/diagnostics" "github.com/influxdata/influxdb/services/meta" @@ -59,8 +58,11 @@ type Monitor struct { NodeID uint64 + // Writer for pushing stats back into the database. + // This causes a circular dependency if it depends on cluster directly so it + // is wrapped in a simpler interface. PointsWriter interface { - WritePoints(p *cluster.WritePointsRequest) error + WritePoints(database, retentionPolicy string, points models.Points) error } Logger *log.Logger @@ -338,13 +340,7 @@ func (m *Monitor) storeStatistics() { points = append(points, pt) } - err = m.PointsWriter.WritePoints(&cluster.WritePointsRequest{ - Database: m.storeDatabase, - RetentionPolicy: m.storeRetentionPolicy, - ConsistencyLevel: cluster.ConsistencyLevelOne, - Points: points, - }) - if err != nil { + if err := m.PointsWriter.WritePoints(m.storeDatabase, m.storeRetentionPolicy, points); err != nil { m.Logger.Printf("failed to store statistics: %s", err) } case <-m.done: @@ -372,7 +368,7 @@ func newStatistic(name string, tags map[string]string, values map[string]interfa } // valueNames returns a sorted list of the value names, if any. -func (s *Statistic) valueNames() []string { +func (s *Statistic) ValueNames() []string { a := make([]string, 0, len(s.Values)) for k := range s.Values { a = append(a, k) diff --git a/monitor/service_test.go b/monitor/service_test.go deleted file mode 100644 index 8549e95745a..00000000000 --- a/monitor/service_test.go +++ /dev/null @@ -1,69 +0,0 @@ -package monitor - -import ( - "strings" - "testing" - - "github.com/influxdata/influxdb" - "github.com/influxdata/influxdb/influxql" - "github.com/influxdata/influxdb/services/meta" -) - -// Test that a registered stats client results in the correct SHOW STATS output. -func Test_RegisterStats(t *testing.T) { - monitor := openMonitor(t) - executor := &StatementExecutor{Monitor: monitor} - - // Register stats without tags. - statMap := influxdb.NewStatistics("foo", "foo", nil) - statMap.Add("bar", 1) - statMap.AddFloat("qux", 2.4) - json := executeShowStatsJSON(t, executor) - if !strings.Contains(json, `"columns":["bar","qux"],"values":[[1,2.4]]`) || !strings.Contains(json, `"name":"foo"`) { - t.Fatalf("SHOW STATS response incorrect, got: %s\n", json) - } - - // Register a client with tags. - statMap = influxdb.NewStatistics("bar", "baz", map[string]string{"proto": "tcp"}) - statMap.Add("bar", 1) - statMap.AddFloat("qux", 2.4) - json = executeShowStatsJSON(t, executor) - if !strings.Contains(json, `"columns":["bar","qux"],"values":[[1,2.4]]`) || - !strings.Contains(json, `"name":"baz"`) || - !strings.Contains(json, `"proto":"tcp"`) { - t.Fatalf("SHOW STATS response incorrect, got: %s\n", json) - - } -} - -type mockMetaClient struct{} - -func (m *mockMetaClient) ClusterID() uint64 { return 1 } -func (m *mockMetaClient) IsLeader() bool { return true } -func (m *mockMetaClient) SetDefaultRetentionPolicy(database, name string) error { return nil } -func (m *mockMetaClient) DropRetentionPolicy(database, name string) error { return nil } -func (m *mockMetaClient) CreateDatabase(name string) (*meta.DatabaseInfo, error) { - return nil, nil -} -func (m *mockMetaClient) CreateRetentionPolicy(database string, rpi *meta.RetentionPolicyInfo) (*meta.RetentionPolicyInfo, error) { - return nil, nil -} - -func openMonitor(t *testing.T) *Monitor { - monitor := New(NewConfig()) - monitor.MetaClient = &mockMetaClient{} - err := monitor.Open() - if err != nil { - t.Fatalf("failed to open monitor: %s", err.Error()) - } - return monitor -} - -func executeShowStatsJSON(t *testing.T, s *StatementExecutor) string { - r := s.ExecuteStatement(&influxql.ShowStatsStatement{}) - b, err := r.MarshalJSON() - if err != nil { - t.Fatalf("failed to decode SHOW STATS response: %s", err.Error()) - } - return string(b) -} diff --git a/monitor/statement_executor.go b/monitor/statement_executor.go deleted file mode 100644 index 901f33c6054..00000000000 --- a/monitor/statement_executor.go +++ /dev/null @@ -1,82 +0,0 @@ -package monitor - -import ( - "fmt" - "sort" - - "github.com/influxdata/influxdb/influxql" - "github.com/influxdata/influxdb/models" - "github.com/influxdata/influxdb/monitor/diagnostics" -) - -// StatementExecutor translates InfluxQL queries to Monitor methods. -type StatementExecutor struct { - Monitor interface { - Statistics(map[string]string) ([]*Statistic, error) - Diagnostics() (map[string]*diagnostics.Diagnostics, error) - } -} - -// ExecuteStatement executes monitor-related query statements. -func (s *StatementExecutor) ExecuteStatement(stmt influxql.Statement) *influxql.Result { - switch stmt := stmt.(type) { - case *influxql.ShowStatsStatement: - return s.executeShowStatistics(stmt.Module) - case *influxql.ShowDiagnosticsStatement: - return s.executeShowDiagnostics(stmt.Module) - default: - panic(fmt.Sprintf("unsupported statement type: %T", stmt)) - } -} - -func (s *StatementExecutor) executeShowStatistics(module string) *influxql.Result { - stats, err := s.Monitor.Statistics(nil) - if err != nil { - return &influxql.Result{Err: err} - } - - var rows []*models.Row - for _, stat := range stats { - if module != "" && stat.Name != module { - continue - } - row := &models.Row{Name: stat.Name, Tags: stat.Tags} - - values := make([]interface{}, 0, len(stat.Values)) - for _, k := range stat.valueNames() { - row.Columns = append(row.Columns, k) - values = append(values, stat.Values[k]) - } - row.Values = [][]interface{}{values} - rows = append(rows, row) - } - return &influxql.Result{Series: rows} -} - -func (s *StatementExecutor) executeShowDiagnostics(module string) *influxql.Result { - diags, err := s.Monitor.Diagnostics() - if err != nil { - return &influxql.Result{Err: err} - } - rows := make([]*models.Row, 0, len(diags)) - - // Get a sorted list of diagnostics keys. - sortedKeys := make([]string, 0, len(diags)) - for k := range diags { - sortedKeys = append(sortedKeys, k) - } - sort.Strings(sortedKeys) - - for _, k := range sortedKeys { - if module != "" && k != module { - continue - } - - row := &models.Row{Name: k} - - row.Columns = diags[k].Columns - row.Values = diags[k].Rows - rows = append(rows, row) - } - return &influxql.Result{Series: rows} -} diff --git a/services/continuous_querier/service.go b/services/continuous_querier/service.go index 7847c90c413..9368a648485 100644 --- a/services/continuous_querier/service.go +++ b/services/continuous_querier/service.go @@ -13,7 +13,6 @@ import ( "github.com/influxdata/influxdb" "github.com/influxdata/influxdb/influxql" "github.com/influxdata/influxdb/services/meta" - "github.com/influxdata/influxdb/tsdb" ) const ( @@ -36,11 +35,6 @@ type ContinuousQuerier interface { Run(database, name string, t time.Time) error } -// queryExecutor is an internal interface to make testing easier. -type queryExecutor interface { - ExecuteQuery(query *influxql.Query, database string, chunkSize int, closing chan struct{}) (<-chan *influxql.Result, error) -} - // metaClient is an internal interface to make testing easier. type metaClient interface { AcquireLease(name string) (l *meta.Lease, err error) @@ -74,7 +68,7 @@ func (rr *RunRequest) matches(cq *meta.ContinuousQueryInfo) bool { // Service manages continuous query execution. type Service struct { MetaClient metaClient - QueryExecutor queryExecutor + QueryExecutor influxql.QueryExecutor Config *Config RunInterval time.Duration // RunCh can be used by clients to signal service to run CQs. @@ -149,7 +143,7 @@ func (s *Service) Run(database, name string, t time.Time) error { if err != nil { return err } else if db == nil { - return tsdb.ErrDatabaseNotFound(database) + return influxql.ErrDatabaseNotFound(database) } dbs = append(dbs, *db) } else { @@ -351,10 +345,8 @@ func (s *Service) runContinuousQueryAndWriteResult(cq *ContinuousQuery) error { defer close(closing) // Execute the SELECT. - ch, err := s.QueryExecutor.ExecuteQuery(q, cq.Database, NoChunkingSize, closing) - if err != nil { - return err - } + ch := s.QueryExecutor.ExecuteQuery(q, cq.Database, NoChunkingSize, closing) + // There is only one statement, so we will only ever receive one result res, ok := <-ch if !ok { diff --git a/services/continuous_querier/service_test.go b/services/continuous_querier/service_test.go index 6131da8a43d..dd3adaa98ae 100644 --- a/services/continuous_querier/service_test.go +++ b/services/continuous_querier/service_test.go @@ -50,14 +50,14 @@ func TestContinuousQueryService_Run(t *testing.T) { // Set a callback for ExecuteQuery. qe := s.QueryExecutor.(*QueryExecutor) - qe.ExecuteQueryFn = func(query *influxql.Query, database string, chunkSize int, closing chan struct{}) (<-chan *influxql.Result, error) { + qe.ExecuteQueryFn = func(query *influxql.Query, database string, chunkSize int, closing chan struct{}) <-chan *influxql.Result { callCnt++ if callCnt >= expectCallCnt { done <- struct{}{} } dummych := make(chan *influxql.Result, 1) dummych <- &influxql.Result{} - return dummych, nil + return dummych } // Use a custom "now" time since the internals of last run care about @@ -123,14 +123,14 @@ func TestContinuousQueryService_ResampleOptions(t *testing.T) { // Set a callback for ExecuteQuery. qe := s.QueryExecutor.(*QueryExecutor) - qe.ExecuteQueryFn = func(query *influxql.Query, database string, chunkSize int, closing chan struct{}) (<-chan *influxql.Result, error) { + qe.ExecuteQueryFn = func(query *influxql.Query, database string, chunkSize int, closing chan struct{}) <-chan *influxql.Result { callCnt++ if callCnt >= expectCallCnt { done <- struct{}{} } dummych := make(chan *influxql.Result, 1) dummych <- &influxql.Result{} - return dummych, nil + return dummych } s.Open() @@ -187,14 +187,14 @@ func TestContinuousQueryService_EveryHigherThanInterval(t *testing.T) { // Set a callback for ExecuteQuery. qe := s.QueryExecutor.(*QueryExecutor) - qe.ExecuteQueryFn = func(query *influxql.Query, database string, chunkSize int, closing chan struct{}) (<-chan *influxql.Result, error) { + qe.ExecuteQueryFn = func(query *influxql.Query, database string, chunkSize int, closing chan struct{}) <-chan *influxql.Result { callCnt++ if callCnt >= expectCallCnt { done <- struct{}{} } dummych := make(chan *influxql.Result, 1) dummych <- &influxql.Result{} - return dummych, nil + return dummych } s.Open() @@ -242,9 +242,11 @@ func TestContinuousQueryService_NotLeader(t *testing.T) { done := make(chan struct{}) qe := s.QueryExecutor.(*QueryExecutor) // Set a callback for ExecuteQuery. Shouldn't get called because we're not the leader. - qe.ExecuteQueryFn = func(query *influxql.Query, database string, chunkSize int, closing chan struct{}) (<-chan *influxql.Result, error) { + qe.ExecuteQueryFn = func(query *influxql.Query, database string, chunkSize int, closing chan struct{}) <-chan *influxql.Result { done <- struct{}{} - return nil, errUnexpected + dummych := make(chan *influxql.Result, 1) + dummych <- &influxql.Result{Err: errUnexpected} + return dummych } s.Open() @@ -267,9 +269,11 @@ func TestContinuousQueryService_MetaClientFailsToGetDatabases(t *testing.T) { done := make(chan struct{}) qe := s.QueryExecutor.(*QueryExecutor) // Set ExecuteQuery callback, which shouldn't get called because of meta store failure. - qe.ExecuteQueryFn = func(query *influxql.Query, database string, chunkSize int, closing chan struct{}) (<-chan *influxql.Result, error) { + qe.ExecuteQueryFn = func(query *influxql.Query, database string, chunkSize int, closing chan struct{}) <-chan *influxql.Result { done <- struct{}{} - return nil, errUnexpected + dummych := make(chan *influxql.Result, 1) + dummych <- &influxql.Result{Err: errUnexpected} + return dummych } s.Open() @@ -469,7 +473,7 @@ func (ms *MetaClient) CreateContinuousQuery(database, name, query string) error // QueryExecutor is a mock query executor. type QueryExecutor struct { - ExecuteQueryFn func(query *influxql.Query, database string, chunkSize int, closing chan struct{}) (<-chan *influxql.Result, error) + ExecuteQueryFn func(query *influxql.Query, database string, chunkSize int, closing chan struct{}) <-chan *influxql.Result Results []*influxql.Result ResultInterval time.Duration Err error @@ -486,22 +490,21 @@ func NewQueryExecutor(t *testing.T) *QueryExecutor { } // ExecuteQuery returns a channel that the caller can read query results from. -func (qe *QueryExecutor) ExecuteQuery(query *influxql.Query, database string, chunkSize int, closing chan struct{}) (<-chan *influxql.Result, error) { - +func (qe *QueryExecutor) ExecuteQuery(query *influxql.Query, database string, chunkSize int, closing chan struct{}) <-chan *influxql.Result { // If the test set a callback, call it. if qe.ExecuteQueryFn != nil { - if _, err := qe.ExecuteQueryFn(query, database, chunkSize, make(chan struct{})); err != nil { - return nil, err - } + return qe.ExecuteQueryFn(query, database, chunkSize, make(chan struct{})) } + ch := make(chan *influxql.Result, 1) + // Are we supposed to error immediately? if qe.ErrAfterResult == -1 && qe.Err != nil { - return nil, qe.Err + ch <- &influxql.Result{Err: qe.Err} + close(ch) + return ch } - ch := make(chan *influxql.Result) - // Start a go routine to send results and / or error. go func() { n := 0 @@ -523,7 +526,7 @@ func (qe *QueryExecutor) ExecuteQuery(query *influxql.Query, database string, ch close(ch) }() - return ch, nil + return ch } // PointsWriter is a mock points writer. diff --git a/services/httpd/handler.go b/services/httpd/handler.go index 90b93614757..d173e0443c5 100644 --- a/services/httpd/handler.go +++ b/services/httpd/handler.go @@ -64,10 +64,8 @@ type Handler struct { Ping(checkAllMetaServers bool) error } - QueryExecutor interface { - Authorize(u *meta.UserInfo, q *influxql.Query, db string) error - ExecuteQuery(q *influxql.Query, db string, chunkSize int, closing chan struct{}) (<-chan *influxql.Result, error) - } + QueryAuthorizer *meta.QueryAuthorizer + QueryExecutor influxql.QueryExecutor PointsWriter interface { WritePoints(p *cluster.WritePointsRequest) error @@ -264,8 +262,10 @@ func (h *Handler) serveQuery(w http.ResponseWriter, r *http.Request, user *meta. // Check authorization. if h.requireAuthentication { - err = h.QueryExecutor.Authorize(user, query, db) - if err != nil { + if err := h.QueryAuthorizer.AuthorizeQuery(user, query, db); err != nil { + if err, ok := err.(meta.ErrAuthorize); ok { + h.Logger.Printf("unauthorized request | user: %q | query: %q | database %q\n", err.User, err.Query.String(), err.Database) + } httpError(w, "error authorizing query: "+err.Error(), pretty, http.StatusUnauthorized) return } @@ -292,12 +292,7 @@ func (h *Handler) serveQuery(w http.ResponseWriter, r *http.Request, user *meta. // Execute query. w.Header().Add("content-type", "application/json") - results, err := h.QueryExecutor.ExecuteQuery(query, db, chunkSize, closing) - - if err != nil { - w.WriteHeader(http.StatusInternalServerError) - return - } + results := h.QueryExecutor.ExecuteQuery(query, db, chunkSize, closing) // if we're not chunking, this will be the in memory buffer for all results before sending to client resp := Response{Results: make([]*influxql.Result, 0)} diff --git a/services/httpd/handler_test.go b/services/httpd/handler_test.go index 12813be5aee..12878544859 100644 --- a/services/httpd/handler_test.go +++ b/services/httpd/handler_test.go @@ -135,7 +135,7 @@ func TestBatchWrite_UnmarshalRFC(t *testing.T) { // Ensure the handler returns results from a query (including nil results). func TestHandler_Query(t *testing.T) { h := NewHandler(false) - h.QueryExecutor.ExecuteQueryFn = func(q *influxql.Query, db string, chunkSize int, closing chan struct{}) (<-chan *influxql.Result, error) { + h.QueryExecutor.ExecuteQueryFn = func(q *influxql.Query, db string, chunkSize int, closing chan struct{}) <-chan *influxql.Result { if q.String() != `SELECT * FROM bar` { t.Fatalf("unexpected query: %s", q.String()) } else if db != `foo` { @@ -145,7 +145,7 @@ func TestHandler_Query(t *testing.T) { &influxql.Result{StatementID: 1, Series: models.Rows([]*models.Row{{Name: "series0"}})}, &influxql.Result{StatementID: 2, Series: models.Rows([]*models.Row{{Name: "series1"}})}, nil, - ), nil + ) } w := httptest.NewRecorder() @@ -160,15 +160,13 @@ func TestHandler_Query(t *testing.T) { // Ensure the handler returns results from a query (including nil results). func TestHandler_QueryRegex(t *testing.T) { h := NewHandler(false) - h.QueryExecutor.ExecuteQueryFn = func(q *influxql.Query, db string, chunkSize int, closing chan struct{}) (<-chan *influxql.Result, error) { + h.QueryExecutor.ExecuteQueryFn = func(q *influxql.Query, db string, chunkSize int, closing chan struct{}) <-chan *influxql.Result { if q.String() != `SELECT * FROM test WHERE url =~ /http\:\/\/www.akamai\.com/` { t.Fatalf("unexpected query: %s", q.String()) } else if db != `test` { t.Fatalf("unexpected db: %s", db) } - return NewResultChan( - nil, - ), nil + return NewResultChan(nil) } w := httptest.NewRecorder() @@ -178,11 +176,11 @@ func TestHandler_QueryRegex(t *testing.T) { // Ensure the handler merges results from the same statement. func TestHandler_Query_MergeResults(t *testing.T) { h := NewHandler(false) - h.QueryExecutor.ExecuteQueryFn = func(q *influxql.Query, db string, chunkSize int, closing chan struct{}) (<-chan *influxql.Result, error) { + h.QueryExecutor.ExecuteQueryFn = func(q *influxql.Query, db string, chunkSize int, closing chan struct{}) <-chan *influxql.Result { return NewResultChan( &influxql.Result{StatementID: 1, Series: models.Rows([]*models.Row{{Name: "series0"}})}, &influxql.Result{StatementID: 1, Series: models.Rows([]*models.Row{{Name: "series1"}})}, - ), nil + ) } w := httptest.NewRecorder() @@ -197,11 +195,11 @@ func TestHandler_Query_MergeResults(t *testing.T) { // Ensure the handler merges results from the same statement. func TestHandler_Query_MergeEmptyResults(t *testing.T) { h := NewHandler(false) - h.QueryExecutor.ExecuteQueryFn = func(q *influxql.Query, db string, chunkSize int, closing chan struct{}) (<-chan *influxql.Result, error) { + h.QueryExecutor.ExecuteQueryFn = func(q *influxql.Query, db string, chunkSize int, closing chan struct{}) <-chan *influxql.Result { return NewResultChan( &influxql.Result{StatementID: 1, Series: models.Rows{}}, &influxql.Result{StatementID: 1, Series: models.Rows([]*models.Row{{Name: "series1"}})}, - ), nil + ) } w := httptest.NewRecorder() @@ -216,14 +214,14 @@ func TestHandler_Query_MergeEmptyResults(t *testing.T) { // Ensure the handler can parse chunked and chunk size query parameters. func TestHandler_Query_Chunked(t *testing.T) { h := NewHandler(false) - h.QueryExecutor.ExecuteQueryFn = func(q *influxql.Query, db string, chunkSize int, closing chan struct{}) (<-chan *influxql.Result, error) { + h.QueryExecutor.ExecuteQueryFn = func(q *influxql.Query, db string, chunkSize int, closing chan struct{}) <-chan *influxql.Result { if chunkSize != 2 { t.Fatalf("unexpected chunk size: %d", chunkSize) } return NewResultChan( &influxql.Result{StatementID: 1, Series: models.Rows([]*models.Row{{Name: "series0"}})}, &influxql.Result{StatementID: 1, Series: models.Rows([]*models.Row{{Name: "series1"}})}, - ), nil + ) } w := httptest.NewRecorder() @@ -273,25 +271,11 @@ func TestHandler_Query_ErrInvalidQuery(t *testing.T) { // } // } -// Ensure the handler returns a status 500 if an error is returned from the query executor. -func TestHandler_Query_ErrExecuteQuery(t *testing.T) { - h := NewHandler(false) - h.QueryExecutor.ExecuteQueryFn = func(q *influxql.Query, db string, chunkSize int, closing chan struct{}) (<-chan *influxql.Result, error) { - return nil, errors.New("marker") - } - - w := httptest.NewRecorder() - h.ServeHTTP(w, MustNewJSONRequest("GET", "/query?db=foo&q=SHOW+SERIES+FROM+bar", nil)) - if w.Code != http.StatusInternalServerError { - t.Fatalf("unexpected status: %d", w.Code) - } -} - // Ensure the handler returns a status 200 if an error is returned in the result. func TestHandler_Query_ErrResult(t *testing.T) { h := NewHandler(false) - h.QueryExecutor.ExecuteQueryFn = func(q *influxql.Query, db string, chunkSize int, closing chan struct{}) (<-chan *influxql.Result, error) { - return NewResultChan(&influxql.Result{Err: errors.New("measurement not found")}), nil + h.QueryExecutor.ExecuteQueryFn = func(q *influxql.Query, db string, chunkSize int, closing chan struct{}) <-chan *influxql.Result { + return NewResultChan(&influxql.Result{Err: errors.New("measurement not found")}) } w := httptest.NewRecorder() @@ -522,14 +506,14 @@ func (s *HandlerMetaStore) Users() []meta.UserInfo { // HandlerQueryExecutor is a mock implementation of Handler.QueryExecutor. type HandlerQueryExecutor struct { AuthorizeFn func(u *meta.UserInfo, q *influxql.Query, db string) error - ExecuteQueryFn func(q *influxql.Query, db string, chunkSize int, closing chan struct{}) (<-chan *influxql.Result, error) + ExecuteQueryFn func(q *influxql.Query, db string, chunkSize int, closing chan struct{}) <-chan *influxql.Result } func (e *HandlerQueryExecutor) Authorize(u *meta.UserInfo, q *influxql.Query, db string) error { return e.AuthorizeFn(u, q, db) } -func (e *HandlerQueryExecutor) ExecuteQuery(q *influxql.Query, db string, chunkSize int, closing chan struct{}) (<-chan *influxql.Result, error) { +func (e *HandlerQueryExecutor) ExecuteQuery(q *influxql.Query, db string, chunkSize int, closing chan struct{}) <-chan *influxql.Result { return e.ExecuteQueryFn(q, db, chunkSize, closing) } diff --git a/services/meta/client.go b/services/meta/client.go index 7f06426a36e..d87a6cc0e82 100644 --- a/services/meta/client.go +++ b/services/meta/client.go @@ -60,8 +60,6 @@ type Client struct { closing chan struct{} cacheData *Data - executor *StatementExecutor - // Authentication cache. authCache map[string]authUser } @@ -73,16 +71,12 @@ type authUser struct { } // NewClient returns a new *Client. -func NewClient(metaServers []string, tls bool) *Client { - client := &Client{ - cacheData: &Data{}, - metaServers: metaServers, - tls: tls, - logger: log.New(os.Stderr, "[metaclient] ", log.LstdFlags), - authCache: make(map[string]authUser, 0), +func NewClient() *Client { + return &Client{ + cacheData: &Data{}, + logger: log.New(os.Stderr, "[metaclient] ", log.LstdFlags), + authCache: make(map[string]authUser, 0), } - client.executor = &StatementExecutor{Store: client} - return client } // Open a connection to a meta service cluster. @@ -118,6 +112,17 @@ func (c *Client) Close() error { // GetNodeID returns the client's node ID. func (c *Client) NodeID() uint64 { return c.nodeID } +// SetMetaServers updates the meta servers on the client. +func (c *Client) SetMetaServers(a []string) { + c.mu.Lock() + defer c.mu.Unlock() + c.metaServers = a +} + +// SetTLS sets whether the client should use TLS when connecting. +// This function is not safe for concurrent use. +func (c *Client) SetTLS(v bool) { c.tls = v } + // Ping will hit the ping endpoint for the metaservice and return nil if // it returns 200. If checkAllMetaServers is set to true, it will hit the // ping endpoint and tell it to verify the health of all metaservers in the @@ -923,10 +928,6 @@ func (c *Client) SetData(data *Data) error { ) } -func (c *Client) ExecuteStatement(stmt influxql.Statement) *influxql.Result { - return c.executor.ExecuteStatement(stmt) -} - // WaitForDataChanged will return a channel that will get closed when // the metastore data has changed func (c *Client) WaitForDataChanged() chan struct{} { diff --git a/services/meta/query_authorizer.go b/services/meta/query_authorizer.go new file mode 100644 index 00000000000..6b1bc5c9836 --- /dev/null +++ b/services/meta/query_authorizer.go @@ -0,0 +1,101 @@ +package meta + +import ( + "fmt" + + "github.com/influxdata/influxdb/influxql" +) + +type QueryAuthorizer struct { + Client *Client +} + +// AuthorizeQuery authorizes u to execute q on database. +// Database can be "" for queries that do not require a database. +// If no user is provided it will return an error unless the query's first statement is to create +// a root user. +func (a *QueryAuthorizer) AuthorizeQuery(u *UserInfo, query *influxql.Query, database string) error { + // Special case if no users exist. + if n := a.Client.UserCount(); n == 0 { + // Ensure there is at least one statement. + if len(query.Statements) > 0 { + // First statement in the query must create a user with admin privilege. + cu, ok := query.Statements[0].(*influxql.CreateUserStatement) + if ok && cu.Admin == true { + return nil + } + } + return &ErrAuthorize{ + Query: query, + Database: database, + Message: "create admin user first or disable authentication", + } + } + + if u == nil { + return &ErrAuthorize{ + Query: query, + Database: database, + Message: "no user provided", + } + } + + // Admin privilege allows the user to execute all statements. + if u.Admin { + return nil + } + + // Check each statement in the query. + for _, stmt := range query.Statements { + // Get the privileges required to execute the statement. + privs := stmt.RequiredPrivileges() + + // Make sure the user has the privileges required to execute + // each statement. + for _, p := range privs { + if p.Admin { + // Admin privilege already checked so statement requiring admin + // privilege cannot be run. + return &ErrAuthorize{ + Query: query, + User: u.Name, + Database: database, + Message: fmt.Sprintf("statement '%s', requires admin privilege", stmt), + } + } + + // Use the db name specified by the statement or the db + // name passed by the caller if one wasn't specified by + // the statement. + db := p.Name + if db == "" { + db = database + } + if !u.Authorize(p.Privilege, db) { + return &ErrAuthorize{ + Query: query, + User: u.Name, + Database: database, + Message: fmt.Sprintf("statement '%s', requires %s on %s", stmt, p.Privilege.String(), db), + } + } + } + } + return nil +} + +// ErrAuthorize represents an authorization error. +type ErrAuthorize struct { + Query *influxql.Query + User string + Database string + Message string +} + +// Error returns the text of the error. +func (e ErrAuthorize) Error() string { + if e.User == "" { + return fmt.Sprint(e.Message) + } + return fmt.Sprintf("%s not authorized to execute %s", e.User, e.Message) +} diff --git a/services/meta/service_test.go b/services/meta/service_test.go index 7f1892306f5..4bd5b10d327 100644 --- a/services/meta/service_test.go +++ b/services/meta/service_test.go @@ -30,8 +30,8 @@ func TestMetaService_CreateDatabase(t *testing.T) { defer s.Close() defer c.Close() - if res := c.ExecuteStatement(mustParseStatement("CREATE DATABASE db0")); res.Err != nil { - t.Fatal(res.Err) + if _, err := c.CreateDatabase("db0"); err != nil { + t.Fatal(err) } db, err := c.Database("db0") @@ -58,9 +58,8 @@ func TestMetaService_CreateDatabaseIfNotExists(t *testing.T) { defer s.Close() defer c.Close() - qry := `CREATE DATABASE IF NOT EXISTS db0` - if res := c.ExecuteStatement(mustParseStatement(qry)); res.Err != nil { - t.Fatal(res.Err) + if _, err := c.CreateDatabase("db0"); err != nil { + t.Fatal(err) } db, err := c.Database("db0") @@ -70,8 +69,8 @@ func TestMetaService_CreateDatabaseIfNotExists(t *testing.T) { t.Fatalf("db name wrong: %s", db.Name) } - if res := c.ExecuteStatement(mustParseStatement(qry)); res.Err != nil { - t.Fatal(res.Err) + if _, err := c.CreateDatabase("db0"); err != nil { + t.Fatal(err) } } @@ -83,9 +82,12 @@ func TestMetaService_CreateDatabaseWithRetentionPolicy(t *testing.T) { defer s.Close() defer c.Close() - qry := `CREATE DATABASE db0 WITH DURATION 1h REPLICATION 1 NAME rp0` - if res := c.ExecuteStatement(mustParseStatement(qry)); res.Err != nil { - t.Fatal(res.Err) + if _, err := c.CreateDatabaseWithRetentionPolicy("db0", &meta.RetentionPolicyInfo{ + Name: "rp0", + Duration: 1 * time.Hour, + ReplicaN: 1, + }); err != nil { + t.Fatal(err) } db, err := c.Database("db0") @@ -151,9 +153,8 @@ func TestMetaService_DropDatabase(t *testing.T) { defer s.Close() defer c.Close() - qry := `CREATE DATABASE db0` - if res := c.ExecuteStatement(mustParseStatement(qry)); res.Err != nil { - t.Fatal(res.Err) + if _, err := c.CreateDatabase("db0"); err != nil { + t.Fatal(err) } db, err := c.Database("db0") @@ -163,9 +164,8 @@ func TestMetaService_DropDatabase(t *testing.T) { t.Fatalf("db name wrong: %s", db.Name) } - qry = `DROP DATABASE db0` - if res := c.ExecuteStatement(mustParseStatement(qry)); res.Err != nil { - t.Fatal(res.Err) + if err := c.DropDatabase("db0"); err != nil { + t.Fatal(err) } if db, _ = c.Database("db0"); db != nil { @@ -181,8 +181,8 @@ func TestMetaService_CreateRetentionPolicy(t *testing.T) { defer s.Close() defer c.Close() - if res := c.ExecuteStatement(mustParseStatement("CREATE DATABASE db0")); res.Err != nil { - t.Fatal(res.Err) + if _, err := c.CreateDatabase("db0"); err != nil { + t.Fatal(err) } db, err := c.Database("db0") @@ -192,9 +192,12 @@ func TestMetaService_CreateRetentionPolicy(t *testing.T) { t.Fatalf("db name wrong: %s", db.Name) } - qry := `CREATE RETENTION POLICY rp0 ON db0 DURATION 1h REPLICATION 1` - if res := c.ExecuteStatement(mustParseStatement(qry)); res.Err != nil { - t.Fatal(res.Err) + if _, err := c.CreateRetentionPolicy("db0", &meta.RetentionPolicyInfo{ + Name: "rp0", + Duration: 1 * time.Hour, + ReplicaN: 1, + }); err != nil { + t.Fatal(err) } rp, err := c.RetentionPolicy("db0", "rp0") @@ -209,8 +212,12 @@ func TestMetaService_CreateRetentionPolicy(t *testing.T) { } // Create the same policy. Should not error. - if res := c.ExecuteStatement(mustParseStatement(qry)); res.Err != nil { - t.Fatal(res.Err) + if _, err := c.CreateRetentionPolicy("db0", &meta.RetentionPolicyInfo{ + Name: "rp0", + Duration: 1 * time.Hour, + ReplicaN: 1, + }); err != nil { + t.Fatal(err) } rp, err = c.RetentionPolicy("db0", "rp0") @@ -233,9 +240,12 @@ func TestMetaService_SetDefaultRetentionPolicy(t *testing.T) { defer s.Close() defer c.Close() - qry := `CREATE DATABASE db0 WITH DURATION 1h REPLICATION 1 NAME rp0` - if res := c.ExecuteStatement(mustParseStatement(qry)); res.Err != nil { - t.Fatal(res.Err) + if _, err := c.CreateDatabaseWithRetentionPolicy("db0", &meta.RetentionPolicyInfo{ + Name: "rp0", + Duration: 1 * time.Hour, + ReplicaN: 1, + }); err != nil { + t.Fatal(err) } db, err := c.Database("db0") @@ -270,8 +280,8 @@ func TestMetaService_DropRetentionPolicy(t *testing.T) { defer s.Close() defer c.Close() - if res := c.ExecuteStatement(mustParseStatement("CREATE DATABASE db0")); res.Err != nil { - t.Fatal(res.Err) + if _, err := c.CreateDatabase("db0"); err != nil { + t.Fatal(err) } db, err := c.Database("db0") @@ -281,9 +291,12 @@ func TestMetaService_DropRetentionPolicy(t *testing.T) { t.Fatalf("db name wrong: %s", db.Name) } - qry := `CREATE RETENTION POLICY rp0 ON db0 DURATION 1h REPLICATION 1` - if res := c.ExecuteStatement(mustParseStatement(qry)); res.Err != nil { - t.Fatal(res.Err) + if _, err := c.CreateRetentionPolicy("db0", &meta.RetentionPolicyInfo{ + Name: "rp0", + Duration: 1 * time.Hour, + ReplicaN: 1, + }); err != nil { + t.Fatal(err) } rp, err := c.RetentionPolicy("db0", "rp0") @@ -297,9 +310,8 @@ func TestMetaService_DropRetentionPolicy(t *testing.T) { t.Fatalf("rp replication wrong: %d", rp.ReplicaN) } - qry = `DROP RETENTION POLICY rp0 ON db0` - if res := c.ExecuteStatement(mustParseStatement(qry)); res.Err != nil { - t.Fatal(res.Err) + if err := c.DropRetentionPolicy("db0", "rp0"); err != nil { + t.Fatal(err) } rp, err = c.RetentionPolicy("db0", "rp0") @@ -319,13 +331,13 @@ func TestMetaService_CreateUser(t *testing.T) { defer c.Close() // Create an admin user - if res := c.ExecuteStatement(mustParseStatement("CREATE USER fred WITH PASSWORD 'supersecure' WITH ALL PRIVILEGES")); res.Err != nil { - t.Fatal(res.Err) + if _, err := c.CreateUser("fred", "supersecure", true); err != nil { + t.Fatal(err) } // Create a non-admin user - if res := c.ExecuteStatement(mustParseStatement("CREATE USER wilma WITH PASSWORD 'password'")); res.Err != nil { - t.Fatal(res.Err) + if _, err := c.CreateUser("wilma", "password", false); err != nil { + t.Fatal(err) } u, err := c.User("fred") @@ -357,8 +369,8 @@ func TestMetaService_CreateUser(t *testing.T) { } // Change password should succeed. - if res := c.ExecuteStatement(mustParseStatement("SET PASSWORD FOR fred = 'moresupersecure'")); res.Err != nil { - t.Fatal(res.Err) + if err := c.UpdateUser("fred", "moresupersecure"); err != nil { + t.Fatal(err) } // Auth for old password should fail @@ -395,8 +407,8 @@ func TestMetaService_CreateUser(t *testing.T) { } // Grant privilidges to a non-admin user - if res := c.ExecuteStatement(mustParseStatement("GRANT ALL PRIVILEGES TO wilma")); res.Err != nil { - t.Fatal(res.Err) + if err := c.SetAdminPrivilege("wilma", true); err != nil { + t.Fatal(err) } u, err = c.User("wilma") @@ -411,8 +423,8 @@ func TestMetaService_CreateUser(t *testing.T) { } // Revoke privilidges from user - if res := c.ExecuteStatement(mustParseStatement("REVOKE ALL PRIVILEGES FROM wilma")); res.Err != nil { - t.Fatal(res.Err) + if err := c.SetAdminPrivilege("wilma", false); err != nil { + t.Fatal(err) } u, err = c.User("wilma") @@ -427,8 +439,8 @@ func TestMetaService_CreateUser(t *testing.T) { } // Create a database to use for assiging privileges to. - if res := c.ExecuteStatement(mustParseStatement("CREATE DATABASE db0")); res.Err != nil { - t.Fatal(res.Err) + if _, err := c.CreateDatabase("db0"); err != nil { + t.Fatal(err) } db, err := c.Database("db0") @@ -439,8 +451,8 @@ func TestMetaService_CreateUser(t *testing.T) { } // Assign a single privilege at the database level - if res := c.ExecuteStatement(mustParseStatement("GRANT READ ON db0 TO wilma")); res.Err != nil { - t.Fatal(res.Err) + if err := c.SetPrivilege("wilma", "db0", influxql.ReadPrivilege); err != nil { + t.Fatal(err) } p, err := c.UserPrivilege("wilma", "db0") @@ -455,8 +467,8 @@ func TestMetaService_CreateUser(t *testing.T) { } // Remove a single privilege at the database level - if res := c.ExecuteStatement(mustParseStatement("REVOKE READ ON db0 FROM wilma")); res.Err != nil { - t.Fatal(res.Err) + if err := c.SetPrivilege("wilma", "db0", influxql.NoPrivileges); err != nil { + t.Fatal(err) } p, err = c.UserPrivilege("wilma", "db0") if err != nil { @@ -470,8 +482,8 @@ func TestMetaService_CreateUser(t *testing.T) { } // Drop a user - if res := c.ExecuteStatement(mustParseStatement("DROP USER wilma")); res.Err != nil { - t.Fatal(res.Err) + if err := c.DropUser("wilma"); err != nil { + t.Fatal(err) } u, err = c.User("wilma") @@ -493,8 +505,8 @@ func TestMetaService_ContinuousQueries(t *testing.T) { defer c.Close() // Create a database to use - if res := c.ExecuteStatement(mustParseStatement("CREATE DATABASE db0")); res.Err != nil { - t.Fatal(res.Err) + if _, err := c.CreateDatabase("db0"); err != nil { + t.Fatal(err) } db, err := c.Database("db0") if err != nil { @@ -504,46 +516,26 @@ func TestMetaService_ContinuousQueries(t *testing.T) { } // Create a CQ - if res := c.ExecuteStatement(mustParseStatement("CREATE CONTINUOUS QUERY cq0 ON db0 BEGIN SELECT count(value) INTO foo_count FROM foo GROUP BY time(10m) END")); res.Err != nil { - t.Fatal(res.Err) - } - - res := c.ExecuteStatement(mustParseStatement("SHOW CONTINUOUS QUERIES")) - if res.Err != nil { - t.Fatal(res.Err) - } - exp := `{"series":[{"name":"db0","columns":["name","query"],"values":[["cq0","CREATE CONTINUOUS QUERY cq0 ON db0 BEGIN SELECT count(value) INTO foo_count FROM foo GROUP BY time(10m) END"]]}]}` - got := mustMarshalJSON(res) - if exp != got { - t.Fatalf("unexpected response.\n\nexp: %s\ngot: %s\n", exp, got) + if err := c.CreateContinuousQuery("db0", "cq0", `SELECT count(value) INTO foo_count FROM foo GROUP BY time(10m)`); err != nil { + t.Fatal(err) } // Recreate an existing CQ - if res := c.ExecuteStatement(mustParseStatement("CREATE CONTINUOUS QUERY cq0 ON db0 BEGIN SELECT max(value) INTO foo_max FROM foo GROUP BY time(10m) END")); res.Err == nil { - t.Fatalf("expected error: got %v", res.Err) + if err := c.CreateContinuousQuery("db0", "cq0", `SELECT max(value) INTO foo_max FROM foo GROUP BY time(10m)`); err == nil || err.Error() != `continuous query already exists` { + t.Fatalf("unexpected error: %s", err) } // Create a few more CQ's - if res := c.ExecuteStatement(mustParseStatement("CREATE CONTINUOUS QUERY cq1 ON db0 BEGIN SELECT max(value) INTO foo_max FROM foo GROUP BY time(10m) END")); res.Err != nil { - t.Fatal(res.Err) + if err := c.CreateContinuousQuery("db0", "cq1", `SELECT max(value) INTO foo_max FROM foo GROUP BY time(10m)`); err != nil { + t.Fatal(err) } - if res := c.ExecuteStatement(mustParseStatement("CREATE CONTINUOUS QUERY cq2 ON db0 BEGIN SELECT min(value) INTO foo_min FROM foo GROUP BY time(10m) END")); res.Err != nil { - t.Fatal(res.Err) + if err := c.CreateContinuousQuery("db0", "cq2", `SELECT min(value) INTO foo_min FROM foo GROUP BY time(10m)`); err != nil { + t.Fatal(err) } // Drop a single CQ - if res := c.ExecuteStatement(mustParseStatement("DROP CONTINUOUS QUERY cq1 ON db0")); res.Err != nil { - t.Fatal(res.Err) - } - - res = c.ExecuteStatement(mustParseStatement("SHOW CONTINUOUS QUERIES")) - if res.Err != nil { - t.Fatal(res.Err) - } - exp = `{"series":[{"name":"db0","columns":["name","query"],"values":[["cq0","CREATE CONTINUOUS QUERY cq0 ON db0 BEGIN SELECT count(value) INTO foo_count FROM foo GROUP BY time(10m) END"],["cq2","CREATE CONTINUOUS QUERY cq2 ON db0 BEGIN SELECT min(value) INTO foo_min FROM foo GROUP BY time(10m) END"]]}]}` - got = mustMarshalJSON(res) - if exp != got { - t.Fatalf("unexpected response.\n\nexp: %s\ngot: %s\n", exp, got) + if err := c.DropContinuousQuery("db0", "cq1"); err != nil { + t.Fatal(err) } } @@ -556,8 +548,8 @@ func TestMetaService_Subscriptions_Create(t *testing.T) { defer c.Close() // Create a database to use - if res := c.ExecuteStatement(mustParseStatement("CREATE DATABASE db0")); res.Err != nil { - t.Fatal(res.Err) + if _, err := c.CreateDatabase("db0"); err != nil { + t.Fatal(err) } db, err := c.Database("db0") if err != nil { @@ -567,82 +559,18 @@ func TestMetaService_Subscriptions_Create(t *testing.T) { } // Create a subscription - if res := c.ExecuteStatement(mustParseStatement(`CREATE SUBSCRIPTION sub0 ON db0."default" DESTINATIONS ALL 'udp://example.com:9090'`)); res.Err != nil { - t.Fatal(res.Err) + if err := c.CreateSubscription("db0", "default", "sub0", "ALL", []string{"udp://example.com:9090"}); err != nil { + t.Fatal(err) } // Re-create a subscription - if res := c.ExecuteStatement(mustParseStatement(`CREATE SUBSCRIPTION sub0 ON db0."default" DESTINATIONS ALL 'udp://example.com:9090'`)); res.Err == nil { - t.Fatal(res.Err) - } - - res := c.ExecuteStatement(mustParseStatement(`SHOW SUBSCRIPTIONS`)) - if res.Err != nil { - t.Fatal(res.Err) - } else if got, exp := len(res.Series), 1; got != exp { - t.Fatalf("unexpected response.\n\ngot: %d series\nexp: %d\n", got, exp) + if err := c.CreateSubscription("db0", "default", "sub0", "ALL", []string{"udp://example.com:9090"}); err == nil || err.Error() != `subscription already exists` { + t.Fatalf("unexpected error: %s", err) } // Create another subscription. - if res := c.ExecuteStatement(mustParseStatement(`CREATE SUBSCRIPTION sub1 ON db0."default" DESTINATIONS ALL 'udp://example.com:6060'`)); res.Err != nil { - t.Fatal(res.Err) - } - - // The subscriptions are correctly created. - if res = c.ExecuteStatement(mustParseStatement(`SHOW SUBSCRIPTIONS`)); res.Err != nil { - t.Fatal(res.Err) - } - - exp := `{"series":[{"name":"db0","columns":["retention_policy","name","mode","destinations"],"values":[["default","sub0","ALL",["udp://example.com:9090"]],["default","sub1","ALL",["udp://example.com:6060"]]]}]}` - got := mustMarshalJSON(res) - if got != exp { - t.Fatalf("unexpected response.\n\ngot: %s\nexp: %s\n", exp, got) - } -} - -func TestMetaService_Subscriptions_Show(t *testing.T) { - t.Parallel() - - d, s, c := newServiceAndClient() - defer os.RemoveAll(d) - defer s.Close() - defer c.Close() - - // Create a database to use - if res := c.ExecuteStatement(mustParseStatement("CREATE DATABASE db0")); res.Err != nil { - t.Fatal(res.Err) - } - db, err := c.Database("db0") - if err != nil { + if err := c.CreateSubscription("db0", "default", "sub1", "ALL", []string{"udp://example.com:6060"}); err != nil { t.Fatal(err) - } else if db.Name != "db0" { - t.Fatalf("db name wrong: %s", db.Name) - } - - // SHOW SUBSCRIPTIONS returns no subscriptions when there are none. - res := c.ExecuteStatement(mustParseStatement(`SHOW SUBSCRIPTIONS`)) - if res.Err != nil { - t.Fatal(res.Err) - } else if got, exp := len(res.Series), 0; got != exp { - t.Fatalf("got %d series, expected %d", got, exp) - } - - // Create a subscription. - if res = c.ExecuteStatement(mustParseStatement(`CREATE SUBSCRIPTION sub0 ON db0."default" DESTINATIONS ALL 'udp://example.com:9090'`)); res.Err != nil { - t.Fatal(res.Err) - } - - // SHOW SUBSCRIPTIONS returns the created subscription. - if res = c.ExecuteStatement(mustParseStatement(`SHOW SUBSCRIPTIONS`)); res.Err != nil { - t.Fatal(res.Err) - } else if got, exp := len(res.Series), 1; got != exp { - t.Fatalf("got %d series, expected %d", got, exp) - } - - exp := `{"series":[{"name":"db0","columns":["retention_policy","name","mode","destinations"],"values":[["default","sub0","ALL",["udp://example.com:9090"]]]}]}` - got := mustMarshalJSON(res) - if got != exp { - t.Fatalf("unexpected response.\n\ngot: %s\nexp: %s\n", got, exp) } } @@ -655,53 +583,41 @@ func TestMetaService_Subscriptions_Drop(t *testing.T) { defer c.Close() // Create a database to use - if res := c.ExecuteStatement(mustParseStatement("CREATE DATABASE db0")); res.Err != nil { - t.Fatal(res.Err) - } - db, err := c.Database("db0") - if err != nil { + if _, err := c.CreateDatabase("db0"); err != nil { t.Fatal(err) - } else if db.Name != "db0" { - t.Fatalf("db name wrong: %s", db.Name) } // DROP SUBSCRIPTION returns ErrSubscriptionNotFound when the // subscription is unknown. - res := c.ExecuteStatement(mustParseStatement(`DROP SUBSCRIPTION foo ON db0."default"`)) - if got, exp := res.Err, meta.ErrSubscriptionNotFound; got.Error() != exp.Error() { + err := c.DropSubscription("db0", "default", "foo") + if got, exp := err, meta.ErrSubscriptionNotFound; got.Error() != exp.Error() { t.Fatalf("got: %s, exp: %s", got, exp) } // Create a subscription. - if res = c.ExecuteStatement(mustParseStatement(`CREATE SUBSCRIPTION sub0 ON db0."default" DESTINATIONS ALL 'udp://example.com:9090'`)); res.Err != nil { - t.Fatal(res.Err) + if err := c.CreateSubscription("db0", "default", "sub0", "ALL", []string{"udp://example.com:9090"}); err != nil { + t.Fatal(err) } // DROP SUBSCRIPTION returns an influxdb.ErrDatabaseNotFound when // the database is unknown. - res = c.ExecuteStatement(mustParseStatement(`DROP SUBSCRIPTION sub0 ON foo."default"`)) - if got, exp := res.Err, influxdb.ErrDatabaseNotFound("foo"); got.Error() != exp.Error() { + err = c.DropSubscription("foo", "default", "sub0") + if got, exp := err, influxdb.ErrDatabaseNotFound("foo"); got.Error() != exp.Error() { t.Fatalf("got: %s, exp: %s", got, exp) } // DROP SUBSCRIPTION returns an influxdb.ErrRetentionPolicyNotFound // when the retention policy is unknown. - res = c.ExecuteStatement(mustParseStatement(`DROP SUBSCRIPTION sub0 ON db0."foo_policy"`)) - if got, exp := res.Err, influxdb.ErrRetentionPolicyNotFound("foo_policy"); got.Error() != exp.Error() { + err = c.DropSubscription("db0", "foo_policy", "sub0") + if got, exp := err, influxdb.ErrRetentionPolicyNotFound("foo_policy"); got.Error() != exp.Error() { t.Fatalf("got: %s, exp: %s", got, exp) } // DROP SUBSCRIPTION drops the subsciption if it can find it. - res = c.ExecuteStatement(mustParseStatement(`DROP SUBSCRIPTION sub0 ON db0."default"`)) - if got := res.Err; got != nil { + err = c.DropSubscription("db0", "default", "sub0") + if got := err; got != nil { t.Fatalf("got: %s, exp: %v", got, nil) } - - if res = c.ExecuteStatement(mustParseStatement(`SHOW SUBSCRIPTIONS`)); res.Err != nil { - t.Fatal(res.Err) - } else if got, exp := len(res.Series), 0; got != exp { - t.Fatalf("got %d series, expected %d", got, exp) - } } func TestMetaService_Shards(t *testing.T) { @@ -822,7 +738,8 @@ func TestMetaService_CreateRemoveMetaNode(t *testing.T) { } defer s3.Close() - c1 := meta.NewClient(joinPeers[0:3], false) + c1 := meta.NewClient() + c1.SetMetaServers(joinPeers[0:3]) if err := c1.Open(); err != nil { t.Fatal(err) } @@ -833,14 +750,15 @@ func TestMetaService_CreateRemoveMetaNode(t *testing.T) { t.Fatalf("meta nodes wrong: %v", metaNodes) } - c := meta.NewClient([]string{s1.HTTPAddr()}, false) + c := meta.NewClient() + c.SetMetaServers([]string{s1.HTTPAddr()}) if err := c.Open(); err != nil { t.Fatal(err) } defer c.Close() - if res := c.ExecuteStatement(mustParseStatement("DROP META SERVER 3")); res.Err != nil { - t.Fatal(res.Err) + if err := c.DeleteMetaNode(3); err != nil { + t.Fatal(err) } metaNodes, _ = c.MetaNodes() @@ -859,7 +777,8 @@ func TestMetaService_CreateRemoveMetaNode(t *testing.T) { } defer s4.Close() - c2 := meta.NewClient(cfg4.JoinPeers, false) + c2 := meta.NewClient() + c2.SetMetaServers(cfg4.JoinPeers) if err := c2.Open(); err != nil { t.Fatal(err) } @@ -903,7 +822,8 @@ func TestMetaService_CommandAgainstNonLeader(t *testing.T) { wg.Wait() for i := range cfgs { - c := meta.NewClient([]string{joinPeers[i]}, false) + c := meta.NewClient() + c.SetMetaServers([]string{joinPeers[i]}) if err := c.Open(); err != nil { t.Fatal(err) } @@ -957,7 +877,8 @@ func TestMetaService_FailureAndRestartCluster(t *testing.T) { } swg.Wait() - c := meta.NewClient(joinPeers, false) + c := meta.NewClient() + c.SetMetaServers(joinPeers) if err := c.Open(); err != nil { t.Fatal(err) } @@ -1015,7 +936,8 @@ func TestMetaService_FailureAndRestartCluster(t *testing.T) { wg.Wait() time.Sleep(time.Second) - c2 := meta.NewClient(joinPeers, false) + c2 := meta.NewClient() + c2.SetMetaServers(joinPeers) if err := c2.Open(); err != nil { t.Fatal(err) } @@ -1056,7 +978,8 @@ func TestMetaService_NameChangeSingleNode(t *testing.T) { } defer s.Close() - c := meta.NewClient([]string{s.HTTPAddr()}, false) + c := meta.NewClient() + c.SetMetaServers([]string{s.HTTPAddr()}) if err := c.Open(); err != nil { t.Fatal(err) } @@ -1077,7 +1000,8 @@ func TestMetaService_NameChangeSingleNode(t *testing.T) { } defer s.Close() - c2 := meta.NewClient([]string{s.HTTPAddr()}, false) + c2 := meta.NewClient() + c2.SetMetaServers([]string{s.HTTPAddr()}) if err := c2.Open(); err != nil { t.Fatal(err) } @@ -1177,8 +1101,8 @@ func TestMetaService_DropDataNode(t *testing.T) { t.Fatalf("expected owners to be [1]: %v", sg.Shards[0].Owners) } - if res := c.ExecuteStatement(mustParseStatement("DROP DATA SERVER 1")); res.Err != nil { - t.Fatal(res.Err) + if err := c.DeleteDataNode(1); err != nil { + t.Fatal(err) } rp, _ := c.RetentionPolicy("foo", "default") @@ -1198,7 +1122,8 @@ func TestMetaService_PersistClusterIDAfterRestart(t *testing.T) { } defer s.Close() - c := meta.NewClient([]string{s.HTTPAddr()}, false) + c := meta.NewClient() + c.SetMetaServers([]string{s.HTTPAddr()}) if err := c.Open(); err != nil { t.Fatal(err) } @@ -1213,7 +1138,8 @@ func TestMetaService_PersistClusterIDAfterRestart(t *testing.T) { t.Fatal(err) } - c = meta.NewClient([]string{s.HTTPAddr()}, false) + c = meta.NewClient() + c.SetMetaServers([]string{s.HTTPAddr()}) if err := c.Open(); err != nil { t.Fatal(err) } @@ -1253,7 +1179,8 @@ func TestMetaService_Ping(t *testing.T) { } swg.Wait() - c := meta.NewClient(joinPeers, false) + c := meta.NewClient() + c.SetMetaServers(joinPeers) if err := c.Open(); err != nil { t.Fatal(err) } @@ -1349,7 +1276,8 @@ func newServiceAndClient() (string, *testService, *meta.Client) { } func newClient(s *testService) *meta.Client { - c := meta.NewClient([]string{s.HTTPAddr()}, false) + c := meta.NewClient() + c.SetMetaServers([]string{s.HTTPAddr()}) if err := c.Open(); err != nil { panic(err) } diff --git a/services/meta/statement_executor.go b/services/meta/statement_executor.go deleted file mode 100644 index 06c253df6b8..00000000000 --- a/services/meta/statement_executor.go +++ /dev/null @@ -1,456 +0,0 @@ -package meta - -import ( - "bytes" - "fmt" - "strconv" - "time" - - "github.com/influxdata/influxdb" - "github.com/influxdata/influxdb/influxql" - "github.com/influxdata/influxdb/models" -) - -// StatementExecutor translates InfluxQL queries to meta store methods. -type StatementExecutor struct { - Store interface { - DataNode(id uint64) (ni *NodeInfo, err error) - DataNodes() ([]NodeInfo, error) - MetaNodes() ([]NodeInfo, error) - DeleteDataNode(nodeID uint64) error - DeleteMetaNode(nodeID uint64) error - - Database(name string) (*DatabaseInfo, error) - Databases() ([]DatabaseInfo, error) - CreateDatabase(name string) (*DatabaseInfo, error) - CreateDatabaseWithRetentionPolicy(name string, rpi *RetentionPolicyInfo) (*DatabaseInfo, error) - DropDatabase(name string) error - - CreateRetentionPolicy(database string, rpi *RetentionPolicyInfo) (*RetentionPolicyInfo, error) - UpdateRetentionPolicy(database, name string, rpu *RetentionPolicyUpdate) error - SetDefaultRetentionPolicy(database, name string) error - DropRetentionPolicy(database, name string) error - - Users() []UserInfo - CreateUser(name, password string, admin bool) (*UserInfo, error) - UpdateUser(name, password string) error - DropUser(name string) error - SetPrivilege(username, database string, p influxql.Privilege) error - SetAdminPrivilege(username string, admin bool) error - UserPrivileges(username string) (map[string]influxql.Privilege, error) - UserPrivilege(username, database string) (*influxql.Privilege, error) - - CreateContinuousQuery(database, name, query string) error - DropContinuousQuery(database, name string) error - - CreateSubscription(database, rp, name, mode string, destinations []string) error - DropSubscription(database, rp, name string) error - } -} - -// ExecuteStatement executes stmt against the meta store as user. -func (e *StatementExecutor) ExecuteStatement(stmt influxql.Statement) *influxql.Result { - switch stmt := stmt.(type) { - case *influxql.CreateDatabaseStatement: - return e.executeCreateDatabaseStatement(stmt) - case *influxql.DropDatabaseStatement: - return e.executeDropDatabaseStatement(stmt) - case *influxql.ShowDatabasesStatement: - return e.executeShowDatabasesStatement(stmt) - case *influxql.ShowGrantsForUserStatement: - return e.executeShowGrantsForUserStatement(stmt) - case *influxql.ShowServersStatement: - return e.executeShowServersStatement(stmt) - case *influxql.CreateUserStatement: - return e.executeCreateUserStatement(stmt) - case *influxql.SetPasswordUserStatement: - return e.executeSetPasswordUserStatement(stmt) - case *influxql.DropUserStatement: - return e.executeDropUserStatement(stmt) - case *influxql.ShowUsersStatement: - return e.executeShowUsersStatement(stmt) - case *influxql.GrantStatement: - return e.executeGrantStatement(stmt) - case *influxql.GrantAdminStatement: - return e.executeGrantAdminStatement(stmt) - case *influxql.RevokeStatement: - return e.executeRevokeStatement(stmt) - case *influxql.RevokeAdminStatement: - return e.executeRevokeAdminStatement(stmt) - case *influxql.CreateRetentionPolicyStatement: - return e.executeCreateRetentionPolicyStatement(stmt) - case *influxql.AlterRetentionPolicyStatement: - return e.executeAlterRetentionPolicyStatement(stmt) - case *influxql.DropRetentionPolicyStatement: - return e.executeDropRetentionPolicyStatement(stmt) - case *influxql.ShowRetentionPoliciesStatement: - return e.executeShowRetentionPoliciesStatement(stmt) - case *influxql.CreateContinuousQueryStatement: - return e.executeCreateContinuousQueryStatement(stmt) - case *influxql.DropContinuousQueryStatement: - return e.executeDropContinuousQueryStatement(stmt) - case *influxql.ShowContinuousQueriesStatement: - return e.executeShowContinuousQueriesStatement(stmt) - case *influxql.ShowShardsStatement: - return e.executeShowShardsStatement(stmt) - case *influxql.ShowShardGroupsStatement: - return e.executeShowShardGroupsStatement(stmt) - case *influxql.ShowStatsStatement: - return e.executeShowStatsStatement(stmt) - case *influxql.DropServerStatement: - return e.executeDropServerStatement(stmt) - case *influxql.CreateSubscriptionStatement: - return e.executeCreateSubscriptionStatement(stmt) - case *influxql.DropSubscriptionStatement: - return e.executeDropSubscriptionStatement(stmt) - case *influxql.ShowSubscriptionsStatement: - return e.executeShowSubscriptionsStatement(stmt) - default: - panic(fmt.Sprintf("unsupported statement type: %T", stmt)) - } -} - -func (e *StatementExecutor) executeCreateDatabaseStatement(q *influxql.CreateDatabaseStatement) *influxql.Result { - var err error - if q.RetentionPolicyCreate { - rpi := NewRetentionPolicyInfo(q.RetentionPolicyName) - rpi.Duration = q.RetentionPolicyDuration - rpi.ReplicaN = q.RetentionPolicyReplication - _, err = e.Store.CreateDatabaseWithRetentionPolicy(q.Name, rpi) - } else { - _, err = e.Store.CreateDatabase(q.Name) - } - - return &influxql.Result{Err: err} -} - -func (e *StatementExecutor) executeDropDatabaseStatement(q *influxql.DropDatabaseStatement) *influxql.Result { - if q.IfExists { - if db, _ := e.Store.Database(q.Name); db == nil { - return &influxql.Result{} - } - } - return &influxql.Result{Err: e.Store.DropDatabase(q.Name)} -} - -func (e *StatementExecutor) executeShowDatabasesStatement(q *influxql.ShowDatabasesStatement) *influxql.Result { - dis, err := e.Store.Databases() - if err != nil { - return &influxql.Result{Err: err} - } - - row := &models.Row{Name: "databases", Columns: []string{"name"}} - for _, di := range dis { - row.Values = append(row.Values, []interface{}{di.Name}) - } - return &influxql.Result{Series: []*models.Row{row}} -} - -func (e *StatementExecutor) executeShowGrantsForUserStatement(q *influxql.ShowGrantsForUserStatement) *influxql.Result { - priv, err := e.Store.UserPrivileges(q.Name) - if err != nil { - return &influxql.Result{Err: err} - } - - row := &models.Row{Columns: []string{"database", "privilege"}} - for d, p := range priv { - row.Values = append(row.Values, []interface{}{d, p.String()}) - } - return &influxql.Result{Series: []*models.Row{row}} -} - -func (e *StatementExecutor) executeShowServersStatement(q *influxql.ShowServersStatement) *influxql.Result { - nis, err := e.Store.DataNodes() - if err != nil { - return &influxql.Result{Err: err} - } - - dataNodes := &models.Row{Columns: []string{"id", "http_addr", "tcp_addr"}} - dataNodes.Name = "data_nodes" - for _, ni := range nis { - dataNodes.Values = append(dataNodes.Values, []interface{}{ni.ID, ni.Host, ni.TCPHost}) - } - - nis, err = e.Store.MetaNodes() - if err != nil { - return &influxql.Result{Err: err} - } - - metaNodes := &models.Row{Columns: []string{"id", "http_addr", "tcp_addr"}} - metaNodes.Name = "meta_nodes" - for _, ni := range nis { - metaNodes.Values = append(metaNodes.Values, []interface{}{ni.ID, ni.Host, ni.TCPHost}) - } - - return &influxql.Result{Series: []*models.Row{dataNodes, metaNodes}} -} - -func (e *StatementExecutor) executeDropServerStatement(q *influxql.DropServerStatement) *influxql.Result { - var err error - if q.Meta { - err = e.Store.DeleteMetaNode(q.NodeID) - } else { - err = e.Store.DeleteDataNode(q.NodeID) - } - - return &influxql.Result{Err: err} -} - -func (e *StatementExecutor) executeCreateUserStatement(q *influxql.CreateUserStatement) *influxql.Result { - _, err := e.Store.CreateUser(q.Name, q.Password, q.Admin) - return &influxql.Result{Err: err} -} - -func (e *StatementExecutor) executeSetPasswordUserStatement(q *influxql.SetPasswordUserStatement) *influxql.Result { - return &influxql.Result{Err: e.Store.UpdateUser(q.Name, q.Password)} -} - -func (e *StatementExecutor) executeDropUserStatement(q *influxql.DropUserStatement) *influxql.Result { - return &influxql.Result{Err: e.Store.DropUser(q.Name)} -} - -func (e *StatementExecutor) executeShowUsersStatement(q *influxql.ShowUsersStatement) *influxql.Result { - uis := e.Store.Users() - - row := &models.Row{Columns: []string{"user", "admin"}} - for _, ui := range uis { - row.Values = append(row.Values, []interface{}{ui.Name, ui.Admin}) - } - return &influxql.Result{Series: []*models.Row{row}} -} - -func (e *StatementExecutor) executeGrantStatement(stmt *influxql.GrantStatement) *influxql.Result { - return &influxql.Result{Err: e.Store.SetPrivilege(stmt.User, stmt.On, stmt.Privilege)} -} - -func (e *StatementExecutor) executeGrantAdminStatement(stmt *influxql.GrantAdminStatement) *influxql.Result { - return &influxql.Result{Err: e.Store.SetAdminPrivilege(stmt.User, true)} -} - -func (e *StatementExecutor) executeRevokeStatement(stmt *influxql.RevokeStatement) *influxql.Result { - priv := influxql.NoPrivileges - - // Revoking all privileges means there's no need to look at existing user privileges. - if stmt.Privilege != influxql.AllPrivileges { - p, err := e.Store.UserPrivilege(stmt.User, stmt.On) - if err != nil { - return &influxql.Result{Err: err} - } - // Bit clear (AND NOT) the user's privilege with the revoked privilege. - priv = *p &^ stmt.Privilege - } - - return &influxql.Result{Err: e.Store.SetPrivilege(stmt.User, stmt.On, priv)} -} - -func (e *StatementExecutor) executeRevokeAdminStatement(stmt *influxql.RevokeAdminStatement) *influxql.Result { - return &influxql.Result{Err: e.Store.SetAdminPrivilege(stmt.User, false)} -} - -func (e *StatementExecutor) executeCreateRetentionPolicyStatement(stmt *influxql.CreateRetentionPolicyStatement) *influxql.Result { - rpi := NewRetentionPolicyInfo(stmt.Name) - rpi.Duration = stmt.Duration - rpi.ReplicaN = stmt.Replication - - // Create new retention policy. - _, err := e.Store.CreateRetentionPolicy(stmt.Database, rpi) - if err != nil { - return &influxql.Result{Err: err} - } - - // If requested, set new policy as the default. - if stmt.Default { - err = e.Store.SetDefaultRetentionPolicy(stmt.Database, stmt.Name) - } - - return &influxql.Result{Err: err} -} - -func (e *StatementExecutor) executeAlterRetentionPolicyStatement(stmt *influxql.AlterRetentionPolicyStatement) *influxql.Result { - rpu := &RetentionPolicyUpdate{ - Duration: stmt.Duration, - ReplicaN: stmt.Replication, - } - - // Update the retention policy. - err := e.Store.UpdateRetentionPolicy(stmt.Database, stmt.Name, rpu) - if err != nil { - return &influxql.Result{Err: err} - } - - // If requested, set as default retention policy. - if stmt.Default { - err = e.Store.SetDefaultRetentionPolicy(stmt.Database, stmt.Name) - } - - return &influxql.Result{Err: err} -} - -func (e *StatementExecutor) executeDropRetentionPolicyStatement(q *influxql.DropRetentionPolicyStatement) *influxql.Result { - return &influxql.Result{Err: e.Store.DropRetentionPolicy(q.Database, q.Name)} -} - -func (e *StatementExecutor) executeShowRetentionPoliciesStatement(q *influxql.ShowRetentionPoliciesStatement) *influxql.Result { - di, err := e.Store.Database(q.Database) - if err != nil { - return &influxql.Result{Err: err} - } else if di == nil { - return &influxql.Result{Err: influxdb.ErrDatabaseNotFound(q.Database)} - } - - row := &models.Row{Columns: []string{"name", "duration", "replicaN", "default"}} - for _, rpi := range di.RetentionPolicies { - row.Values = append(row.Values, []interface{}{rpi.Name, rpi.Duration.String(), rpi.ReplicaN, di.DefaultRetentionPolicy == rpi.Name}) - } - return &influxql.Result{Series: []*models.Row{row}} -} - -func (e *StatementExecutor) executeCreateContinuousQueryStatement(q *influxql.CreateContinuousQueryStatement) *influxql.Result { - return &influxql.Result{ - Err: e.Store.CreateContinuousQuery(q.Database, q.Name, q.String()), - } -} - -func (e *StatementExecutor) executeDropContinuousQueryStatement(q *influxql.DropContinuousQueryStatement) *influxql.Result { - return &influxql.Result{ - Err: e.Store.DropContinuousQuery(q.Database, q.Name), - } -} - -func (e *StatementExecutor) executeShowContinuousQueriesStatement(stmt *influxql.ShowContinuousQueriesStatement) *influxql.Result { - dis, err := e.Store.Databases() - if err != nil { - return &influxql.Result{Err: err} - } - - rows := []*models.Row{} - for _, di := range dis { - row := &models.Row{Columns: []string{"name", "query"}, Name: di.Name} - for _, cqi := range di.ContinuousQueries { - row.Values = append(row.Values, []interface{}{cqi.Name, cqi.Query}) - } - rows = append(rows, row) - } - return &influxql.Result{Series: rows} -} - -func (e *StatementExecutor) executeCreateSubscriptionStatement(q *influxql.CreateSubscriptionStatement) *influxql.Result { - return &influxql.Result{ - Err: e.Store.CreateSubscription(q.Database, q.RetentionPolicy, q.Name, q.Mode, q.Destinations), - } -} - -func (e *StatementExecutor) executeDropSubscriptionStatement(q *influxql.DropSubscriptionStatement) *influxql.Result { - return &influxql.Result{ - Err: e.Store.DropSubscription(q.Database, q.RetentionPolicy, q.Name), - } -} - -func (e *StatementExecutor) executeShowSubscriptionsStatement(stmt *influxql.ShowSubscriptionsStatement) *influxql.Result { - dis, err := e.Store.Databases() - if err != nil { - return &influxql.Result{Err: err} - } - - rows := []*models.Row{} - for _, di := range dis { - row := &models.Row{Columns: []string{"retention_policy", "name", "mode", "destinations"}, Name: di.Name} - for _, rpi := range di.RetentionPolicies { - for _, si := range rpi.Subscriptions { - row.Values = append(row.Values, []interface{}{rpi.Name, si.Name, si.Mode, si.Destinations}) - } - } - if len(row.Values) > 0 { - rows = append(rows, row) - } - } - return &influxql.Result{Series: rows} -} - -func (e *StatementExecutor) executeShowShardGroupsStatement(stmt *influxql.ShowShardGroupsStatement) *influxql.Result { - dis, err := e.Store.Databases() - if err != nil { - return &influxql.Result{Err: err} - } - - row := &models.Row{Columns: []string{"id", "database", "retention_policy", "start_time", "end_time", "expiry_time"}, Name: "shard groups"} - for _, di := range dis { - for _, rpi := range di.RetentionPolicies { - for _, sgi := range rpi.ShardGroups { - // Shards associated with deleted shard groups are effectively deleted. - // Don't list them. - if sgi.Deleted() { - continue - } - - row.Values = append(row.Values, []interface{}{ - sgi.ID, - di.Name, - rpi.Name, - sgi.StartTime.UTC().Format(time.RFC3339), - sgi.EndTime.UTC().Format(time.RFC3339), - sgi.EndTime.Add(rpi.Duration).UTC().Format(time.RFC3339), - }) - } - } - } - - return &influxql.Result{Series: []*models.Row{row}} -} - -func (e *StatementExecutor) executeShowShardsStatement(stmt *influxql.ShowShardsStatement) *influxql.Result { - dis, err := e.Store.Databases() - if err != nil { - return &influxql.Result{Err: err} - } - - rows := []*models.Row{} - for _, di := range dis { - row := &models.Row{Columns: []string{"id", "database", "retention_policy", "shard_group", "start_time", "end_time", "expiry_time", "owners"}, Name: di.Name} - for _, rpi := range di.RetentionPolicies { - for _, sgi := range rpi.ShardGroups { - // Shards associated with deleted shard groups are effectively deleted. - // Don't list them. - if sgi.Deleted() { - continue - } - - for _, si := range sgi.Shards { - ownerIDs := make([]uint64, len(si.Owners)) - for i, owner := range si.Owners { - ownerIDs[i] = owner.NodeID - } - - row.Values = append(row.Values, []interface{}{ - si.ID, - di.Name, - rpi.Name, - sgi.ID, - sgi.StartTime.UTC().Format(time.RFC3339), - sgi.EndTime.UTC().Format(time.RFC3339), - sgi.EndTime.Add(rpi.Duration).UTC().Format(time.RFC3339), - joinUint64(ownerIDs), - }) - } - } - } - rows = append(rows, row) - } - return &influxql.Result{Series: rows} -} - -func (e *StatementExecutor) executeShowStatsStatement(stmt *influxql.ShowStatsStatement) *influxql.Result { - return &influxql.Result{Err: fmt.Errorf("SHOW STATS is not implemented yet")} -} - -// joinUint64 returns a comma-delimited string of uint64 numbers. -func joinUint64(a []uint64) string { - var buf bytes.Buffer - for i, x := range a { - buf.WriteString(strconv.FormatUint(x, 10)) - if i < len(a)-1 { - buf.WriteRune(',') - } - } - return buf.String() -} diff --git a/services/meta/statement_executor_test.go.fixme b/services/meta/statement_executor_test.go.fixme deleted file mode 100644 index 8a0f8e4d9ff..00000000000 --- a/services/meta/statement_executor_test.go.fixme +++ /dev/null @@ -1,1192 +0,0 @@ -package meta_test - -import ( - "errors" - "reflect" - "testing" - "time" - - "github.com/davecgh/go-spew/spew" - "github.com/influxdata/influxdb" - "github.com/influxdata/influxdb/influxql" - "github.com/influxdata/influxdb/models" - "github.com/influxdata/influxdb/services/meta" -) - -// Ensure a CREATE DATABASE statement can be executed. -func TestStatementExecutor_ExecuteStatement_CreateDatabase(t *testing.T) { - e := NewStatementExecutor() - e.Store.CreateDatabaseFn = func(name string) (*meta.DatabaseInfo, error) { - if name != "foo" { - t.Fatalf("unexpected name: %s", name) - } - return &meta.DatabaseInfo{Name: name}, nil - } - - if res := e.ExecuteStatement(influxql.MustParseStatement(`CREATE DATABASE foo`)); res.Err != nil { - t.Fatal(res.Err) - } else if res.Series != nil { - t.Fatalf("unexpected rows: %#v", res.Series) - } -} - -// Ensure a DROP DATABASE statement can be executed. -func TestStatementExecutor_ExecuteStatement_DropDatabase(t *testing.T) { - e := NewStatementExecutor() - e.Store.DropDatabaseFn = func(name string) error { - if name != "foo" { - t.Fatalf("unexpected name: %s", name) - } - return nil - } - - if res := e.ExecuteStatement(influxql.MustParseStatement(`DROP DATABASE foo`)); res.Err != nil { - t.Fatal(res.Err) - } else if res.Series != nil { - t.Fatalf("unexpected rows: %#v", res.Series) - } -} - -// Ensure a SHOW DATABASES statement can be executed. -func TestStatementExecutor_ExecuteStatement_ShowDatabases(t *testing.T) { - e := NewStatementExecutor() - e.Store.DatabasesFn = func() ([]meta.DatabaseInfo, error) { - return []meta.DatabaseInfo{ - {Name: "foo"}, - {Name: "bar"}, - }, nil - } - - if res := e.ExecuteStatement(influxql.MustParseStatement(`SHOW DATABASES`)); res.Err != nil { - t.Fatal(res.Err) - } else if !reflect.DeepEqual(res.Series, models.Rows{ - { - Name: "databases", - Columns: []string{"name"}, - Values: [][]interface{}{ - {"foo"}, - {"bar"}, - }, - }, - }) { - t.Fatalf("unexpected rows: %s", spew.Sdump(res.Series)) - } -} - -// Ensure a SHOW DATABASES statement returns errors from the store. -func TestStatementExecutor_ExecuteStatement_ShowDatabases_Err(t *testing.T) { - e := NewStatementExecutor() - e.Store.DatabasesFn = func() ([]meta.DatabaseInfo, error) { - return nil, errors.New("marker") - } - - if res := e.ExecuteStatement(influxql.MustParseStatement(`SHOW DATABASES`)); res.Err == nil || res.Err.Error() != "marker" { - t.Fatalf("unexpected error: %s", res.Err) - } -} - -// Ensure a SHOW GRANTS FOR statement can be executed. -func TestStatementExecutor_ExecuteStatement_ShowGrantsFor(t *testing.T) { - t.Skip("Intermittent test failure: issue 3028") - e := NewStatementExecutor() - e.Store.UserPrivilegesFn = func(username string) (map[string]influxql.Privilege, error) { - if username != "dejan" { - t.Fatalf("unexpected username: %s", username) - } - return map[string]influxql.Privilege{ - "dejan": influxql.ReadPrivilege, - "golja": influxql.WritePrivilege, - }, nil - } - - if res := e.ExecuteStatement(influxql.MustParseStatement(`SHOW GRANTS FOR dejan`)); res.Err != nil { - t.Fatal(res.Err) - } else if !reflect.DeepEqual(res.Series, models.Rows{ - { - Columns: []string{"database", "privilege"}, - Values: [][]interface{}{ - {"dejan", "READ"}, - {"golja", "WRITE"}, - }, - }, - }) { - t.Fatalf("unexpected rows: %s", spew.Sdump(res.Series)) - } -} - -// Ensure a SHOW SERVERS statement can be executed. -func TestStatementExecutor_ExecuteStatement_ShowServers(t *testing.T) { - e := NewStatementExecutor() - e.Store.NodesFn = func() ([]meta.NodeInfo, error) { - return []meta.NodeInfo{ - {ID: 1, Host: "node0"}, - {ID: 2, Host: "node1"}, - }, nil - } - e.Store.PeersFn = func() ([]string, error) { - return []string{"node0"}, nil - } - e.Store.LeaderFn = func() string { - return "node0" - } - - if res := e.ExecuteStatement(influxql.MustParseStatement(`SHOW SERVERS`)); res.Err != nil { - t.Fatal(res.Err) - } else if !reflect.DeepEqual(res.Series, models.Rows{ - { - Columns: []string{"id", "cluster_addr", "raft", "raft-leader"}, - Values: [][]interface{}{ - {uint64(1), "node0", true, true}, - {uint64(2), "node1", false, false}, - }, - }, - }) { - t.Fatalf("unexpected rows: %s", spew.Sdump(res.Series)) - } -} - -// Ensure a SHOW SERVERS statement returns errors from the store. -func TestStatementExecutor_ExecuteStatement_ShowServers_Err(t *testing.T) { - e := NewStatementExecutor() - e.Store.NodesFn = func() ([]meta.NodeInfo, error) { - return nil, errors.New("marker") - } - - if res := e.ExecuteStatement(influxql.MustParseStatement(`SHOW SERVERS`)); res.Err == nil || res.Err.Error() != "marker" { - t.Fatalf("unexpected error: %s", res.Err) - } -} - -// Ensure a CREATE USER statement can be executed. -func TestStatementExecutor_ExecuteStatement_CreateUser(t *testing.T) { - e := NewStatementExecutor() - e.Store.CreateUserFn = func(name, password string, admin bool) (*meta.UserInfo, error) { - if name != "susy" { - t.Fatalf("unexpected name: %s", name) - } else if password != "pass" { - t.Fatalf("unexpected password: %s", password) - } else if admin != true { - t.Fatalf("unexpected admin: %v", admin) - } - return &meta.UserInfo{Name: name, Admin: admin}, nil - } - - if res := e.ExecuteStatement(influxql.MustParseStatement(`CREATE USER susy WITH PASSWORD 'pass' WITH ALL PRIVILEGES`)); res.Err != nil { - t.Fatal(res.Err) - } else if res.Series != nil { - t.Fatalf("unexpected rows: %#v", res.Series) - } -} - -// Ensure a CREATE USER statement returns errors from the store. -func TestStatementExecutor_ExecuteStatement_CreateUser_Err(t *testing.T) { - e := NewStatementExecutor() - e.Store.CreateUserFn = func(name, password string, admin bool) (*meta.UserInfo, error) { - return nil, errors.New("marker") - } - - if res := e.ExecuteStatement(influxql.MustParseStatement(`CREATE USER susy WITH PASSWORD 'pass'`)); res.Err == nil || res.Err.Error() != "marker" { - t.Fatalf("unexpected error: %s", res.Err) - } -} - -// Ensure a SET PASSWORD statement can be executed. -func TestStatementExecutor_ExecuteStatement_SetPassword(t *testing.T) { - e := NewStatementExecutor() - e.Store.UpdateUserFn = func(name, password string) error { - if name != "susy" { - t.Fatalf("unexpected name: %s", name) - } else if password != "pass" { - t.Fatalf("unexpected password: %s", password) - } - return nil - } - - if res := e.ExecuteStatement(influxql.MustParseStatement(`SET PASSWORD FOR susy = 'pass' WITH ALL PRIVILEGES`)); res.Err != nil { - t.Fatal(res.Err) - } else if res.Series != nil { - t.Fatalf("unexpected rows: %#v", res.Series) - } -} - -// Ensure a SET PASSWORD statement returns errors from the store. -func TestStatementExecutor_ExecuteStatement_SetPassword_Err(t *testing.T) { - e := NewStatementExecutor() - e.Store.UpdateUserFn = func(name, password string) error { - return errors.New("marker") - } - - if res := e.ExecuteStatement(influxql.MustParseStatement(`SET PASSWORD FOR susy = 'pass'`)); res.Err == nil || res.Err.Error() != "marker" { - t.Fatalf("unexpected error: %s", res.Err) - } -} - -// Ensure a DROP USER statement can be executed. -func TestStatementExecutor_ExecuteStatement_DropUser(t *testing.T) { - e := NewStatementExecutor() - e.Store.DropUserFn = func(name string) error { - if name != "susy" { - t.Fatalf("unexpected name: %s", name) - } - return nil - } - - if res := e.ExecuteStatement(influxql.MustParseStatement(`DROP USER susy`)); res.Err != nil { - t.Fatal(res.Err) - } else if res.Series != nil { - t.Fatalf("unexpected rows: %#v", res.Series) - } -} - -// Ensure a DROP USER statement returns errors from the store. -func TestStatementExecutor_ExecuteStatement_DropUser_Err(t *testing.T) { - e := NewStatementExecutor() - e.Store.DropUserFn = func(name string) error { - return errors.New("marker") - } - - if res := e.ExecuteStatement(influxql.MustParseStatement(`DROP USER susy`)); res.Err == nil || res.Err.Error() != "marker" { - t.Fatalf("unexpected error: %s", res.Err) - } -} - -// Ensure a SHOW USERS statement can be executed. -func TestStatementExecutor_ExecuteStatement_ShowUsers(t *testing.T) { - e := NewStatementExecutor() - e.Store.UsersFn = func() ([]meta.UserInfo, error) { - return []meta.UserInfo{ - {Name: "susy", Admin: true}, - {Name: "bob", Admin: false}, - }, nil - } - - if res := e.ExecuteStatement(influxql.MustParseStatement(`SHOW USERS`)); res.Err != nil { - t.Fatal(res.Err) - } else if !reflect.DeepEqual(res.Series, models.Rows{ - { - Columns: []string{"user", "admin"}, - Values: [][]interface{}{ - {"susy", true}, - {"bob", false}, - }, - }, - }) { - t.Fatalf("unexpected rows: %s", spew.Sdump(res.Series)) - } -} - -// Ensure a SHOW USERS statement returns an error from the store. -func TestStatementExecutor_ExecuteStatement_ShowUsers_Err(t *testing.T) { - e := NewStatementExecutor() - e.Store.UsersFn = func() ([]meta.UserInfo, error) { - return nil, errors.New("marker") - } - - if res := e.ExecuteStatement(influxql.MustParseStatement(`SHOW USERS`)); res.Err == nil || res.Err.Error() != "marker" { - t.Fatalf("unexpected error: %s", res.Err) - } -} - -// Ensure a GRANT statement can be executed. -func TestStatementExecutor_ExecuteStatement_Grant(t *testing.T) { - e := NewStatementExecutor() - e.Store.SetPrivilegeFn = func(username, database string, p influxql.Privilege) error { - if username != "susy" { - t.Fatalf("unexpected username: %s", username) - } else if database != "foo" { - t.Fatalf("unexpected database: %s", database) - } else if p != influxql.WritePrivilege { - t.Fatalf("unexpected privilege: %s", p) - } - return nil - } - - if res := e.ExecuteStatement(influxql.MustParseStatement(`GRANT WRITE ON foo TO susy`)); res.Err != nil { - t.Fatal(res.Err) - } else if res.Series != nil { - t.Fatalf("unexpected rows: %#v", res.Series) - } -} - -// Ensure a GRANT statement returns errors from the store. -func TestStatementExecutor_ExecuteStatement_Grant_Err(t *testing.T) { - e := NewStatementExecutor() - e.Store.SetPrivilegeFn = func(username, database string, p influxql.Privilege) error { - return errors.New("marker") - } - - if res := e.ExecuteStatement(influxql.MustParseStatement(`GRANT READ ON foo TO susy`)); res.Err == nil || res.Err.Error() != "marker" { - t.Fatalf("unexpected error: %s", res.Err) - } -} - -// Ensure a GRANT statement for admin privilege can be executed. -func TestStatementExecutor_ExecuteStatement_GrantAdmin(t *testing.T) { - e := NewStatementExecutor() - e.Store.SetAdminPrivilegeFn = func(username string, admin bool) error { - if username != "susy" { - t.Fatalf("unexpected username: %s", username) - } else if admin != true { - t.Fatalf("unexpected admin privilege: %t", admin) - } - return nil - } - - if res := e.ExecuteStatement(influxql.MustParseStatement(`GRANT ALL TO susy`)); res.Err != nil { - t.Fatal(res.Err) - } else if res.Series != nil { - t.Fatalf("unexpected rows: %#v", res.Series) - } -} - -// Ensure a GRANT statement for admin privilege returns errors from the store. -func TestStatementExecutor_ExecuteStatement_GrantAdmin_Err(t *testing.T) { - e := NewStatementExecutor() - e.Store.SetAdminPrivilegeFn = func(username string, admin bool) error { - return errors.New("marker") - } - - if res := e.ExecuteStatement(influxql.MustParseStatement(`GRANT ALL PRIVILEGES TO susy`)); res.Err == nil || res.Err.Error() != "marker" { - t.Fatalf("unexpected error: %s", res.Err) - } -} - -// Ensure a REVOKE statement can be executed. -func TestStatementExecutor_ExecuteStatement_Revoke(t *testing.T) { - e := NewStatementExecutor() - e.Store.SetPrivilegeFn = func(username, database string, p influxql.Privilege) error { - if username != "susy" { - t.Fatalf("unexpected username: %s", username) - } else if database != "foo" { - t.Fatalf("unexpected database: %s", database) - } else if p != influxql.NoPrivileges { - t.Fatalf("unexpected privilege: %s", p) - } - return nil - } - - if res := e.ExecuteStatement(influxql.MustParseStatement(`REVOKE ALL PRIVILEGES ON foo FROM susy`)); res.Err != nil { - t.Fatal(res.Err) - } else if res.Series != nil { - t.Fatalf("unexpected rows: %#v", res.Series) - } -} - -// Ensure a REVOKE statement returns errors from the store. -func TestStatementExecutor_ExecuteStatement_Revoke_Err(t *testing.T) { - e := NewStatementExecutor() - e.Store.SetPrivilegeFn = func(username, database string, p influxql.Privilege) error { - return errors.New("marker") - } - - if res := e.ExecuteStatement(influxql.MustParseStatement(`REVOKE ALL PRIVILEGES ON foo FROM susy`)); res.Err == nil || res.Err.Error() != "marker" { - t.Fatalf("unexpected error: %s", res.Err) - } -} - -// Ensure a REVOKE statement for admin privilege can be executed. -func TestStatementExecutor_ExecuteStatement_RevokeAdmin(t *testing.T) { - e := NewStatementExecutor() - e.Store.SetAdminPrivilegeFn = func(username string, admin bool) error { - if username != "susy" { - t.Fatalf("unexpected username: %s", username) - } else if admin != false { - t.Fatalf("unexpected admin privilege: %t", admin) - } - return nil - } - - if res := e.ExecuteStatement(influxql.MustParseStatement(`REVOKE ALL PRIVILEGES FROM susy`)); res.Err != nil { - t.Fatal(res.Err) - } else if res.Series != nil { - t.Fatalf("unexpected rows: %#v", res.Series) - } -} - -// Ensure a REVOKE statement for admin privilege returns errors from the store. -func TestStatementExecutor_ExecuteStatement_RevokeAdmin_Err(t *testing.T) { - e := NewStatementExecutor() - e.Store.SetAdminPrivilegeFn = func(username string, admin bool) error { - return errors.New("marker") - } - - if res := e.ExecuteStatement(influxql.MustParseStatement(`REVOKE ALL PRIVILEGES FROM susy`)); res.Err == nil || res.Err.Error() != "marker" { - t.Fatalf("unexpected error: %s", res.Err) - } -} - -// Ensure a CREATE RETENTION POLICY statement can be executed. -func TestStatementExecutor_ExecuteStatement_CreateRetentionPolicy(t *testing.T) { - e := NewStatementExecutor() - e.Store.CreateRetentionPolicyFn = func(database string, rpi *meta.RetentionPolicyInfo) (*meta.RetentionPolicyInfo, error) { - if database != "foo" { - t.Fatalf("unexpected database: %s", database) - } else if rpi.Name != "rp0" { - t.Fatalf("unexpected name: %s", rpi.Name) - } else if rpi.Duration != 2*time.Hour { - t.Fatalf("unexpected duration: %v", rpi.Duration) - } else if rpi.ReplicaN != 3 { - t.Fatalf("unexpected replication factor: %v", rpi.ReplicaN) - } - return nil, nil - } - e.Store.SetDefaultRetentionPolicyFn = func(database, name string) error { - if database != "foo" { - t.Fatalf("unexpected database: %s", database) - } else if name != "rp0" { - t.Fatalf("unexpected name: %s", name) - } - return nil - } - - if res := e.ExecuteStatement(influxql.MustParseStatement(`CREATE RETENTION POLICY rp0 ON foo DURATION 2h REPLICATION 3 DEFAULT`)); res.Err != nil { - t.Fatal(res.Err) - } else if res.Series != nil { - t.Fatalf("unexpected rows: %#v", res.Series) - } -} - -// Ensure a CREATE RETENTION POLICY statement returns errors from the store. -func TestStatementExecutor_ExecuteStatement_CreateRetentionPolicy_Err(t *testing.T) { - e := NewStatementExecutor() - e.Store.CreateRetentionPolicyFn = func(database string, rpi *meta.RetentionPolicyInfo) (*meta.RetentionPolicyInfo, error) { - return nil, errors.New("marker") - } - - if res := e.ExecuteStatement(influxql.MustParseStatement(`CREATE RETENTION POLICY rp0 ON foo DURATION 2h REPLICATION 1`)); res.Err == nil || res.Err.Error() != "marker" { - t.Fatalf("unexpected error: %s", res.Err) - } -} - -// Ensure an ALTER RETENTION POLICY statement can execute. -func TestStatementExecutor_ExecuteStatement_AlterRetentionPolicy(t *testing.T) { - e := NewStatementExecutor() - e.Store.UpdateRetentionPolicyFn = func(database, name string, rpu *meta.RetentionPolicyUpdate) error { - if database != "foo" { - t.Fatalf("unexpected database: %s", database) - } else if name != "rp0" { - t.Fatalf("unexpected name: %s", name) - } else if rpu.Duration != nil && *rpu.Duration != 7*24*time.Hour { - t.Fatalf("unexpected duration: %v", *rpu.Duration) - } else if rpu.ReplicaN != nil && *rpu.ReplicaN != 2 { - t.Fatalf("unexpected replication factor: %v", *rpu.ReplicaN) - } - return nil - } - e.Store.SetDefaultRetentionPolicyFn = func(database, name string) error { - if database != "foo" { - t.Fatalf("unexpected database: %s", database) - } else if name != "rp0" { - t.Fatalf("unexpected name: %s", name) - } - return nil - } - - stmt := influxql.MustParseStatement(`ALTER RETENTION POLICY rp0 ON foo DURATION 7d REPLICATION 2 DEFAULT`) - if res := e.ExecuteStatement(stmt); res.Err != nil { - t.Fatalf("unexpected error: %s", res.Err) - } - - stmt = influxql.MustParseStatement(`ALTER RETENTION POLICY rp0 ON foo DURATION 7d`) - if res := e.ExecuteStatement(stmt); res.Err != nil { - t.Fatalf("unexpected error: %s", res.Err) - } - - stmt = influxql.MustParseStatement(`ALTER RETENTION POLICY rp0 ON foo REPLICATION 2`) - if res := e.ExecuteStatement(stmt); res.Err != nil { - t.Fatalf("unexpected error: %s", res.Err) - } -} - -// Ensure a ALTER RETENTION POLICY statement returns errors from the store. -func TestStatementExecutor_ExecuteStatement_AlterRetentionPolicy_Err(t *testing.T) { - e := NewStatementExecutor() - e.Store.UpdateRetentionPolicyFn = func(database, name string, rpu *meta.RetentionPolicyUpdate) error { - return errors.New("marker") - } - - stmt := influxql.MustParseStatement(`ALTER RETENTION POLICY rp0 ON foo DURATION 1m REPLICATION 4 DEFAULT`) - if res := e.ExecuteStatement(stmt); res.Err == nil || res.Err.Error() != "marker" { - t.Fatalf("unexpected error: %s", res.Err) - } -} - -// Ensure a ALTER RETENTION POLICY statement returns errors from the store. -func TestStatementExecutor_ExecuteStatement_AlterRetentionPolicy_ErrSetDefault(t *testing.T) { - e := NewStatementExecutor() - e.Store.UpdateRetentionPolicyFn = func(database, name string, rpu *meta.RetentionPolicyUpdate) error { - return nil - } - e.Store.SetDefaultRetentionPolicyFn = func(database, name string) error { - return errors.New("marker") - } - - stmt := influxql.MustParseStatement(`ALTER RETENTION POLICY rp0 ON foo DURATION 1m REPLICATION 4 DEFAULT`) - if res := e.ExecuteStatement(stmt); res.Err == nil || res.Err.Error() != "marker" { - t.Fatalf("unexpected error: %s", res.Err) - } -} - -// Ensure a DROP RETENTION POLICY statement can execute. -func TestStatementExecutor_ExecuteStatement_DropRetentionPolicy(t *testing.T) { - e := NewStatementExecutor() - e.Store.DropRetentionPolicyFn = func(database, name string) error { - if database != "foo" { - t.Fatalf("unexpected database: %s", database) - } else if name != "rp0" { - t.Fatalf("unexpected name: %s", name) - } - return nil - } - - stmt := influxql.MustParseStatement(`DROP RETENTION POLICY rp0 ON foo`) - if res := e.ExecuteStatement(stmt); res.Err != nil { - t.Fatalf("unexpected error: %s", res.Err) - } -} - -// Ensure a DROP RETENTION POLICY statement returns errors from the store. -func TestStatementExecutor_ExecuteStatement_DropRetentionPolicy_Err(t *testing.T) { - e := NewStatementExecutor() - e.Store.DropRetentionPolicyFn = func(database, name string) error { - return errors.New("marker") - } - - stmt := influxql.MustParseStatement(`DROP RETENTION POLICY rp0 ON foo`) - if res := e.ExecuteStatement(stmt); res.Err == nil || res.Err.Error() != "marker" { - t.Fatalf("unexpected error: %s", res.Err) - } -} - -// Ensure a SHOW RETENTION POLICIES statement can be executed. -func TestStatementExecutor_ExecuteStatement_ShowRetentionPolicies(t *testing.T) { - e := NewStatementExecutor() - e.Store.DatabaseFn = func(name string) (*meta.DatabaseInfo, error) { - if name != "db0" { - t.Fatalf("unexpected name: %s", name) - } - return &meta.DatabaseInfo{ - Name: name, - DefaultRetentionPolicy: "rp1", - RetentionPolicies: []meta.RetentionPolicyInfo{ - { - Name: "rp0", - Duration: 2 * time.Hour, - ReplicaN: 3, - }, - { - Name: "rp1", - Duration: 24 * time.Hour, - ReplicaN: 1, - }, - }, - }, nil - } - - if res := e.ExecuteStatement(influxql.MustParseStatement(`SHOW RETENTION POLICIES ON db0`)); res.Err != nil { - t.Fatal(res.Err) - } else if !reflect.DeepEqual(res.Series, models.Rows{ - { - Columns: []string{"name", "duration", "replicaN", "default"}, - Values: [][]interface{}{ - {"rp0", "2h0m0s", 3, false}, - {"rp1", "24h0m0s", 1, true}, - }, - }, - }) { - t.Fatalf("unexpected rows: %s", spew.Sdump(res.Series)) - } -} - -// Ensure a SHOW RETENTION POLICIES statement can return an error from the store. -func TestStatementExecutor_ExecuteStatement_ShowRetentionPolicies_Err(t *testing.T) { - e := NewStatementExecutor() - e.Store.DatabaseFn = func(name string) (*meta.DatabaseInfo, error) { - return nil, errors.New("marker") - } - - if res := e.ExecuteStatement(influxql.MustParseStatement(`SHOW RETENTION POLICIES ON db0`)); res.Err == nil || res.Err.Error() != "marker" { - t.Fatalf("unexpected error: %s", res.Err) - } -} - -// Ensure a SHOW RETENTION POLICIES statement can return an error if the database doesn't exist. -func TestStatementExecutor_ExecuteStatement_ShowRetentionPolicies_ErrDatabaseNotFound(t *testing.T) { - e := NewStatementExecutor() - e.Store.DatabaseFn = func(name string) (*meta.DatabaseInfo, error) { - return nil, nil - } - - expErr := influxdb.ErrDatabaseNotFound("db0") - if res := e.ExecuteStatement(influxql.MustParseStatement(`SHOW RETENTION POLICIES ON db0`)); res.Err.Error() != expErr.Error() { - t.Fatalf("unexpected error: %s", res.Err) - } -} - -// Ensure a CREATE CONTINUOUS QUERY statement can be executed. -func TestStatementExecutor_ExecuteStatement_CreateContinuousQuery(t *testing.T) { - e := NewStatementExecutor() - e.Store.CreateContinuousQueryFn = func(database, name, query string) error { - if database != "db0" { - t.Fatalf("unexpected database: %s", database) - } else if name != "cq0" { - t.Fatalf("unexpected name: %s", name) - } else if query != `CREATE CONTINUOUS QUERY cq0 ON db0 BEGIN SELECT count(field1) INTO db1 FROM db0 GROUP BY time(1h) END` { - t.Fatalf("unexpected query: %s", query) - } - return nil - } - - stmt := influxql.MustParseStatement(`CREATE CONTINUOUS QUERY cq0 ON db0 BEGIN SELECT count(field1) INTO db1 FROM db0 GROUP BY time(1h) END`) - if res := e.ExecuteStatement(stmt); res.Err != nil { - t.Fatal(res.Err) - } else if res.Series != nil { - t.Fatalf("unexpected rows: %#v", res.Series) - } -} - -// Ensure a CREATE CONTINUOUS QUERY statement can return an error from the store. -func TestStatementExecutor_ExecuteStatement_CreateContinuousQuery_Err(t *testing.T) { - e := NewStatementExecutor() - e.Store.CreateContinuousQueryFn = func(database, name, query string) error { - return errors.New("marker") - } - - stmt := influxql.MustParseStatement(`CREATE CONTINUOUS QUERY cq0 ON db0 BEGIN SELECT count(field1) INTO db1 FROM db0 GROUP BY time(1h) END`) - if res := e.ExecuteStatement(stmt); res.Err == nil || res.Err.Error() != "marker" { - t.Fatalf("unexpected error: %s", res.Err) - } -} - -// Ensure a DROP CONTINUOUS QUERY statement can be executed. -func TestStatementExecutor_ExecuteStatement_DropContinuousQuery(t *testing.T) { - e := NewStatementExecutor() - e.Store.DropContinuousQueryFn = func(database, name string) error { - if database != "db0" { - t.Fatalf("unexpected database: %s", database) - } else if name != "cq0" { - t.Fatalf("unexpected name: %s", name) - } - return nil - } - - stmt := influxql.MustParseStatement(`DROP CONTINUOUS QUERY cq0 ON db0`) - if res := e.ExecuteStatement(stmt); res.Err != nil { - t.Fatal(res.Err) - } else if res.Series != nil { - t.Fatalf("unexpected rows: %#v", res.Series) - } -} - -// Ensure a DROP CONTINUOUS QUERY statement can return an error from the store. -func TestStatementExecutor_ExecuteStatement_DropContinuousQuery_Err(t *testing.T) { - e := NewStatementExecutor() - e.Store.DropContinuousQueryFn = func(database, name string) error { - return errors.New("marker") - } - - stmt := influxql.MustParseStatement(`DROP CONTINUOUS QUERY cq0 ON db0`) - if res := e.ExecuteStatement(stmt); res.Err == nil || res.Err.Error() != "marker" { - t.Fatalf("unexpected error: %s", res.Err) - } -} - -// Ensure a SHOW CONTINUOUS QUERIES statement can be executed. -func TestStatementExecutor_ExecuteStatement_ShowContinuousQueries(t *testing.T) { - e := NewStatementExecutor() - e.Store.DatabasesFn = func() ([]meta.DatabaseInfo, error) { - return []meta.DatabaseInfo{ - { - Name: "db0", - ContinuousQueries: []meta.ContinuousQueryInfo{ - {Name: "cq0", Query: "SELECT count(field1) INTO db1 FROM db0"}, - {Name: "cq1", Query: "SELECT count(field1) INTO db2 FROM db0"}, - }, - }, - { - Name: "db1", - ContinuousQueries: []meta.ContinuousQueryInfo{ - {Name: "cq2", Query: "SELECT count(field1) INTO db3 FROM db1"}, - }, - }, - }, nil - } - - stmt := influxql.MustParseStatement(`SHOW CONTINUOUS QUERIES`) - if res := e.ExecuteStatement(stmt); res.Err != nil { - t.Fatal(res.Err) - } else if !reflect.DeepEqual(res.Series, models.Rows{ - { - Name: "db0", - Columns: []string{"name", "query"}, - Values: [][]interface{}{ - {"cq0", "SELECT count(field1) INTO db1 FROM db0"}, - {"cq1", "SELECT count(field1) INTO db2 FROM db0"}, - }, - }, - { - Name: "db1", - Columns: []string{"name", "query"}, - Values: [][]interface{}{ - {"cq2", "SELECT count(field1) INTO db3 FROM db1"}, - }, - }, - }) { - t.Fatalf("unexpected rows: %s", spew.Sdump(res.Series)) - } -} - -// Ensure a SHOW CONTINUOUS QUERIES statement can return an error from the store. -func TestStatementExecutor_ExecuteStatement_ShowContinuousQueries_Err(t *testing.T) { - e := NewStatementExecutor() - e.Store.DatabasesFn = func() ([]meta.DatabaseInfo, error) { - return nil, errors.New("marker") - } - - stmt := influxql.MustParseStatement(`SHOW CONTINUOUS QUERIES`) - if res := e.ExecuteStatement(stmt); res.Err == nil || res.Err.Error() != "marker" { - t.Fatal(res.Err) - } -} - -// Ensure a CREATE SUBSCRIPTION statement can be executed. -func TestStatementExecutor_ExecuteStatement_CreateSubscription(t *testing.T) { - e := NewStatementExecutor() - e.Store.CreateSubscriptionFn = func(database, rp, name, mode string, destinations []string) error { - if database != "db0" { - t.Fatalf("unexpected database: %s", database) - } else if rp != "rp0" { - t.Fatalf("unexpected rp: %s", rp) - } else if name != "s0" { - t.Fatalf("unexpected name: %s", name) - } else if mode != "ANY" { - t.Fatalf("unexpected mode: %s", mode) - } else if len(destinations) != 2 { - t.Fatalf("unexpected destinations: %s", destinations) - } else if destinations[0] != "udp://h0:1234" { - t.Fatalf("unexpected destinations[0]: %s", destinations[0]) - } else if destinations[1] != "udp://h1:1234" { - t.Fatalf("unexpected destinations[1]: %s", destinations[1]) - } - return nil - } - - stmt := influxql.MustParseStatement(`CREATE SUBSCRIPTION s0 ON db0.rp0 DESTINATIONS ANY 'udp://h0:1234', 'udp://h1:1234'`) - if res := e.ExecuteStatement(stmt); res.Err != nil { - t.Fatal(res.Err) - } else if res.Series != nil { - t.Fatalf("unexpected rows: %#v", res.Series) - } -} - -// Ensure a CREATE SUBSCRIPTION statement can return an error from the store. -func TestStatementExecutor_ExecuteStatement_CreateSubscription_Err(t *testing.T) { - e := NewStatementExecutor() - e.Store.CreateSubscriptionFn = func(database, rp, name, mode string, destinations []string) error { - return errors.New("marker") - } - - stmt := influxql.MustParseStatement(`CREATE SUBSCRIPTION s0 ON db0.rp0 DESTINATIONS ANY 'udp://h0:1234', 'udp://h1:1234'`) - if res := e.ExecuteStatement(stmt); res.Err == nil || res.Err.Error() != "marker" { - t.Fatalf("unexpected error: %s", res.Err) - } -} - -// Ensure a DROP SUBSCRIPTION statement can be executed. -func TestStatementExecutor_ExecuteStatement_DropSubscription(t *testing.T) { - e := NewStatementExecutor() - e.Store.DropSubscriptionFn = func(database, rp, name string) error { - if database != "db0" { - t.Fatalf("unexpected database: %s", database) - } else if rp != "rp0" { - t.Fatalf("unexpected rp: %s", rp) - } else if name != "s0" { - t.Fatalf("unexpected name: %s", name) - } - return nil - } - - stmt := influxql.MustParseStatement(`DROP SUBSCRIPTION s0 ON db0.rp0`) - if res := e.ExecuteStatement(stmt); res.Err != nil { - t.Fatal(res.Err) - } else if res.Series != nil { - t.Fatalf("unexpected rows: %#v", res.Series) - } -} - -// Ensure a DROP SUBSCRIPTION statement can return an error from the store. -func TestStatementExecutor_ExecuteStatement_DropSubscription_Err(t *testing.T) { - e := NewStatementExecutor() - e.Store.DropSubscriptionFn = func(database, rp, name string) error { - return errors.New("marker") - } - - stmt := influxql.MustParseStatement(`DROP SUBSCRIPTION s0 ON db0.rp0`) - if res := e.ExecuteStatement(stmt); res.Err == nil || res.Err.Error() != "marker" { - t.Fatalf("unexpected error: %s", res.Err) - } -} - -// Ensure a SHOW SUBSCRIPTIONS statement can be executed. -func TestStatementExecutor_ExecuteStatement_ShowSubscriptions(t *testing.T) { - e := NewStatementExecutor() - e.Store.DatabasesFn = func() ([]meta.DatabaseInfo, error) { - return []meta.DatabaseInfo{ - { - Name: "db0", - RetentionPolicies: []meta.RetentionPolicyInfo{ - { - Name: "rp0", - Subscriptions: []meta.SubscriptionInfo{ - {Name: "s0", Mode: "ALL", Destinations: []string{"udp://h0:1234", "udp://h1:1234"}}, - {Name: "s1", Mode: "ANY", Destinations: []string{"udp://h2:1234", "udp://h3:1234"}}, - }, - }, - { - Name: "rp1", - Subscriptions: []meta.SubscriptionInfo{ - {Name: "s2", Mode: "ALL", Destinations: []string{"udp://h4:1234", "udp://h5:1234"}}, - }, - }, - }, - }, - { - Name: "db1", - RetentionPolicies: []meta.RetentionPolicyInfo{ - { - Name: "rp2", - Subscriptions: []meta.SubscriptionInfo{ - {Name: "s3", Mode: "ANY", Destinations: []string{"udp://h6:1234", "udp://h7:1234"}}, - }, - }, - }, - }, - }, nil - } - - stmt := influxql.MustParseStatement(`SHOW SUBSCRIPTIONS`) - if res := e.ExecuteStatement(stmt); res.Err != nil { - t.Fatal(res.Err) - } else if !reflect.DeepEqual(res.Series, models.Rows{ - { - Name: "db0", - Columns: []string{"retention_policy", "name", "mode", "destinations"}, - Values: [][]interface{}{ - {"rp0", "s0", "ALL", []string{"udp://h0:1234", "udp://h1:1234"}}, - {"rp0", "s1", "ANY", []string{"udp://h2:1234", "udp://h3:1234"}}, - {"rp1", "s2", "ALL", []string{"udp://h4:1234", "udp://h5:1234"}}, - }, - }, - { - Name: "db1", - Columns: []string{"retention_policy", "name", "mode", "destinations"}, - Values: [][]interface{}{ - {"rp2", "s3", "ANY", []string{"udp://h6:1234", "udp://h7:1234"}}, - }, - }, - }) { - t.Fatalf("unexpected rows: %s", spew.Sdump(res.Series)) - } -} - -// Ensure a SHOW SUBSCRIPTIONS statement can return an error from the store. -func TestStatementExecutor_ExecuteStatement_ShowSubscriptions_Err(t *testing.T) { - e := NewStatementExecutor() - e.Store.DatabasesFn = func() ([]meta.DatabaseInfo, error) { - return nil, errors.New("marker") - } - - stmt := influxql.MustParseStatement(`SHOW SUBSCRIPTIONS`) - if res := e.ExecuteStatement(stmt); res.Err == nil || res.Err.Error() != "marker" { - t.Fatal(res.Err) - } -} - -// Ensure that executing an unsupported statement will panic. -func TestStatementExecutor_ExecuteStatement_Unsupported(t *testing.T) { - var panicked bool - func() { - defer func() { - if r := recover(); r != nil { - panicked = true - } - }() - - // Execute a SELECT statement. - NewStatementExecutor().ExecuteStatement( - influxql.MustParseStatement(`SELECT count(field1) FROM db0`), - ) - }() - - // Ensure that the executor panicked. - if !panicked { - t.Fatal("executor did not panic") - } -} - -// Ensure a SHOW SHARD GROUPS statement can be executed. -func TestStatementExecutor_ExecuteStatement_ShowShardGroups(t *testing.T) { - e := NewStatementExecutor() - e.Store.DatabasesFn = func() ([]meta.DatabaseInfo, error) { - return []meta.DatabaseInfo{ - { - Name: "foo", - RetentionPolicies: []meta.RetentionPolicyInfo{ - { - Name: "rpi_foo", - Duration: time.Second, - ShardGroups: []meta.ShardGroupInfo{ - { - ID: 66, - StartTime: time.Unix(0, 0), - EndTime: time.Unix(1, 0), - }, - }, - }, - }, - }, - { - Name: "foo", - RetentionPolicies: []meta.RetentionPolicyInfo{ - { - Name: "rpi_foo", - Duration: time.Second, - ShardGroups: []meta.ShardGroupInfo{ - { - ID: 77, - StartTime: time.Unix(2, 0), - EndTime: time.Unix(3, 0), - }, - }, - }, - }, - }, - }, nil - } - - if res := e.ExecuteStatement(influxql.MustParseStatement(`SHOW SHARD GROUPS`)); res.Err != nil { - t.Fatal(res.Err) - } else if !reflect.DeepEqual(res.Series, models.Rows{ - { - Name: "shard groups", - Columns: []string{"id", "database", "retention_policy", "start_time", "end_time", "expiry_time"}, - Values: [][]interface{}{ - {uint64(66), "foo", "rpi_foo", "1970-01-01T00:00:00Z", "1970-01-01T00:00:01Z", "1970-01-01T00:00:02Z"}, - {uint64(77), "foo", "rpi_foo", "1970-01-01T00:00:02Z", "1970-01-01T00:00:03Z", "1970-01-01T00:00:04Z"}, - }, - }, - }) { - t.Fatalf("unexpected rows: %s", spew.Sdump(res.Series)) - } -} - -// Ensure a SHOW SHARDS statement can be executed. -func TestStatementExecutor_ExecuteStatement_ShowShards(t *testing.T) { - e := NewStatementExecutor() - e.Store.DatabasesFn = func() ([]meta.DatabaseInfo, error) { - return []meta.DatabaseInfo{ - { - Name: "foo", - RetentionPolicies: []meta.RetentionPolicyInfo{ - { - Name: "rpi_foo", - Duration: time.Second, - ShardGroups: []meta.ShardGroupInfo{ - { - ID: 66, - StartTime: time.Unix(0, 0), - EndTime: time.Unix(1, 0), - Shards: []meta.ShardInfo{ - { - ID: 1, - Owners: []meta.ShardOwner{ - {NodeID: 1}, - {NodeID: 2}, - {NodeID: 3}, - }, - }, - { - ID: 2, - }, - }, - }, - }, - }, - }, - }, - }, nil - } - - if res := e.ExecuteStatement(influxql.MustParseStatement(`SHOW SHARDS`)); res.Err != nil { - t.Fatal(res.Err) - } else if !reflect.DeepEqual(res.Series, models.Rows{ - { - Name: "foo", - Columns: []string{"id", "database", "retention_policy", "shard_group", "start_time", "end_time", "expiry_time", "owners"}, - Values: [][]interface{}{ - {uint64(1), "foo", "rpi_foo", uint64(66), "1970-01-01T00:00:00Z", "1970-01-01T00:00:01Z", "1970-01-01T00:00:02Z", "1,2,3"}, - {uint64(2), "foo", "rpi_foo", uint64(66), "1970-01-01T00:00:00Z", "1970-01-01T00:00:01Z", "1970-01-01T00:00:02Z", ""}, - }, - }, - }) { - t.Fatalf("unexpected rows: %s", spew.Sdump(res.Series)) - } -} - -// StatementExecutor represents a test wrapper for meta.StatementExecutor. -type StatementExecutor struct { - *meta.StatementExecutor - Store StatementExecutorStore -} - -// NewStatementExecutor returns a new instance of StatementExecutor with a mock store. -func NewStatementExecutor() *StatementExecutor { - e := &StatementExecutor{} - e.StatementExecutor = &meta.StatementExecutor{Store: &e.Store} - return e -} - -// StatementExecutorStore represents a mock implementation of StatementExecutor.Store. -type StatementExecutorStore struct { - NodeFn func(id uint64) (*meta.NodeInfo, error) - NodesFn func() ([]meta.NodeInfo, error) - PeersFn func() ([]string, error) - LeaderFn func() string - DatabaseFn func(name string) (*meta.DatabaseInfo, error) - DatabasesFn func() ([]meta.DatabaseInfo, error) - CreateDatabaseFn func(name string) (*meta.DatabaseInfo, error) - CreateDatabaseWithRetentionPolicyFn func(name string, rpi *meta.RetentionPolicyInfo) (*meta.DatabaseInfo, error) - DropDatabaseFn func(name string) error - DeleteDataNodeFn func(nodeID uint64) error - DeleteMetaNodeFn func(nodeID uint64) error - DefaultRetentionPolicyFn func(database string) (*meta.RetentionPolicyInfo, error) - CreateRetentionPolicyFn func(database string, rpi *meta.RetentionPolicyInfo) (*meta.RetentionPolicyInfo, error) - UpdateRetentionPolicyFn func(database, name string, rpu *meta.RetentionPolicyUpdate) error - SetDefaultRetentionPolicyFn func(database, name string) error - DropRetentionPolicyFn func(database, name string) error - UsersFn func() ([]meta.UserInfo, error) - CreateUserFn func(name, password string, admin bool) (*meta.UserInfo, error) - UpdateUserFn func(name, password string) error - DropUserFn func(name string) error - SetPrivilegeFn func(username, database string, p influxql.Privilege) error - SetAdminPrivilegeFn func(username string, admin bool) error - UserPrivilegesFn func(username string) (map[string]influxql.Privilege, error) - UserPrivilegeFn func(username, database string) (*influxql.Privilege, error) - ContinuousQueriesFn func() ([]meta.ContinuousQueryInfo, error) - CreateContinuousQueryFn func(database, name, query string) error - DropContinuousQueryFn func(database, name string) error - CreateSubscriptionFn func(database, rp, name, typ string, hosts []string) error - DropSubscriptionFn func(database, rp, name string) error -} - -func (s *StatementExecutorStore) DataNode(id uint64) (*meta.NodeInfo, error) { - return s.NodeFn(id) -} - -func (s *StatementExecutorStore) DataNodes() ([]meta.NodeInfo, error) { - return s.NodesFn() -} - -func (s *StatementExecutorStore) MetaNodes() ([]meta.NodeInfo, error) { - return s.NodesFn() -} - -func (s *StatementExecutorStore) DeleteDataNode(nodeID uint64) error { - return s.DeleteDataNodeFn(nodeID) -} - -func (s *StatementExecutorStore) DeleteMetaNode(nodeID uint64) error { - return s.DeleteMetaNodeFn(nodeID) -} - -func (s *StatementExecutorStore) Database(name string) (*meta.DatabaseInfo, error) { - return s.DatabaseFn(name) -} - -func (s *StatementExecutorStore) Databases() ([]meta.DatabaseInfo, error) { - return s.DatabasesFn() -} - -func (s *StatementExecutorStore) CreateDatabase(name string) (*meta.DatabaseInfo, error) { - return s.CreateDatabaseFn(name) -} - -func (s *StatementExecutorStore) CreateDatabaseWithRetentionPolicy(name string, rpi *meta.RetentionPolicyInfo) (*meta.DatabaseInfo, error) { - return s.CreateDatabaseWithRetentionPolicy(name, rpi) -} - -func (s *StatementExecutorStore) DropDatabase(name string) error { - return s.DropDatabaseFn(name) -} - -func (s *StatementExecutorStore) DefaultRetentionPolicy(database string) (*meta.RetentionPolicyInfo, error) { - return s.DefaultRetentionPolicyFn(database) -} - -func (s *StatementExecutorStore) CreateRetentionPolicy(database string, rpi *meta.RetentionPolicyInfo) (*meta.RetentionPolicyInfo, error) { - return s.CreateRetentionPolicyFn(database, rpi) -} - -func (s *StatementExecutorStore) UpdateRetentionPolicy(database, name string, rpu *meta.RetentionPolicyUpdate) error { - return s.UpdateRetentionPolicyFn(database, name, rpu) -} - -func (s *StatementExecutorStore) SetDefaultRetentionPolicy(database, name string) error { - return s.SetDefaultRetentionPolicyFn(database, name) -} - -func (s *StatementExecutorStore) DropRetentionPolicy(database, name string) error { - return s.DropRetentionPolicyFn(database, name) -} - -func (s *StatementExecutorStore) Users() ([]meta.UserInfo, error) { - return s.UsersFn() -} - -func (s *StatementExecutorStore) CreateUser(name, password string, admin bool) (*meta.UserInfo, error) { - return s.CreateUserFn(name, password, admin) -} - -func (s *StatementExecutorStore) UpdateUser(name, password string) error { - return s.UpdateUserFn(name, password) -} - -func (s *StatementExecutorStore) DropUser(name string) error { - return s.DropUserFn(name) -} - -func (s *StatementExecutorStore) SetPrivilege(username, database string, p influxql.Privilege) error { - return s.SetPrivilegeFn(username, database, p) -} - -func (s *StatementExecutorStore) SetAdminPrivilege(username string, admin bool) error { - return s.SetAdminPrivilegeFn(username, admin) -} - -func (s *StatementExecutorStore) UserPrivileges(username string) (map[string]influxql.Privilege, error) { - return s.UserPrivilegesFn(username) -} - -func (s *StatementExecutorStore) UserPrivilege(username, database string) (*influxql.Privilege, error) { - return s.UserPrivilegeFn(username, database) -} - -func (s *StatementExecutorStore) ContinuousQueries() ([]meta.ContinuousQueryInfo, error) { - return s.ContinuousQueriesFn() -} - -func (s *StatementExecutorStore) CreateContinuousQuery(database, name, query string) error { - return s.CreateContinuousQueryFn(database, name, query) -} - -func (s *StatementExecutorStore) DropContinuousQuery(database, name string) error { - return s.DropContinuousQueryFn(database, name) -} - -func (s *StatementExecutorStore) CreateSubscription(database, rp, name, typ string, hosts []string) error { - return s.CreateSubscriptionFn(database, rp, name, typ, hosts) -} - -func (s *StatementExecutorStore) DropSubscription(database, rp, name string) error { - return s.DropSubscriptionFn(database, rp, name) -} diff --git a/services/meta/store.go b/services/meta/store.go index c9f511e29c3..7c8c859f29b 100644 --- a/services/meta/store.go +++ b/services/meta/store.go @@ -82,7 +82,9 @@ func (s *store) open(raftln net.Listener) error { var initializePeers []string if len(joinPeers) > 0 { - c := NewClient(joinPeers, s.config.HTTPSEnabled) + c := NewClient() + c.SetMetaServers(joinPeers) + c.SetTLS(s.config.HTTPSEnabled) for { peers := c.peers() if !Peers(peers).Contains(s.raftAddr) { @@ -117,7 +119,9 @@ func (s *store) open(raftln net.Listener) error { } if len(joinPeers) > 0 { - c := NewClient(joinPeers, s.config.HTTPSEnabled) + c := NewClient() + c.SetMetaServers(joinPeers) + c.SetTLS(s.config.HTTPSEnabled) if err := c.Open(); err != nil { return err } diff --git a/tsdb/query_executor.go b/tsdb/query_executor.go deleted file mode 100644 index d5e87fdc54f..00000000000 --- a/tsdb/query_executor.go +++ /dev/null @@ -1,1086 +0,0 @@ -package tsdb - -import ( - "errors" - "fmt" - "log" - "os" - "sort" - "time" - - "github.com/influxdata/influxdb/influxql" - "github.com/influxdata/influxdb/models" - "github.com/influxdata/influxdb/services/meta" -) - -// QueryExecutor executes every statement in an influxdb Query. It is responsible for -// coordinating between the local influxql.Store, the meta.Store, and the other nodes in -// the cluster to run the query against their local tsdb.Stores. There should be one executor -// in a running process -type QueryExecutor struct { - // Local data store. - Store interface { - DatabaseIndex(name string) *DatabaseIndex - Shards(ids []uint64) []*Shard - ExpandSources(sources influxql.Sources) (influxql.Sources, error) - DeleteDatabase(name string, shardIDs []uint64) error - DeleteMeasurement(database, name string) error - DeleteSeries(database string, seriesKeys []string) error - } - - // The meta store for accessing and updating cluster and schema data. - MetaClient interface { - Database(name string) (*meta.DatabaseInfo, error) - Databases() ([]meta.DatabaseInfo, error) - User(name string) (*meta.UserInfo, error) - AdminUserExists() bool - Authenticate(username, password string) (*meta.UserInfo, error) - RetentionPolicy(database, name string) (rpi *meta.RetentionPolicyInfo, err error) - UserCount() int - ShardIDsByTimeRange(sources influxql.Sources, tmin, tmax time.Time) (a []uint64, err error) - ExecuteStatement(stmt influxql.Statement) *influxql.Result - } - - // Execute statements relating to statistics and diagnostics. - MonitorStatementExecutor interface { - ExecuteStatement(stmt influxql.Statement) *influxql.Result - } - - IntoWriter interface { - WritePointsInto(p *IntoWriteRequest) error - } - - Logger *log.Logger - QueryLogEnabled bool -} - -// IntoWriteRequest is a partial copy of cluster.WriteRequest -type IntoWriteRequest struct { - Database string - RetentionPolicy string - Points []models.Point -} - -// NewQueryExecutor returns a new instance of QueryExecutor. -func NewQueryExecutor() *QueryExecutor { - return &QueryExecutor{ - Logger: log.New(os.Stderr, "[query] ", log.LstdFlags), - } -} - -// SetLogger sets the internal logger to the logger passed in. -func (q *QueryExecutor) SetLogger(l *log.Logger) { - q.Logger = l -} - -// Authorize user u to execute query q on database. -// database can be "" for queries that do not require a database. -// If no user is provided it will return an error unless the query's first statement is to create -// a root user. -func (q *QueryExecutor) Authorize(u *meta.UserInfo, query *influxql.Query, database string) error { - // Special case if no users exist. - if n := q.MetaClient.UserCount(); n == 0 { - // Ensure there is at least one statement. - if len(query.Statements) > 0 { - // First statement in the query must create a user with admin privilege. - cu, ok := query.Statements[0].(*influxql.CreateUserStatement) - if ok && cu.Admin == true { - return nil - } - } - return NewErrAuthorize(q, query, "", database, "create admin user first or disable authentication") - } - - if u == nil { - return NewErrAuthorize(q, query, "", database, "no user provided") - } - - // Admin privilege allows the user to execute all statements. - if u.Admin { - return nil - } - - // Check each statement in the query. - for _, stmt := range query.Statements { - // Get the privileges required to execute the statement. - privs := stmt.RequiredPrivileges() - - // Make sure the user has the privileges required to execute - // each statement. - for _, p := range privs { - if p.Admin { - // Admin privilege already checked so statement requiring admin - // privilege cannot be run. - msg := fmt.Sprintf("statement '%s', requires admin privilege", stmt) - return NewErrAuthorize(q, query, u.Name, database, msg) - } - - // Use the db name specified by the statement or the db - // name passed by the caller if one wasn't specified by - // the statement. - db := p.Name - if db == "" { - db = database - } - if !u.Authorize(p.Privilege, db) { - msg := fmt.Sprintf("statement '%s', requires %s on %s", stmt, p.Privilege.String(), db) - return NewErrAuthorize(q, query, u.Name, database, msg) - } - } - } - return nil -} - -// ExecuteQuery executes an InfluxQL query against the server. -// It sends results down the passed in chan and closes it when done. It will close the chan -// on the first statement that throws an error. -func (q *QueryExecutor) ExecuteQuery(query *influxql.Query, database string, chunkSize int, closing chan struct{}) (<-chan *influxql.Result, error) { - // Execute each statement. Keep the iterator external so we can - // track how many of the statements were executed - results := make(chan *influxql.Result) - - go func() { - defer close(results) - - var i int - var stmt influxql.Statement - for i, stmt = range query.Statements { - // If a default database wasn't passed in by the caller, check the statement. - // Some types of statements have an associated default database, even if it - // is not explicitly included. - defaultDB := database - if defaultDB == "" { - if s, ok := stmt.(influxql.HasDefaultDatabase); ok { - defaultDB = s.DefaultDatabase() - } - } - - // Normalize each statement. - if err := q.normalizeStatement(stmt, defaultDB); err != nil { - results <- &influxql.Result{Err: err} - break - } - - // Log each normalized statement. - if q.QueryLogEnabled { - q.Logger.Println(stmt.String()) - } - - var res *influxql.Result - switch stmt := stmt.(type) { - case *influxql.SelectStatement: - if err := q.executeStatement(i, stmt, database, results, chunkSize, closing); err != nil { - results <- &influxql.Result{Err: err} - break - } - case *influxql.DropSeriesStatement: - // TODO: handle this in a cluster - res = q.executeDropSeriesStatement(stmt, database) - case *influxql.ShowSeriesStatement: - res = q.executeShowSeriesStatement(stmt, database) - case *influxql.DropMeasurementStatement: - // TODO: handle this in a cluster - res = q.executeDropMeasurementStatement(stmt, database) - case *influxql.ShowMeasurementsStatement: - if err := q.executeStatement(i, stmt, database, results, chunkSize, closing); err != nil { - results <- &influxql.Result{Err: err} - break - } - case *influxql.ShowTagKeysStatement: - if err := q.executeStatement(i, stmt, database, results, chunkSize, closing); err != nil { - results <- &influxql.Result{Err: err} - break - } - case *influxql.ShowTagValuesStatement: - res = q.executeShowTagValuesStatement(stmt, database) - case *influxql.ShowFieldKeysStatement: - res = q.executeShowFieldKeysStatement(stmt, database) - case *influxql.DeleteStatement: - res = &influxql.Result{Err: ErrInvalidQuery} - case *influxql.DropDatabaseStatement: - // TODO: handle this in a cluster - res = q.executeDropDatabaseStatement(stmt) - case *influxql.ShowStatsStatement, *influxql.ShowDiagnosticsStatement: - // Send monitor-related queries to the monitor service. - res = q.MonitorStatementExecutor.ExecuteStatement(stmt) - default: - // Delegate all other meta statements to a separate executor. They don't hit tsdb storage. - res = q.MetaClient.ExecuteStatement(stmt) - } - - if res != nil { - // set the StatementID for the handler on the other side to combine results - res.StatementID = i - - // If an error occurs then stop processing remaining statements. - results <- res - if res.Err != nil { - break - } - } - } - - // if there was an error send results that the remaining statements weren't executed - for ; i < len(query.Statements)-1; i++ { - results <- &influxql.Result{Err: ErrNotExecuted} - } - }() - - return results, nil -} - -// PlanSelect creates an execution plan for the given SelectStatement and returns an Executor. -func (q *QueryExecutor) PlanSelect(stmt *influxql.SelectStatement, chunkSize int) (Executor, error) { - // It is important to "stamp" this time so that everywhere we evaluate `now()` in the statement is EXACTLY the same `now` - now := time.Now().UTC() - opt := influxql.SelectOptions{} - - // Replace instances of "now()" with the current time, and check the resultant times. - stmt.Condition = influxql.Reduce(stmt.Condition, &influxql.NowValuer{Now: now}) - opt.MinTime, opt.MaxTime = influxql.TimeRange(stmt.Condition) - if opt.MaxTime.IsZero() { - opt.MaxTime = now - } - if opt.MinTime.IsZero() { - opt.MinTime = time.Unix(0, 0) - } - - // Expand regex sources to their actual source names. - sources, err := q.Store.ExpandSources(stmt.Sources) - if err != nil { - return nil, err - } - stmt.Sources = sources - - // Convert DISTINCT into a call. - stmt.RewriteDistinct() - - // Remove "time" from fields list. - stmt.RewriteTimeFields() - - // Filter only shards that contain date range. - shardIDs, err := q.MetaClient.ShardIDsByTimeRange(stmt.Sources, opt.MinTime, opt.MaxTime) - if err != nil { - return nil, err - } - shards := q.Store.Shards(shardIDs) - - // Rewrite wildcards, if any exist. - tmp, err := stmt.RewriteWildcards(Shards(shards)) - if err != nil { - return nil, err - } - stmt = tmp - - // Create a set of iterators from a selection. - itrs, err := influxql.Select(stmt, Shards(shards), &opt) - if err != nil { - return nil, err - } - - // Generate a row emitter from the iterator set. - em := influxql.NewEmitter(itrs, stmt.TimeAscending()) - em.Columns = stmt.ColumnNames() - em.OmitTime = stmt.OmitTime - - // Wrap emitter in an adapter to conform to the Executor interface. - return (*emitterExecutor)(em), nil -} - -// executeDropDatabaseStatement closes all local shards for the database and removes the directory. It then calls to the metastore to remove the database from there. -// TODO: make this work in a cluster/distributed -func (q *QueryExecutor) executeDropDatabaseStatement(stmt *influxql.DropDatabaseStatement) *influxql.Result { - dbi, err := q.MetaClient.Database(stmt.Name) - if err != nil { - return &influxql.Result{Err: err} - } else if dbi == nil { - if stmt.IfExists { - return &influxql.Result{} - } - return &influxql.Result{Err: ErrDatabaseNotFound(stmt.Name)} - } - - var shardIDs []uint64 - for _, rp := range dbi.RetentionPolicies { - for _, sg := range rp.ShardGroups { - for _, s := range sg.Shards { - shardIDs = append(shardIDs, s.ID) - } - } - } - - // Remove database from meta-store first so that in-flight writes can complete without error, but new ones will - // be rejected. - res := q.MetaClient.ExecuteStatement(stmt) - - // Remove the database from the local store - err = q.Store.DeleteDatabase(stmt.Name, shardIDs) - if err != nil { - return &influxql.Result{Err: err} - } - - return res -} - -// executeDropMeasurementStatement removes the measurement and all series data from the local store for the given measurement -func (q *QueryExecutor) executeDropMeasurementStatement(stmt *influxql.DropMeasurementStatement, database string) *influxql.Result { - if err := q.Store.DeleteMeasurement(database, stmt.Name); err != nil { - return &influxql.Result{Err: err} - } - return &influxql.Result{} -} - -// executeDropSeriesStatement removes all series from the local store that match the drop query -func (q *QueryExecutor) executeDropSeriesStatement(stmt *influxql.DropSeriesStatement, database string) *influxql.Result { - // Check for time in WHERE clause (not supported). - if influxql.HasTimeExpr(stmt.Condition) { - return &influxql.Result{Err: errors.New("DROP SERIES doesn't support time in WHERE clause")} - } - - // Find the database. - db := q.Store.DatabaseIndex(database) - if db == nil { - return &influxql.Result{} - } - - // Expand regex expressions in the FROM clause. - sources, err := q.Store.ExpandSources(stmt.Sources) - if err != nil { - return &influxql.Result{Err: err} - } else if stmt.Sources != nil && len(stmt.Sources) != 0 && len(sources) == 0 { - return &influxql.Result{} - } - - measurements, err := measurementsFromSourcesOrDB(db, sources...) - if err != nil { - return &influxql.Result{Err: err} - } - - var seriesKeys []string - for _, m := range measurements { - var ids SeriesIDs - var filters FilterExprs - if stmt.Condition != nil { - // Get series IDs that match the WHERE clause. - ids, filters, err = m.walkWhereForSeriesIds(stmt.Condition) - if err != nil { - return &influxql.Result{Err: err} - } - - // Delete boolean literal true filter expressions. - // These are returned for `WHERE tagKey = 'tagVal'` type expressions and are okay. - filters.DeleteBoolLiteralTrues() - - // Check for unsupported field filters. - // Any remaining filters means there were fields (e.g., `WHERE value = 1.2`). - if filters.Len() > 0 { - return &influxql.Result{Err: errors.New("DROP SERIES doesn't support fields in WHERE clause")} - } - } else { - // No WHERE clause so get all series IDs for this measurement. - ids = m.seriesIDs - } - - for _, id := range ids { - seriesKeys = append(seriesKeys, m.seriesByID[id].Key) - } - } - - // delete the raw series data - if err := q.Store.DeleteSeries(database, seriesKeys); err != nil { - return &influxql.Result{Err: err} - } - // remove them from the index - db.DropSeries(seriesKeys) - - return &influxql.Result{} -} - -func (q *QueryExecutor) executeShowSeriesStatement(stmt *influxql.ShowSeriesStatement, database string) *influxql.Result { - // Check for time in WHERE clause (not supported). - if influxql.HasTimeExpr(stmt.Condition) { - return &influxql.Result{Err: errors.New("SHOW SERIES doesn't support time in WHERE clause")} - } - - // Find the database. - db := q.Store.DatabaseIndex(database) - if db == nil { - return &influxql.Result{} - } - - // Expand regex expressions in the FROM clause. - sources, err := q.Store.ExpandSources(stmt.Sources) - if err != nil { - return &influxql.Result{Err: err} - } - - // Get the list of measurements we're interested in. - measurements, err := measurementsFromSourcesOrDB(db, sources...) - if err != nil { - return &influxql.Result{Err: err} - } - - // Create result struct that will be populated and returned. - result := &influxql.Result{ - Series: make(models.Rows, 0, len(measurements)), - } - - // Loop through measurements to build result. One result row / measurement. - for _, m := range measurements { - var ids SeriesIDs - var filters FilterExprs - - if stmt.Condition != nil { - // Get series IDs that match the WHERE clause. - ids, filters, err = m.walkWhereForSeriesIds(stmt.Condition) - if err != nil { - return &influxql.Result{Err: err} - } - - // Delete boolean literal true filter expressions. - filters.DeleteBoolLiteralTrues() - - // Check for unsupported field filters. - if filters.Len() > 0 { - return &influxql.Result{Err: errors.New("SHOW SERIES doesn't support fields in WHERE clause")} - } - - // If no series matched, then go to the next measurement. - if len(ids) == 0 { - continue - } - } else { - // No WHERE clause so get all series IDs for this measurement. - ids = m.seriesIDs - } - - // Make a new row for this measurement. - r := &models.Row{ - Name: m.Name, - Columns: m.TagKeys(), - } - - // Loop through series IDs getting matching tag sets. - for _, id := range ids { - if s, ok := m.seriesByID[id]; ok { - values := make([]interface{}, 0, len(r.Columns)) - - // make the series key the first value - values = append(values, s.Key) - - for _, column := range r.Columns { - values = append(values, s.Tags[column]) - } - - // Add the tag values to the row. - r.Values = append(r.Values, values) - } - } - // make the id the first column - r.Columns = append([]string{"_key"}, r.Columns...) - - // Append the row to the result. - result.Series = append(result.Series, r) - } - - if stmt.Limit > 0 || stmt.Offset > 0 { - result.Series = q.filterShowSeriesResult(stmt.Limit, stmt.Offset, result.Series) - } - - return result -} - -// filterShowSeriesResult will limit the number of series returned based on the limit and the offset. -// Unlike limit and offset on SELECT statements, the limit and offset don't apply to the number of Rows, but -// to the number of total Values returned, since each Value represents a unique series. -func (q *QueryExecutor) filterShowSeriesResult(limit, offset int, rows models.Rows) models.Rows { - var filteredSeries models.Rows - seriesCount := 0 - for _, r := range rows { - var currentSeries [][]interface{} - - // filter the values - for _, v := range r.Values { - if seriesCount >= offset && seriesCount-offset < limit { - currentSeries = append(currentSeries, v) - } - seriesCount++ - } - - // only add the row back in if there are some values in it - if len(currentSeries) > 0 { - r.Values = currentSeries - filteredSeries = append(filteredSeries, r) - if seriesCount > limit+offset { - return filteredSeries - } - } - } - return filteredSeries -} - -func (q *QueryExecutor) planStatement(stmt influxql.Statement, database string, chunkSize int) (Executor, error) { - switch stmt := stmt.(type) { - case *influxql.SelectStatement: - return q.PlanSelect(stmt, chunkSize) - case *influxql.ShowMeasurementsStatement: - return q.planShowMeasurements(stmt, database, chunkSize) - case *influxql.ShowTagKeysStatement: - return q.planShowTagKeys(stmt, database, chunkSize) - default: - return nil, fmt.Errorf("can't plan statement type: %v", stmt) - } -} - -// planShowMeasurements converts the statement to a SELECT and executes it. -func (q *QueryExecutor) planShowMeasurements(stmt *influxql.ShowMeasurementsStatement, database string, chunkSize int) (Executor, error) { - // Check for time in WHERE clause (not supported). - if influxql.HasTimeExpr(stmt.Condition) { - return nil, errors.New("SHOW MEASUREMENTS doesn't support time in WHERE clause") - } - - condition := stmt.Condition - if source, ok := stmt.Source.(*influxql.Measurement); ok { - var expr influxql.Expr - if source.Regex != nil { - expr = &influxql.BinaryExpr{ - Op: influxql.EQREGEX, - LHS: &influxql.VarRef{Val: "name"}, - RHS: &influxql.RegexLiteral{Val: source.Regex.Val}, - } - } else if source.Name != "" { - expr = &influxql.BinaryExpr{ - Op: influxql.EQ, - LHS: &influxql.VarRef{Val: "name"}, - RHS: &influxql.StringLiteral{Val: source.Name}, - } - } - - // Set condition or "AND" together. - if condition == nil { - condition = expr - } else { - condition = &influxql.BinaryExpr{Op: influxql.AND, LHS: expr, RHS: condition} - } - } - - ss := &influxql.SelectStatement{ - Fields: influxql.Fields([]*influxql.Field{ - {Expr: &influxql.VarRef{Val: "name"}}, - }), - Sources: influxql.Sources([]influxql.Source{ - &influxql.Measurement{Name: "_measurements"}, - }), - Condition: condition, - Offset: stmt.Offset, - Limit: stmt.Limit, - SortFields: stmt.SortFields, - OmitTime: true, - Dedupe: true, - } - - // Normalize the statement. - if err := q.normalizeStatement(ss, database); err != nil { - return nil, err - } - - return q.PlanSelect(ss, chunkSize) -} - -// planShowTagKeys creates an execution plan for a SHOW MEASUREMENTS statement and returns an Executor. -func (q *QueryExecutor) planShowTagKeys(stmt *influxql.ShowTagKeysStatement, database string, chunkSize int) (Executor, error) { - // Check for time in WHERE clause (not supported). - if influxql.HasTimeExpr(stmt.Condition) { - return nil, errors.New("SHOW TAG KEYS doesn't support time in WHERE clause") - } - - condition := stmt.Condition - if len(stmt.Sources) > 0 { - if source, ok := stmt.Sources[0].(*influxql.Measurement); ok { - var expr influxql.Expr - if source.Regex != nil { - expr = &influxql.BinaryExpr{ - Op: influxql.EQREGEX, - LHS: &influxql.VarRef{Val: "name"}, - RHS: &influxql.RegexLiteral{Val: source.Regex.Val}, - } - } else if source.Name != "" { - expr = &influxql.BinaryExpr{ - Op: influxql.EQ, - LHS: &influxql.VarRef{Val: "name"}, - RHS: &influxql.StringLiteral{Val: source.Name}, - } - } - - // Set condition or "AND" together. - if condition == nil { - condition = expr - } else { - condition = &influxql.BinaryExpr{Op: influxql.AND, LHS: expr, RHS: condition} - } - } - } - - ss := &influxql.SelectStatement{ - Fields: []*influxql.Field{ - {Expr: &influxql.VarRef{Val: "tagKey"}}, - }, - Sources: []influxql.Source{ - &influxql.Measurement{Name: "_tagKeys"}, - }, - Condition: condition, - Offset: stmt.Offset, - Limit: stmt.Limit, - SortFields: stmt.SortFields, - OmitTime: true, - Dedupe: true, - } - - // Normalize the statement. - if err := q.normalizeStatement(ss, database); err != nil { - return nil, err - } - - return q.PlanSelect(ss, chunkSize) -} - -func (q *QueryExecutor) executeStatement(statementID int, stmt influxql.Statement, database string, results chan *influxql.Result, chunkSize int, closing chan struct{}) error { - // Plan statement execution. - e, err := q.planStatement(stmt, database, chunkSize) - if err != nil { - return err - } - - // Execute plan. - ch := e.Execute(closing) - var writeerr error - var intoNum int64 - var isinto bool - // Stream results from the channel. We should send an empty result if nothing comes through. - resultSent := false - for row := range ch { - // We had a write error. Continue draining results from the channel - // so we don't hang the goroutine in the executor. - if writeerr != nil { - continue - } - if row.Err != nil { - return row.Err - } - selectstmt, ok := stmt.(*influxql.SelectStatement) - if ok && selectstmt.Target != nil { - isinto = true - // this is a into query. Write results back to database - writeerr = q.writeInto(row, selectstmt) - intoNum += int64(len(row.Values)) - } else { - resultSent = true - results <- &influxql.Result{StatementID: statementID, Series: []*models.Row{row}} - } - } - if writeerr != nil { - return writeerr - } else if isinto { - results <- &influxql.Result{ - StatementID: statementID, - Series: []*models.Row{{ - Name: "result", - // it seems weird to give a time here, but so much stuff breaks if you don't - Columns: []string{"time", "written"}, - Values: [][]interface{}{{ - time.Unix(0, 0).UTC(), - intoNum, - }}, - }}, - } - return nil - } - - if !resultSent { - results <- &influxql.Result{StatementID: statementID, Series: make([]*models.Row, 0)} - } - - return nil -} - -func (q *QueryExecutor) writeInto(row *models.Row, selectstmt *influxql.SelectStatement) error { - // It might seem a bit weird that this is where we do this, since we will have to - // convert rows back to points. The Executors (both aggregate and raw) are complex - // enough that changing them to write back to the DB is going to be clumsy - // - // it might seem weird to have the write be in the QueryExecutor, but the interweaving of - // limitedRowWriter and ExecuteAggregate/Raw makes it ridiculously hard to make sure that the - // results will be the same as when queried normally. - measurement := intoMeasurement(selectstmt) - if measurement == "" { - measurement = row.Name - } - intodb, err := intoDB(selectstmt) - if err != nil { - return err - } - rp := intoRP(selectstmt) - points, err := convertRowToPoints(measurement, row) - if err != nil { - return err - } - req := &IntoWriteRequest{ - Database: intodb, - RetentionPolicy: rp, - Points: points, - } - err = q.IntoWriter.WritePointsInto(req) - if err != nil { - return err - } - return nil -} - -func (q *QueryExecutor) executeShowTagValuesStatement(stmt *influxql.ShowTagValuesStatement, database string) *influxql.Result { - // Check for time in WHERE clause (not supported). - if influxql.HasTimeExpr(stmt.Condition) { - return &influxql.Result{Err: errors.New("SHOW TAG VALUES doesn't support time in WHERE clause")} - } - - // Find the database. - db := q.Store.DatabaseIndex(database) - if db == nil { - return &influxql.Result{} - } - - // Expand regex expressions in the FROM clause. - sources, err := q.Store.ExpandSources(stmt.Sources) - if err != nil { - return &influxql.Result{Err: err} - } - - // Get the list of measurements we're interested in. - measurements, err := measurementsFromSourcesOrDB(db, sources...) - if err != nil { - return &influxql.Result{Err: err} - } - - // Make result. - result := &influxql.Result{ - Series: make(models.Rows, 0), - } - - tagValues := make(map[string]stringSet) - for _, m := range measurements { - var ids SeriesIDs - - if stmt.Condition != nil { - // Get series IDs that match the WHERE clause. - ids, _, err = m.walkWhereForSeriesIds(stmt.Condition) - if err != nil { - return &influxql.Result{Err: err} - } - - // If no series matched, then go to the next measurement. - if len(ids) == 0 { - continue - } - - // TODO: check return of walkWhereForSeriesIds for fields - } else { - // No WHERE clause so get all series IDs for this measurement. - ids = m.seriesIDs - } - - for k, v := range m.tagValuesByKeyAndSeriesID(stmt.TagKeys, ids) { - _, ok := tagValues[k] - if !ok { - tagValues[k] = v - } - tagValues[k] = tagValues[k].union(v) - } - } - - for k, v := range tagValues { - r := &models.Row{ - Name: k + "TagValues", - Columns: []string{k}, - } - - vals := v.list() - sort.Strings(vals) - - for _, val := range vals { - v := interface{}(val) - r.Values = append(r.Values, []interface{}{v}) - } - - result.Series = append(result.Series, r) - } - - sort.Sort(result.Series) - return result -} - -func (q *QueryExecutor) executeShowFieldKeysStatement(stmt *influxql.ShowFieldKeysStatement, database string) *influxql.Result { - var err error - - // Find the database. - db := q.Store.DatabaseIndex(database) - if db == nil { - return &influxql.Result{} - } - - // Expand regex expressions in the FROM clause. - sources, err := q.Store.ExpandSources(stmt.Sources) - if err != nil { - return &influxql.Result{Err: err} - } - - measurements, err := measurementsFromSourcesOrDB(db, sources...) - if err != nil { - return &influxql.Result{Err: err} - } - - // Make result. - result := &influxql.Result{ - Series: make(models.Rows, 0, len(measurements)), - } - - // Loop through measurements, adding a result row for each. - for _, m := range measurements { - // Create a new row. - r := &models.Row{ - Name: m.Name, - Columns: []string{"fieldKey"}, - } - - // Get a list of field names from the measurement then sort them. - names := m.FieldNames() - sort.Strings(names) - - // Add the field names to the result row values. - for _, n := range names { - v := interface{}(n) - r.Values = append(r.Values, []interface{}{v}) - } - - // Append the row to the result. - result.Series = append(result.Series, r) - } - - return result -} - -// measurementsFromSourcesOrDB returns a list of measurements from the -// sources passed in or, if sources is empty, a list of all -// measurement names from the database passed in. -func measurementsFromSourcesOrDB(db *DatabaseIndex, sources ...influxql.Source) (Measurements, error) { - var measurements Measurements - if len(sources) > 0 { - for _, source := range sources { - if m, ok := source.(*influxql.Measurement); ok { - measurement := db.measurements[m.Name] - if measurement == nil { - continue - } - - measurements = append(measurements, measurement) - } else { - return nil, errors.New("identifiers in FROM clause must be measurement names") - } - } - } else { - // No measurements specified in FROM clause so get all measurements that have series. - for _, m := range db.Measurements() { - if m.HasSeries() { - measurements = append(measurements, m) - } - } - } - sort.Sort(measurements) - - return measurements, nil -} - -// normalizeStatement adds a default database and policy to the measurements in statement. -func (q *QueryExecutor) normalizeStatement(stmt influxql.Statement, defaultDatabase string) (err error) { - // Track prefixes for replacing field names. - prefixes := make(map[string]string) - - // Qualify all measurements. - influxql.WalkFunc(stmt, func(n influxql.Node) { - if err != nil { - return - } - switch n := n.(type) { - case *influxql.Measurement: - e := q.normalizeMeasurement(n, defaultDatabase) - if e != nil { - err = e - return - } - prefixes[n.Name] = n.Name - } - }) - return -} - -// normalizeMeasurement inserts the default database or policy into all measurement names, -// if required. -func (q *QueryExecutor) normalizeMeasurement(m *influxql.Measurement, defaultDatabase string) error { - // Targets (measurements in an INTO clause) can have blank names, which means it will be - // the same as the measurement name it came from in the FROM clause. - if !m.IsTarget && m.Name == "" && m.Regex == nil { - return errors.New("invalid measurement") - } - - // Measurement does not have an explicit database? Insert default. - if m.Database == "" { - m.Database = defaultDatabase - } - - // The database must now be specified by this point. - if m.Database == "" { - return errors.New("database name required") - } - - // Find database. - di, err := q.MetaClient.Database(m.Database) - if err != nil { - return err - } else if di == nil { - return ErrDatabaseNotFound(m.Database) - } - - // If no retention policy was specified, use the default. - if m.RetentionPolicy == "" { - if di.DefaultRetentionPolicy == "" { - return fmt.Errorf("default retention policy not set for: %s", di.Name) - } - m.RetentionPolicy = di.DefaultRetentionPolicy - } - - return nil -} - -// ErrAuthorize represents an authorization error. -type ErrAuthorize struct { - q *QueryExecutor - query *influxql.Query - user string - database string - message string -} - -const authErrLogFmt string = "unauthorized request | user: %q | query: %q | database %q\n" - -// NewErrAuthorize returns a new instance of AuthorizationError. -func NewErrAuthorize(qe *QueryExecutor, q *influxql.Query, u, db, m string) *ErrAuthorize { - return &ErrAuthorize{q: qe, query: q, user: u, database: db, message: m} -} - -// Error returns the text of the error. -func (e ErrAuthorize) Error() string { - e.q.Logger.Printf(authErrLogFmt, e.user, e.query.String(), e.database) - if e.user == "" { - return fmt.Sprint(e.message) - } - return fmt.Sprintf("%s not authorized to execute %s", e.user, e.message) -} - -var ( - // ErrInvalidQuery is returned when executing an unknown query type. - ErrInvalidQuery = errors.New("invalid query") - - // ErrNotExecuted is returned when a statement is not executed in a query. - // This can occur when a previous statement in the same query has errored. - ErrNotExecuted = errors.New("not executed") -) - -// ErrDatabaseNotFound returns a database not found error for the given database name. -func ErrDatabaseNotFound(name string) error { return fmt.Errorf("database not found: %s", name) } - -// ErrMeasurementNotFound returns a measurement not found error for the given measurement name. -func ErrMeasurementNotFound(name string) error { return fmt.Errorf("measurement not found: %s", name) } - -type uint64Slice []uint64 - -func (a uint64Slice) Len() int { return len(a) } -func (a uint64Slice) Swap(i, j int) { a[i], a[j] = a[j], a[i] } -func (a uint64Slice) Less(i, j int) bool { return a[i] < a[j] } - -// convertRowToPoints will convert a query result Row into Points that can be written back in. -// Used for INTO queries -func convertRowToPoints(measurementName string, row *models.Row) ([]models.Point, error) { - // figure out which parts of the result are the time and which are the fields - timeIndex := -1 - fieldIndexes := make(map[string]int) - for i, c := range row.Columns { - if c == "time" { - timeIndex = i - } else { - fieldIndexes[c] = i - } - } - - if timeIndex == -1 { - return nil, errors.New("error finding time index in result") - } - - points := make([]models.Point, 0, len(row.Values)) - for _, v := range row.Values { - vals := make(map[string]interface{}) - for fieldName, fieldIndex := range fieldIndexes { - val := v[fieldIndex] - if val != nil { - vals[fieldName] = v[fieldIndex] - } - } - - p, err := models.NewPoint(measurementName, row.Tags, vals, v[timeIndex].(time.Time)) - if err != nil { - // Drop points that can't be stored - continue - } - - points = append(points, p) - } - - return points, nil -} - -func intoDB(stmt *influxql.SelectStatement) (string, error) { - if stmt.Target.Measurement.Database != "" { - return stmt.Target.Measurement.Database, nil - } - return "", errNoDatabaseInTarget -} - -var errNoDatabaseInTarget = errors.New("no database in target") - -func intoRP(stmt *influxql.SelectStatement) string { return stmt.Target.Measurement.RetentionPolicy } -func intoMeasurement(stmt *influxql.SelectStatement) string { return stmt.Target.Measurement.Name } - -// emitterExecutor represents an adapter for emitters to implement Executor. -type emitterExecutor influxql.Emitter - -func (e *emitterExecutor) Execute(closing <-chan struct{}) <-chan *models.Row { - out := make(chan *models.Row, 0) - go e.execute(out, closing) - return out -} - -func (e *emitterExecutor) execute(out chan *models.Row, closing <-chan struct{}) { - defer close(out) - - // Continually read rows from emitter until no more are available. - em := (*influxql.Emitter)(e) - for { - row := em.Emit() - if row == nil { - break - } - - select { - case <-closing: - break - case out <- row: - } - } -} diff --git a/tsdb/query_executor_test.go b/tsdb/query_executor_test.go deleted file mode 100644 index 0ce971cdb0c..00000000000 --- a/tsdb/query_executor_test.go +++ /dev/null @@ -1,490 +0,0 @@ -package tsdb_test - -import ( - "encoding/json" - "reflect" - "strings" - "testing" - "time" - - "github.com/davecgh/go-spew/spew" - "github.com/influxdata/influxdb/influxql" - "github.com/influxdata/influxdb/services/meta" - "github.com/influxdata/influxdb/tsdb" -) - -const ( - // DefaultDatabase is the default database name used by tests. - DefaultDatabase = "db0" - - // DefaultRetentionPolicy is the default retention policy name used by tests. - DefaultRetentionPolicy = "rp0" -) - -// Ensure the query executor can execute a basic query. -func TestQueryExecutor_ExecuteQuery_Select(t *testing.T) { - sh := MustOpenShard() - defer sh.Close() - sh.MustWritePointsString(` -cpu,region=serverA value=1 0 -cpu,region=serverA value=2 10 -cpu,region=serverB value=3 20 -`) - - e := NewQueryExecutor() - e.MetaClient.ShardIDsByTimeRangeFn = func(sources influxql.Sources, tmin, tmax time.Time) (a []uint64, err error) { - if !reflect.DeepEqual(sources, influxql.Sources([]influxql.Source{&influxql.Measurement{Database: "db0", RetentionPolicy: "rp0", Name: "cpu"}})) { - t.Fatalf("unexpected sources: %s", spew.Sdump(sources)) - } else if tmin.IsZero() { - t.Fatalf("unexpected tmin: %s", tmin) - } else if tmax.IsZero() { - t.Fatalf("unexpected tmax: %s", tmax) - } - return []uint64{100}, nil - } - e.Store.ShardsFn = func(ids []uint64) []*tsdb.Shard { - if !reflect.DeepEqual(ids, []uint64{100}) { - t.Fatalf("unexpected shard ids: %+v", ids) - } - return []*tsdb.Shard{sh.Shard} - } - - res := e.MustExecuteQueryString("db0", `SELECT value FROM cpu`) - if s := MustMarshalJSON(res); s != `[{"series":[{"name":"cpu","columns":["time","value"],"values":[["1970-01-01T00:00:00Z",1],["1970-01-01T00:00:10Z",2],["1970-01-01T00:00:20Z",3]]}]}]` { - t.Fatalf("unexpected results: %s", s) - } -} - -// Ensure the query executor can select from a tsdb.Store. -func TestQueryExecutor_ExecuteQuery_Select_Wildcard_Intg(t *testing.T) { - s := MustOpenStore() - defer s.Close() - - s.MustCreateShardWithData("db0", "rp0", 0, - `cpu,host=serverA value=1 0`, - `cpu,host=serverA value=2 10`, - `cpu,host=serverB value=3 20`, - ) - - res := NewQueryExecutorStore(s).MustExecuteQueryStringJSON("db0", `SELECT * FROM cpu`) - if res != `[{"series":[{"name":"cpu","columns":["time","host","value"],"values":[["1970-01-01T00:00:00Z","serverA",1],["1970-01-01T00:00:10Z","serverA",2],["1970-01-01T00:00:20Z","serverB",3]]}]}]` { - t.Fatalf("unexpected results: %s", res) - } -} - -// Ensure the query executor returns an empty set if no points are returned. -/* -func TestQueryExecutor_ExecuteQuery_Select_Empty(t *testing.T) { - e := NewQueryExecutor() - - // Return an empty iterator. - e.IteratorCreator.CreateIteratorFn = func(opt influxql.IteratorOptions) (influxql.Iterator, error) { - return &FloatIterator{}, nil - } - - res := e.MustExecuteQueryString("db0", `SELECT value FROM cpu`) - if MustMarshalJSON(res) != `[{}]` { - t.Fatalf("unexpected results: %s", spew.Sdump(res)) - } -} -*/ - -// Ensure the query executor can execute a DROP MEASUREMENT statement. -func TestQueryExecutor_ExecuteQuery_DropMeasurement(t *testing.T) { - e := NewQueryExecutor() - e.Store.DeleteMeasurementFn = func(database, name string) error { - if database != `db0` { - t.Fatalf("unexpected database: %s", database) - } else if name != `memory` { - t.Fatalf("unexpected name: %s", name) - } - return nil - } - - res := e.MustExecuteQueryString("db0", `drop measurement memory`) - if s := MustMarshalJSON(res); s != `[{}]` { - t.Fatalf("unexpected results: %s", s) - } -} - -// Ensure the query executor can execute a DROP DATABASE statement. -// -// Dropping a database involves executing against the meta store as well as -// removing all associated shards from the local TSDB storage. -func TestQueryExecutor_ExecuteQuery_DropDatabase(t *testing.T) { - e := NewQueryExecutor() - e.MetaClient.DatabaseFn = func(name string) (*meta.DatabaseInfo, error) { - return &meta.DatabaseInfo{ - Name: name, - DefaultRetentionPolicy: "rp0", - RetentionPolicies: []meta.RetentionPolicyInfo{ - { - Name: "rp0", - ShardGroups: []meta.ShardGroupInfo{ - { - ID: 1, - Shards: []meta.ShardInfo{{ID: 10}, {ID: 20}}, - }, - { - ID: 2, - Shards: []meta.ShardInfo{{ID: 50}}, - }, - }, - }, - { - Name: "rp1", - ShardGroups: []meta.ShardGroupInfo{ - { - ID: 3, - Shards: []meta.ShardInfo{{ID: 60}}, - }, - }, - }, - }, - }, nil - } - e.MetaClient.ExecuteStatementFn = func(stmt influxql.Statement) *influxql.Result { - if s := stmt.String(); s != `DROP DATABASE db0` { - t.Fatalf("unexpected meta statement: %s", s) - } - return &influxql.Result{} - } - - e.Store.DeleteDatabaseFn = func(name string, shardIDs []uint64) error { - if name != `db0` { - t.Fatalf("unexpected name: %s", name) - } else if !reflect.DeepEqual(shardIDs, []uint64{10, 20, 50, 60}) { - t.Fatalf("unexpected shard ids: %+v", shardIDs) - } - return nil - } - - res := e.MustExecuteQueryString("db0", `drop database db0`) - if s := MustMarshalJSON(res); s != `[{}]` { - t.Fatalf("unexpected results: %s", s) - } -} - -// Ensure that the query executor doesn't return an error when user count is zero -// and the user is attempting to create a user. -func TestQueryExecutor_Authorize_CreateUser_NoUsers(t *testing.T) { - /* - store, executor := testStoreAndExecutor("") - defer os.RemoveAll(store.Path()) - ms := &testMetastore{userCount: 0} - executor.MetaStore = ms - - if err := executor.Authorize(nil, MustParseQuery("create user foo with password 'asdf' with all privileges"), ""); err != nil { - t.Fatalf("should have authenticated if no users and attempting to create a user but got error: %s", err.Error()) - } - - if executor.Authorize(nil, MustParseQuery("create user foo with password 'asdf'"), "") == nil { - t.Fatalf("should have failed authentication if no user given and no users exist for create user query that doesn't grant all privileges") - } - - if executor.Authorize(nil, MustParseQuery("select * from foo"), "") == nil { - t.Fatalf("should have failed authentication if no user given and no users exist for any query other than create user") - } - - ms.userCount = 1 - - if executor.Authorize(nil, MustParseQuery("create user foo with password 'asdf'"), "") == nil { - t.Fatalf("should have failed authentication if no user given and users exist") - } - - if executor.Authorize(nil, MustParseQuery("select * from foo"), "") == nil { - t.Fatalf("should have failed authentication if no user given and users exist") - } - */ -} - -func TestDropDatabase(t *testing.T) { - /* - store, executor := testStoreAndExecutor("") - defer os.RemoveAll(store.Path()) - - pt := models.MustNewPoint( - "cpu", - map[string]string{"host": "server"}, - map[string]interface{}{"value": 1.0}, - time.Unix(1, 2), - ) - - if err := store.WriteToShard(shardID, []models.Point{pt}); err != nil { - t.Fatal(err) - } - - got := executeAndGetJSON("SELECT * FROM cpu GROUP BY *", executor) - expected := `[{"series":[{"name":"cpu","tags":{"host":"server"},"columns":["time","value"],"values":[["1970-01-01T00:00:01.000000002Z",1]]}]}]` - if expected != got { - t.Fatalf("exp: %s\ngot: %s", expected, got) - } - - var name string - executor.MetaClient = &testMetaClient{ - ExecuteStatemenFn: func(stmt influxql.Statement) *influxql.Result { - name = stmt.(*influxql.DropDatabaseStatement).Name - return &influxql.Result{} - }, - } - // verify the database is there on disk - dbPath := filepath.Join(store.Path(), "foo") - if _, err := os.Stat(dbPath); err != nil { - t.Fatalf("execpted database dir %s to exist", dbPath) - } - - got = executeAndGetJSON("drop database foo", executor) - expected = `[{}]` - if got != expected { - t.Fatalf("exp: %s\ngot: %s", expected, got) - } - */ -} - -// ensure that authenticate doesn't return an error if the user count is zero and they're attempting -// to create a user. -func TestAuthenticateIfUserCountZeroAndCreateUser(t *testing.T) { - /* - store, executor := testStoreAndExecutor("") - defer os.RemoveAll(store.Path()) - ms := &testMetaClient{userCount: 0} - executor.MetaClient = ms - - if err := executor.Authorize(nil, mustParseQuery("create user foo with password 'asdf' with all privileges"), ""); err != nil { - t.Fatalf("should have authenticated if no users and attempting to create a user but got error: %s", err.Error()) - } - - if executor.Authorize(nil, mustParseQuery("create user foo with password 'asdf'"), "") == nil { - t.Fatalf("should have failed authentication if no user given and no users exist for create user query that doesn't grant all privileges") - } - - if executor.Authorize(nil, mustParseQuery("select * from foo"), "") == nil { - t.Fatalf("should have failed authentication if no user given and no users exist for any query other than create user") - } - - ms.userCount = 1 - */ -} - -// QueryExecutor represents a test wrapper for tsdb.QueryExecutor. -type QueryExecutor struct { - *tsdb.QueryExecutor - - Store QueryExecutorStore - MetaClient QueryExecutorMetaClient - MonitorStatementExecutor StatementExecutor - IntoWriter IntoWriter -} - -// NewQueryExecutor returns a new instance of QueryExecutor. -func NewQueryExecutor() *QueryExecutor { - e := &QueryExecutor{} - e.QueryExecutor = tsdb.NewQueryExecutor() - e.QueryExecutor.Store = &e.Store - e.QueryExecutor.MetaClient = &e.MetaClient - e.QueryExecutor.MonitorStatementExecutor = &e.MonitorStatementExecutor - e.QueryExecutor.IntoWriter = &e.IntoWriter - - // By default, always return a database when looking it up. - e.MetaClient.DatabaseFn = MetaClientDatabaseFoundFn - - // By default, returns the same sources when expanding. - e.Store.ExpandSourcesFn = DefaultStoreExpandSourcesFn - - return e -} - -// NewQueryExecutorStore returns a new instance of QueryExecutor attached to a store. -func NewQueryExecutorStore(s *Store) *QueryExecutor { - e := NewQueryExecutor() - e.QueryExecutor.Store = s - - // Always return all shards from store. - e.MetaClient.ShardIDsByTimeRangeFn = func(sources influxql.Sources, tmin, tmax time.Time) ([]uint64, error) { - return s.ShardIDs(), nil - } - - return e -} - -// MustExecuteQuery executes a query. Panic on error. -func (e *QueryExecutor) MustExecuteQueryString(database string, s string) []*influxql.Result { - q := MustParseQuery(s) - - // Execute query. - ch, err := e.ExecuteQuery(q, database, 1000, make(chan struct{})) - if err != nil { - panic(err) - } - - // Read all results from the channel. - var a []*influxql.Result - for { - select { - case result, ok := <-ch: - if !ok { - return a - } - a = append(a, result) - case <-time.After(10 * time.Second): - panic("query timeout") - } - } -} - -// MustExecuteQueryStringJSON executes a query and returns JSON. Panic on error. -func (e *QueryExecutor) MustExecuteQueryStringJSON(database string, s string) string { - return MustMarshalJSON(e.MustExecuteQueryString(database, s)) -} - -// QueryExecutorStore is a mockable implementation of QueryExecutor.Store. -type QueryExecutorStore struct { - DatabaseIndexFn func(name string) *tsdb.DatabaseIndex - ShardsFn func(ids []uint64) []*tsdb.Shard - ExpandSourcesFn func(sources influxql.Sources) (influxql.Sources, error) - DeleteDatabaseFn func(name string, shardIDs []uint64) error - DeleteMeasurementFn func(database, name string) error - DeleteSeriesFn func(database string, seriesKeys []string) error -} - -func (s *QueryExecutorStore) DatabaseIndex(name string) *tsdb.DatabaseIndex { - return s.DatabaseIndexFn(name) -} -func (s *QueryExecutorStore) Shards(ids []uint64) []*tsdb.Shard { - return s.ShardsFn(ids) -} -func (s *QueryExecutorStore) ExpandSources(sources influxql.Sources) (influxql.Sources, error) { - return s.ExpandSourcesFn(sources) -} -func (s *QueryExecutorStore) DeleteDatabase(name string, shardIDs []uint64) error { - return s.DeleteDatabaseFn(name, shardIDs) -} -func (s *QueryExecutorStore) DeleteMeasurement(database, name string) error { - return s.DeleteMeasurementFn(database, name) -} -func (s *QueryExecutorStore) DeleteSeries(database string, seriesKeys []string) error { - return s.DeleteSeriesFn(database, seriesKeys) -} - -// DefaultStoreExpandSourcesFn returns the original sources unchanged. -func DefaultStoreExpandSourcesFn(sources influxql.Sources) (influxql.Sources, error) { - return sources, nil -} - -// QueryExecutorMetaClient is a mockable implementation of QueryExecutor.MetaClient. -type QueryExecutorMetaClient struct { - DatabaseFn func(name string) (*meta.DatabaseInfo, error) - DatabasesFn func() ([]meta.DatabaseInfo, error) - UserFn func(name string) (*meta.UserInfo, error) - AdminUserExistsFn func() bool - AuthenticateFn func(username, password string) (*meta.UserInfo, error) - RetentionPolicyFn func(database, name string) (rpi *meta.RetentionPolicyInfo, err error) - UserCountFn func() int - ShardIDsByTimeRangeFn func(sources influxql.Sources, tmin, tmax time.Time) (a []uint64, err error) - ExecuteStatementFn func(stmt influxql.Statement) *influxql.Result -} - -func (s *QueryExecutorMetaClient) Database(name string) (*meta.DatabaseInfo, error) { - return s.DatabaseFn(name) -} -func (s *QueryExecutorMetaClient) Databases() ([]meta.DatabaseInfo, error) { - return s.DatabasesFn() -} -func (s *QueryExecutorMetaClient) User(name string) (*meta.UserInfo, error) { - return s.UserFn(name) -} -func (s *QueryExecutorMetaClient) AdminUserExists() bool { - return s.AdminUserExistsFn() -} -func (s *QueryExecutorMetaClient) Authenticate(username, password string) (*meta.UserInfo, error) { - return s.AuthenticateFn(username, password) -} -func (s *QueryExecutorMetaClient) RetentionPolicy(database, name string) (rpi *meta.RetentionPolicyInfo, err error) { - return s.RetentionPolicyFn(database, name) -} -func (s *QueryExecutorMetaClient) UserCount() int { - return s.UserCountFn() -} -func (s *QueryExecutorMetaClient) ShardIDsByTimeRange(sources influxql.Sources, tmin, tmax time.Time) (a []uint64, err error) { - return s.ShardIDsByTimeRangeFn(sources, tmin, tmax) -} -func (s *QueryExecutorMetaClient) ExecuteStatement(stmt influxql.Statement) *influxql.Result { - return s.ExecuteStatementFn(stmt) -} - -// MetaClientDatabaseFoundFn always returns a database for a database name. -func MetaClientDatabaseFoundFn(name string) (*meta.DatabaseInfo, error) { - return &meta.DatabaseInfo{ - Name: name, - DefaultRetentionPolicy: DefaultRetentionPolicy, - }, nil -} - -// StatementExecutor is a mockable implementation of QueryExecutor.StatementExecutor. -type StatementExecutor struct { - ExecuteStatementFn func(stmt influxql.Statement) *influxql.Result -} - -func (e *StatementExecutor) ExecuteStatement(stmt influxql.Statement) *influxql.Result { - return e.ExecuteStatementFn(stmt) -} - -// IteratorCreator is a mockable implementation of SelectStatementExecutor.IteratorCreator. -type IteratorCreator struct { - CreateIteratorFn func(opt influxql.IteratorOptions) (influxql.Iterator, error) - FieldDimensionsFn func(sources influxql.Sources) (field, dimensions map[string]struct{}, err error) -} - -func (ic *IteratorCreator) CreateIterator(opt influxql.IteratorOptions) (influxql.Iterator, error) { - return ic.CreateIteratorFn(opt) -} - -func (ic *IteratorCreator) FieldDimensions(sources influxql.Sources) (field, dimensions map[string]struct{}, err error) { - return ic.FieldDimensionsFn(sources) -} - -// IntoWriter is a mockable implementation of QueryExecutor.IntoWriter. -type IntoWriter struct { - WritePointsIntoFn func(p *tsdb.IntoWriteRequest) error -} - -func (w *IntoWriter) WritePointsInto(p *tsdb.IntoWriteRequest) error { - return w.WritePointsIntoFn(p) -} - -// FloatIterator is a test implementation of influxql.FloatIterator. -type FloatIterator struct { - Points []influxql.FloatPoint -} - -// Close is a no-op. -func (itr *FloatIterator) Close() error { return nil } - -// Next returns the next value and shifts it off the beginning of the points slice. -func (itr *FloatIterator) Next() *influxql.FloatPoint { - if len(itr.Points) == 0 { - return nil - } - - v := &itr.Points[0] - itr.Points = itr.Points[1:] - return v -} - -// MustParseQuery parses an InfluxQL query. Panic on error. -func MustParseQuery(s string) *influxql.Query { - q, err := influxql.NewParser(strings.NewReader(s)).ParseQuery() - if err != nil { - panic(err.Error()) - } - return q -} - -// MustMarshalJSON marshals a value to a JSON string. Panic on error. -func MustMarshalJSON(v interface{}) string { - buf, err := json.Marshal(v) - if err != nil { - panic(err) - } - return string(buf) -} diff --git a/tsdb/store.go b/tsdb/store.go index c6cc4355ec1..7183ba14d1f 100644 --- a/tsdb/store.go +++ b/tsdb/store.go @@ -1,6 +1,7 @@ package tsdb // import "github.com/influxdata/influxdb/tsdb" import ( + "errors" "fmt" "io" "io/ioutil" @@ -323,7 +324,7 @@ func (s *Store) DeleteMeasurement(database, name string) error { // Find the measurement. m := db.Measurement(name) if m == nil { - return ErrMeasurementNotFound(name) + return influxql.ErrMeasurementNotFound(name) } // Remove measurement from index. @@ -437,13 +438,75 @@ func (s *Store) ShardRelativePath(id uint64) (string, error) { } // DeleteSeries loops through the local shards and deletes the series data and metadata for the passed in series keys -func (s *Store) DeleteSeries(database string, seriesKeys []string) error { +func (s *Store) DeleteSeries(database string, sources []influxql.Source, condition influxql.Expr) error { s.mu.RLock() defer s.mu.RUnlock() + // Find the database. + db := s.DatabaseIndex(database) + if db == nil { + return nil + } + + // Expand regex expressions in the FROM clause. + a, err := s.expandSources(sources) + if err != nil { + return err + } else if sources != nil && len(sources) != 0 && len(a) == 0 { + return nil + } + sources = a + + measurements, err := measurementsFromSourcesOrDB(db, sources...) + if err != nil { + return err + } + + var seriesKeys []string + for _, m := range measurements { + var ids SeriesIDs + var filters FilterExprs + if condition != nil { + // Get series IDs that match the WHERE clause. + ids, filters, err = m.walkWhereForSeriesIds(condition) + if err != nil { + return err + } + + // Delete boolean literal true filter expressions. + // These are returned for `WHERE tagKey = 'tagVal'` type expressions and are okay. + filters.DeleteBoolLiteralTrues() + + // Check for unsupported field filters. + // Any remaining filters means there were fields (e.g., `WHERE value = 1.2`). + if filters.Len() > 0 { + return errors.New("DROP SERIES doesn't support fields in WHERE clause") + } + } else { + // No WHERE clause so get all series IDs for this measurement. + ids = m.seriesIDs + } + + for _, id := range ids { + seriesKeys = append(seriesKeys, m.seriesByID[id].Key) + } + } + + // delete the raw series data + if err := s.deleteSeries(database, seriesKeys); err != nil { + return err + } + + // remove them from the index + db.DropSeries(seriesKeys) + + return nil +} + +func (s *Store) deleteSeries(database string, seriesKeys []string) error { db, ok := s.databaseIndexes[database] if !ok { - return ErrDatabaseNotFound(database) + return influxql.ErrDatabaseNotFound(database) } for _, sh := range s.shards { @@ -497,7 +560,10 @@ func (s *Store) performMaintenanceOnShard(shard *Shard) { func (s *Store) ExpandSources(sources influxql.Sources) (influxql.Sources, error) { s.mu.Lock() defer s.mu.Unlock() + return s.expandSources(sources) +} +func (s *Store) expandSources(sources influxql.Sources) (influxql.Sources, error) { // Use a map as a set to prevent duplicates. set := map[string]influxql.Source{} @@ -567,6 +633,260 @@ func (s *Store) WriteToShard(shardID uint64, points []models.Point) error { return sh.WritePoints(points) } +func (s *Store) ExecuteShowFieldKeysStatement(stmt *influxql.ShowFieldKeysStatement, database string) (models.Rows, error) { + // NOTE(benbjohnson): + // This function is temporarily moved here until reimplemented in the new query engine. + + // Find the database. + db := s.DatabaseIndex(database) + if db == nil { + return nil, nil + } + + // Expand regex expressions in the FROM clause. + sources, err := s.ExpandSources(stmt.Sources) + if err != nil { + return nil, err + } + + measurements, err := measurementsFromSourcesOrDB(db, sources...) + if err != nil { + return nil, err + } + + // Make result. + rows := make(models.Rows, 0, len(measurements)) + + // Loop through measurements, adding a result row for each. + for _, m := range measurements { + // Create a new row. + r := &models.Row{ + Name: m.Name, + Columns: []string{"fieldKey"}, + } + + // Get a list of field names from the measurement then sort them. + names := m.FieldNames() + sort.Strings(names) + + // Add the field names to the result row values. + for _, n := range names { + v := interface{}(n) + r.Values = append(r.Values, []interface{}{v}) + } + + // Append the row to the result. + rows = append(rows, r) + } + + return rows, nil +} + +func (s *Store) ExecuteShowSeriesStatement(stmt *influxql.ShowSeriesStatement, database string) (models.Rows, error) { + // NOTE(benbjohnson): + // This function is temporarily moved here until reimplemented in the new query engine. + + // Check for time in WHERE clause (not supported). + if influxql.HasTimeExpr(stmt.Condition) { + return nil, errors.New("SHOW SERIES doesn't support time in WHERE clause") + } + + // Find the database. + db := s.DatabaseIndex(database) + if db == nil { + return nil, nil + } + + // Expand regex expressions in the FROM clause. + sources, err := s.ExpandSources(stmt.Sources) + if err != nil { + return nil, err + } + + // Get the list of measurements we're interested in. + measurements, err := measurementsFromSourcesOrDB(db, sources...) + if err != nil { + return nil, err + } + + // Create result struct that will be populated and returned. + rows := make(models.Rows, 0, len(measurements)) + + // Loop through measurements to build result. One result row / measurement. + for _, m := range measurements { + var ids SeriesIDs + var filters FilterExprs + + if stmt.Condition != nil { + // Get series IDs that match the WHERE clause. + ids, filters, err = m.walkWhereForSeriesIds(stmt.Condition) + if err != nil { + return nil, err + } + + // Delete boolean literal true filter expressions. + filters.DeleteBoolLiteralTrues() + + // Check for unsupported field filters. + if filters.Len() > 0 { + return nil, errors.New("SHOW SERIES doesn't support fields in WHERE clause") + } + + // If no series matched, then go to the next measurement. + if len(ids) == 0 { + continue + } + } else { + // No WHERE clause so get all series IDs for this measurement. + ids = m.seriesIDs + } + + // Make a new row for this measurement. + r := &models.Row{ + Name: m.Name, + Columns: m.TagKeys(), + } + + // Loop through series IDs getting matching tag sets. + for _, id := range ids { + if s, ok := m.seriesByID[id]; ok { + values := make([]interface{}, 0, len(r.Columns)) + + // make the series key the first value + values = append(values, s.Key) + + for _, column := range r.Columns { + values = append(values, s.Tags[column]) + } + + // Add the tag values to the row. + r.Values = append(r.Values, values) + } + } + // make the id the first column + r.Columns = append([]string{"_key"}, r.Columns...) + + // Append the row. + rows = append(rows, r) + } + + if stmt.Limit > 0 || stmt.Offset > 0 { + rows = s.filterShowSeriesResult(stmt.Limit, stmt.Offset, rows) + } + + return rows, nil +} + +// filterShowSeriesResult will limit the number of series returned based on the limit and the offset. +// Unlike limit and offset on SELECT statements, the limit and offset don't apply to the number of Rows, but +// to the number of total Values returned, since each Value represents a unique series. +func (e *Store) filterShowSeriesResult(limit, offset int, rows models.Rows) models.Rows { + var filteredSeries models.Rows + seriesCount := 0 + for _, r := range rows { + var currentSeries [][]interface{} + + // filter the values + for _, v := range r.Values { + if seriesCount >= offset && seriesCount-offset < limit { + currentSeries = append(currentSeries, v) + } + seriesCount++ + } + + // only add the row back in if there are some values in it + if len(currentSeries) > 0 { + r.Values = currentSeries + filteredSeries = append(filteredSeries, r) + if seriesCount > limit+offset { + return filteredSeries + } + } + } + return filteredSeries +} + +func (s *Store) ExecuteShowTagValuesStatement(stmt *influxql.ShowTagValuesStatement, database string) (models.Rows, error) { + // NOTE(benbjohnson): + // This function is temporarily moved here until reimplemented in the new query engine. + + // Check for time in WHERE clause (not supported). + if influxql.HasTimeExpr(stmt.Condition) { + return nil, errors.New("SHOW TAG VALUES doesn't support time in WHERE clause") + } + + // Find the database. + db := s.DatabaseIndex(database) + if db == nil { + return nil, nil + } + + // Expand regex expressions in the FROM clause. + sources, err := s.ExpandSources(stmt.Sources) + if err != nil { + return nil, err + } + + // Get the list of measurements we're interested in. + measurements, err := measurementsFromSourcesOrDB(db, sources...) + if err != nil { + return nil, err + } + + // Make result. + var rows models.Rows + tagValues := make(map[string]stringSet) + for _, m := range measurements { + var ids SeriesIDs + + if stmt.Condition != nil { + // Get series IDs that match the WHERE clause. + ids, _, err = m.walkWhereForSeriesIds(stmt.Condition) + if err != nil { + return nil, err + } + + // If no series matched, then go to the next measurement. + if len(ids) == 0 { + continue + } + + // TODO: check return of walkWhereForSeriesIds for fields + } else { + // No WHERE clause so get all series IDs for this measurement. + ids = m.seriesIDs + } + + for k, v := range m.tagValuesByKeyAndSeriesID(stmt.TagKeys, ids) { + _, ok := tagValues[k] + if !ok { + tagValues[k] = v + } + tagValues[k] = tagValues[k].union(v) + } + } + + for k, v := range tagValues { + r := &models.Row{ + Name: k + "TagValues", + Columns: []string{k}, + } + + vals := v.list() + sort.Strings(vals) + + for _, val := range vals { + v := interface{}(val) + r.Values = append(r.Values, []interface{}{v}) + } + + rows = append(rows, r) + } + + sort.Sort(rows) + return rows, nil +} + // IsRetryable returns true if this error is temporary and could be retried func IsRetryable(err error) bool { if err == nil { @@ -599,3 +919,34 @@ func relativePath(storePath, shardPath string) (string, error) { return name, nil } + +// measurementsFromSourcesOrDB returns a list of measurements from the +// sources passed in or, if sources is empty, a list of all +// measurement names from the database passed in. +func measurementsFromSourcesOrDB(db *DatabaseIndex, sources ...influxql.Source) (Measurements, error) { + var measurements Measurements + if len(sources) > 0 { + for _, source := range sources { + if m, ok := source.(*influxql.Measurement); ok { + measurement := db.measurements[m.Name] + if measurement == nil { + continue + } + + measurements = append(measurements, measurement) + } else { + return nil, errors.New("identifiers in FROM clause must be measurement names") + } + } + } else { + // No measurements specified in FROM clause so get all measurements that have series. + for _, m := range db.Measurements() { + if m.HasSeries() { + measurements = append(measurements, m) + } + } + } + sort.Sort(measurements) + + return measurements, nil +}