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.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 {
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,25 +197,31 @@ 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,
}
quota := int(c.ConcurrencyQuota)
ctrl.wg.Add(quota)
for i := 0; i < quota; i++ {
go func() {
defer ctrl.wg.Done()
ctrl.processQueryQueue()
}()
if c.ConcurrencyQuota != 0 {
quota := int(c.ConcurrencyQuota)
ctrl.wg.Add(quota)
for i := 0; i < quota; i++ {
go func() {
defer ctrl.wg.Done()
ctrl.processQueryQueue()
}()
}
}
return ctrl, nil
}
@ -385,12 +399,32 @@ func (c *Controller) enqueueQuery(q *Query) error {
}
}
select {
case c.queryQueue <- q:
default:
return &flux.Error{
Code: codes.ResourceExhausted,
Msg: "queue length exceeded",
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:
return &flux.Error{
Code: codes.ResourceExhausted,
Msg: "queue length exceeded",
}
}
}
@ -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.
c.queriesMu.Lock()
c.shutdown = true
if len(c.queries) == 0 {
c.queriesMu.Unlock()
return nil
}
c.queriesMu.Unlock()
func() {
c.queriesMu.Lock()
defer c.queriesMu.Unlock()
if !c.shutdown {
c.shutdown = true
if len(c.queries) == 0 {
// 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)
}
}
}()
// Cancel all of the currently active queries.
c.queriesMu.RLock()

File diff suppressed because it is too large Load Diff

View File

@ -29,91 +29,64 @@ 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",
Help: "Count of the query requests",
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",
Help: "Count of functions in queries",
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",
Help: "Number of queries in all states",
Name: "qc_all_active",
Help: "Number of queries in all states",
}, labels),
compiling: prometheus.NewGaugeVec(prometheus.GaugeOpts{
Namespace: namespace,
Subsystem: subsystem,
Name: "compiling_active",
Help: "Number of queries actively compiling",
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",
Help: "Number of queries actively queueing",
Name: "qc_queueing_active",
Help: "Number of queries actively queueing",
}, labels),
executing: prometheus.NewGaugeVec(prometheus.GaugeOpts{
Namespace: namespace,
Subsystem: subsystem,
Name: "executing_active",
Help: "Number of queries actively executing",
Name: "qc_executing_active",
Help: "Number of queries actively executing",
}, labels),
memoryUnused: prometheus.NewGaugeVec(prometheus.GaugeOpts{
Namespace: namespace,
Subsystem: subsystem,
Name: "memory_unused_bytes",
Help: "The free memory as seen by the internal memory manager",
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",
Help: "Histogram of total times spent in all query states",
Buckets: prometheus.ExponentialBuckets(1e-3, 5, 7),
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",
Help: "Histogram of times spent compiling queries",
Buckets: prometheus.ExponentialBuckets(1e-3, 5, 7),
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",
Help: "Histogram of times spent queueing queries",
Buckets: prometheus.ExponentialBuckets(1e-3, 5, 7),
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",
Help: "Histogram of times spent executing queries",
Buckets: prometheus.ExponentialBuckets(1e-3, 5, 7),
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)
}