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`
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]

View File

@ -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,

View File

@ -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

View File

@ -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

View File

@ -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()

View File

@ -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)
}

View File

@ -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),

View File

@ -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)
}