feat(flux): allow values of 0 to disable controller limits (#21127)

Co-authored-by: Sam Arnold <sarnold@influxdata.com>
pull/21150/head
Daniel Moran 2021-04-05 16:58:27 -04:00 committed by GitHub
parent 28ae9eac75
commit ff6b55aca4
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
8 changed files with 612 additions and 428 deletions

View File

@ -17,6 +17,17 @@ the endpoint has been removed. Use the `/metrics` endpoint to collect system sta
The `transpile` command has been retired. Users can send InfluxQL directly to the server via the `/api/v2/query` The `transpile` command has been retired. Users can send InfluxQL directly to the server via the `/api/v2/query`
or `/query` HTTP endpoints. or `/query` HTTP endpoints.
#### Default query concurrency changed
The default setting for the max number of concurrent Flux queries has been changed from 10 to unlimited. Set the
`query-concurrency` config parameter to > 0 when running `influxd` to re-limit the maximum running query count,
and the `query-queue-size` config parameter to > 0 to set the max number of queries that can be queued before the
server starts rejecting requests.
#### Prefix for query-controller metrics changed
The prefix used for Prometheus metrics from the query controller has changed from `query_control_` to `qc_`.
### Features ### Features
1. [19811](https://github.com/influxdata/influxdb/pull/19811): Add Geo graph type to be able to store in Dashboard cells. 1. [19811](https://github.com/influxdata/influxdb/pull/19811): Add Geo graph type to be able to store in Dashboard cells.
@ -38,6 +49,7 @@ or `/query` HTTP endpoints.
1. [21046](https://github.com/influxdata/influxdb/pull/21046): Write to standard out when `--output-path -` is passed to `influxd inspect export-lp`. 1. [21046](https://github.com/influxdata/influxdb/pull/21046): Write to standard out when `--output-path -` is passed to `influxd inspect export-lp`.
1. [21006](https://github.com/influxdata/influxdb/pull/21006): Add `-p, --profilers` flag to `influx query` command. 1. [21006](https://github.com/influxdata/influxdb/pull/21006): Add `-p, --profilers` flag to `influx query` command.
1. [21090](https://github.com/influxdata/influxdb/pull/21090): Update UI to match InfluxDB Cloud. 1. [21090](https://github.com/influxdata/influxdb/pull/21090): Update UI to match InfluxDB Cloud.
1. [21127](https://github.com/influxdata/influxdb/pull/21127): Allow for disabling concurrency-limits in Flux controller.
### Bug Fixes ### Bug Fixes
@ -59,6 +71,7 @@ or `/query` HTTP endpoints.
1. [20921](https://github.com/influxdata/influxdb/pull/20921): Fix the cipher suite used when TLS strict ciphers are enabled in `influxd`. 1. [20921](https://github.com/influxdata/influxdb/pull/20921): Fix the cipher suite used when TLS strict ciphers are enabled in `influxd`.
1. [20925](https://github.com/influxdata/influxdb/pull/20925): Fix parse error in UI for tag filters containing regex meta characters. 1. [20925](https://github.com/influxdata/influxdb/pull/20925): Fix parse error in UI for tag filters containing regex meta characters.
1. [21042](https://github.com/influxdata/influxdb/pull/21042): Prevent concurrent access panic when gathering bolt metrics. 1. [21042](https://github.com/influxdata/influxdb/pull/21042): Prevent concurrent access panic when gathering bolt metrics.
1. [21127](https://github.com/influxdata/influxdb/pull/21127): Fix race condition in Flux controller shutdown.
## v2.0.4 [2021-02-08] ## v2.0.4 [2021-02-08]

View File

@ -205,11 +205,11 @@ func newOpts(viper *viper.Viper) *InfluxdOpts {
NoTasks: false, NoTasks: false,
ConcurrencyQuota: 10, ConcurrencyQuota: 0,
InitialMemoryBytesQuotaPerQuery: 0, InitialMemoryBytesQuotaPerQuery: 0,
MemoryBytesQuotaPerQuery: MaxInt, MemoryBytesQuotaPerQuery: MaxInt,
MaxMemoryBytes: 0, MaxMemoryBytes: 0,
QueueSize: 10, QueueSize: 0,
Testing: false, Testing: false,
TestingAlwaysAllowSetup: false, TestingAlwaysAllowSetup: false,
@ -405,7 +405,7 @@ func (o *InfluxdOpts) bindCliOpts() []cli.Opt {
DestP: &o.ConcurrencyQuota, DestP: &o.ConcurrencyQuota,
Flag: "query-concurrency", Flag: "query-concurrency",
Default: o.ConcurrencyQuota, Default: o.ConcurrencyQuota,
Desc: "the number of queries that are allowed to execute concurrently", Desc: "the number of queries that are allowed to execute concurrently. Set to 0 to allow an unlimited number of concurrent queries",
}, },
{ {
DestP: &o.InitialMemoryBytesQuotaPerQuery, DestP: &o.InitialMemoryBytesQuotaPerQuery,
@ -423,13 +423,13 @@ func (o *InfluxdOpts) bindCliOpts() []cli.Opt {
DestP: &o.MaxMemoryBytes, DestP: &o.MaxMemoryBytes,
Flag: "query-max-memory-bytes", Flag: "query-max-memory-bytes",
Default: o.MaxMemoryBytes, Default: o.MaxMemoryBytes,
Desc: "the maximum amount of memory used for queries. If this is unset, then this number is query-concurrency * query-memory-bytes", Desc: "the maximum amount of memory used for queries. Can only be set when query-concurrency is limited. If this is unset, then this number is query-concurrency * query-memory-bytes",
}, },
{ {
DestP: &o.QueueSize, DestP: &o.QueueSize,
Flag: "query-queue-size", Flag: "query-queue-size",
Default: o.QueueSize, Default: o.QueueSize,
Desc: "the number of queries that are allowed to be awaiting execution before new queries are rejected", Desc: "the number of queries that are allowed to be awaiting execution before new queries are rejected. Must be > 0 if query-concurrency is not unlimited",
}, },
{ {
DestP: &o.FeatureFlags, DestP: &o.FeatureFlags,

View File

@ -435,9 +435,8 @@ func (m *Launcher) run(ctx context.Context, opts *InfluxdOpts) (err error) {
MemoryBytesQuotaPerQuery: opts.MemoryBytesQuotaPerQuery, MemoryBytesQuotaPerQuery: opts.MemoryBytesQuotaPerQuery,
MaxMemoryBytes: opts.MaxMemoryBytes, MaxMemoryBytes: opts.MaxMemoryBytes,
QueueSize: opts.QueueSize, QueueSize: opts.QueueSize,
Logger: m.log.With(zap.String("service", "storage-reads")),
ExecutorDependencies: dependencyList, ExecutorDependencies: dependencyList,
}) }, m.log.With(zap.String("service", "storage-reads")))
if err != nil { if err != nil {
m.log.Error("Failed to create query controller", zap.Error(err)) m.log.Error("Failed to create query controller", zap.Error(err))
return err return err

View File

@ -5,7 +5,6 @@ import (
"context" "context"
"errors" "errors"
"fmt" "fmt"
errors2 "github.com/influxdata/influxdb/v2/kit/platform/errors"
"html/template" "html/template"
"io" "io"
"io/ioutil" "io/ioutil"
@ -16,6 +15,8 @@ import (
"testing" "testing"
"time" "time"
errors2 "github.com/influxdata/influxdb/v2/kit/platform/errors"
"github.com/influxdata/flux" "github.com/influxdata/flux"
"github.com/influxdata/flux/csv" "github.com/influxdata/flux/csv"
"github.com/influxdata/flux/execute" "github.com/influxdata/flux/execute"
@ -122,7 +123,7 @@ func getMemoryUnused(t *testing.T, reg *prom.Registry) int64 {
t.Fatal(err) t.Fatal(err)
} }
for _, m := range ms { for _, m := range ms {
if m.GetName() == "query_control_memory_unused_bytes" { if m.GetName() == "qc_memory_unused_bytes" {
return int64(*m.GetMetric()[0].Gauge.Value) return int64(*m.GetMetric()[0].Gauge.Value)
} }
} }
@ -253,6 +254,7 @@ func TestLauncher_QueryMemoryLimits(t *testing.T) {
name: "ok - initial memory bytes, memory bytes, and max memory set", name: "ok - initial memory bytes, memory bytes, and max memory set",
setOpts: func(o *launcher.InfluxdOpts) { setOpts: func(o *launcher.InfluxdOpts) {
o.ConcurrencyQuota = 1 o.ConcurrencyQuota = 1
o.QueueSize = 1
o.InitialMemoryBytesQuotaPerQuery = 100 o.InitialMemoryBytesQuotaPerQuery = 100
o.MaxMemoryBytes = 1048576 // 1MB o.MaxMemoryBytes = 1048576 // 1MB
}, },
@ -264,6 +266,7 @@ func TestLauncher_QueryMemoryLimits(t *testing.T) {
name: "error - memory bytes and max memory set", name: "error - memory bytes and max memory set",
setOpts: func(o *launcher.InfluxdOpts) { setOpts: func(o *launcher.InfluxdOpts) {
o.ConcurrencyQuota = 1 o.ConcurrencyQuota = 1
o.QueueSize = 1
o.MemoryBytesQuotaPerQuery = 1 o.MemoryBytesQuotaPerQuery = 1
o.MaxMemoryBytes = 100 o.MaxMemoryBytes = 100
}, },
@ -275,6 +278,7 @@ func TestLauncher_QueryMemoryLimits(t *testing.T) {
name: "error - initial memory bytes and max memory set", name: "error - initial memory bytes and max memory set",
setOpts: func(o *launcher.InfluxdOpts) { setOpts: func(o *launcher.InfluxdOpts) {
o.ConcurrencyQuota = 1 o.ConcurrencyQuota = 1
o.QueueSize = 1
o.InitialMemoryBytesQuotaPerQuery = 1 o.InitialMemoryBytesQuotaPerQuery = 1
o.MaxMemoryBytes = 100 o.MaxMemoryBytes = 100
}, },
@ -286,6 +290,7 @@ func TestLauncher_QueryMemoryLimits(t *testing.T) {
name: "error - initial memory bytes, memory bytes, and max memory set", name: "error - initial memory bytes, memory bytes, and max memory set",
setOpts: func(o *launcher.InfluxdOpts) { setOpts: func(o *launcher.InfluxdOpts) {
o.ConcurrencyQuota = 1 o.ConcurrencyQuota = 1
o.QueueSize = 1
o.InitialMemoryBytesQuotaPerQuery = 1 o.InitialMemoryBytesQuotaPerQuery = 1
o.MemoryBytesQuotaPerQuery = 50 o.MemoryBytesQuotaPerQuery = 50
o.MaxMemoryBytes = 100 o.MaxMemoryBytes = 100

View File

@ -109,7 +109,6 @@ type Config struct {
// this to follow suit. // this to follow suit.
QueueSize int32 QueueSize int32
Logger *zap.Logger
// MetricLabelKeys is a list of labels to add to the metrics produced by the controller. // MetricLabelKeys is a list of labels to add to the metrics produced by the controller.
// The value for a given key will be read off the context. // The value for a given key will be read off the context.
// The context value must be a string or an implementation of the Stringer interface. // The context value must be a string or an implementation of the Stringer interface.
@ -133,10 +132,23 @@ func (c *Config) complete() (Config, error) {
} }
func (c *Config) validate(isComplete bool) error { func (c *Config) validate(isComplete bool) error {
if c.ConcurrencyQuota <= 0 { if c.ConcurrencyQuota < 0 {
return errors.New("ConcurrencyQuota must be positive") return errors.New("ConcurrencyQuota must not be negative")
} else if c.ConcurrencyQuota == 0 {
if c.QueueSize != 0 {
return errors.New("QueueSize must be unlimited when ConcurrencyQuota is unlimited")
} }
if c.MemoryBytesQuotaPerQuery <= 0 { if c.MaxMemoryBytes != 0 {
// This is because we have to account for the per-query reserved memory and remove it from
// the max total memory. If there is not a maximum number of queries this is not possible.
return errors.New("Cannot limit max memory when ConcurrencyQuota is unlimited")
}
} else {
if c.QueueSize <= 0 {
return errors.New("QueueSize must be positive when ConcurrencyQuota is limited")
}
}
if c.MemoryBytesQuotaPerQuery < 0 || (isComplete && c.MemoryBytesQuotaPerQuery == 0) {
return errors.New("MemoryBytesQuotaPerQuery must be positive") return errors.New("MemoryBytesQuotaPerQuery must be positive")
} }
if c.InitialMemoryBytesQuotaPerQuery < 0 || (isComplete && c.InitialMemoryBytesQuotaPerQuery == 0) { if c.InitialMemoryBytesQuotaPerQuery < 0 || (isComplete && c.InitialMemoryBytesQuotaPerQuery == 0) {
@ -150,9 +162,6 @@ func (c *Config) validate(isComplete bool) error {
return fmt.Errorf("MaxMemoryBytes must be greater than or equal to the ConcurrencyQuota * InitialMemoryBytesQuotaPerQuery: %d < %d (%d * %d)", c.MaxMemoryBytes, minMemory, c.ConcurrencyQuota, c.InitialMemoryBytesQuotaPerQuery) return fmt.Errorf("MaxMemoryBytes must be greater than or equal to the ConcurrencyQuota * InitialMemoryBytesQuotaPerQuery: %d < %d (%d * %d)", c.MaxMemoryBytes, minMemory, c.ConcurrencyQuota, c.InitialMemoryBytesQuotaPerQuery)
} }
} }
if c.QueueSize <= 0 {
return errors.New("QueueSize must be positive")
}
return nil return nil
} }
@ -163,13 +172,12 @@ func (c *Config) Validate() error {
type QueryID uint64 type QueryID uint64
func New(config Config) (*Controller, error) { func New(config Config, logger *zap.Logger) (*Controller, error) {
c, err := config.complete() c, err := config.complete()
if err != nil { if err != nil {
return nil, errors.Wrap(err, "invalid controller config") return nil, errors.Wrap(err, "invalid controller config")
} }
c.MetricLabelKeys = append(c.MetricLabelKeys, orgLabel) metricLabelKeys := append(c.MetricLabelKeys, orgLabel)
logger := c.Logger
if logger == nil { if logger == nil {
logger = zap.NewNop() logger = zap.NewNop()
} }
@ -189,18 +197,23 @@ func New(config Config) (*Controller, error) {
} else { } else {
mm.unlimited = true mm.unlimited = true
} }
queryQueue := make(chan *Query, c.QueueSize)
if c.ConcurrencyQuota == 0 {
queryQueue = nil
}
ctrl := &Controller{ ctrl := &Controller{
config: c, config: c,
queries: make(map[QueryID]*Query), queries: make(map[QueryID]*Query),
queryQueue: make(chan *Query, c.QueueSize), queryQueue: queryQueue,
done: make(chan struct{}), done: make(chan struct{}),
abort: make(chan struct{}), abort: make(chan struct{}),
memory: mm, memory: mm,
log: logger, log: logger,
metrics: newControllerMetrics(c.MetricLabelKeys), metrics: newControllerMetrics(metricLabelKeys),
labelKeys: c.MetricLabelKeys, labelKeys: metricLabelKeys,
dependencies: c.ExecutorDependencies, dependencies: c.ExecutorDependencies,
} }
if c.ConcurrencyQuota != 0 {
quota := int(c.ConcurrencyQuota) quota := int(c.ConcurrencyQuota)
ctrl.wg.Add(quota) ctrl.wg.Add(quota)
for i := 0; i < quota; i++ { for i := 0; i < quota; i++ {
@ -209,6 +222,7 @@ func New(config Config) (*Controller, error) {
ctrl.processQueryQueue() ctrl.processQueryQueue()
}() }()
} }
}
return ctrl, nil return ctrl, nil
} }
@ -385,6 +399,25 @@ func (c *Controller) enqueueQuery(q *Query) error {
} }
} }
if c.queryQueue == nil {
// unlimited queries case
c.queriesMu.RLock()
defer c.queriesMu.RUnlock()
if c.shutdown {
return &flux.Error{
Code: codes.Internal,
Msg: "controller is shutting down, query not runnable",
}
}
// we can't start shutting down until unlock, so it is safe to add to the waitgroup
c.wg.Add(1)
// unlimited queries, so start a goroutine for every query
go func() {
defer c.wg.Done()
c.executeQuery(q)
}()
} else {
select { select {
case c.queryQueue <- q: case c.queryQueue <- q:
default: default:
@ -393,6 +426,7 @@ func (c *Controller) enqueueQuery(q *Query) error {
Msg: "queue length exceeded", Msg: "queue length exceeded",
} }
} }
}
return nil return nil
} }
@ -487,15 +521,24 @@ func (c *Controller) Queries() []*Query {
// This will return once the Controller's run loop has been exited and all // This will return once the Controller's run loop has been exited and all
// queries have been finished or until the Context has been canceled. // queries have been finished or until the Context has been canceled.
func (c *Controller) Shutdown(ctx context.Context) error { func (c *Controller) Shutdown(ctx context.Context) error {
// Wait for query processing goroutines to finish.
defer c.wg.Wait()
// Mark that the controller is shutdown so it does not // Mark that the controller is shutdown so it does not
// accept new queries. // accept new queries.
func() {
c.queriesMu.Lock() c.queriesMu.Lock()
defer c.queriesMu.Unlock()
if !c.shutdown {
c.shutdown = true c.shutdown = true
if len(c.queries) == 0 { if len(c.queries) == 0 {
c.queriesMu.Unlock() // We hold the lock. No other queries can be spawned.
return nil // No other queries are waiting to be finished, so we have to
// close the done channel here instead of in finish(*Query)
close(c.done)
} }
c.queriesMu.Unlock() }
}()
// Cancel all of the currently active queries. // Cancel all of the currently active queries.
c.queriesMu.RLock() c.queriesMu.RLock()

View File

@ -4,6 +4,7 @@ import (
"context" "context"
"errors" "errors"
"fmt" "fmt"
"math"
"strings" "strings"
"sync" "sync"
"testing" "testing"
@ -20,12 +21,11 @@ import (
"github.com/influxdata/flux/plan" "github.com/influxdata/flux/plan"
"github.com/influxdata/flux/plan/plantest" "github.com/influxdata/flux/plan/plantest"
"github.com/influxdata/flux/stdlib/universe" "github.com/influxdata/flux/stdlib/universe"
_ "github.com/influxdata/influxdb/v2/fluxinit/static"
"github.com/influxdata/influxdb/v2/kit/feature" "github.com/influxdata/influxdb/v2/kit/feature"
pmock "github.com/influxdata/influxdb/v2/mock" pmock "github.com/influxdata/influxdb/v2/mock"
"github.com/influxdata/influxdb/v2/query" "github.com/influxdata/influxdb/v2/query"
_ "github.com/influxdata/influxdb/v2/fluxinit/static"
"github.com/influxdata/influxdb/v2/query/control" "github.com/influxdata/influxdb/v2/query/control"
"github.com/influxdata/influxdb/v2/query/stdlib/influxdata/influxdb"
"github.com/opentracing/opentracing-go" "github.com/opentracing/opentracing-go"
"github.com/opentracing/opentracing-go/mocktracer" "github.com/opentracing/opentracing-go/mocktracer"
"github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus"
@ -48,15 +48,14 @@ var (
}, },
} }
config = control.Config{ config = control.Config{
ConcurrencyQuota: 1, MemoryBytesQuotaPerQuery: math.MaxInt64,
MemoryBytesQuotaPerQuery: 1024,
QueueSize: 1,
ExecutorDependencies: []flux.Dependency{
influxdb.Dependencies{
FluxDeps: executetest.NewTestExecuteDependencies(),
},
},
} }
limitedConfig = control.Config{
MemoryBytesQuotaPerQuery: math.MaxInt64,
ConcurrencyQuota: 1,
QueueSize: 1,
}
bothConfigs = map[string]control.Config{"unlimited": config, "limited": limitedConfig}
) )
func setupPromRegistry(c *control.Controller) *prometheus.Registry { func setupPromRegistry(c *control.Controller) *prometheus.Registry {
@ -79,7 +78,7 @@ func validateRequestTotals(t testing.TB, reg *prometheus.Registry, success, comp
validate := func(name string, want int) { validate := func(name string, want int) {
m := FindMetric( m := FindMetric(
metrics, metrics,
"query_control_requests_total", "qc_requests_total",
map[string]string{ map[string]string{
"result": name, "result": name,
"org": "", "org": "",
@ -107,7 +106,7 @@ func validateUnusedMemory(t testing.TB, reg *prometheus.Registry, c control.Conf
} }
m := FindMetric( m := FindMetric(
metrics, metrics,
"query_control_memory_unused_bytes", "qc_memory_unused_bytes",
map[string]string{ map[string]string{
"org": "", "org": "",
}, },
@ -123,7 +122,9 @@ func validateUnusedMemory(t testing.TB, reg *prometheus.Registry, c control.Conf
} }
func TestController_QuerySuccess(t *testing.T) { func TestController_QuerySuccess(t *testing.T) {
ctrl, err := control.New(config) for name, config := range bothConfigs {
t.Run(name, func(t *testing.T) {
ctrl, err := control.New(config, zaptest.NewLogger(t))
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
} }
@ -159,10 +160,14 @@ func TestController_QuerySuccess(t *testing.T) {
t.Error("expected total duration to be above zero") t.Error("expected total duration to be above zero")
} }
validateRequestTotals(t, reg, 1, 0, 0, 0) validateRequestTotals(t, reg, 1, 0, 0, 0)
})
}
} }
func TestController_QueryCompileError(t *testing.T) { func TestController_QueryCompileError(t *testing.T) {
ctrl, err := control.New(config) for name, config := range bothConfigs {
t.Run(name, func(t *testing.T) {
ctrl, err := control.New(config, zaptest.NewLogger(t))
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
} }
@ -185,10 +190,14 @@ func TestController_QueryCompileError(t *testing.T) {
} }
validateRequestTotals(t, reg, 0, 1, 0, 0) validateRequestTotals(t, reg, 0, 1, 0, 0)
})
}
} }
func TestController_QueryRuntimeError(t *testing.T) { func TestController_QueryRuntimeError(t *testing.T) {
ctrl, err := control.New(config) for name, config := range bothConfigs {
t.Run(name, func(t *testing.T) {
ctrl, err := control.New(config, zaptest.NewLogger(t))
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
} }
@ -232,12 +241,14 @@ func TestController_QueryRuntimeError(t *testing.T) {
t.Error("expected total duration to be above zero") t.Error("expected total duration to be above zero")
} }
validateRequestTotals(t, reg, 0, 0, 1, 0) validateRequestTotals(t, reg, 0, 0, 1, 0)
})
}
} }
func TestController_QueryQueueError(t *testing.T) { func TestController_QueryQueueError(t *testing.T) {
t.Skip("This test exposed several race conditions, its not clear if the races are specific to the test case") t.Skip("This test exposed several race conditions, its not clear if the races are specific to the test case")
ctrl, err := control.New(config) ctrl, err := control.New(config, zaptest.NewLogger(t))
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
} }
@ -341,7 +352,9 @@ func findMetric(mfs []*dto.MetricFamily, name string, labels map[string]string)
} }
func TestController_AfterShutdown(t *testing.T) { func TestController_AfterShutdown(t *testing.T) {
ctrl, err := control.New(config) for name, config := range bothConfigs {
t.Run(name, func(t *testing.T) {
ctrl, err := control.New(config, zaptest.NewLogger(t))
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
} }
@ -358,10 +371,14 @@ func TestController_AfterShutdown(t *testing.T) {
} else if got, want := err.Error(), "query controller shutdown"; got != want { } else if got, want := err.Error(), "query controller shutdown"; got != want {
t.Errorf("unexpected error -want/+got\n\t- %q\n\t+ %q", want, got) t.Errorf("unexpected error -want/+got\n\t- %q\n\t+ %q", want, got)
} }
})
}
} }
func TestController_CompileError(t *testing.T) { func TestController_CompileError(t *testing.T) {
ctrl, err := control.New(config) for name, config := range bothConfigs {
t.Run(name, func(t *testing.T) {
ctrl, err := control.New(config, zaptest.NewLogger(t))
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
} }
@ -380,10 +397,14 @@ func TestController_CompileError(t *testing.T) {
} else if got, want := err.Error(), "compilation failed: expected error"; got != want { } else if got, want := err.Error(), "compilation failed: expected error"; got != want {
t.Errorf("unexpected error -want/+got\n\t- %q\n\t+ %q", want, got) t.Errorf("unexpected error -want/+got\n\t- %q\n\t+ %q", want, got)
} }
})
}
} }
func TestController_ExecuteError(t *testing.T) { func TestController_ExecuteError(t *testing.T) {
ctrl, err := control.New(config) for name, config := range bothConfigs {
t.Run(name, func(t *testing.T) {
ctrl, err := control.New(config, zaptest.NewLogger(t))
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
} }
@ -420,13 +441,16 @@ func TestController_ExecuteError(t *testing.T) {
} else if got, want := err.Error(), "expected error"; got != want { } else if got, want := err.Error(), "expected error"; got != want {
t.Errorf("unexpected error -want/+got\n\t- %q\n\t+ %q", want, got) t.Errorf("unexpected error -want/+got\n\t- %q\n\t+ %q", want, got)
} }
})
}
} }
func TestController_LimitExceededError(t *testing.T) { func TestController_LimitExceededError(t *testing.T) {
const memoryBytesQuotaPerQuery = 64 const memoryBytesQuotaPerQuery = 64
config := config config := config
config.MemoryBytesQuotaPerQuery = memoryBytesQuotaPerQuery config.MemoryBytesQuotaPerQuery = memoryBytesQuotaPerQuery
ctrl, err := control.New(config) ctrl, err := control.New(config, zaptest.NewLogger(t))
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
} }
@ -497,7 +521,9 @@ func TestController_LimitExceededError(t *testing.T) {
} }
func TestController_CompilePanic(t *testing.T) { func TestController_CompilePanic(t *testing.T) {
ctrl, err := control.New(config) for name, config := range bothConfigs {
t.Run(name, func(t *testing.T) {
ctrl, err := control.New(config, zaptest.NewLogger(t))
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
} }
@ -515,10 +541,14 @@ func TestController_CompilePanic(t *testing.T) {
} else if !strings.Contains(err.Error(), "panic during compile step") { } else if !strings.Contains(err.Error(), "panic during compile step") {
t.Fatalf(`expected error to contain "panic during compile step" instead it contains "%v"`, err.Error()) t.Fatalf(`expected error to contain "panic during compile step" instead it contains "%v"`, err.Error())
} }
})
}
} }
func TestController_StartPanic(t *testing.T) { func TestController_StartPanic(t *testing.T) {
ctrl, err := control.New(config) for name, config := range bothConfigs {
t.Run(name, func(t *testing.T) {
ctrl, err := control.New(config, zaptest.NewLogger(t))
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
} }
@ -548,10 +578,14 @@ func TestController_StartPanic(t *testing.T) {
} else if !strings.Contains(err.Error(), "panic during start step") { } else if !strings.Contains(err.Error(), "panic during start step") {
t.Fatalf(`expected error to contain "panic during start step" instead it contains "%v"`, err.Error()) t.Fatalf(`expected error to contain "panic during start step" instead it contains "%v"`, err.Error())
} }
})
}
} }
func TestController_ShutdownWithRunningQuery(t *testing.T) { func TestController_ShutdownWithRunningQuery(t *testing.T) {
ctrl, err := control.New(config) for name, config := range bothConfigs {
t.Run(name, func(t *testing.T) {
ctrl, err := control.New(config, zaptest.NewLogger(t))
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
} }
@ -594,10 +628,14 @@ func TestController_ShutdownWithRunningQuery(t *testing.T) {
// query should be canceled and then shutdown should return. // query should be canceled and then shutdown should return.
shutdown(t, ctrl) shutdown(t, ctrl)
wg.Wait() wg.Wait()
})
}
} }
func TestController_ShutdownWithTimeout(t *testing.T) { func TestController_ShutdownWithTimeout(t *testing.T) {
ctrl, err := control.New(config) for name, config := range bothConfigs {
t.Run(name, func(t *testing.T) {
ctrl, err := control.New(config, zaptest.NewLogger(t))
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
} }
@ -645,10 +683,14 @@ func TestController_ShutdownWithTimeout(t *testing.T) {
t.Errorf("unexpected error -want/+got\n\t- %q\n\t+ %q", want, got) t.Errorf("unexpected error -want/+got\n\t- %q\n\t+ %q", want, got)
} }
cancel() cancel()
})
}
} }
func TestController_PerQueryMemoryLimit(t *testing.T) { func TestController_PerQueryMemoryLimit(t *testing.T) {
ctrl, err := control.New(config) for name, config := range bothConfigs {
t.Run(name, func(t *testing.T) {
ctrl, err := control.New(config, zaptest.NewLogger(t))
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
} }
@ -686,6 +728,8 @@ func TestController_PerQueryMemoryLimit(t *testing.T) {
if q.Err() == nil { if q.Err() == nil {
t.Fatal("expected error about memory limit exceeded") t.Fatal("expected error about memory limit exceeded")
} }
})
}
} }
func TestController_ConcurrencyQuota(t *testing.T) { func TestController_ConcurrencyQuota(t *testing.T) {
@ -697,7 +741,7 @@ func TestController_ConcurrencyQuota(t *testing.T) {
config := config config := config
config.ConcurrencyQuota = concurrencyQuota config.ConcurrencyQuota = concurrencyQuota
config.QueueSize = numQueries config.QueueSize = numQueries
ctrl, err := control.New(config) ctrl, err := control.New(config, zaptest.NewLogger(t))
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
} }
@ -763,7 +807,7 @@ func TestController_QueueSize(t *testing.T) {
config := config config := config
config.ConcurrencyQuota = concurrencyQuota config.ConcurrencyQuota = concurrencyQuota
config.QueueSize = queueSize config.QueueSize = queueSize
ctrl, err := control.New(config) ctrl, err := control.New(config, zaptest.NewLogger(t))
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
} }
@ -824,6 +868,101 @@ func TestController_QueueSize(t *testing.T) {
} }
} }
// Test that rapidly starting and canceling the query and then calling done will correctly
// cancel the query and not result in a race condition.
func TestController_CancelDone_Unlimited(t *testing.T) {
config := config
ctrl, err := control.New(config, zaptest.NewLogger(t))
if err != nil {
t.Fatal(err)
}
defer shutdown(t, ctrl)
compiler := &mock.Compiler{
CompileFn: func(ctx context.Context) (flux.Program, error) {
return &mock.Program{
ExecuteFn: func(ctx context.Context, q *mock.Query, alloc *memory.Allocator) {
// Ensure the query takes a little bit of time so the cancel actually cancels something.
t := time.NewTimer(time.Second)
defer t.Stop()
select {
case <-t.C:
case <-ctx.Done():
}
},
}, nil
},
}
var wg sync.WaitGroup
for i := 0; i < 100; i++ {
wg.Add(1)
go func() {
defer wg.Done()
q, err := ctrl.Query(context.Background(), makeRequest(compiler))
if err != nil {
t.Errorf("unexpected error: %s", err)
return
}
q.Cancel()
q.Done()
}()
}
wg.Wait()
}
// Test that rapidly starts and calls done on queries without reading the result.
func TestController_DoneWithoutRead_Unlimited(t *testing.T) {
config := config
ctrl, err := control.New(config, zaptest.NewLogger(t))
if err != nil {
t.Fatal(err)
}
defer shutdown(t, ctrl)
compiler := &mock.Compiler{
CompileFn: func(ctx context.Context) (flux.Program, error) {
return &mock.Program{
ExecuteFn: func(ctx context.Context, q *mock.Query, alloc *memory.Allocator) {
// Ensure the query takes a little bit of time so the cancel actually cancels something.
t := time.NewTimer(time.Second)
defer t.Stop()
select {
case <-t.C:
q.ResultsCh <- &executetest.Result{
Nm: "_result",
Tbls: []*executetest.Table{},
}
case <-ctx.Done():
}
},
}, nil
},
}
var wg sync.WaitGroup
for i := 0; i < 100; i++ {
wg.Add(1)
go func() {
defer wg.Done()
q, err := ctrl.Query(context.Background(), makeRequest(compiler))
if err != nil {
t.Errorf("unexpected error: %s", err)
return
}
// If we call done without reading anything it should work just fine.
q.Done()
}()
}
wg.Wait()
}
// Test that rapidly starting and canceling the query and then calling done will correctly // Test that rapidly starting and canceling the query and then calling done will correctly
// cancel the query and not result in a race condition. // cancel the query and not result in a race condition.
func TestController_CancelDone(t *testing.T) { func TestController_CancelDone(t *testing.T) {
@ -831,7 +970,7 @@ func TestController_CancelDone(t *testing.T) {
config.ConcurrencyQuota = 10 config.ConcurrencyQuota = 10
config.QueueSize = 200 config.QueueSize = 200
ctrl, err := control.New(config) ctrl, err := control.New(config, zaptest.NewLogger(t))
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
} }
@ -878,7 +1017,7 @@ func TestController_DoneWithoutRead(t *testing.T) {
config.ConcurrencyQuota = 10 config.ConcurrencyQuota = 10
config.QueueSize = 200 config.QueueSize = 200
ctrl, err := control.New(config) ctrl, err := control.New(config, zaptest.NewLogger(t))
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
} }
@ -927,10 +1066,13 @@ func TestController_DoneWithoutRead(t *testing.T) {
// but we would go above the maximum amount of available memory. // but we would go above the maximum amount of available memory.
func TestController_Error_MaxMemory(t *testing.T) { func TestController_Error_MaxMemory(t *testing.T) {
config := config config := config
config.InitialMemoryBytesQuotaPerQuery = config.MemoryBytesQuotaPerQuery / 2 config.InitialMemoryBytesQuotaPerQuery = 512
config.MaxMemoryBytes = config.MemoryBytesQuotaPerQuery * 2 config.MaxMemoryBytes = 2048
config.MemoryBytesQuotaPerQuery = 512
config.QueueSize = 1
config.ConcurrencyQuota = 1
ctrl, err := control.New(config) ctrl, err := control.New(config, zaptest.NewLogger(t))
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
} }
@ -988,7 +1130,7 @@ func TestController_NoisyNeighbor(t *testing.T) {
// Set the queue length to something that can accommodate the input. // Set the queue length to something that can accommodate the input.
config.QueueSize = 1000 config.QueueSize = 1000
ctrl, err := control.New(config) ctrl, err := control.New(config, zaptest.NewLogger(t))
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
} }
@ -1090,7 +1232,11 @@ func TestController_Error_NoRemainingMemory(t *testing.T) {
// The maximum memory available on the system is double the initial quota. // The maximum memory available on the system is double the initial quota.
config.MaxMemoryBytes = config.InitialMemoryBytesQuotaPerQuery * 2 config.MaxMemoryBytes = config.InitialMemoryBytesQuotaPerQuery * 2
ctrl, err := control.New(config) // Need to limit concurrency along with max memory or the config validation complains
config.ConcurrencyQuota = 1
config.QueueSize = 1
ctrl, err := control.New(config, zaptest.NewLogger(t))
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
} }
@ -1131,9 +1277,12 @@ func TestController_Error_NoRemainingMemory(t *testing.T) {
func TestController_MemoryRelease(t *testing.T) { func TestController_MemoryRelease(t *testing.T) {
config := config config := config
config.InitialMemoryBytesQuotaPerQuery = 16 config.InitialMemoryBytesQuotaPerQuery = 16
config.MemoryBytesQuotaPerQuery = 1024
config.MaxMemoryBytes = config.MemoryBytesQuotaPerQuery * 2 config.MaxMemoryBytes = config.MemoryBytesQuotaPerQuery * 2
config.QueueSize = 1
config.ConcurrencyQuota = 1
ctrl, err := control.New(config) ctrl, err := control.New(config, zaptest.NewLogger(t))
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
} }
@ -1179,8 +1328,10 @@ func TestController_IrregularMemoryQuota(t *testing.T) {
config.InitialMemoryBytesQuotaPerQuery = 64 config.InitialMemoryBytesQuotaPerQuery = 64
config.MemoryBytesQuotaPerQuery = 768 config.MemoryBytesQuotaPerQuery = 768
config.MaxMemoryBytes = config.MemoryBytesQuotaPerQuery * 2 config.MaxMemoryBytes = config.MemoryBytesQuotaPerQuery * 2
config.QueueSize = 1
config.ConcurrencyQuota = 1
ctrl, err := control.New(config) ctrl, err := control.New(config, zaptest.NewLogger(t))
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
} }
@ -1239,7 +1390,7 @@ func TestController_ReserveMemoryWithoutExceedingMax(t *testing.T) {
// Set the queue length to something that can accommodate the input. // Set the queue length to something that can accommodate the input.
config.QueueSize = 1000 config.QueueSize = 1000
ctrl, err := control.New(config) ctrl, err := control.New(config, zaptest.NewLogger(t))
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
} }
@ -1303,7 +1454,7 @@ func TestController_QueryTracing(t *testing.T) {
const memoryBytesQuotaPerQuery = 64 const memoryBytesQuotaPerQuery = 64
config := config config := config
config.MemoryBytesQuotaPerQuery = memoryBytesQuotaPerQuery config.MemoryBytesQuotaPerQuery = memoryBytesQuotaPerQuery
ctrl, err := control.New(config) ctrl, err := control.New(config, zaptest.NewLogger(t))
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
} }

View File

@ -29,89 +29,62 @@ const (
) )
func newControllerMetrics(labels []string) *controllerMetrics { func newControllerMetrics(labels []string) *controllerMetrics {
const (
namespace = "query"
subsystem = "control"
)
return &controllerMetrics{ return &controllerMetrics{
requests: prometheus.NewCounterVec(prometheus.CounterOpts{ requests: prometheus.NewCounterVec(prometheus.CounterOpts{
Namespace: namespace, Name: "qc_requests_total",
Subsystem: subsystem,
Name: "requests_total",
Help: "Count of the query requests", Help: "Count of the query requests",
}, append(labels, "result")), }, append(labels, "result")),
functions: prometheus.NewCounterVec(prometheus.CounterOpts{ functions: prometheus.NewCounterVec(prometheus.CounterOpts{
Namespace: namespace, Name: "qc_functions_total",
Subsystem: subsystem,
Name: "functions_total",
Help: "Count of functions in queries", Help: "Count of functions in queries",
}, append(labels, "function")), }, append(labels, "function")),
all: prometheus.NewGaugeVec(prometheus.GaugeOpts{ all: prometheus.NewGaugeVec(prometheus.GaugeOpts{
Namespace: namespace, Name: "qc_all_active",
Subsystem: subsystem,
Name: "all_active",
Help: "Number of queries in all states", Help: "Number of queries in all states",
}, labels), }, labels),
compiling: prometheus.NewGaugeVec(prometheus.GaugeOpts{ compiling: prometheus.NewGaugeVec(prometheus.GaugeOpts{
Namespace: namespace, Name: "qc_compiling_active",
Subsystem: subsystem,
Name: "compiling_active",
Help: "Number of queries actively compiling", Help: "Number of queries actively compiling",
}, append(labels, "compiler_type")), }, append(labels, "compiler_type")),
queueing: prometheus.NewGaugeVec(prometheus.GaugeOpts{ queueing: prometheus.NewGaugeVec(prometheus.GaugeOpts{
Namespace: namespace, Name: "qc_queueing_active",
Subsystem: subsystem,
Name: "queueing_active",
Help: "Number of queries actively queueing", Help: "Number of queries actively queueing",
}, labels), }, labels),
executing: prometheus.NewGaugeVec(prometheus.GaugeOpts{ executing: prometheus.NewGaugeVec(prometheus.GaugeOpts{
Namespace: namespace, Name: "qc_executing_active",
Subsystem: subsystem,
Name: "executing_active",
Help: "Number of queries actively executing", Help: "Number of queries actively executing",
}, labels), }, labels),
memoryUnused: prometheus.NewGaugeVec(prometheus.GaugeOpts{ memoryUnused: prometheus.NewGaugeVec(prometheus.GaugeOpts{
Namespace: namespace, Name: "qc_memory_unused_bytes",
Subsystem: subsystem,
Name: "memory_unused_bytes",
Help: "The free memory as seen by the internal memory manager", Help: "The free memory as seen by the internal memory manager",
}, labels), }, labels),
allDur: prometheus.NewHistogramVec(prometheus.HistogramOpts{ allDur: prometheus.NewHistogramVec(prometheus.HistogramOpts{
Namespace: namespace, Name: "qc_all_duration_seconds",
Subsystem: subsystem,
Name: "all_duration_seconds",
Help: "Histogram of total times spent in all query states", Help: "Histogram of total times spent in all query states",
Buckets: prometheus.ExponentialBuckets(1e-3, 5, 7), Buckets: prometheus.ExponentialBuckets(1e-3, 5, 7),
}, labels), }, labels),
compilingDur: prometheus.NewHistogramVec(prometheus.HistogramOpts{ compilingDur: prometheus.NewHistogramVec(prometheus.HistogramOpts{
Namespace: namespace, Name: "qc_compiling_duration_seconds",
Subsystem: subsystem,
Name: "compiling_duration_seconds",
Help: "Histogram of times spent compiling queries", Help: "Histogram of times spent compiling queries",
Buckets: prometheus.ExponentialBuckets(1e-3, 5, 7), Buckets: prometheus.ExponentialBuckets(1e-3, 5, 7),
}, append(labels, "compiler_type")), }, append(labels, "compiler_type")),
queueingDur: prometheus.NewHistogramVec(prometheus.HistogramOpts{ queueingDur: prometheus.NewHistogramVec(prometheus.HistogramOpts{
Namespace: namespace, Name: "qc_queueing_duration_seconds",
Subsystem: subsystem,
Name: "queueing_duration_seconds",
Help: "Histogram of times spent queueing queries", Help: "Histogram of times spent queueing queries",
Buckets: prometheus.ExponentialBuckets(1e-3, 5, 7), Buckets: prometheus.ExponentialBuckets(1e-3, 5, 7),
}, labels), }, labels),
executingDur: prometheus.NewHistogramVec(prometheus.HistogramOpts{ executingDur: prometheus.NewHistogramVec(prometheus.HistogramOpts{
Namespace: namespace, Name: "qc_executing_duration_seconds",
Subsystem: subsystem,
Name: "executing_duration_seconds",
Help: "Histogram of times spent executing queries", Help: "Histogram of times spent executing queries",
Buckets: prometheus.ExponentialBuckets(1e-3, 5, 7), Buckets: prometheus.ExponentialBuckets(1e-3, 5, 7),
}, labels), }, labels),

View File

@ -2,12 +2,13 @@ package backend_test
import ( import (
"context" "context"
"github.com/influxdata/influxdb/v2/kit/platform"
"io/ioutil" "io/ioutil"
"os" "os"
"testing" "testing"
"time" "time"
"github.com/influxdata/influxdb/v2/kit/platform"
"github.com/golang/mock/gomock" "github.com/golang/mock/gomock"
"github.com/influxdata/flux" "github.com/influxdata/flux"
"github.com/influxdata/influxdb/v2" "github.com/influxdata/influxdb/v2"
@ -216,10 +217,9 @@ func newAnalyticalBackend(t *testing.T, orgSvc influxdb.OrganizationService, buc
ConcurrencyQuota: concurrencyQuota, ConcurrencyQuota: concurrencyQuota,
MemoryBytesQuotaPerQuery: int64(memoryBytesQuotaPerQuery), MemoryBytesQuotaPerQuery: int64(memoryBytesQuotaPerQuery),
QueueSize: queueSize, QueueSize: queueSize,
Logger: logger.With(zap.String("service", "storage-reads")),
} }
queryController, err := control.New(cc) queryController, err := control.New(cc, logger.With(zap.String("service", "storage-reads")))
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
} }