feat(flux): allow values of 0 to disable controller limits (#21127)
Co-authored-by: Sam Arnold <sarnold@influxdata.com>pull/21150/head
parent
28ae9eac75
commit
ff6b55aca4
13
CHANGELOG.md
13
CHANGELOG.md
|
@ -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`
|
||||
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
|
||||
|
||||
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. [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. [21127](https://github.com/influxdata/influxdb/pull/21127): Allow for disabling concurrency-limits in Flux controller.
|
||||
|
||||
### 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. [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. [21127](https://github.com/influxdata/influxdb/pull/21127): Fix race condition in Flux controller shutdown.
|
||||
|
||||
## v2.0.4 [2021-02-08]
|
||||
|
||||
|
|
|
@ -205,11 +205,11 @@ func newOpts(viper *viper.Viper) *InfluxdOpts {
|
|||
|
||||
NoTasks: false,
|
||||
|
||||
ConcurrencyQuota: 10,
|
||||
ConcurrencyQuota: 0,
|
||||
InitialMemoryBytesQuotaPerQuery: 0,
|
||||
MemoryBytesQuotaPerQuery: MaxInt,
|
||||
MaxMemoryBytes: 0,
|
||||
QueueSize: 10,
|
||||
QueueSize: 0,
|
||||
|
||||
Testing: false,
|
||||
TestingAlwaysAllowSetup: false,
|
||||
|
@ -405,7 +405,7 @@ func (o *InfluxdOpts) bindCliOpts() []cli.Opt {
|
|||
DestP: &o.ConcurrencyQuota,
|
||||
Flag: "query-concurrency",
|
||||
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,
|
||||
|
@ -423,13 +423,13 @@ func (o *InfluxdOpts) bindCliOpts() []cli.Opt {
|
|||
DestP: &o.MaxMemoryBytes,
|
||||
Flag: "query-max-memory-bytes",
|
||||
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,
|
||||
Flag: "query-queue-size",
|
||||
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,
|
||||
|
|
|
@ -435,9 +435,8 @@ func (m *Launcher) run(ctx context.Context, opts *InfluxdOpts) (err error) {
|
|||
MemoryBytesQuotaPerQuery: opts.MemoryBytesQuotaPerQuery,
|
||||
MaxMemoryBytes: opts.MaxMemoryBytes,
|
||||
QueueSize: opts.QueueSize,
|
||||
Logger: m.log.With(zap.String("service", "storage-reads")),
|
||||
ExecutorDependencies: dependencyList,
|
||||
})
|
||||
}, m.log.With(zap.String("service", "storage-reads")))
|
||||
if err != nil {
|
||||
m.log.Error("Failed to create query controller", zap.Error(err))
|
||||
return err
|
||||
|
|
|
@ -5,7 +5,6 @@ import (
|
|||
"context"
|
||||
"errors"
|
||||
"fmt"
|
||||
errors2 "github.com/influxdata/influxdb/v2/kit/platform/errors"
|
||||
"html/template"
|
||||
"io"
|
||||
"io/ioutil"
|
||||
|
@ -16,6 +15,8 @@ import (
|
|||
"testing"
|
||||
"time"
|
||||
|
||||
errors2 "github.com/influxdata/influxdb/v2/kit/platform/errors"
|
||||
|
||||
"github.com/influxdata/flux"
|
||||
"github.com/influxdata/flux/csv"
|
||||
"github.com/influxdata/flux/execute"
|
||||
|
@ -122,7 +123,7 @@ func getMemoryUnused(t *testing.T, reg *prom.Registry) int64 {
|
|||
t.Fatal(err)
|
||||
}
|
||||
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)
|
||||
}
|
||||
}
|
||||
|
@ -253,6 +254,7 @@ func TestLauncher_QueryMemoryLimits(t *testing.T) {
|
|||
name: "ok - initial memory bytes, memory bytes, and max memory set",
|
||||
setOpts: func(o *launcher.InfluxdOpts) {
|
||||
o.ConcurrencyQuota = 1
|
||||
o.QueueSize = 1
|
||||
o.InitialMemoryBytesQuotaPerQuery = 100
|
||||
o.MaxMemoryBytes = 1048576 // 1MB
|
||||
},
|
||||
|
@ -264,6 +266,7 @@ func TestLauncher_QueryMemoryLimits(t *testing.T) {
|
|||
name: "error - memory bytes and max memory set",
|
||||
setOpts: func(o *launcher.InfluxdOpts) {
|
||||
o.ConcurrencyQuota = 1
|
||||
o.QueueSize = 1
|
||||
o.MemoryBytesQuotaPerQuery = 1
|
||||
o.MaxMemoryBytes = 100
|
||||
},
|
||||
|
@ -275,6 +278,7 @@ func TestLauncher_QueryMemoryLimits(t *testing.T) {
|
|||
name: "error - initial memory bytes and max memory set",
|
||||
setOpts: func(o *launcher.InfluxdOpts) {
|
||||
o.ConcurrencyQuota = 1
|
||||
o.QueueSize = 1
|
||||
o.InitialMemoryBytesQuotaPerQuery = 1
|
||||
o.MaxMemoryBytes = 100
|
||||
},
|
||||
|
@ -286,6 +290,7 @@ func TestLauncher_QueryMemoryLimits(t *testing.T) {
|
|||
name: "error - initial memory bytes, memory bytes, and max memory set",
|
||||
setOpts: func(o *launcher.InfluxdOpts) {
|
||||
o.ConcurrencyQuota = 1
|
||||
o.QueueSize = 1
|
||||
o.InitialMemoryBytesQuotaPerQuery = 1
|
||||
o.MemoryBytesQuotaPerQuery = 50
|
||||
o.MaxMemoryBytes = 100
|
||||
|
|
|
@ -109,7 +109,6 @@ type Config struct {
|
|||
// this to follow suit.
|
||||
QueueSize int32
|
||||
|
||||
Logger *zap.Logger
|
||||
// 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 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 {
|
||||
if c.ConcurrencyQuota <= 0 {
|
||||
return errors.New("ConcurrencyQuota must be positive")
|
||||
if c.ConcurrencyQuota < 0 {
|
||||
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")
|
||||
}
|
||||
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)
|
||||
}
|
||||
}
|
||||
if c.QueueSize <= 0 {
|
||||
return errors.New("QueueSize must be positive")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
|
@ -163,13 +172,12 @@ func (c *Config) Validate() error {
|
|||
|
||||
type QueryID uint64
|
||||
|
||||
func New(config Config) (*Controller, error) {
|
||||
func New(config Config, logger *zap.Logger) (*Controller, error) {
|
||||
c, err := config.complete()
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "invalid controller config")
|
||||
}
|
||||
c.MetricLabelKeys = append(c.MetricLabelKeys, orgLabel)
|
||||
logger := c.Logger
|
||||
metricLabelKeys := append(c.MetricLabelKeys, orgLabel)
|
||||
if logger == nil {
|
||||
logger = zap.NewNop()
|
||||
}
|
||||
|
@ -189,18 +197,23 @@ func New(config Config) (*Controller, error) {
|
|||
} else {
|
||||
mm.unlimited = true
|
||||
}
|
||||
queryQueue := make(chan *Query, c.QueueSize)
|
||||
if c.ConcurrencyQuota == 0 {
|
||||
queryQueue = nil
|
||||
}
|
||||
ctrl := &Controller{
|
||||
config: c,
|
||||
queries: make(map[QueryID]*Query),
|
||||
queryQueue: make(chan *Query, c.QueueSize),
|
||||
queryQueue: queryQueue,
|
||||
done: make(chan struct{}),
|
||||
abort: make(chan struct{}),
|
||||
memory: mm,
|
||||
log: logger,
|
||||
metrics: newControllerMetrics(c.MetricLabelKeys),
|
||||
labelKeys: c.MetricLabelKeys,
|
||||
metrics: newControllerMetrics(metricLabelKeys),
|
||||
labelKeys: metricLabelKeys,
|
||||
dependencies: c.ExecutorDependencies,
|
||||
}
|
||||
if c.ConcurrencyQuota != 0 {
|
||||
quota := int(c.ConcurrencyQuota)
|
||||
ctrl.wg.Add(quota)
|
||||
for i := 0; i < quota; i++ {
|
||||
|
@ -209,6 +222,7 @@ func New(config Config) (*Controller, error) {
|
|||
ctrl.processQueryQueue()
|
||||
}()
|
||||
}
|
||||
}
|
||||
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 {
|
||||
case c.queryQueue <- q:
|
||||
default:
|
||||
|
@ -393,6 +426,7 @@ func (c *Controller) enqueueQuery(q *Query) error {
|
|||
Msg: "queue length exceeded",
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
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
|
||||
// queries have been finished or until the Context has been canceled.
|
||||
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
|
||||
// accept new queries.
|
||||
func() {
|
||||
c.queriesMu.Lock()
|
||||
defer c.queriesMu.Unlock()
|
||||
if !c.shutdown {
|
||||
c.shutdown = true
|
||||
if len(c.queries) == 0 {
|
||||
c.queriesMu.Unlock()
|
||||
return nil
|
||||
// We hold the lock. No other queries can be spawned.
|
||||
// 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.
|
||||
c.queriesMu.RLock()
|
||||
|
|
|
@ -4,6 +4,7 @@ import (
|
|||
"context"
|
||||
"errors"
|
||||
"fmt"
|
||||
"math"
|
||||
"strings"
|
||||
"sync"
|
||||
"testing"
|
||||
|
@ -20,12 +21,11 @@ import (
|
|||
"github.com/influxdata/flux/plan"
|
||||
"github.com/influxdata/flux/plan/plantest"
|
||||
"github.com/influxdata/flux/stdlib/universe"
|
||||
_ "github.com/influxdata/influxdb/v2/fluxinit/static"
|
||||
"github.com/influxdata/influxdb/v2/kit/feature"
|
||||
pmock "github.com/influxdata/influxdb/v2/mock"
|
||||
"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/stdlib/influxdata/influxdb"
|
||||
"github.com/opentracing/opentracing-go"
|
||||
"github.com/opentracing/opentracing-go/mocktracer"
|
||||
"github.com/prometheus/client_golang/prometheus"
|
||||
|
@ -48,15 +48,14 @@ var (
|
|||
},
|
||||
}
|
||||
config = control.Config{
|
||||
ConcurrencyQuota: 1,
|
||||
MemoryBytesQuotaPerQuery: 1024,
|
||||
QueueSize: 1,
|
||||
ExecutorDependencies: []flux.Dependency{
|
||||
influxdb.Dependencies{
|
||||
FluxDeps: executetest.NewTestExecuteDependencies(),
|
||||
},
|
||||
},
|
||||
MemoryBytesQuotaPerQuery: math.MaxInt64,
|
||||
}
|
||||
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 {
|
||||
|
@ -79,7 +78,7 @@ func validateRequestTotals(t testing.TB, reg *prometheus.Registry, success, comp
|
|||
validate := func(name string, want int) {
|
||||
m := FindMetric(
|
||||
metrics,
|
||||
"query_control_requests_total",
|
||||
"qc_requests_total",
|
||||
map[string]string{
|
||||
"result": name,
|
||||
"org": "",
|
||||
|
@ -107,7 +106,7 @@ func validateUnusedMemory(t testing.TB, reg *prometheus.Registry, c control.Conf
|
|||
}
|
||||
m := FindMetric(
|
||||
metrics,
|
||||
"query_control_memory_unused_bytes",
|
||||
"qc_memory_unused_bytes",
|
||||
map[string]string{
|
||||
"org": "",
|
||||
},
|
||||
|
@ -123,7 +122,9 @@ func validateUnusedMemory(t testing.TB, reg *prometheus.Registry, c control.Conf
|
|||
}
|
||||
|
||||
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 {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
@ -159,10 +160,14 @@ func TestController_QuerySuccess(t *testing.T) {
|
|||
t.Error("expected total duration to be above zero")
|
||||
}
|
||||
validateRequestTotals(t, reg, 1, 0, 0, 0)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
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 {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
@ -185,10 +190,14 @@ func TestController_QueryCompileError(t *testing.T) {
|
|||
}
|
||||
|
||||
validateRequestTotals(t, reg, 0, 1, 0, 0)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
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 {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
@ -232,12 +241,14 @@ func TestController_QueryRuntimeError(t *testing.T) {
|
|||
t.Error("expected total duration to be above zero")
|
||||
}
|
||||
validateRequestTotals(t, reg, 0, 0, 1, 0)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
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")
|
||||
|
||||
ctrl, err := control.New(config)
|
||||
ctrl, err := control.New(config, zaptest.NewLogger(t))
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
@ -341,7 +352,9 @@ func findMetric(mfs []*dto.MetricFamily, name string, labels map[string]string)
|
|||
}
|
||||
|
||||
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 {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
@ -358,10 +371,14 @@ func TestController_AfterShutdown(t *testing.T) {
|
|||
} 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)
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
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 {
|
||||
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 {
|
||||
t.Errorf("unexpected error -want/+got\n\t- %q\n\t+ %q", want, got)
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
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 {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
@ -420,13 +441,16 @@ func TestController_ExecuteError(t *testing.T) {
|
|||
} else if got, want := err.Error(), "expected error"; got != want {
|
||||
t.Errorf("unexpected error -want/+got\n\t- %q\n\t+ %q", want, got)
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestController_LimitExceededError(t *testing.T) {
|
||||
|
||||
const memoryBytesQuotaPerQuery = 64
|
||||
config := config
|
||||
config.MemoryBytesQuotaPerQuery = memoryBytesQuotaPerQuery
|
||||
ctrl, err := control.New(config)
|
||||
ctrl, err := control.New(config, zaptest.NewLogger(t))
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
@ -497,7 +521,9 @@ func TestController_LimitExceededError(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 {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
@ -515,10 +541,14 @@ func TestController_CompilePanic(t *testing.T) {
|
|||
} 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())
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
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 {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
@ -548,10 +578,14 @@ func TestController_StartPanic(t *testing.T) {
|
|||
} 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())
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
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 {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
@ -594,10 +628,14 @@ func TestController_ShutdownWithRunningQuery(t *testing.T) {
|
|||
// query should be canceled and then shutdown should return.
|
||||
shutdown(t, ctrl)
|
||||
wg.Wait()
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
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 {
|
||||
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)
|
||||
}
|
||||
cancel()
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
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 {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
@ -686,6 +728,8 @@ func TestController_PerQueryMemoryLimit(t *testing.T) {
|
|||
if q.Err() == nil {
|
||||
t.Fatal("expected error about memory limit exceeded")
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestController_ConcurrencyQuota(t *testing.T) {
|
||||
|
@ -697,7 +741,7 @@ func TestController_ConcurrencyQuota(t *testing.T) {
|
|||
config := config
|
||||
config.ConcurrencyQuota = concurrencyQuota
|
||||
config.QueueSize = numQueries
|
||||
ctrl, err := control.New(config)
|
||||
ctrl, err := control.New(config, zaptest.NewLogger(t))
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
@ -763,7 +807,7 @@ func TestController_QueueSize(t *testing.T) {
|
|||
config := config
|
||||
config.ConcurrencyQuota = concurrencyQuota
|
||||
config.QueueSize = queueSize
|
||||
ctrl, err := control.New(config)
|
||||
ctrl, err := control.New(config, zaptest.NewLogger(t))
|
||||
if err != nil {
|
||||
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
|
||||
// cancel the query and not result in a race condition.
|
||||
func TestController_CancelDone(t *testing.T) {
|
||||
|
@ -831,7 +970,7 @@ func TestController_CancelDone(t *testing.T) {
|
|||
config.ConcurrencyQuota = 10
|
||||
config.QueueSize = 200
|
||||
|
||||
ctrl, err := control.New(config)
|
||||
ctrl, err := control.New(config, zaptest.NewLogger(t))
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
@ -878,7 +1017,7 @@ func TestController_DoneWithoutRead(t *testing.T) {
|
|||
config.ConcurrencyQuota = 10
|
||||
config.QueueSize = 200
|
||||
|
||||
ctrl, err := control.New(config)
|
||||
ctrl, err := control.New(config, zaptest.NewLogger(t))
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
@ -927,10 +1066,13 @@ func TestController_DoneWithoutRead(t *testing.T) {
|
|||
// but we would go above the maximum amount of available memory.
|
||||
func TestController_Error_MaxMemory(t *testing.T) {
|
||||
config := config
|
||||
config.InitialMemoryBytesQuotaPerQuery = config.MemoryBytesQuotaPerQuery / 2
|
||||
config.MaxMemoryBytes = config.MemoryBytesQuotaPerQuery * 2
|
||||
config.InitialMemoryBytesQuotaPerQuery = 512
|
||||
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 {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
@ -988,7 +1130,7 @@ func TestController_NoisyNeighbor(t *testing.T) {
|
|||
// Set the queue length to something that can accommodate the input.
|
||||
config.QueueSize = 1000
|
||||
|
||||
ctrl, err := control.New(config)
|
||||
ctrl, err := control.New(config, zaptest.NewLogger(t))
|
||||
if err != nil {
|
||||
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.
|
||||
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 {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
@ -1131,9 +1277,12 @@ func TestController_Error_NoRemainingMemory(t *testing.T) {
|
|||
func TestController_MemoryRelease(t *testing.T) {
|
||||
config := config
|
||||
config.InitialMemoryBytesQuotaPerQuery = 16
|
||||
config.MemoryBytesQuotaPerQuery = 1024
|
||||
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 {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
@ -1179,8 +1328,10 @@ func TestController_IrregularMemoryQuota(t *testing.T) {
|
|||
config.InitialMemoryBytesQuotaPerQuery = 64
|
||||
config.MemoryBytesQuotaPerQuery = 768
|
||||
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 {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
@ -1239,7 +1390,7 @@ func TestController_ReserveMemoryWithoutExceedingMax(t *testing.T) {
|
|||
// Set the queue length to something that can accommodate the input.
|
||||
config.QueueSize = 1000
|
||||
|
||||
ctrl, err := control.New(config)
|
||||
ctrl, err := control.New(config, zaptest.NewLogger(t))
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
@ -1303,7 +1454,7 @@ func TestController_QueryTracing(t *testing.T) {
|
|||
const memoryBytesQuotaPerQuery = 64
|
||||
config := config
|
||||
config.MemoryBytesQuotaPerQuery = memoryBytesQuotaPerQuery
|
||||
ctrl, err := control.New(config)
|
||||
ctrl, err := control.New(config, zaptest.NewLogger(t))
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
|
|
@ -29,89 +29,62 @@ const (
|
|||
)
|
||||
|
||||
func newControllerMetrics(labels []string) *controllerMetrics {
|
||||
const (
|
||||
namespace = "query"
|
||||
subsystem = "control"
|
||||
)
|
||||
|
||||
return &controllerMetrics{
|
||||
requests: prometheus.NewCounterVec(prometheus.CounterOpts{
|
||||
Namespace: namespace,
|
||||
Subsystem: subsystem,
|
||||
Name: "requests_total",
|
||||
Name: "qc_requests_total",
|
||||
Help: "Count of the query requests",
|
||||
}, append(labels, "result")),
|
||||
|
||||
functions: prometheus.NewCounterVec(prometheus.CounterOpts{
|
||||
Namespace: namespace,
|
||||
Subsystem: subsystem,
|
||||
Name: "functions_total",
|
||||
Name: "qc_functions_total",
|
||||
Help: "Count of functions in queries",
|
||||
}, append(labels, "function")),
|
||||
|
||||
all: prometheus.NewGaugeVec(prometheus.GaugeOpts{
|
||||
Namespace: namespace,
|
||||
Subsystem: subsystem,
|
||||
Name: "all_active",
|
||||
Name: "qc_all_active",
|
||||
Help: "Number of queries in all states",
|
||||
}, labels),
|
||||
|
||||
compiling: prometheus.NewGaugeVec(prometheus.GaugeOpts{
|
||||
Namespace: namespace,
|
||||
Subsystem: subsystem,
|
||||
Name: "compiling_active",
|
||||
Name: "qc_compiling_active",
|
||||
Help: "Number of queries actively compiling",
|
||||
}, append(labels, "compiler_type")),
|
||||
|
||||
queueing: prometheus.NewGaugeVec(prometheus.GaugeOpts{
|
||||
Namespace: namespace,
|
||||
Subsystem: subsystem,
|
||||
Name: "queueing_active",
|
||||
Name: "qc_queueing_active",
|
||||
Help: "Number of queries actively queueing",
|
||||
}, labels),
|
||||
|
||||
executing: prometheus.NewGaugeVec(prometheus.GaugeOpts{
|
||||
Namespace: namespace,
|
||||
Subsystem: subsystem,
|
||||
Name: "executing_active",
|
||||
Name: "qc_executing_active",
|
||||
Help: "Number of queries actively executing",
|
||||
}, labels),
|
||||
|
||||
memoryUnused: prometheus.NewGaugeVec(prometheus.GaugeOpts{
|
||||
Namespace: namespace,
|
||||
Subsystem: subsystem,
|
||||
Name: "memory_unused_bytes",
|
||||
Name: "qc_memory_unused_bytes",
|
||||
Help: "The free memory as seen by the internal memory manager",
|
||||
}, labels),
|
||||
|
||||
allDur: prometheus.NewHistogramVec(prometheus.HistogramOpts{
|
||||
Namespace: namespace,
|
||||
Subsystem: subsystem,
|
||||
Name: "all_duration_seconds",
|
||||
Name: "qc_all_duration_seconds",
|
||||
Help: "Histogram of total times spent in all query states",
|
||||
Buckets: prometheus.ExponentialBuckets(1e-3, 5, 7),
|
||||
}, labels),
|
||||
|
||||
compilingDur: prometheus.NewHistogramVec(prometheus.HistogramOpts{
|
||||
Namespace: namespace,
|
||||
Subsystem: subsystem,
|
||||
Name: "compiling_duration_seconds",
|
||||
Name: "qc_compiling_duration_seconds",
|
||||
Help: "Histogram of times spent compiling queries",
|
||||
Buckets: prometheus.ExponentialBuckets(1e-3, 5, 7),
|
||||
}, append(labels, "compiler_type")),
|
||||
|
||||
queueingDur: prometheus.NewHistogramVec(prometheus.HistogramOpts{
|
||||
Namespace: namespace,
|
||||
Subsystem: subsystem,
|
||||
Name: "queueing_duration_seconds",
|
||||
Name: "qc_queueing_duration_seconds",
|
||||
Help: "Histogram of times spent queueing queries",
|
||||
Buckets: prometheus.ExponentialBuckets(1e-3, 5, 7),
|
||||
}, labels),
|
||||
|
||||
executingDur: prometheus.NewHistogramVec(prometheus.HistogramOpts{
|
||||
Namespace: namespace,
|
||||
Subsystem: subsystem,
|
||||
Name: "executing_duration_seconds",
|
||||
Name: "qc_executing_duration_seconds",
|
||||
Help: "Histogram of times spent executing queries",
|
||||
Buckets: prometheus.ExponentialBuckets(1e-3, 5, 7),
|
||||
}, labels),
|
||||
|
|
|
@ -2,12 +2,13 @@ package backend_test
|
|||
|
||||
import (
|
||||
"context"
|
||||
"github.com/influxdata/influxdb/v2/kit/platform"
|
||||
"io/ioutil"
|
||||
"os"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/influxdata/influxdb/v2/kit/platform"
|
||||
|
||||
"github.com/golang/mock/gomock"
|
||||
"github.com/influxdata/flux"
|
||||
"github.com/influxdata/influxdb/v2"
|
||||
|
@ -216,10 +217,9 @@ func newAnalyticalBackend(t *testing.T, orgSvc influxdb.OrganizationService, buc
|
|||
ConcurrencyQuota: concurrencyQuota,
|
||||
MemoryBytesQuotaPerQuery: int64(memoryBytesQuotaPerQuery),
|
||||
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 {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue