chore: Upgrade 1.x to the latest flux (#20767)
* chore: Update flux to 0.67 * chore: Builds against 0.68 flux * chore: Builds against 0.80.0 * chore: Builds against 0.90.0 * chore: Everything builds on latest flux * chore: goimports fixed * chore: fix tests locally * chore: fix CI dockerfiles * chore: clean up some unused code * chore: remove flux repl and Spec in flux query json * chore: port flux end to end tests from 2.x * chore: fix up goimports * chore: remove 32 bit build supportpull/20873/head
parent
de491dab97
commit
7210fa6355
|
@ -6,8 +6,8 @@ jobs:
|
|||
enabled: true
|
||||
docker_layer_caching: true
|
||||
environment:
|
||||
- PARALLELISM: 4 # Input to influxdb/build.py
|
||||
parallelism: 5 # How many CircleCI test containers
|
||||
- PARALLELISM: 4 # Amount of parallelism within each test (not the number of tests)
|
||||
parallelism: 4 # How many CircleCI test containers
|
||||
steps:
|
||||
- checkout
|
||||
- run:
|
||||
|
|
|
@ -1,39 +0,0 @@
|
|||
FROM ioft/i386-ubuntu:xenial
|
||||
|
||||
RUN apt-get update && DEBIAN_FRONTEND=noninteractive apt-get install -y \
|
||||
python-software-properties \
|
||||
software-properties-common \
|
||||
wget \
|
||||
git \
|
||||
mercurial \
|
||||
make \
|
||||
ruby \
|
||||
ruby-dev \
|
||||
rubygems \
|
||||
autoconf \
|
||||
libtool \
|
||||
build-essential \
|
||||
rpm \
|
||||
zip \
|
||||
python \
|
||||
python-boto
|
||||
|
||||
RUN gem install fpm
|
||||
|
||||
# setup environment
|
||||
ENV GO_VERSION 1.13.8
|
||||
ENV GOARCH 386
|
||||
ENV GOROOT /usr/local/go
|
||||
ENV GOPATH /root/go
|
||||
ENV PATH $GOPATH/bin:$GOROOT/bin:$PATH
|
||||
ENV PROJECT_DIR /root/influxdb
|
||||
|
||||
# install go
|
||||
RUN wget --no-verbose https://storage.googleapis.com/golang/go${GO_VERSION}.linux-${GOARCH}.tar.gz -O- | tar -C /usr/local/ -zxf-
|
||||
|
||||
RUN mkdir -p $PROJECT_DIR
|
||||
|
||||
WORKDIR $PROJECT_DIR
|
||||
VOLUME $PROJECT_DIR
|
||||
|
||||
ENTRYPOINT [ "/root/influxdb/build.py" ]
|
|
@ -1,34 +1,39 @@
|
|||
FROM ubuntu:xenial
|
||||
|
||||
RUN apt-get update && DEBIAN_FRONTEND=noninteractive apt-get install -y \
|
||||
python-software-properties \
|
||||
software-properties-common \
|
||||
wget \
|
||||
git \
|
||||
mercurial \
|
||||
make \
|
||||
ruby \
|
||||
ruby-dev \
|
||||
asciidoc \
|
||||
autoconf \
|
||||
libtool \
|
||||
build-essential \
|
||||
rpm \
|
||||
zip \
|
||||
curl \
|
||||
docbook-xsl \
|
||||
git \
|
||||
libtool \
|
||||
make \
|
||||
mercurial \
|
||||
pkg-config \
|
||||
python \
|
||||
python-boto \
|
||||
asciidoc \
|
||||
python-software-properties \
|
||||
rpm \
|
||||
ruby \
|
||||
ruby-dev \
|
||||
software-properties-common \
|
||||
wget \
|
||||
xmlto \
|
||||
docbook-xsl
|
||||
zip
|
||||
|
||||
RUN gem install fpm
|
||||
|
||||
RUN curl --proto '=https' --tlsv1.2 -sSf https://sh.rustup.rs | sh -s -- -y
|
||||
|
||||
# setup environment
|
||||
ENV GO_VERSION 1.13.8
|
||||
ENV GOARCH amd64
|
||||
ENV GOROOT /usr/local/go
|
||||
ENV GOPATH /root/go
|
||||
ENV PATH $GOPATH/bin:$GOROOT/bin:$PATH
|
||||
ENV PATH $GOPATH/bin:$GOROOT/bin:/root/.cargo/bin:$PATH
|
||||
ENV PROJECT_DIR /root/influxdb
|
||||
ENV PKG_CONFIG /root/influxdb/pkg-config.sh
|
||||
|
||||
# install go
|
||||
RUN wget --no-verbose https://storage.googleapis.com/golang/go${GO_VERSION}.linux-${GOARCH}.tar.gz -O- | tar -C /usr/local/ -zxf-
|
||||
|
|
|
@ -1,34 +1,39 @@
|
|||
FROM ubuntu:xenial
|
||||
|
||||
RUN apt-get update && DEBIAN_FRONTEND=noninteractive apt-get install -y \
|
||||
python-software-properties \
|
||||
software-properties-common \
|
||||
wget \
|
||||
git \
|
||||
mercurial \
|
||||
make \
|
||||
ruby \
|
||||
ruby-dev \
|
||||
asciidoc \
|
||||
autoconf \
|
||||
libtool \
|
||||
build-essential \
|
||||
rpm \
|
||||
zip \
|
||||
curl \
|
||||
docbook-xsl \
|
||||
git \
|
||||
libtool \
|
||||
make \
|
||||
mercurial \
|
||||
pkg-config \
|
||||
python \
|
||||
python-boto \
|
||||
asciidoc \
|
||||
python-software-properties \
|
||||
rpm \
|
||||
ruby \
|
||||
ruby-dev \
|
||||
software-properties-common \
|
||||
wget \
|
||||
xmlto \
|
||||
docbook-xsl
|
||||
zip
|
||||
|
||||
RUN gem install fpm
|
||||
|
||||
RUN curl --proto '=https' --tlsv1.2 -sSf https://sh.rustup.rs | sh -s -- -y
|
||||
|
||||
# setup environment
|
||||
ENV GO_VERSION 1.13
|
||||
ENV GOARCH amd64
|
||||
ENV GOROOT /usr/local/go
|
||||
ENV GOPATH /root/go
|
||||
ENV PATH $GOPATH/bin:$GOROOT/bin:$PATH
|
||||
ENV PATH $GOPATH/bin:$GOROOT/bin:/root/.cargo/bin:$PATH
|
||||
ENV PROJECT_DIR /root/influxdb
|
||||
ENV PKG_CONFIG /root/influxdb/pkg-config.sh
|
||||
|
||||
# install go
|
||||
RUN wget --no-verbose https://storage.googleapis.com/golang/go${GO_VERSION}.linux-${GOARCH}.tar.gz -O- | tar -C /usr/local/ -zxf-
|
||||
|
|
|
@ -1,20 +0,0 @@
|
|||
FROM ioft/i386-ubuntu:xenial
|
||||
|
||||
RUN DEBIAN_FRONTEND=noninteractive apt-get update && \
|
||||
DEBIAN_FRONTEND=noninteractive apt-get install -y \
|
||||
wget \
|
||||
mercurial \
|
||||
gcc \
|
||||
git && \
|
||||
rm -rf /var/lib/apt/lists/*
|
||||
|
||||
# Install go
|
||||
ENV GOCACHE=/tmp
|
||||
ENV GOPATH /go
|
||||
ENV GO_VERSION 1.13
|
||||
ENV GO_ARCH 386
|
||||
RUN wget --no-verbose -q https://storage.googleapis.com/golang/go${GO_VERSION}.linux-${GO_ARCH}.tar.gz && \
|
||||
tar -C /usr/local/ -xf /go${GO_VERSION}.linux-${GO_ARCH}.tar.gz && \
|
||||
mkdir -p "$GOPATH/src" "$GOPATH/bin" && chmod -R 777 "$GOPATH" && \
|
||||
rm /go${GO_VERSION}.linux-${GO_ARCH}.tar.gz
|
||||
ENV PATH $GOPATH/bin:/usr/local/go/bin:$PATH
|
|
@ -1,12 +0,0 @@
|
|||
FROM 32bit/ubuntu:14.04
|
||||
|
||||
RUN apt-get update && DEBIAN_FRONTEND=noninteractive apt-get install -y python-software-properties software-properties-common git
|
||||
RUN add-apt-repository ppa:evarlast/golang1.4
|
||||
RUN apt-get update && DEBIAN_FRONTEND=noninteractive apt-get install -y -o Dpkg::Options::="--force-overwrite" golang-go
|
||||
|
||||
ENV GOPATH=/root/go
|
||||
RUN mkdir -p /root/go/src/github.com/influxdata/influxdb
|
||||
RUN mkdir -p /tmp/artifacts
|
||||
|
||||
VOLUME /root/go/src/github.com/influxdata/influxdb
|
||||
VOLUME /tmp/artifacts
|
|
@ -1,5 +1,3 @@
|
|||
readTrusted 'Dockerfile_jenkins_ubuntu32'
|
||||
|
||||
pipeline {
|
||||
agent none
|
||||
|
||||
|
@ -64,25 +62,5 @@ pipeline {
|
|||
"""
|
||||
}
|
||||
}
|
||||
|
||||
stage('32bit') {
|
||||
agent {
|
||||
dockerfile {
|
||||
filename 'Dockerfile_jenkins_ubuntu32'
|
||||
}
|
||||
}
|
||||
|
||||
steps {
|
||||
sh """
|
||||
cd $WORKSPACE
|
||||
go mod download
|
||||
"""
|
||||
|
||||
sh """
|
||||
cd $WORKSPACE
|
||||
go test -parallel=1 ./...
|
||||
"""
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,4 @@
|
|||
export PKG_CONFIG=./pkg-config.sh
|
||||
|
||||
build:
|
||||
go build ./...
|
|
@ -211,12 +211,8 @@ func (c *CommandLine) Run() error {
|
|||
c.Version()
|
||||
|
||||
if c.Type == QueryLanguageFlux {
|
||||
repl, err := getFluxREPL(c.URL, c.ClientConfig.Username, c.ClientConfig.Password)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
repl.Run()
|
||||
os.Exit(0)
|
||||
// TODO(lesam): steal 2.x flux client
|
||||
return fmt.Errorf("ERROR: flux repl missing due to flux upgrade")
|
||||
}
|
||||
|
||||
c.Line = liner.NewLiner()
|
||||
|
@ -1182,13 +1178,8 @@ func (c *CommandLine) ExecuteFluxQuery(query string) error {
|
|||
}
|
||||
}()
|
||||
}
|
||||
|
||||
repl, err := getFluxREPL(c.URL, c.ClientConfig.Username, c.ClientConfig.Password)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return repl.Input(query)
|
||||
// TODO(lesam): steal 2.x flux client
|
||||
return fmt.Errorf("ERROR: flux repl missing due to flux upgrade")
|
||||
}
|
||||
|
||||
type QueryLanguage uint8
|
||||
|
|
|
@ -1,44 +0,0 @@
|
|||
package cli
|
||||
|
||||
import (
|
||||
"context"
|
||||
"net/url"
|
||||
|
||||
"github.com/influxdata/flux"
|
||||
"github.com/influxdata/flux/csv"
|
||||
"github.com/influxdata/flux/repl"
|
||||
"github.com/influxdata/influxdb/flux/builtin"
|
||||
"github.com/influxdata/influxdb/flux/client"
|
||||
)
|
||||
|
||||
// QueryService represents a type capable of performing queries.
|
||||
type fluxClient interface {
|
||||
// Query submits a query for execution returning a results iterator.
|
||||
// Cancel must be called on any returned results to free resources.
|
||||
Query(ctx context.Context, req *client.ProxyRequest) (flux.ResultIterator, error)
|
||||
}
|
||||
|
||||
// replQuerier implements the repl.Querier interface while consuming a fluxClient
|
||||
type replQuerier struct {
|
||||
client fluxClient
|
||||
}
|
||||
|
||||
func (q *replQuerier) Query(ctx context.Context, deps flux.Dependencies, compiler flux.Compiler) (flux.ResultIterator, error) {
|
||||
req := &client.ProxyRequest{
|
||||
Compiler: compiler,
|
||||
Dialect: csv.DefaultDialect(),
|
||||
}
|
||||
return q.client.Query(ctx, req)
|
||||
}
|
||||
|
||||
func getFluxREPL(u url.URL, username, password string) (*repl.REPL, error) {
|
||||
builtin.Initialize()
|
||||
|
||||
c, err := client.NewHTTP(u)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
c.Username = username
|
||||
c.Password = password
|
||||
return repl.New(context.Background(), flux.NewDefaultDependencies(), &replQuerier{client: c}), nil
|
||||
}
|
|
@ -296,7 +296,7 @@ func (s *Server) appendHTTPDService(c httpd.Config) {
|
|||
ss := storage.NewStore(s.TSDBStore, s.MetaClient)
|
||||
srv.Handler.Store = ss
|
||||
if s.config.HTTPD.FluxEnabled {
|
||||
srv.Handler.Controller = control.NewController(s.MetaClient, reads.NewReader(ss), authorizer, c.AuthEnabled, s.Logger)
|
||||
srv.Handler.Controller = control.NewController(s.MetaClient, reads.NewReader(ss), authorizer, c.AuthEnabled, s.PointsWriter, s.Logger)
|
||||
}
|
||||
|
||||
s.Services = append(s.Services, srv)
|
||||
|
|
|
@ -6,7 +6,7 @@ package builtin
|
|||
import (
|
||||
"sync"
|
||||
|
||||
"github.com/influxdata/flux"
|
||||
"github.com/influxdata/flux/runtime"
|
||||
_ "github.com/influxdata/flux/stdlib"
|
||||
_ "github.com/influxdata/influxdb/flux/stdlib"
|
||||
)
|
||||
|
@ -18,6 +18,6 @@ var once sync.Once
|
|||
// and is idempotent.
|
||||
func Initialize() {
|
||||
once.Do(func() {
|
||||
flux.FinalizeBuiltIns()
|
||||
runtime.FinalizeBuiltIns()
|
||||
})
|
||||
}
|
||||
|
|
|
@ -1,21 +1,9 @@
|
|||
package client
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
"crypto/tls"
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
"io/ioutil"
|
||||
"mime"
|
||||
"net/http"
|
||||
"net/url"
|
||||
|
||||
"github.com/influxdata/flux"
|
||||
"github.com/influxdata/flux/csv"
|
||||
"github.com/influxdata/flux/lang"
|
||||
"github.com/influxdata/flux/repl"
|
||||
"github.com/pkg/errors"
|
||||
)
|
||||
|
||||
const (
|
||||
|
@ -46,40 +34,16 @@ func NewHTTP(u url.URL) (*HTTP, error) {
|
|||
}
|
||||
|
||||
// Query runs a flux query against a influx server and decodes the result
|
||||
func (s *HTTP) Query(ctx context.Context, r *ProxyRequest) (flux.ResultIterator, error) {
|
||||
qreq, err := QueryRequestFromProxyRequest(r)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
var body bytes.Buffer
|
||||
if err := json.NewEncoder(&body).Encode(qreq); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
hreq, err := http.NewRequest("POST", s.url.String(), &body)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
func (s *HTTP) Do(hreq *http.Request) (*http.Response, error) {
|
||||
if s.Username != "" {
|
||||
hreq.SetBasicAuth(s.Username, s.Password)
|
||||
}
|
||||
|
||||
hreq.Header.Set("Content-Type", "application/json")
|
||||
hreq.Header.Set("Accept", "text/csv")
|
||||
hreq = hreq.WithContext(ctx)
|
||||
|
||||
hc := newClient(s.url.Scheme, s.InsecureSkipVerify)
|
||||
resp, err := hc.Do(hreq)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
if err := checkError(resp); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
decoder := csv.NewMultiResultDecoder(csv.ResultDecoderConfig{})
|
||||
return decoder.Decode(resp.Body)
|
||||
return hc.Do(hreq)
|
||||
}
|
||||
|
||||
func newClient(scheme string, insecure bool) *http.Client {
|
||||
|
@ -91,56 +55,3 @@ func newClient(scheme string, insecure bool) *http.Client {
|
|||
}
|
||||
return hc
|
||||
}
|
||||
|
||||
// CheckError reads the http.Response and returns an error if one exists.
|
||||
// It will automatically recognize the errors returned by Influx services
|
||||
// and decode the error into an internal error type. If the error cannot
|
||||
// be determined in that way, it will create a generic error message.
|
||||
//
|
||||
// If there is no error, then this returns nil.
|
||||
func checkError(resp *http.Response) error {
|
||||
switch resp.StatusCode / 100 {
|
||||
case 4:
|
||||
// We will attempt to parse this error outside of this block.
|
||||
msg := "client error"
|
||||
data, _ := ioutil.ReadAll(resp.Body)
|
||||
mt, _, err := mime.ParseMediaType(resp.Header.Get("content-type"))
|
||||
if err == nil && mt == "text/plain" && len(msg) > 0 {
|
||||
msg = string(data)
|
||||
}
|
||||
return errors.Wrap(errors.New(resp.Status), msg)
|
||||
case 1, 2:
|
||||
return nil
|
||||
|
||||
default:
|
||||
msg := "unknown server error"
|
||||
return errors.Wrap(errors.New(resp.Status), msg)
|
||||
}
|
||||
}
|
||||
|
||||
func QueryRequestFromProxyRequest(req *ProxyRequest) (*QueryRequest, error) {
|
||||
qr := new(QueryRequest)
|
||||
switch c := req.Compiler.(type) {
|
||||
case lang.FluxCompiler:
|
||||
qr.Type = "flux"
|
||||
qr.Query = c.Query
|
||||
case repl.Compiler:
|
||||
qr.Type = "flux"
|
||||
qr.Spec = c.Spec
|
||||
default:
|
||||
return nil, fmt.Errorf("unsupported compiler %T", c)
|
||||
}
|
||||
switch d := req.Dialect.(type) {
|
||||
case *csv.Dialect:
|
||||
var header = !d.ResultEncoderConfig.NoHeader
|
||||
qr.Dialect.Header = &header
|
||||
qr.Dialect.Delimiter = string(d.ResultEncoderConfig.Delimiter)
|
||||
qr.Dialect.CommentPrefix = "#"
|
||||
qr.Dialect.DateTimeFormat = "RFC3339"
|
||||
qr.Dialect.Annotations = d.ResultEncoderConfig.Annotations
|
||||
default:
|
||||
return nil, fmt.Errorf("unsupported dialect %T", d)
|
||||
}
|
||||
|
||||
return qr, nil
|
||||
}
|
||||
|
|
|
@ -2,13 +2,15 @@ package client
|
|||
|
||||
import (
|
||||
"context"
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
"time"
|
||||
"unicode/utf8"
|
||||
|
||||
"github.com/influxdata/flux"
|
||||
"github.com/influxdata/flux/csv"
|
||||
"github.com/influxdata/flux/lang"
|
||||
"github.com/influxdata/flux/repl"
|
||||
"github.com/influxdata/influxdb/query"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/prometheus/client_golang/prometheus"
|
||||
)
|
||||
|
@ -20,10 +22,31 @@ type Controller interface {
|
|||
|
||||
// QueryRequest is a flux query request.
|
||||
type QueryRequest struct {
|
||||
Spec *flux.Spec `json:"spec,omitempty"`
|
||||
Query string `json:"query"`
|
||||
Type string `json:"type"`
|
||||
Dialect QueryDialect `json:"dialect"`
|
||||
Type string `json:"type"`
|
||||
Query string `json:"query"`
|
||||
|
||||
// Flux fields
|
||||
Extern json.RawMessage `json:"extern,omitempty"`
|
||||
AST json.RawMessage `json:"ast,omitempty"`
|
||||
Dialect QueryDialect `json:"dialect"`
|
||||
Now time.Time `json:"now"`
|
||||
|
||||
// PreferNoContent specifies if the Response to this request should
|
||||
// contain any result. This is done for avoiding unnecessary
|
||||
// bandwidth consumption in certain cases. For example, when the
|
||||
// query produces side effects and the results do not matter. E.g.:
|
||||
// from(...) |> ... |> to()
|
||||
// For example, tasks do not use the results of queries, but only
|
||||
// care about their side effects.
|
||||
// To obtain a QueryRequest with no result, add the header
|
||||
// `Prefer: return-no-content` to the HTTP request.
|
||||
PreferNoContent bool
|
||||
// PreferNoContentWithError is the same as above, but it forces the
|
||||
// Response to contain an error if that is a Flux runtime error encoded
|
||||
// in the response body.
|
||||
// To obtain a QueryRequest with no result but runtime errors,
|
||||
// add the header `Prefer: return-no-content-with-error` to the HTTP request.
|
||||
PreferNoContentWithError bool
|
||||
}
|
||||
|
||||
// QueryDialect is the formatting options for the query response.
|
||||
|
@ -55,8 +78,8 @@ func (r QueryRequest) WithDefaults() QueryRequest {
|
|||
|
||||
// Validate checks the query request and returns an error if the request is invalid.
|
||||
func (r QueryRequest) Validate() error {
|
||||
if r.Query == "" && r.Spec == nil {
|
||||
return errors.New(`request body requires either spec or query`)
|
||||
if r.Query == "" && r.AST == nil {
|
||||
return errors.New(`request body requires either query or AST`)
|
||||
}
|
||||
|
||||
if r.Type != "flux" {
|
||||
|
@ -104,16 +127,26 @@ type ProxyRequest struct {
|
|||
|
||||
// ProxyRequest returns a request to proxy from the flux.
|
||||
func (r QueryRequest) ProxyRequest() *ProxyRequest {
|
||||
n := r.Now
|
||||
if n.IsZero() {
|
||||
n = time.Now()
|
||||
}
|
||||
|
||||
// Query is preferred over spec
|
||||
var compiler flux.Compiler
|
||||
if r.Query != "" {
|
||||
compiler = lang.FluxCompiler{
|
||||
Query: r.Query,
|
||||
Query: r.Query,
|
||||
Extern: r.Extern,
|
||||
Now: n,
|
||||
}
|
||||
} else if r.Spec != nil {
|
||||
compiler = repl.Compiler{
|
||||
Spec: r.Spec,
|
||||
} else if len(r.AST) > 0 {
|
||||
c := lang.ASTCompiler{
|
||||
Extern: r.Extern,
|
||||
AST: r.AST,
|
||||
Now: n,
|
||||
}
|
||||
compiler = c
|
||||
}
|
||||
|
||||
delimiter, _ := utf8.DecodeRuneInString(r.Dialect.Delimiter)
|
||||
|
@ -123,14 +156,30 @@ func (r QueryRequest) ProxyRequest() *ProxyRequest {
|
|||
noHeader = !*r.Dialect.Header
|
||||
}
|
||||
|
||||
cfg := csv.DefaultEncoderConfig()
|
||||
cfg.NoHeader = noHeader
|
||||
cfg.Delimiter = delimiter
|
||||
var dialect flux.Dialect
|
||||
if r.PreferNoContent {
|
||||
dialect = &query.NoContentDialect{}
|
||||
} else {
|
||||
// TODO(nathanielc): Use commentPrefix and dateTimeFormat
|
||||
// once they are supported.
|
||||
encConfig := csv.ResultEncoderConfig{
|
||||
NoHeader: noHeader,
|
||||
Delimiter: delimiter,
|
||||
Annotations: r.Dialect.Annotations,
|
||||
}
|
||||
if r.PreferNoContentWithError {
|
||||
dialect = &query.NoContentWithErrorDialect{
|
||||
ResultEncoderConfig: encConfig,
|
||||
}
|
||||
} else {
|
||||
dialect = &csv.Dialect{
|
||||
ResultEncoderConfig: encConfig,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return &ProxyRequest{
|
||||
Compiler: compiler,
|
||||
Dialect: csv.Dialect{
|
||||
ResultEncoderConfig: cfg,
|
||||
},
|
||||
Dialect: dialect,
|
||||
}
|
||||
}
|
||||
|
|
|
@ -6,6 +6,7 @@ import (
|
|||
"github.com/influxdata/flux"
|
||||
"github.com/influxdata/flux/lang"
|
||||
"github.com/influxdata/flux/memory"
|
||||
"github.com/influxdata/flux/runtime"
|
||||
"github.com/influxdata/influxdb/coordinator"
|
||||
"github.com/influxdata/influxdb/flux/builtin"
|
||||
"github.com/influxdata/influxdb/flux/stdlib/influxdata/influxdb"
|
||||
|
@ -16,10 +17,10 @@ import (
|
|||
type MetaClient = coordinator.MetaClient
|
||||
type Authorizer = influxdb.Authorizer
|
||||
|
||||
func NewController(mc MetaClient, reader influxdb.Reader, auth Authorizer, authEnabled bool, logger *zap.Logger) *Controller {
|
||||
func NewController(mc MetaClient, reader influxdb.Reader, auth Authorizer, authEnabled bool, writer influxdb.PointsWriter, logger *zap.Logger) *Controller {
|
||||
builtin.Initialize()
|
||||
|
||||
storageDeps, err := influxdb.NewDependencies(mc, reader, auth, authEnabled)
|
||||
storageDeps, err := influxdb.NewDependencies(mc, reader, auth, authEnabled, writer)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
@ -40,7 +41,7 @@ func (c *Controller) Query(ctx context.Context, compiler flux.Compiler) (flux.Qu
|
|||
ctx = dep.Inject(ctx)
|
||||
}
|
||||
|
||||
p, err := compiler.Compile(ctx)
|
||||
p, err := compiler.Compile(ctx, runtime.Default)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
|
|
@ -5,17 +5,23 @@ import (
|
|||
"errors"
|
||||
|
||||
"github.com/influxdata/flux"
|
||||
"github.com/influxdata/influxdb/coordinator"
|
||||
)
|
||||
|
||||
type key int
|
||||
|
||||
const dependenciesKey key = iota
|
||||
|
||||
type PointsWriter interface {
|
||||
WritePointsInto(request *coordinator.IntoWriteRequest) error
|
||||
}
|
||||
|
||||
type StorageDependencies struct {
|
||||
Reader Reader
|
||||
MetaClient MetaClient
|
||||
Authorizer Authorizer
|
||||
AuthEnabled bool
|
||||
Reader Reader
|
||||
MetaClient MetaClient
|
||||
Authorizer Authorizer
|
||||
AuthEnabled bool
|
||||
PointsWriter PointsWriter
|
||||
}
|
||||
|
||||
func (d StorageDependencies) Inject(ctx context.Context) context.Context {
|
||||
|
@ -32,6 +38,9 @@ func (d StorageDependencies) Validate() error {
|
|||
if d.AuthEnabled && d.Authorizer == nil {
|
||||
return errors.New("missing authorizer dependency")
|
||||
}
|
||||
if d.PointsWriter == nil {
|
||||
return errors.New("missing points writer dependency")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
|
@ -54,14 +63,16 @@ func NewDependencies(
|
|||
reader Reader,
|
||||
auth Authorizer,
|
||||
authEnabled bool,
|
||||
writer PointsWriter,
|
||||
) (Dependencies, error) {
|
||||
fdeps := flux.NewDefaultDependencies()
|
||||
deps := Dependencies{FluxDeps: fdeps}
|
||||
deps.StorageDeps = StorageDependencies{
|
||||
Reader: reader,
|
||||
MetaClient: mc,
|
||||
Authorizer: auth,
|
||||
AuthEnabled: authEnabled,
|
||||
Reader: reader,
|
||||
MetaClient: mc,
|
||||
Authorizer: auth,
|
||||
AuthEnabled: authEnabled,
|
||||
PointsWriter: writer,
|
||||
}
|
||||
if err := deps.StorageDeps.Validate(); err != nil {
|
||||
return Dependencies{}, err
|
||||
|
|
|
@ -1,4 +1,4 @@
|
|||
package reads
|
||||
package influxdb
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
|
@ -16,12 +16,8 @@ const (
|
|||
valueKey = "_value"
|
||||
)
|
||||
|
||||
func toStoragePredicate(f *semantic.FunctionExpression) (*datatypes.Predicate, error) {
|
||||
if f.Block.Parameters == nil || len(f.Block.Parameters.List) != 1 {
|
||||
return nil, errors.New("storage predicate functions must have exactly one parameter")
|
||||
}
|
||||
|
||||
root, err := toStoragePredicateHelper(f.Block.Body.(semantic.Expression), f.Block.Parameters.List[0].Key.Name)
|
||||
func ToStoragePredicate(n semantic.Expression, objectName string) (*datatypes.Predicate, error) {
|
||||
root, err := toStoragePredicateHelper(n, objectName)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -31,6 +27,38 @@ func toStoragePredicate(f *semantic.FunctionExpression) (*datatypes.Predicate, e
|
|||
}, nil
|
||||
}
|
||||
|
||||
func MergePredicates(op ast.LogicalOperatorKind, predicates ...*datatypes.Predicate) (*datatypes.Predicate, error) {
|
||||
if len(predicates) == 0 {
|
||||
return nil, errors.New("at least one predicate is needed")
|
||||
}
|
||||
var value datatypes.Node_Logical
|
||||
switch op {
|
||||
case ast.AndOperator:
|
||||
value = datatypes.LogicalAnd
|
||||
case ast.OrOperator:
|
||||
value = datatypes.LogicalOr
|
||||
default:
|
||||
return nil, fmt.Errorf("unknown logical operator %v", op)
|
||||
}
|
||||
|
||||
// Nest the predicates backwards. This way we get a tree like this:
|
||||
// a AND (b AND c)
|
||||
root := predicates[len(predicates)-1].Root
|
||||
for i := len(predicates) - 2; i >= 0; i-- {
|
||||
root = &datatypes.Node{
|
||||
NodeType: datatypes.NodeTypeLogicalExpression,
|
||||
Value: &datatypes.Node_Logical_{Logical: value},
|
||||
Children: []*datatypes.Node{
|
||||
predicates[i].Root,
|
||||
root,
|
||||
},
|
||||
}
|
||||
}
|
||||
return &datatypes.Predicate{
|
||||
Root: root,
|
||||
}, nil
|
||||
}
|
||||
|
||||
func toStoragePredicateHelper(n semantic.Expression, objectName string) (*datatypes.Node, error) {
|
||||
switch n := n.(type) {
|
||||
case *semantic.LogicalExpression:
|
|
@ -6,7 +6,7 @@ import (
|
|||
"github.com/influxdata/flux"
|
||||
"github.com/influxdata/flux/codes"
|
||||
"github.com/influxdata/flux/plan"
|
||||
"github.com/influxdata/flux/semantic"
|
||||
"github.com/influxdata/flux/runtime"
|
||||
"github.com/influxdata/flux/stdlib/influxdata/influxdb"
|
||||
)
|
||||
|
||||
|
@ -18,16 +18,8 @@ type FromOpSpec struct {
|
|||
}
|
||||
|
||||
func init() {
|
||||
fromSignature := semantic.FunctionPolySignature{
|
||||
Parameters: map[string]semantic.PolyType{
|
||||
"bucket": semantic.String,
|
||||
"bucketID": semantic.String,
|
||||
},
|
||||
Required: nil,
|
||||
Return: flux.TableObjectType,
|
||||
}
|
||||
|
||||
flux.ReplacePackageValue("influxdata/influxdb", influxdb.FromKind, flux.FunctionValue(FromKind, createFromOpSpec, fromSignature))
|
||||
fromSignature := runtime.MustLookupBuiltinType("influxdata/influxdb", "from")
|
||||
runtime.ReplacePackageValue("influxdata/influxdb", influxdb.FromKind, flux.MustValue(flux.FunctionValue(FromKind, createFromOpSpec, fromSignature)))
|
||||
flux.RegisterOpSpec(FromKind, newFromOp)
|
||||
plan.RegisterProcedureSpec(FromKind, newFromProcedure, FromKind)
|
||||
}
|
||||
|
|
|
@ -8,9 +8,9 @@ import (
|
|||
"github.com/influxdata/flux"
|
||||
"github.com/influxdata/flux/execute"
|
||||
"github.com/influxdata/flux/plan"
|
||||
"github.com/influxdata/flux/semantic"
|
||||
"github.com/influxdata/flux/values"
|
||||
"github.com/influxdata/influxdb/services/meta"
|
||||
"github.com/influxdata/influxdb/storage/reads/datatypes"
|
||||
"github.com/influxdata/influxql"
|
||||
)
|
||||
|
||||
|
@ -52,12 +52,9 @@ type ReadRangePhysSpec struct {
|
|||
Bucket string
|
||||
BucketID string
|
||||
|
||||
// FilterSet is set to true if there is a filter.
|
||||
FilterSet bool
|
||||
// Filter is the filter to use when calling into
|
||||
// storage. It must be possible to push down this
|
||||
// filter.
|
||||
Filter *semantic.FunctionExpression
|
||||
// Predicate is the filtering predicate for calling into storage.
|
||||
// It must not be mutated.
|
||||
Predicate *datatypes.Predicate
|
||||
|
||||
Bounds flux.Bounds
|
||||
}
|
||||
|
@ -71,55 +68,51 @@ func (s *ReadRangePhysSpec) Copy() plan.ProcedureSpec {
|
|||
ns.Bucket = s.Bucket
|
||||
ns.BucketID = s.BucketID
|
||||
|
||||
ns.FilterSet = s.FilterSet
|
||||
if ns.FilterSet {
|
||||
ns.Filter = s.Filter.Copy().(*semantic.FunctionExpression)
|
||||
}
|
||||
ns.Predicate = s.Predicate
|
||||
|
||||
ns.Bounds = s.Bounds
|
||||
|
||||
return ns
|
||||
}
|
||||
|
||||
func (s *ReadRangePhysSpec) LookupDatabase(ctx context.Context, deps StorageDependencies, a execute.Administration) (string, string, error) {
|
||||
if len(s.BucketID) != 0 {
|
||||
return "", "", errors.New("cannot refer to buckets by their id in 1.x")
|
||||
}
|
||||
|
||||
func lookupDatabase(ctx context.Context, bucketName string, deps StorageDependencies, privilege influxql.Privilege) (string, string, error) {
|
||||
var db, rp string
|
||||
if i := strings.IndexByte(s.Bucket, '/'); i == -1 {
|
||||
db = s.Bucket
|
||||
if i := strings.IndexByte(bucketName, '/'); i == -1 {
|
||||
db = bucketName
|
||||
} else {
|
||||
rp = s.Bucket[i+1:]
|
||||
db = s.Bucket[:i]
|
||||
rp = bucketName[i+1:]
|
||||
db = bucketName[:i]
|
||||
}
|
||||
|
||||
// validate and resolve db/rp
|
||||
di := deps.MetaClient.Database(db)
|
||||
if di == nil {
|
||||
return "", "", errors.New("no database")
|
||||
}
|
||||
|
||||
if deps.AuthEnabled {
|
||||
user := meta.UserFromContext(a.Context())
|
||||
user := meta.UserFromContext(ctx)
|
||||
if user == nil {
|
||||
return "", "", errors.New("createFromSource: no user")
|
||||
return "", "", errors.New("no user for auth-enabled flux access")
|
||||
}
|
||||
if err := deps.Authorizer.AuthorizeDatabase(user, influxql.ReadPrivilege, db); err != nil {
|
||||
if err := deps.Authorizer.AuthorizeDatabase(user, privilege, db); err != nil {
|
||||
return "", "", err
|
||||
}
|
||||
}
|
||||
|
||||
if rp == "" {
|
||||
rp = di.DefaultRetentionPolicy
|
||||
}
|
||||
|
||||
if rpi := di.RetentionPolicy(rp); rpi == nil {
|
||||
return "", "", errors.New("invalid retention policy")
|
||||
}
|
||||
return db, rp, nil
|
||||
}
|
||||
|
||||
func (s *ReadRangePhysSpec) LookupDatabase(_ context.Context, deps StorageDependencies, a execute.Administration) (string, string, error) {
|
||||
if len(s.BucketID) != 0 {
|
||||
return "", "", errors.New("cannot refer to buckets by their id in 1.x")
|
||||
}
|
||||
return lookupDatabase(a.Context(), s.Bucket, deps, influxql.ReadPrivilege)
|
||||
}
|
||||
|
||||
// TimeBounds implements plan.BoundsAwareProcedureSpec.
|
||||
func (s *ReadRangePhysSpec) TimeBounds(predecessorBounds *plan.Bounds) *plan.Bounds {
|
||||
return &plan.Bounds{
|
||||
|
|
|
@ -1,12 +1,16 @@
|
|||
package influxdb
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/influxdata/flux"
|
||||
"github.com/influxdata/flux/ast"
|
||||
"github.com/influxdata/flux/execute"
|
||||
"github.com/influxdata/flux/plan"
|
||||
"github.com/influxdata/flux/semantic"
|
||||
"github.com/influxdata/flux/stdlib/universe"
|
||||
"github.com/influxdata/influxdb/models"
|
||||
"github.com/influxdata/influxdb/storage/reads/datatypes"
|
||||
)
|
||||
|
||||
func init() {
|
||||
|
@ -31,7 +35,7 @@ func (rule PushDownGroupRule) Pattern() plan.Pattern {
|
|||
return plan.Pat(universe.GroupKind, plan.Pat(ReadRangePhysKind))
|
||||
}
|
||||
|
||||
func (rule PushDownGroupRule) Rewrite(node plan.Node) (plan.Node, bool, error) {
|
||||
func (rule PushDownGroupRule) Rewrite(ctx context.Context, node plan.Node) (plan.Node, bool, error) {
|
||||
src := node.Predecessors()[0].ProcedureSpec().(*ReadRangePhysSpec)
|
||||
grp := node.ProcedureSpec().(*universe.GroupProcedureSpec)
|
||||
|
||||
|
@ -71,7 +75,7 @@ func (rule PushDownRangeRule) Pattern() plan.Pattern {
|
|||
}
|
||||
|
||||
// Rewrite converts 'from |> range' into 'ReadRange'
|
||||
func (rule PushDownRangeRule) Rewrite(node plan.Node) (plan.Node, bool, error) {
|
||||
func (rule PushDownRangeRule) Rewrite(ctx context.Context, node plan.Node) (plan.Node, bool, error) {
|
||||
fromNode := node.Predecessors()[0]
|
||||
fromSpec := fromNode.ProcedureSpec().(*FromProcedureSpec)
|
||||
|
||||
|
@ -96,7 +100,7 @@ func (PushDownFilterRule) Pattern() plan.Pattern {
|
|||
return plan.Pat(universe.FilterKind, plan.Pat(ReadRangePhysKind))
|
||||
}
|
||||
|
||||
func (PushDownFilterRule) Rewrite(pn plan.Node) (plan.Node, bool, error) {
|
||||
func (PushDownFilterRule) Rewrite(ctx context.Context, pn plan.Node) (plan.Node, bool, error) {
|
||||
filterSpec := pn.ProcedureSpec().(*universe.FilterProcedureSpec)
|
||||
fromNode := pn.Predecessors()[0]
|
||||
fromSpec := fromNode.ProcedureSpec().(*ReadRangePhysSpec)
|
||||
|
@ -106,17 +110,17 @@ func (PushDownFilterRule) Rewrite(pn plan.Node) (plan.Node, bool, error) {
|
|||
return pn, false, nil
|
||||
}
|
||||
|
||||
bodyExpr, ok := filterSpec.Fn.Fn.Block.Body.(semantic.Expression)
|
||||
bodyExpr, ok := filterSpec.Fn.Fn.GetFunctionBodyExpression()
|
||||
if !ok {
|
||||
return pn, false, nil
|
||||
}
|
||||
|
||||
if len(filterSpec.Fn.Fn.Block.Parameters.List) != 1 {
|
||||
if len(filterSpec.Fn.Fn.Parameters.List) != 1 {
|
||||
// I would expect that type checking would catch this, but just to be safe...
|
||||
return pn, false, nil
|
||||
}
|
||||
|
||||
paramName := filterSpec.Fn.Fn.Block.Parameters.List[0].Key.Name
|
||||
paramName := filterSpec.Fn.Fn.Parameters.List[0].Key.Name
|
||||
|
||||
pushable, notPushable, err := semantic.PartitionPredicates(bodyExpr, func(e semantic.Expression) (bool, error) {
|
||||
return isPushableExpr(paramName, e)
|
||||
|
@ -131,15 +135,16 @@ func (PushDownFilterRule) Rewrite(pn plan.Node) (plan.Node, bool, error) {
|
|||
}
|
||||
pushable, _ = rewritePushableExpr(pushable)
|
||||
|
||||
pushablePredicate, err := ToStoragePredicate(pushable, paramName)
|
||||
|
||||
newFromSpec := fromSpec.Copy().(*ReadRangePhysSpec)
|
||||
if newFromSpec.FilterSet {
|
||||
newBody := semantic.ExprsToConjunction(newFromSpec.Filter.Block.Body.(semantic.Expression), pushable)
|
||||
newFromSpec.Filter.Block.Body = newBody
|
||||
if newFromSpec.Predicate != nil {
|
||||
newFromSpec.Predicate, err = MergePredicates(ast.AndOperator, fromSpec.Predicate, pushablePredicate)
|
||||
if err != nil {
|
||||
return nil, false, err
|
||||
}
|
||||
} else {
|
||||
newFromSpec.FilterSet = true
|
||||
// NOTE: We loose the scope here, but that is ok because we can't push down the scope to storage.
|
||||
newFromSpec.Filter = filterSpec.Fn.Fn.Copy().(*semantic.FunctionExpression)
|
||||
newFromSpec.Filter.Block.Body = pushable
|
||||
newFromSpec.Predicate = pushablePredicate
|
||||
}
|
||||
|
||||
if notPushable == nil {
|
||||
|
@ -157,7 +162,11 @@ func (PushDownFilterRule) Rewrite(pn plan.Node) (plan.Node, bool, error) {
|
|||
}
|
||||
|
||||
newFilterSpec := filterSpec.Copy().(*universe.FilterProcedureSpec)
|
||||
newFilterSpec.Fn.Fn.Block.Body = notPushable
|
||||
newFilterSpec.Fn.Fn.Block.Body = []semantic.Statement{
|
||||
&semantic.ReturnStatement{
|
||||
Argument: notPushable,
|
||||
},
|
||||
}
|
||||
if err := pn.ReplaceSpec(newFilterSpec); err != nil {
|
||||
return nil, false, err
|
||||
}
|
||||
|
@ -183,7 +192,23 @@ func (rule PushDownReadTagKeysRule) Pattern() plan.Pattern {
|
|||
plan.Pat(ReadRangePhysKind))))
|
||||
}
|
||||
|
||||
func (rule PushDownReadTagKeysRule) Rewrite(pn plan.Node) (plan.Node, bool, error) {
|
||||
func hasFieldRef(node *datatypes.Node) bool {
|
||||
if node == nil {
|
||||
return false
|
||||
}
|
||||
// NodeType should imply the type, panic if it doesn't
|
||||
if node.NodeType == datatypes.NodeTypeTagRef && node.Value.(*datatypes.Node_TagRefValue).TagRefValue == models.FieldKeyTagKey {
|
||||
return true
|
||||
}
|
||||
for _, c := range node.Children {
|
||||
if hasFieldRef(c) {
|
||||
return true
|
||||
}
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
func (rule PushDownReadTagKeysRule) Rewrite(ctx context.Context, pn plan.Node) (plan.Node, bool, error) {
|
||||
// Retrieve the nodes and specs for all of the predecessors.
|
||||
distinctSpec := pn.ProcedureSpec().(*universe.DistinctProcedureSpec)
|
||||
keepNode := pn.Predecessors()[0]
|
||||
|
@ -200,7 +225,7 @@ func (rule PushDownReadTagKeysRule) Rewrite(pn plan.Node) (plan.Node, bool, erro
|
|||
|
||||
// The tag keys mechanism doesn't know about fields so we cannot
|
||||
// push down _field comparisons in 1.x.
|
||||
if hasFieldExpr(fromSpec.Filter) {
|
||||
if fromSpec.Predicate != nil && hasFieldRef(fromSpec.Predicate.Root) {
|
||||
return pn, false, nil
|
||||
}
|
||||
|
||||
|
@ -265,7 +290,7 @@ func (rule PushDownReadTagValuesRule) Pattern() plan.Pattern {
|
|||
plan.Pat(ReadRangePhysKind))))
|
||||
}
|
||||
|
||||
func (rule PushDownReadTagValuesRule) Rewrite(pn plan.Node) (plan.Node, bool, error) {
|
||||
func (rule PushDownReadTagValuesRule) Rewrite(ctx context.Context, pn plan.Node) (plan.Node, bool, error) {
|
||||
// Retrieve the nodes and specs for all of the predecessors.
|
||||
distinctNode := pn
|
||||
distinctSpec := distinctNode.ProcedureSpec().(*universe.DistinctProcedureSpec)
|
||||
|
@ -579,7 +604,7 @@ func (SortedPivotRule) Pattern() plan.Pattern {
|
|||
return plan.Pat(universe.PivotKind, plan.Pat(ReadRangePhysKind))
|
||||
}
|
||||
|
||||
func (SortedPivotRule) Rewrite(pn plan.Node) (plan.Node, bool, error) {
|
||||
func (SortedPivotRule) Rewrite(ctx context.Context, pn plan.Node) (plan.Node, bool, error) {
|
||||
pivotSpec := pn.ProcedureSpec().Copy().(*universe.PivotProcedureSpec)
|
||||
pivotSpec.IsSortedByFunc = func(cols []string, desc bool) bool {
|
||||
if desc {
|
||||
|
|
|
@ -13,6 +13,7 @@ import (
|
|||
"github.com/influxdata/flux/semantic"
|
||||
"github.com/influxdata/flux/stdlib/universe"
|
||||
"github.com/influxdata/influxdb/flux/stdlib/influxdata/influxdb"
|
||||
"github.com/influxdata/influxdb/storage/reads/datatypes"
|
||||
)
|
||||
|
||||
func fluxTime(t int64) flux.Time {
|
||||
|
@ -164,14 +165,16 @@ func TestPushDownFilterRule(t *testing.T) {
|
|||
statementFn = interpreter.ResolvedFunction{
|
||||
Scope: nil,
|
||||
Fn: &semantic.FunctionExpression{
|
||||
Block: &semantic.FunctionBlock{
|
||||
Parameters: &semantic.FunctionParameters{
|
||||
List: []*semantic.FunctionParameter{
|
||||
{Key: &semantic.Identifier{Name: "r"}},
|
||||
},
|
||||
Parameters: &semantic.FunctionParameters{
|
||||
List: []*semantic.FunctionParameter{
|
||||
{Key: &semantic.Identifier{Name: "r"}},
|
||||
},
|
||||
Body: &semantic.ReturnStatement{
|
||||
Argument: &semantic.BooleanLiteral{Value: true},
|
||||
},
|
||||
Block: &semantic.Block{
|
||||
Body: []semantic.Statement{
|
||||
&semantic.ReturnStatement{
|
||||
Argument: &semantic.BooleanLiteral{Value: true},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
|
@ -181,13 +184,17 @@ func TestPushDownFilterRule(t *testing.T) {
|
|||
makeFilterFn := func(exprs ...semantic.Expression) *semantic.FunctionExpression {
|
||||
body := semantic.ExprsToConjunction(exprs...)
|
||||
return &semantic.FunctionExpression{
|
||||
Block: &semantic.FunctionBlock{
|
||||
Parameters: &semantic.FunctionParameters{
|
||||
List: []*semantic.FunctionParameter{
|
||||
{Key: &semantic.Identifier{Name: "r"}},
|
||||
Parameters: &semantic.FunctionParameters{
|
||||
List: []*semantic.FunctionParameter{
|
||||
{Key: &semantic.Identifier{Name: "r"}},
|
||||
},
|
||||
},
|
||||
Block: &semantic.Block{
|
||||
Body: []semantic.Statement{
|
||||
&semantic.ReturnStatement{
|
||||
Argument: body,
|
||||
},
|
||||
},
|
||||
Body: body,
|
||||
},
|
||||
}
|
||||
}
|
||||
|
@ -197,6 +204,13 @@ func TestPushDownFilterRule(t *testing.T) {
|
|||
Fn: makeFilterFn(exprs...),
|
||||
}
|
||||
}
|
||||
mustStoragePredicate := func(expression semantic.Expression, objectName string) *datatypes.Predicate {
|
||||
pred, err := influxdb.ToStoragePredicate(expression, objectName)
|
||||
if err != nil {
|
||||
t.Fatalf("Error while converting to storage predicate: %v", err)
|
||||
}
|
||||
return pred
|
||||
}
|
||||
|
||||
tests := []plantest.RuleTestCase{
|
||||
{
|
||||
|
@ -220,8 +234,7 @@ func TestPushDownFilterRule(t *testing.T) {
|
|||
Nodes: []plan.Node{
|
||||
plan.CreatePhysicalNode("merged_ReadRange_filter", &influxdb.ReadRangePhysSpec{
|
||||
Bounds: bounds,
|
||||
FilterSet: true,
|
||||
Filter: makeFilterFn(pushableExpr1),
|
||||
Predicate: mustStoragePredicate(pushableExpr1, "r"),
|
||||
}),
|
||||
},
|
||||
},
|
||||
|
@ -250,9 +263,12 @@ func TestPushDownFilterRule(t *testing.T) {
|
|||
After: &plantest.PlanSpec{
|
||||
Nodes: []plan.Node{
|
||||
plan.CreatePhysicalNode("merged_ReadRange_filter1_filter2", &influxdb.ReadRangePhysSpec{
|
||||
Bounds: bounds,
|
||||
FilterSet: true,
|
||||
Filter: makeFilterFn(pushableExpr1, pushableExpr2),
|
||||
Bounds: bounds,
|
||||
Predicate: mustStoragePredicate(&semantic.LogicalExpression{
|
||||
Operator: ast.AndOperator,
|
||||
Left: pushableExpr1,
|
||||
Right: pushableExpr2,
|
||||
}, "r"),
|
||||
}),
|
||||
},
|
||||
},
|
||||
|
@ -278,8 +294,7 @@ func TestPushDownFilterRule(t *testing.T) {
|
|||
Nodes: []plan.Node{
|
||||
plan.CreatePhysicalNode("ReadRange", &influxdb.ReadRangePhysSpec{
|
||||
Bounds: bounds,
|
||||
FilterSet: true,
|
||||
Filter: makeFilterFn(pushableExpr1),
|
||||
Predicate: mustStoragePredicate(pushableExpr1, "r"),
|
||||
}),
|
||||
plan.CreatePhysicalNode("filter", &universe.FilterProcedureSpec{
|
||||
Fn: makeResolvedFilterFn(unpushableExpr),
|
||||
|
@ -316,8 +331,7 @@ func TestPushDownFilterRule(t *testing.T) {
|
|||
Nodes: []plan.Node{
|
||||
plan.CreatePhysicalNode("merged_ReadRange_filter", &influxdb.ReadRangePhysSpec{
|
||||
Bounds: bounds,
|
||||
FilterSet: true,
|
||||
Filter: makeFilterFn(pushableExpr1),
|
||||
Predicate: mustStoragePredicate(pushableExpr1, "r"),
|
||||
}),
|
||||
},
|
||||
},
|
||||
|
@ -385,9 +399,8 @@ func TestPushDownFilterRule(t *testing.T) {
|
|||
After: &plantest.PlanSpec{
|
||||
Nodes: []plan.Node{
|
||||
plan.CreatePhysicalNode("merged_ReadRange_filter", &influxdb.ReadRangePhysSpec{
|
||||
Bounds: bounds,
|
||||
FilterSet: true,
|
||||
Filter: makeFilterFn(&semantic.BinaryExpression{
|
||||
Bounds: bounds,
|
||||
Predicate: mustStoragePredicate(&semantic.BinaryExpression{
|
||||
Operator: ast.NotEqualOperator,
|
||||
Left: &semantic.MemberExpression{
|
||||
Object: &semantic.IdentifierExpression{Name: "r"},
|
||||
|
@ -396,7 +409,7 @@ func TestPushDownFilterRule(t *testing.T) {
|
|||
Right: &semantic.StringLiteral{
|
||||
Value: "",
|
||||
},
|
||||
}),
|
||||
}, "r"),
|
||||
}),
|
||||
},
|
||||
},
|
||||
|
@ -429,9 +442,8 @@ func TestPushDownFilterRule(t *testing.T) {
|
|||
After: &plantest.PlanSpec{
|
||||
Nodes: []plan.Node{
|
||||
plan.CreatePhysicalNode("merged_ReadRange_filter", &influxdb.ReadRangePhysSpec{
|
||||
Bounds: bounds,
|
||||
FilterSet: true,
|
||||
Filter: makeFilterFn(&semantic.BinaryExpression{
|
||||
Bounds: bounds,
|
||||
Predicate: mustStoragePredicate(&semantic.BinaryExpression{
|
||||
Operator: ast.EqualOperator,
|
||||
Left: &semantic.MemberExpression{
|
||||
Object: &semantic.IdentifierExpression{Name: "r"},
|
||||
|
@ -440,7 +452,7 @@ func TestPushDownFilterRule(t *testing.T) {
|
|||
Right: &semantic.StringLiteral{
|
||||
Value: "",
|
||||
},
|
||||
}),
|
||||
}, "r"),
|
||||
}),
|
||||
},
|
||||
},
|
||||
|
@ -496,9 +508,8 @@ func TestPushDownFilterRule(t *testing.T) {
|
|||
After: &plantest.PlanSpec{
|
||||
Nodes: []plan.Node{
|
||||
plan.CreatePhysicalNode("merged_ReadRange_filter", &influxdb.ReadRangePhysSpec{
|
||||
Bounds: bounds,
|
||||
FilterSet: true,
|
||||
Filter: makeFilterFn(&semantic.BinaryExpression{
|
||||
Bounds: bounds,
|
||||
Predicate: mustStoragePredicate(&semantic.BinaryExpression{
|
||||
Operator: ast.NotEqualOperator,
|
||||
Left: &semantic.MemberExpression{
|
||||
Object: &semantic.IdentifierExpression{Name: "r"},
|
||||
|
@ -507,7 +518,7 @@ func TestPushDownFilterRule(t *testing.T) {
|
|||
Right: &semantic.StringLiteral{
|
||||
Value: "",
|
||||
},
|
||||
}),
|
||||
}, "r"),
|
||||
}),
|
||||
},
|
||||
},
|
||||
|
@ -538,16 +549,15 @@ func TestPushDownFilterRule(t *testing.T) {
|
|||
After: &plantest.PlanSpec{
|
||||
Nodes: []plan.Node{
|
||||
plan.CreatePhysicalNode("merged_ReadRange_filter", &influxdb.ReadRangePhysSpec{
|
||||
Bounds: bounds,
|
||||
FilterSet: true,
|
||||
Filter: makeFilterFn(&semantic.BinaryExpression{
|
||||
Bounds: bounds,
|
||||
Predicate: mustStoragePredicate(&semantic.BinaryExpression{
|
||||
Operator: ast.EqualOperator,
|
||||
Left: &semantic.MemberExpression{
|
||||
Object: &semantic.IdentifierExpression{Name: "r"},
|
||||
Property: "_value",
|
||||
},
|
||||
Right: &semantic.StringLiteral{Value: ""},
|
||||
}),
|
||||
}, "r"),
|
||||
}),
|
||||
},
|
||||
},
|
||||
|
@ -619,9 +629,8 @@ func TestPushDownFilterRule(t *testing.T) {
|
|||
After: &plantest.PlanSpec{
|
||||
Nodes: []plan.Node{
|
||||
plan.CreatePhysicalNode("merged_ReadRange_filter", &influxdb.ReadRangePhysSpec{
|
||||
Bounds: bounds,
|
||||
FilterSet: true,
|
||||
Filter: makeFilterFn(&semantic.LogicalExpression{
|
||||
Bounds: bounds,
|
||||
Predicate: mustStoragePredicate(&semantic.LogicalExpression{
|
||||
Operator: ast.AndOperator,
|
||||
Left: &semantic.BinaryExpression{
|
||||
Operator: ast.EqualOperator,
|
||||
|
@ -643,7 +652,7 @@ func TestPushDownFilterRule(t *testing.T) {
|
|||
Value: "",
|
||||
},
|
||||
},
|
||||
}),
|
||||
}, "r"),
|
||||
}),
|
||||
},
|
||||
},
|
||||
|
@ -867,24 +876,28 @@ func TestReadTagKeysRule(t *testing.T) {
|
|||
Fn: interpreter.ResolvedFunction{
|
||||
Scope: nil,
|
||||
Fn: &semantic.FunctionExpression{
|
||||
Block: &semantic.FunctionBlock{
|
||||
Parameters: &semantic.FunctionParameters{
|
||||
List: []*semantic.FunctionParameter{{
|
||||
Key: &semantic.Identifier{
|
||||
Name: "r",
|
||||
},
|
||||
}},
|
||||
},
|
||||
Body: &semantic.BinaryExpression{
|
||||
Operator: ast.EqualOperator,
|
||||
Left: &semantic.MemberExpression{
|
||||
Object: &semantic.IdentifierExpression{
|
||||
Name: "r",
|
||||
},
|
||||
Property: "_measurement",
|
||||
Parameters: &semantic.FunctionParameters{
|
||||
List: []*semantic.FunctionParameter{{
|
||||
Key: &semantic.Identifier{
|
||||
Name: "r",
|
||||
},
|
||||
Right: &semantic.StringLiteral{
|
||||
Value: "cpu",
|
||||
}},
|
||||
},
|
||||
Block: &semantic.Block{
|
||||
Body: []semantic.Statement{
|
||||
&semantic.ReturnStatement{
|
||||
Argument: &semantic.BinaryExpression{
|
||||
Operator: ast.EqualOperator,
|
||||
Left: &semantic.MemberExpression{
|
||||
Object: &semantic.IdentifierExpression{
|
||||
Name: "r",
|
||||
},
|
||||
Property: "_measurement",
|
||||
},
|
||||
Right: &semantic.StringLiteral{
|
||||
Value: "cpu",
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
|
@ -917,8 +930,15 @@ func TestReadTagKeysRule(t *testing.T) {
|
|||
},
|
||||
}
|
||||
if filter {
|
||||
s.FilterSet = true
|
||||
s.Filter = filterSpec.Fn.Fn
|
||||
body, ok := filterSpec.Fn.Fn.GetFunctionBodyExpression()
|
||||
if !ok {
|
||||
t.Fatal("could not get function bodyexpression for filter")
|
||||
}
|
||||
pred, err := influxdb.ToStoragePredicate(body, "r")
|
||||
if err != nil {
|
||||
t.Fatalf("Error while creating storage predicate: %v", err)
|
||||
}
|
||||
s.Predicate = pred
|
||||
}
|
||||
return &s
|
||||
}
|
||||
|
@ -1082,24 +1102,28 @@ func TestReadTagValuesRule(t *testing.T) {
|
|||
Fn: interpreter.ResolvedFunction{
|
||||
Scope: nil,
|
||||
Fn: &semantic.FunctionExpression{
|
||||
Block: &semantic.FunctionBlock{
|
||||
Parameters: &semantic.FunctionParameters{
|
||||
List: []*semantic.FunctionParameter{{
|
||||
Key: &semantic.Identifier{
|
||||
Name: "r",
|
||||
},
|
||||
}},
|
||||
},
|
||||
Body: &semantic.BinaryExpression{
|
||||
Operator: ast.EqualOperator,
|
||||
Left: &semantic.MemberExpression{
|
||||
Object: &semantic.IdentifierExpression{
|
||||
Name: "r",
|
||||
},
|
||||
Property: "_measurement",
|
||||
Parameters: &semantic.FunctionParameters{
|
||||
List: []*semantic.FunctionParameter{{
|
||||
Key: &semantic.Identifier{
|
||||
Name: "r",
|
||||
},
|
||||
Right: &semantic.StringLiteral{
|
||||
Value: "cpu",
|
||||
}},
|
||||
},
|
||||
Block: &semantic.Block{
|
||||
Body: []semantic.Statement{
|
||||
&semantic.ReturnStatement{
|
||||
Argument: &semantic.BinaryExpression{
|
||||
Operator: ast.EqualOperator,
|
||||
Left: &semantic.MemberExpression{
|
||||
Object: &semantic.IdentifierExpression{
|
||||
Name: "r",
|
||||
},
|
||||
Property: "_measurement",
|
||||
},
|
||||
Right: &semantic.StringLiteral{
|
||||
Value: "cpu",
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
|
@ -1134,8 +1158,15 @@ func TestReadTagValuesRule(t *testing.T) {
|
|||
TagKey: "host",
|
||||
}
|
||||
if filter {
|
||||
s.FilterSet = true
|
||||
s.Filter = filterSpec.Fn.Fn
|
||||
body, ok := filterSpec.Fn.Fn.GetFunctionBodyExpression()
|
||||
if !ok {
|
||||
t.Fatal("could not get function bodyexpression for filter")
|
||||
}
|
||||
pred, err := influxdb.ToStoragePredicate(body, "r")
|
||||
if err != nil {
|
||||
t.Fatalf("Error while creating storage predicate: %v", err)
|
||||
}
|
||||
s.Predicate = pred
|
||||
}
|
||||
return &s
|
||||
}
|
||||
|
|
|
@ -8,8 +8,8 @@ import (
|
|||
"github.com/influxdata/flux/codes"
|
||||
"github.com/influxdata/flux/execute"
|
||||
"github.com/influxdata/flux/memory"
|
||||
"github.com/influxdata/flux/metadata"
|
||||
"github.com/influxdata/flux/plan"
|
||||
"github.com/influxdata/flux/semantic"
|
||||
"github.com/influxdata/influxdb/tsdb/cursors"
|
||||
)
|
||||
|
||||
|
@ -31,9 +31,19 @@ type Source struct {
|
|||
alloc *memory.Allocator
|
||||
stats cursors.CursorStats
|
||||
|
||||
label string
|
||||
|
||||
runner runner
|
||||
}
|
||||
|
||||
func (s *Source) Label() string {
|
||||
return s.label
|
||||
}
|
||||
|
||||
func (s *Source) SetLabel(l string) {
|
||||
s.label = l
|
||||
}
|
||||
|
||||
func (s *Source) Run(ctx context.Context) {
|
||||
err := s.runner.run(ctx)
|
||||
for _, t := range s.ts {
|
||||
|
@ -45,8 +55,8 @@ func (s *Source) AddTransformation(t execute.Transformation) {
|
|||
s.ts = append(s.ts, t)
|
||||
}
|
||||
|
||||
func (s *Source) Metadata() flux.Metadata {
|
||||
return flux.Metadata{
|
||||
func (s *Source) Metadata() metadata.Metadata {
|
||||
return metadata.Metadata{
|
||||
"influxdb/scanned-bytes": []interface{}{s.stats.ScannedBytes},
|
||||
"influxdb/scanned-values": []interface{}{s.stats.ScannedValues},
|
||||
}
|
||||
|
@ -73,7 +83,7 @@ func (s *Source) processTables(ctx context.Context, tables TableIterator, waterm
|
|||
return nil
|
||||
}
|
||||
|
||||
func (s *Source) processTable(ctx context.Context, tbl flux.Table) error {
|
||||
func (s *Source) processTable(_ context.Context, tbl flux.Table) error {
|
||||
if len(s.ts) == 0 {
|
||||
tbl.Done()
|
||||
return nil
|
||||
|
@ -149,10 +159,6 @@ func createReadFilterSource(s plan.ProcedureSpec, id execute.DatasetID, a execut
|
|||
return nil, err
|
||||
}
|
||||
|
||||
var filter *semantic.FunctionExpression
|
||||
if spec.FilterSet {
|
||||
filter = spec.Filter
|
||||
}
|
||||
return ReadFilterSource(
|
||||
id,
|
||||
deps.Reader,
|
||||
|
@ -160,7 +166,7 @@ func createReadFilterSource(s plan.ProcedureSpec, id execute.DatasetID, a execut
|
|||
Database: db,
|
||||
RetentionPolicy: rp,
|
||||
Bounds: *bounds,
|
||||
Predicate: filter,
|
||||
Predicate: spec.Predicate,
|
||||
},
|
||||
a,
|
||||
), nil
|
||||
|
@ -215,10 +221,6 @@ func createReadGroupSource(s plan.ProcedureSpec, id execute.DatasetID, a execute
|
|||
return nil, err
|
||||
}
|
||||
|
||||
var filter *semantic.FunctionExpression
|
||||
if spec.FilterSet {
|
||||
filter = spec.Filter
|
||||
}
|
||||
return ReadGroupSource(
|
||||
id,
|
||||
deps.Reader,
|
||||
|
@ -227,7 +229,7 @@ func createReadGroupSource(s plan.ProcedureSpec, id execute.DatasetID, a execute
|
|||
Database: db,
|
||||
RetentionPolicy: rp,
|
||||
Bounds: *bounds,
|
||||
Predicate: filter,
|
||||
Predicate: spec.Predicate,
|
||||
},
|
||||
GroupMode: ToGroupMode(spec.GroupMode),
|
||||
GroupKeys: spec.GroupKeys,
|
||||
|
@ -248,11 +250,6 @@ func createReadTagKeysSource(prSpec plan.ProcedureSpec, dsid execute.DatasetID,
|
|||
return nil, err
|
||||
}
|
||||
|
||||
var filter *semantic.FunctionExpression
|
||||
if spec.FilterSet {
|
||||
filter = spec.Filter
|
||||
}
|
||||
|
||||
bounds := a.StreamContext().Bounds()
|
||||
return ReadTagKeysSource(
|
||||
dsid,
|
||||
|
@ -262,7 +259,7 @@ func createReadTagKeysSource(prSpec plan.ProcedureSpec, dsid execute.DatasetID,
|
|||
Database: db,
|
||||
RetentionPolicy: rp,
|
||||
Bounds: *bounds,
|
||||
Predicate: filter,
|
||||
Predicate: spec.Predicate,
|
||||
},
|
||||
},
|
||||
a,
|
||||
|
@ -307,11 +304,6 @@ func createReadTagValuesSource(prSpec plan.ProcedureSpec, dsid execute.DatasetID
|
|||
return nil, err
|
||||
}
|
||||
|
||||
var filter *semantic.FunctionExpression
|
||||
if spec.FilterSet {
|
||||
filter = spec.Filter
|
||||
}
|
||||
|
||||
bounds := a.StreamContext().Bounds()
|
||||
return ReadTagValuesSource(
|
||||
dsid,
|
||||
|
@ -321,7 +313,7 @@ func createReadTagValuesSource(prSpec plan.ProcedureSpec, dsid execute.DatasetID
|
|||
Database: db,
|
||||
RetentionPolicy: rp,
|
||||
Bounds: *bounds,
|
||||
Predicate: filter,
|
||||
Predicate: spec.Predicate,
|
||||
},
|
||||
TagKey: spec.TagKey,
|
||||
},
|
||||
|
|
|
@ -7,8 +7,8 @@ import (
|
|||
"github.com/influxdata/flux"
|
||||
"github.com/influxdata/flux/execute"
|
||||
"github.com/influxdata/flux/memory"
|
||||
"github.com/influxdata/flux/semantic"
|
||||
"github.com/influxdata/influxdb/services/meta"
|
||||
"github.com/influxdata/influxdb/storage/reads/datatypes"
|
||||
"github.com/influxdata/influxdb/tsdb/cursors"
|
||||
"github.com/influxdata/influxql"
|
||||
"github.com/pkg/errors"
|
||||
|
@ -65,7 +65,7 @@ type ReadFilterSpec struct {
|
|||
|
||||
Bounds execute.Bounds
|
||||
|
||||
Predicate *semantic.FunctionExpression
|
||||
Predicate *datatypes.Predicate
|
||||
}
|
||||
|
||||
type ReadGroupSpec struct {
|
||||
|
|
|
@ -1,4 +1,353 @@
|
|||
package influxdb
|
||||
|
||||
// TODO(jsternberg): Implement the to method in influxdb 1.x.
|
||||
// This file is kept around so it shows up in the patch.
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"time"
|
||||
|
||||
"github.com/influxdata/flux"
|
||||
"github.com/influxdata/flux/codes"
|
||||
"github.com/influxdata/flux/execute"
|
||||
"github.com/influxdata/flux/plan"
|
||||
"github.com/influxdata/flux/runtime"
|
||||
"github.com/influxdata/flux/semantic"
|
||||
"github.com/influxdata/flux/stdlib/influxdata/influxdb"
|
||||
"github.com/influxdata/flux/values"
|
||||
"github.com/influxdata/influxdb/coordinator"
|
||||
"github.com/influxdata/influxdb/models"
|
||||
"github.com/influxdata/influxql"
|
||||
)
|
||||
|
||||
const (
|
||||
ToKind = "influx1x/toKind"
|
||||
DefaultBufferSize = 5000
|
||||
)
|
||||
|
||||
type ToOpSpec struct {
|
||||
Bucket string `json:"bucket"`
|
||||
}
|
||||
|
||||
func init() {
|
||||
toSignature := runtime.MustLookupBuiltinType("influxdata/influxdb", influxdb.ToKind)
|
||||
runtime.ReplacePackageValue("influxdata/influxdb", "to", flux.MustValue(flux.FunctionValueWithSideEffect(ToKind, createToOpSpec, toSignature)))
|
||||
flux.RegisterOpSpec(ToKind, func() flux.OperationSpec { return &ToOpSpec{} })
|
||||
plan.RegisterProcedureSpecWithSideEffect(ToKind, newToProcedure, ToKind)
|
||||
execute.RegisterTransformation(ToKind, createToTransformation)
|
||||
}
|
||||
|
||||
var unsupportedToArgs = []string{"bucketID", "orgID", "host", "timeColumn", "measurementColumn", "tagColumns", "fieldFn"}
|
||||
|
||||
func (o *ToOpSpec) ReadArgs(args flux.Arguments) error {
|
||||
var err error
|
||||
var ok bool
|
||||
|
||||
for _, arg := range unsupportedToArgs {
|
||||
if _, ok = args.Get(arg); ok {
|
||||
return &flux.Error{
|
||||
Code: codes.Invalid,
|
||||
Msg: fmt.Sprintf("argument %s for 'to' not supported in 1.x flux engine", arg),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if o.Bucket, err = args.GetRequiredString("bucket"); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func createToOpSpec(args flux.Arguments, a *flux.Administration) (flux.OperationSpec, error) {
|
||||
if err := a.AddParentFromArgs(args); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
s := &ToOpSpec{}
|
||||
if err := s.ReadArgs(args); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return s, nil
|
||||
}
|
||||
|
||||
func (ToOpSpec) Kind() flux.OperationKind {
|
||||
return ToKind
|
||||
}
|
||||
|
||||
// ToProcedureSpec is the procedure spec for the `to` flux function.
|
||||
type ToProcedureSpec struct {
|
||||
plan.DefaultCost
|
||||
Spec *ToOpSpec
|
||||
}
|
||||
|
||||
// Kind returns the kind for the procedure spec for the `to` flux function.
|
||||
func (o *ToProcedureSpec) Kind() plan.ProcedureKind {
|
||||
return ToKind
|
||||
}
|
||||
|
||||
// Copy clones the procedure spec for `to` flux function.
|
||||
func (o *ToProcedureSpec) Copy() plan.ProcedureSpec {
|
||||
s := o.Spec
|
||||
res := &ToProcedureSpec{
|
||||
Spec: &ToOpSpec{
|
||||
Bucket: s.Bucket,
|
||||
},
|
||||
}
|
||||
return res
|
||||
}
|
||||
|
||||
func newToProcedure(qs flux.OperationSpec, a plan.Administration) (plan.ProcedureSpec, error) {
|
||||
spec, ok := qs.(*ToOpSpec)
|
||||
if !ok && spec != nil {
|
||||
return nil, &flux.Error{
|
||||
Code: codes.Internal,
|
||||
Msg: fmt.Sprintf("invalid spec type %T", qs),
|
||||
}
|
||||
}
|
||||
return &ToProcedureSpec{Spec: spec}, nil
|
||||
}
|
||||
|
||||
func createToTransformation(id execute.DatasetID, mode execute.AccumulationMode, spec plan.ProcedureSpec, a execute.Administration) (execute.Transformation, execute.Dataset, error) {
|
||||
s, ok := spec.(*ToProcedureSpec)
|
||||
if !ok {
|
||||
return nil, nil, &flux.Error{
|
||||
Code: codes.Internal,
|
||||
Msg: fmt.Sprintf("invalid spec type %T", spec),
|
||||
}
|
||||
}
|
||||
cache := execute.NewTableBuilderCache(a.Allocator())
|
||||
d := execute.NewDataset(id, mode, cache)
|
||||
deps := GetStorageDependencies(a.Context())
|
||||
if deps == (StorageDependencies{}) {
|
||||
return nil, nil, &flux.Error{
|
||||
Code: codes.Unimplemented,
|
||||
Msg: "cannot return storage dependencies; storage dependencies are unimplemented",
|
||||
}
|
||||
}
|
||||
t, err := NewToTransformation(a.Context(), d, cache, s, deps)
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
return t, d, nil
|
||||
}
|
||||
|
||||
type ToTransformation struct {
|
||||
execute.ExecutionNode
|
||||
Ctx context.Context
|
||||
DB string
|
||||
RP string
|
||||
d execute.Dataset
|
||||
cache execute.TableBuilderCache
|
||||
deps StorageDependencies
|
||||
buf *coordinator.BufferedPointsWriter
|
||||
}
|
||||
|
||||
func NewToTransformation(ctx context.Context, d execute.Dataset, cache execute.TableBuilderCache, toSpec *ToProcedureSpec, deps StorageDependencies) (*ToTransformation, error) {
|
||||
spec := toSpec.Spec
|
||||
var db, rp string
|
||||
db, rp, err := lookupDatabase(ctx, spec.Bucket, deps, influxql.WritePrivilege)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return &ToTransformation{
|
||||
Ctx: ctx,
|
||||
DB: db,
|
||||
RP: rp,
|
||||
d: d,
|
||||
cache: cache,
|
||||
deps: deps,
|
||||
buf: coordinator.NewBufferedPointsWriter(deps.PointsWriter, db, rp, DefaultBufferSize),
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (t *ToTransformation) RetractTable(id execute.DatasetID, key flux.GroupKey) error {
|
||||
return t.d.RetractTable(key)
|
||||
}
|
||||
|
||||
// Process does the actual work for the ToTransformation.
|
||||
func (t *ToTransformation) Process(id execute.DatasetID, tbl flux.Table) error {
|
||||
//TODO(lesam): this is where 2.x overrides with explicit tag columns
|
||||
measurementColumn := "_measurement"
|
||||
fieldColumn := "_field"
|
||||
excludeColumns := map[string]bool{
|
||||
execute.DefaultValueColLabel: true,
|
||||
fieldColumn: true,
|
||||
measurementColumn: true,
|
||||
}
|
||||
|
||||
isTagColumn := func(column flux.ColMeta) bool {
|
||||
return column.Type == flux.TString && !excludeColumns[column.Label]
|
||||
}
|
||||
|
||||
columns := tbl.Cols()
|
||||
isTag := make([]bool, len(columns))
|
||||
numTags := 0
|
||||
for i, col := range columns {
|
||||
isTag[i] = isTagColumn(col)
|
||||
numTags++
|
||||
}
|
||||
|
||||
// TODO(lesam): this is where 2.x overrides the default time column label
|
||||
timeColLabel := execute.DefaultTimeColLabel
|
||||
timeColIdx := execute.ColIdx(timeColLabel, columns)
|
||||
if timeColIdx < 0 {
|
||||
return &flux.Error{
|
||||
Code: codes.Invalid,
|
||||
Msg: "no time column detected",
|
||||
}
|
||||
}
|
||||
if columns[timeColIdx].Type != flux.TTime {
|
||||
return &flux.Error{
|
||||
Code: codes.Invalid,
|
||||
Msg: fmt.Sprintf("column %s of type %s is not of type %s", timeColLabel, columns[timeColIdx].Type, flux.TTime),
|
||||
}
|
||||
}
|
||||
|
||||
builder, new := t.cache.TableBuilder(tbl.Key())
|
||||
if new {
|
||||
if err := execute.AddTableCols(tbl, builder); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
return tbl.Do(func(er flux.ColReader) error {
|
||||
kv := make([][]byte, 0, numTags*2)
|
||||
var tags models.Tags
|
||||
var points models.Points
|
||||
outer:
|
||||
for i := 0; i < er.Len(); i++ {
|
||||
measurementName := ""
|
||||
fields := make(models.Fields)
|
||||
var pointTime time.Time
|
||||
kv = kv[0:]
|
||||
|
||||
// get the non-field values
|
||||
for j, col := range er.Cols() {
|
||||
switch {
|
||||
case col.Label == measurementColumn:
|
||||
measurementName = string(er.Strings(j).Value(i))
|
||||
case col.Label == timeColLabel:
|
||||
valueTime := execute.ValueForRow(er, i, j)
|
||||
if valueTime.IsNull() {
|
||||
// skip rows with null timestamp
|
||||
continue outer
|
||||
}
|
||||
pointTime = valueTime.Time().Time()
|
||||
|
||||
case isTag[j]:
|
||||
if col.Type != flux.TString {
|
||||
return &flux.Error{
|
||||
Code: codes.Invalid,
|
||||
Msg: "Invalid type for tag column",
|
||||
}
|
||||
}
|
||||
kv = append(kv, []byte(col.Label), er.Strings(j).Value(i))
|
||||
}
|
||||
}
|
||||
|
||||
// validate the point
|
||||
if pointTime.IsZero() {
|
||||
return &flux.Error{
|
||||
Code: codes.Invalid,
|
||||
Msg: "timestamp missing from block",
|
||||
}
|
||||
}
|
||||
if measurementName == "" {
|
||||
return &flux.Error{
|
||||
Code: codes.Invalid,
|
||||
Msg: fmt.Sprintf("no column with label %s exists", measurementColumn),
|
||||
}
|
||||
}
|
||||
|
||||
var fieldValues values.Object
|
||||
var err error
|
||||
// TODO(lesam): this is where we would support the `fn` argument to `to`
|
||||
if fieldValues, err = defaultFieldMapping(er, i); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
fieldValues.Range(func(k string, v values.Value) {
|
||||
if v.IsNull() {
|
||||
fields[k] = nil
|
||||
return
|
||||
}
|
||||
switch v.Type().Nature() {
|
||||
case semantic.Float:
|
||||
fields[k] = v.Float()
|
||||
case semantic.Int:
|
||||
fields[k] = v.Int()
|
||||
case semantic.UInt:
|
||||
fields[k] = v.UInt()
|
||||
case semantic.String:
|
||||
fields[k] = v.Str()
|
||||
case semantic.Time:
|
||||
fields[k] = v.Time()
|
||||
case semantic.Bool:
|
||||
fields[k] = v.Bool()
|
||||
}
|
||||
})
|
||||
|
||||
tags, _ = models.NewTagsKeyValues(tags, kv...)
|
||||
pt, err := models.NewPoint(measurementName, tags, fields, pointTime)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
points = append(points, pt)
|
||||
if err := execute.AppendRecord(i, er, builder); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
return t.buf.WritePointsInto(&coordinator.IntoWriteRequest{
|
||||
Database: t.DB,
|
||||
RetentionPolicy: t.RP,
|
||||
Points: points,
|
||||
})
|
||||
})
|
||||
}
|
||||
|
||||
// UpdateWatermark updates the watermark for the transformation for the `to` flux function.
|
||||
func (t *ToTransformation) UpdateWatermark(id execute.DatasetID, pt execute.Time) error {
|
||||
return t.d.UpdateWatermark(pt)
|
||||
}
|
||||
|
||||
// UpdateProcessingTime updates the processing time for the transformation for the `to` flux function.
|
||||
func (t *ToTransformation) UpdateProcessingTime(id execute.DatasetID, pt execute.Time) error {
|
||||
return t.d.UpdateProcessingTime(pt)
|
||||
}
|
||||
|
||||
// Finish is called after the `to` flux function's transformation is done processing.
|
||||
func (t *ToTransformation) Finish(id execute.DatasetID, err error) {
|
||||
if err == nil {
|
||||
err = t.buf.Flush()
|
||||
}
|
||||
t.d.Finish(err)
|
||||
}
|
||||
|
||||
func defaultFieldMapping(er flux.ColReader, row int) (values.Object, error) {
|
||||
fieldColumnIdx := execute.ColIdx("_field", er.Cols())
|
||||
valueColumnIdx := execute.ColIdx(execute.DefaultValueColLabel, er.Cols())
|
||||
|
||||
if fieldColumnIdx < 0 {
|
||||
return nil, &flux.Error{
|
||||
Code: codes.Invalid,
|
||||
Msg: "table has no _field column",
|
||||
}
|
||||
}
|
||||
|
||||
if valueColumnIdx < 0 {
|
||||
return nil, &flux.Error{
|
||||
Code: codes.Invalid,
|
||||
Msg: "table has no _value column",
|
||||
}
|
||||
}
|
||||
|
||||
value := execute.ValueForRow(er, row, valueColumnIdx)
|
||||
field := execute.ValueForRow(er, row, fieldColumnIdx)
|
||||
props := []semantic.PropertyType{
|
||||
{
|
||||
Key: []byte(field.Str()),
|
||||
Value: value.Type(),
|
||||
},
|
||||
}
|
||||
fieldValueMapping := values.NewObject(semantic.NewObjectType(props))
|
||||
fieldValueMapping.Set(field.Str(), value)
|
||||
return fieldValueMapping, nil
|
||||
}
|
||||
|
|
|
@ -1 +1,208 @@
|
|||
package influxdb_test
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"testing"
|
||||
|
||||
"github.com/influxdata/flux"
|
||||
"github.com/influxdata/flux/dependencies/dependenciestest"
|
||||
"github.com/influxdata/flux/execute"
|
||||
"github.com/influxdata/flux/execute/executetest"
|
||||
"github.com/influxdata/flux/querytest"
|
||||
"github.com/influxdata/flux/runtime"
|
||||
_ "github.com/influxdata/flux/stdlib"
|
||||
"github.com/influxdata/influxdb/coordinator"
|
||||
"github.com/influxdata/influxdb/flux/stdlib/influxdata/influxdb"
|
||||
"github.com/influxdata/influxdb/models"
|
||||
"github.com/influxdata/influxdb/pkg/testing/assert"
|
||||
"github.com/influxdata/influxdb/services/meta"
|
||||
)
|
||||
|
||||
func TestTo_Query(t *testing.T) {
|
||||
runtime.FinalizeBuiltIns()
|
||||
tests := []querytest.NewQueryTestCase{
|
||||
{
|
||||
Name: "from with database with range",
|
||||
Raw: `from(bucket:"mydb") |> to(bucket:"myotherdb/autogen")`,
|
||||
Want: &flux.Spec{
|
||||
Operations: []*flux.Operation{
|
||||
{
|
||||
ID: "influxDBFrom0",
|
||||
Spec: &influxdb.FromOpSpec{
|
||||
Bucket: "mydb",
|
||||
},
|
||||
},
|
||||
{
|
||||
ID: "influx1x/toKind1",
|
||||
Spec: &influxdb.ToOpSpec{
|
||||
Bucket: "myotherdb/autogen",
|
||||
},
|
||||
},
|
||||
},
|
||||
Edges: []flux.Edge{
|
||||
{Parent: "influxDBFrom0", Child: "influx1x/toKind1"},
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
for _, tc := range tests {
|
||||
tc := tc
|
||||
t.Run(tc.Name, func(t *testing.T) {
|
||||
t.Parallel()
|
||||
querytest.NewQueryTestHelper(t, tc)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestTo_Process(t *testing.T) {
|
||||
type wanted struct {
|
||||
tables []*executetest.Table
|
||||
result *mockPointsWriter
|
||||
}
|
||||
testCases := []struct {
|
||||
name string
|
||||
spec *influxdb.ToProcedureSpec
|
||||
data []flux.Table
|
||||
want wanted
|
||||
}{
|
||||
{
|
||||
name: "default case",
|
||||
spec: &influxdb.ToProcedureSpec{
|
||||
Spec: &influxdb.ToOpSpec{
|
||||
Bucket: "my_db",
|
||||
},
|
||||
},
|
||||
data: []flux.Table{executetest.MustCopyTable(&executetest.Table{
|
||||
ColMeta: []flux.ColMeta{
|
||||
{Label: "_start", Type: flux.TTime},
|
||||
{Label: "_stop", Type: flux.TTime},
|
||||
{Label: "_time", Type: flux.TTime},
|
||||
{Label: "_measurement", Type: flux.TString},
|
||||
{Label: "_field", Type: flux.TString},
|
||||
{Label: "_value", Type: flux.TFloat},
|
||||
},
|
||||
Data: [][]interface{}{
|
||||
{execute.Time(0), execute.Time(100), execute.Time(11), "a", "_value", 2.0},
|
||||
{execute.Time(0), execute.Time(100), execute.Time(21), "a", "_value", 2.0},
|
||||
{execute.Time(0), execute.Time(100), execute.Time(21), "b", "_value", 1.0},
|
||||
{execute.Time(0), execute.Time(100), execute.Time(31), "a", "_value", 3.0},
|
||||
{execute.Time(0), execute.Time(100), execute.Time(41), "c", "_value", 4.0},
|
||||
},
|
||||
})},
|
||||
want: wanted{
|
||||
result: &mockPointsWriter{
|
||||
points: mockPoints(`a _value=2 11
|
||||
a _value=2 21
|
||||
b _value=1 21
|
||||
a _value=3 31
|
||||
c _value=4 41`),
|
||||
db: "my_db",
|
||||
rp: "autogen",
|
||||
},
|
||||
tables: []*executetest.Table{{
|
||||
ColMeta: []flux.ColMeta{
|
||||
{Label: "_start", Type: flux.TTime},
|
||||
{Label: "_stop", Type: flux.TTime},
|
||||
{Label: "_time", Type: flux.TTime},
|
||||
{Label: "_measurement", Type: flux.TString},
|
||||
{Label: "_field", Type: flux.TString},
|
||||
{Label: "_value", Type: flux.TFloat},
|
||||
},
|
||||
Data: [][]interface{}{
|
||||
{execute.Time(0), execute.Time(100), execute.Time(11), "a", "_value", 2.0},
|
||||
{execute.Time(0), execute.Time(100), execute.Time(21), "a", "_value", 2.0},
|
||||
{execute.Time(0), execute.Time(100), execute.Time(21), "b", "_value", 1.0},
|
||||
{execute.Time(0), execute.Time(100), execute.Time(31), "a", "_value", 3.0},
|
||||
{execute.Time(0), execute.Time(100), execute.Time(41), "c", "_value", 4.0},
|
||||
},
|
||||
}},
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
for _, tc := range testCases {
|
||||
tc := tc
|
||||
t.Run(tc.name, func(t *testing.T) {
|
||||
deps := influxdb.Dependencies{
|
||||
FluxDeps: dependenciestest.Default(),
|
||||
StorageDeps: influxdb.StorageDependencies{
|
||||
MetaClient: new(mockMetaClient),
|
||||
PointsWriter: &mockPointsWriter{
|
||||
db: tc.want.result.db,
|
||||
rp: tc.want.result.rp,
|
||||
},
|
||||
},
|
||||
}
|
||||
executetest.ProcessTestHelper(
|
||||
t,
|
||||
tc.data,
|
||||
tc.want.tables,
|
||||
nil,
|
||||
func(d execute.Dataset, c execute.TableBuilderCache) execute.Transformation {
|
||||
ctx := deps.Inject(context.Background())
|
||||
newT, err := influxdb.NewToTransformation(ctx, d, c, tc.spec, deps.StorageDeps)
|
||||
if err != nil {
|
||||
t.Error(err)
|
||||
}
|
||||
return newT
|
||||
},
|
||||
)
|
||||
pw := deps.StorageDeps.PointsWriter.(*mockPointsWriter)
|
||||
assert.Equal(t, len(tc.want.result.points), len(pw.points))
|
||||
|
||||
gotStr := pointsToStr(pw.points)
|
||||
wantStr := pointsToStr(tc.want.result.points)
|
||||
|
||||
assert.Equal(t, wantStr, gotStr)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
type mockPointsWriter struct {
|
||||
points models.Points
|
||||
db string
|
||||
rp string
|
||||
}
|
||||
|
||||
func (m *mockPointsWriter) WritePointsInto(request *coordinator.IntoWriteRequest) error {
|
||||
if m.db != request.Database {
|
||||
return fmt.Errorf("Wrong database - %s != %s", m.db, request.Database)
|
||||
}
|
||||
if m.rp != request.RetentionPolicy {
|
||||
return fmt.Errorf("Wrong retention policy - %s != %s", m.rp, request.RetentionPolicy)
|
||||
}
|
||||
m.points = append(m.points, request.Points...)
|
||||
return nil
|
||||
}
|
||||
|
||||
type mockMetaClient struct {
|
||||
}
|
||||
|
||||
func (m *mockMetaClient) Databases() []meta.DatabaseInfo {
|
||||
panic("mockMetaClient.Databases not implemented")
|
||||
}
|
||||
|
||||
func (m *mockMetaClient) Database(name string) *meta.DatabaseInfo {
|
||||
return &meta.DatabaseInfo{
|
||||
Name: name,
|
||||
DefaultRetentionPolicy: "autogen",
|
||||
RetentionPolicies: []meta.RetentionPolicyInfo{{Name: "autogen"}},
|
||||
}
|
||||
}
|
||||
|
||||
func pointsToStr(points []models.Point) string {
|
||||
outStr := ""
|
||||
for _, x := range points {
|
||||
outStr += x.String() + "\n"
|
||||
}
|
||||
return outStr
|
||||
}
|
||||
|
||||
func mockPoints(pointdata string) []models.Point {
|
||||
points, err := models.ParsePoints([]byte(pointdata))
|
||||
if err != nil {
|
||||
return nil
|
||||
}
|
||||
return points
|
||||
}
|
||||
|
|
|
@ -9,20 +9,21 @@ import (
|
|||
"github.com/influxdata/flux/execute"
|
||||
"github.com/influxdata/flux/memory"
|
||||
"github.com/influxdata/flux/plan"
|
||||
v1 "github.com/influxdata/flux/stdlib/influxdata/influxdb/v1"
|
||||
"github.com/influxdata/flux/runtime"
|
||||
"github.com/influxdata/flux/values"
|
||||
"github.com/influxdata/influxdb/flux/stdlib/influxdata/influxdb"
|
||||
"github.com/influxdata/influxdb/services/meta"
|
||||
"github.com/influxdata/influxql"
|
||||
)
|
||||
|
||||
const DatabasesKind = v1.DatabasesKind
|
||||
const DatabasesKind = "influxDBDatabases"
|
||||
|
||||
type DatabasesOpSpec struct {
|
||||
}
|
||||
|
||||
func init() {
|
||||
flux.ReplacePackageValue("influxdata/influxdb/v1", DatabasesKind, flux.FunctionValue(DatabasesKind, createDatabasesOpSpec, v1.DatabasesSignature))
|
||||
databasesSignature := runtime.MustLookupBuiltinType("influxdata/influxdb/v1", "databases")
|
||||
runtime.ReplacePackageValue("influxdata/influxdb/v1", "databases", flux.MustValue(flux.FunctionValue(DatabasesKind, createDatabasesOpSpec, databasesSignature)))
|
||||
flux.RegisterOpSpec(DatabasesKind, newDatabasesOp)
|
||||
plan.RegisterProcedureSpec(DatabasesKind, newDatabasesProcedure, DatabasesKind)
|
||||
}
|
||||
|
|
10
go.mod
10
go.mod
|
@ -5,7 +5,7 @@ go 1.13
|
|||
require (
|
||||
collectd.org v0.3.0
|
||||
github.com/BurntSushi/toml v0.3.1
|
||||
github.com/apache/arrow/go/arrow v0.0.0-20191024131854-af6fa24be0db
|
||||
github.com/apache/arrow/go/arrow v0.0.0-20200923215132-ac86123a3f01
|
||||
github.com/bmizerany/pat v0.0.0-20170815010413-6226ea591a40
|
||||
github.com/boltdb/bolt v1.3.1
|
||||
github.com/cespare/xxhash v1.1.0
|
||||
|
@ -15,8 +15,9 @@ require (
|
|||
github.com/gogo/protobuf v1.3.1
|
||||
github.com/golang/snappy v0.0.1
|
||||
github.com/google/go-cmp v0.5.0
|
||||
github.com/influxdata/flux v0.65.0
|
||||
github.com/influxdata/flux v0.105.1
|
||||
github.com/influxdata/influxql v1.1.1-0.20210223160523-b6ab99450c93
|
||||
github.com/influxdata/pkg-config v0.2.6
|
||||
github.com/influxdata/roaring v0.4.13-0.20180809181101-fc520f41fab6
|
||||
github.com/influxdata/usage-client v0.0.0-20160829180054-6d3895376368
|
||||
github.com/jsternberg/zap-logfmt v1.2.0
|
||||
|
@ -36,10 +37,11 @@ require (
|
|||
github.com/willf/bitset v1.1.9 // indirect
|
||||
github.com/xlab/treeprint v0.0.0-20180616005107-d6fb6747feb6
|
||||
go.uber.org/zap v1.14.1
|
||||
golang.org/x/crypto v0.0.0-20200422194213-44a606286825
|
||||
golang.org/x/sync v0.0.0-20200317015054-43a5402ce75a
|
||||
golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9
|
||||
golang.org/x/sync v0.0.0-20200625203802-6e8e738ad208
|
||||
golang.org/x/sys v0.0.0-20200420163511-1957bb5e6d1f
|
||||
golang.org/x/text v0.3.2
|
||||
golang.org/x/time v0.0.0-20200416051211-89c76fbcd5d1
|
||||
golang.org/x/tools v0.0.0-20200721032237-77f530d86f9a
|
||||
google.golang.org/grpc v1.29.1
|
||||
)
|
||||
|
|
85
go.sum
85
go.sum
|
@ -18,6 +18,8 @@ cloud.google.com/go/bigquery v1.4.0 h1:xE3CPsOgttP4ACBePh79zTKALtXwn/Edhcr16R5hM
|
|||
cloud.google.com/go/bigquery v1.4.0/go.mod h1:S8dzgnTigyfTmLBfrtrhyYhwRxG72rYxvftPBK2Dvzc=
|
||||
cloud.google.com/go/bigtable v1.2.0 h1:F4cCmA4nuV84V5zYQ3MKY+M1Cw1avHDuf3S/LcZPA9c=
|
||||
cloud.google.com/go/bigtable v1.2.0/go.mod h1:JcVAOl45lrTmQfLj7T6TxyMzIN/3FGGcFm+2xVAli2o=
|
||||
cloud.google.com/go/bigtable v1.3.0 h1:PAplkJLXheOLlK5PPyy4/HXtPzHn+1/LaYDWIeGxnio=
|
||||
cloud.google.com/go/bigtable v1.3.0/go.mod h1:z5EyKrPE8OQmeg4h5MNdKvuSnI9CCT49Ki3f23aBzio=
|
||||
cloud.google.com/go/datastore v1.0.0/go.mod h1:LXYbyblFSglQ5pkeyhO+Qmw7ukd3C+pD7TKLgZqpHYE=
|
||||
cloud.google.com/go/datastore v1.1.0 h1:/May9ojXjRkPBNVrq+oWLqmWCkr4OU5uRY29bu0mRyQ=
|
||||
cloud.google.com/go/datastore v1.1.0/go.mod h1:umbIZjpQpHh4hmRpGhH4tLFup+FVzqBi1b3c64qFpCk=
|
||||
|
@ -34,26 +36,44 @@ collectd.org v0.3.0/go.mod h1:A/8DzQBkF6abtvrT2j/AU/4tiBgJWYyh0y/oB/4MlWE=
|
|||
dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9/go.mod h1:H6x//7gZCb22OMCxBHrMx7a5I7Hp++hsVxbQ4BYO7hU=
|
||||
github.com/Azure/azure-sdk-for-go v41.3.0+incompatible/go.mod h1:9XXNKU+eRnpl9moKnB4QOLf1HestfXbmab5FXxiDBjc=
|
||||
github.com/Azure/go-autorest/autorest v0.9.0/go.mod h1:xyHB1BMZT0cuDHU7I0+g046+BFDTQ8rEZB0s4Yfa6bI=
|
||||
github.com/Azure/go-autorest/autorest v0.9.3/go.mod h1:GsRuLYvwzLjjjRoWEIyMUaYq8GNUx2nRB378IPt/1p0=
|
||||
github.com/Azure/go-autorest/autorest v0.10.0/go.mod h1:/FALq9T/kS7b5J5qsQ+RSTUdAmGFqi0vUdVNNx8q630=
|
||||
github.com/Azure/go-autorest/autorest v0.10.1 h1:uaB8A32IZU9YKs9v50+/LWIWTDHJk2vlGzbfd7FfESI=
|
||||
github.com/Azure/go-autorest/autorest v0.10.1/go.mod h1:/FALq9T/kS7b5J5qsQ+RSTUdAmGFqi0vUdVNNx8q630=
|
||||
github.com/Azure/go-autorest/autorest/adal v0.5.0/go.mod h1:8Z9fGy2MpX0PvDjB1pEgQTmVqjGhiHBW7RJJEciWzS0=
|
||||
github.com/Azure/go-autorest/autorest/adal v0.8.0/go.mod h1:Z6vX6WXXuyieHAXwMj0S6HY6e6wcHn37qQMBQlvY3lc=
|
||||
github.com/Azure/go-autorest/autorest/adal v0.8.1/go.mod h1:ZjhuQClTqx435SRJ2iMlOxPYt3d2C/T/7TiQCVZSn3Q=
|
||||
github.com/Azure/go-autorest/autorest/adal v0.8.2/go.mod h1:ZjhuQClTqx435SRJ2iMlOxPYt3d2C/T/7TiQCVZSn3Q=
|
||||
github.com/Azure/go-autorest/autorest/adal v0.8.3 h1:O1AGG9Xig71FxdX9HO5pGNyZ7TbSyHaVg+5eJO/jSGw=
|
||||
github.com/Azure/go-autorest/autorest/adal v0.8.3/go.mod h1:ZjhuQClTqx435SRJ2iMlOxPYt3d2C/T/7TiQCVZSn3Q=
|
||||
github.com/Azure/go-autorest/autorest/azure/auth v0.4.2 h1:iM6UAvjR97ZIeR93qTcwpKNMpV+/FTWjwEbuPD495Tk=
|
||||
github.com/Azure/go-autorest/autorest/azure/auth v0.4.2/go.mod h1:90gmfKdlmKgfjUpnCEpOJzsUEjrWDSLwHIG73tSXddM=
|
||||
github.com/Azure/go-autorest/autorest/azure/cli v0.3.1 h1:LXl088ZQlP0SBppGFsRZonW6hSvwgL5gRByMbvUbx8U=
|
||||
github.com/Azure/go-autorest/autorest/azure/cli v0.3.1/go.mod h1:ZG5p860J94/0kI9mNJVoIoLgXcirM2gF5i2kWloofxw=
|
||||
github.com/Azure/go-autorest/autorest/date v0.1.0/go.mod h1:plvfp3oPSKwf2DNjlBjWF/7vwR+cUD/ELuzDCXwHUVA=
|
||||
github.com/Azure/go-autorest/autorest/date v0.2.0 h1:yW+Zlqf26583pE43KhfnhFcdmSWlm5Ew6bxipnr/tbM=
|
||||
github.com/Azure/go-autorest/autorest/date v0.2.0/go.mod h1:vcORJHLJEh643/Ioh9+vPmf1Ij9AEBM5FuBIXLmIy0g=
|
||||
github.com/Azure/go-autorest/autorest/mocks v0.1.0/go.mod h1:OTyCOPRA2IgIlWxVYxBee2F5Gr4kF2zd2J5cFRaIDN0=
|
||||
github.com/Azure/go-autorest/autorest/mocks v0.2.0/go.mod h1:OTyCOPRA2IgIlWxVYxBee2F5Gr4kF2zd2J5cFRaIDN0=
|
||||
github.com/Azure/go-autorest/autorest/mocks v0.3.0 h1:qJumjCaCudz+OcqE9/XtEPfvtOjOmKaui4EOpFI6zZc=
|
||||
github.com/Azure/go-autorest/autorest/mocks v0.3.0/go.mod h1:a8FDP3DYzQ4RYfVAxAN3SVSiiO77gL2j2ronKKP0syM=
|
||||
github.com/Azure/go-autorest/autorest/to v0.3.0/go.mod h1:MgwOyqaIuKdG4TL/2ywSsIWKAfJfgHDo8ObuUk3t5sA=
|
||||
github.com/Azure/go-autorest/autorest/validation v0.2.0/go.mod h1:3EEqHnBxQGHXRYq3HT1WyXAvT7LLY3tl70hw6tQIbjI=
|
||||
github.com/Azure/go-autorest/logger v0.1.0 h1:ruG4BSDXONFRrZZJ2GUXDiUyVpayPmb1GnWeHDdaNKY=
|
||||
github.com/Azure/go-autorest/logger v0.1.0/go.mod h1:oExouG+K6PryycPJfVSxi/koC6LSNgds39diKLz7Vrc=
|
||||
github.com/Azure/go-autorest/tracing v0.5.0 h1:TRn4WjSnkcSy5AEG3pnbtFSwNtwzjr4VYyQflFE619k=
|
||||
github.com/Azure/go-autorest/tracing v0.5.0/go.mod h1:r/s2XiOKccPW3HrqB+W0TQzfbtp2fGCgRFtBroKn4Dk=
|
||||
github.com/BurntSushi/toml v0.3.1 h1:WXkYYl6Yr3qBf1K79EBnL4mak0OimBfB0XUf9Vl28OQ=
|
||||
github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU=
|
||||
github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym/WlBOVXweHU+Q+/VP0lqqI8lqeDx9IjBqo=
|
||||
github.com/DATA-DOG/go-sqlmock v1.3.3 h1:CWUqKXe0s8A2z6qCgkP4Kru7wC11YoAnoupUKFDnH08=
|
||||
github.com/DATA-DOG/go-sqlmock v1.3.3/go.mod h1:f/Ixk793poVmq4qj/V1dPUg2JEAKC73Q5eFN3EC/SaM=
|
||||
github.com/DATA-DOG/go-sqlmock v1.4.1 h1:ThlnYciV1iM/V0OSF/dtkqWb6xo5qITT1TJBG1MRDJM=
|
||||
github.com/DATA-DOG/go-sqlmock v1.4.1/go.mod h1:f/Ixk793poVmq4qj/V1dPUg2JEAKC73Q5eFN3EC/SaM=
|
||||
github.com/DataDog/datadog-go v3.2.0+incompatible/go.mod h1:LButxg5PwREeZtORoXG3tL4fMGNddJ+vMq1mwgfaqoQ=
|
||||
github.com/Knetic/govaluate v3.0.1-0.20171022003610-9aa49832a739+incompatible/go.mod h1:r7JcOSlj0wfOMncg0iLm8Leh48TZaKVeNIfJntJ2wa0=
|
||||
github.com/Masterminds/semver v1.4.2 h1:WBLTQ37jOCzSLtXNdoo8bNM8876KhNqOKvrlGITgsTc=
|
||||
github.com/Masterminds/semver v1.4.2/go.mod h1:MB6lktGJrhw8PrUyiEoblNEGEQ+RzHPF078ddwwvV3Y=
|
||||
github.com/NYTimes/gziphandler v0.0.0-20170623195520-56545f4a5d46/go.mod h1:3wb06e3pkSAbeQ52E9H9iFoQsEEwGN64994WTCIhntQ=
|
||||
github.com/OneOfOne/xxhash v1.2.2 h1:KMrpdQIwFcEqXDklaen+P1axHaj9BSKzvpUUfnHldSE=
|
||||
github.com/OneOfOne/xxhash v1.2.2/go.mod h1:HSdplMjZKSmBqAxg5vPj2TmRDmfkzw+cTzAElWljhcU=
|
||||
|
@ -62,6 +82,8 @@ github.com/PuerkitoBio/purell v1.1.0/go.mod h1:c11w/QuzBsJSee3cPx9rAFu61PvFxuPbt
|
|||
github.com/PuerkitoBio/purell v1.1.1/go.mod h1:c11w/QuzBsJSee3cPx9rAFu61PvFxuPbtSwDGJws/X0=
|
||||
github.com/PuerkitoBio/urlesc v0.0.0-20160726150825-5bd2802263f2/go.mod h1:uGdkoq3SwY9Y+13GIhn11/XLaGBb4BfwItxLd5jeuXE=
|
||||
github.com/PuerkitoBio/urlesc v0.0.0-20170810143723-de5bf2ad4578/go.mod h1:uGdkoq3SwY9Y+13GIhn11/XLaGBb4BfwItxLd5jeuXE=
|
||||
github.com/SAP/go-hdb v0.14.1 h1:hkw4ozGZ/i4eak7ZuGkY5e0hxiXFdNUBNhr4AvZVNFE=
|
||||
github.com/SAP/go-hdb v0.14.1/go.mod h1:7fdQLVC2lER3urZLjZCm0AuMQfApof92n3aylBPEkMo=
|
||||
github.com/Shopify/sarama v1.19.0/go.mod h1:FVkBWblsNy7DGZRfXLU0O9RCGt5g3g3yEuWXgklEdEo=
|
||||
github.com/Shopify/toxiproxy v2.1.4+incompatible/go.mod h1:OXgGpZ6Cli1/URJOF1DMxUHB2q5Ap20/P/eIdh4G0pI=
|
||||
github.com/VividCortex/gohistogram v1.0.0/go.mod h1:Pf5mBqqDxYaXu3hDrrU+w6nw50o/4+TcAqDqk/vUH7g=
|
||||
|
@ -78,6 +100,8 @@ github.com/andreyvit/diff v0.0.0-20170406064948-c7f18ee00883/go.mod h1:rCTlJbsFo
|
|||
github.com/antihax/optional v0.0.0-20180407024304-ca021399b1a6/go.mod h1:V8iCPQYkqmusNa815XgQio277wI47sdRh1dUOLdyC6Q=
|
||||
github.com/apache/arrow/go/arrow v0.0.0-20191024131854-af6fa24be0db h1:nxAtV4VajJDhKysp2kdcJZsq8Ss1xSA0vZTkVHHJd0E=
|
||||
github.com/apache/arrow/go/arrow v0.0.0-20191024131854-af6fa24be0db/go.mod h1:VTxUBvSJ3s3eHAg65PNgrsn5BtqCRPdmyXh6rAfdxN0=
|
||||
github.com/apache/arrow/go/arrow v0.0.0-20200923215132-ac86123a3f01 h1:FSqtT0UCktIlSU19mxj0YE5HK3HOO4IFMU9BpOif/7A=
|
||||
github.com/apache/arrow/go/arrow v0.0.0-20200923215132-ac86123a3f01/go.mod h1:QNYViu/X0HXDHw7m3KXzWSVXIbfUvJqBFe6Gj8/pYA0=
|
||||
github.com/apache/thrift v0.12.0/go.mod h1:cp2SuWMxlEZw2r+iP2GNCdIi4C1qmUzdZFSVb+bacwQ=
|
||||
github.com/apache/thrift v0.13.0/go.mod h1:cp2SuWMxlEZw2r+iP2GNCdIi4C1qmUzdZFSVb+bacwQ=
|
||||
github.com/armon/circbuf v0.0.0-20150827004946-bbbad097214e/go.mod h1:3U/XgcO3hCbHZ8TKRvWD2dDTCfh9M9ya+I9JpbB7O8o=
|
||||
|
@ -90,8 +114,12 @@ github.com/asaskevich/govalidator v0.0.0-20190424111038-f61b66f89f4a/go.mod h1:l
|
|||
github.com/asaskevich/govalidator v0.0.0-20200108200545-475eaeb16496/go.mod h1:oGkLhpf+kjZl6xBf758TQhh5XrAeiJv/7FRz/2spLIg=
|
||||
github.com/aws/aws-lambda-go v1.13.3/go.mod h1:4UKl9IzQMoD+QF79YdCuzCwp8VbmG4VAQwij/eHl5CU=
|
||||
github.com/aws/aws-sdk-go v1.27.0/go.mod h1:KmX6BPdI08NWTb3/sm4ZGu5ShLoqVDhKgpiN924inxo=
|
||||
github.com/aws/aws-sdk-go v1.29.16/go.mod h1:1KvfttTE3SPKMpo8g2c6jL3ZKfXtFvKscTgahTma5Xg=
|
||||
github.com/aws/aws-sdk-go v1.30.12 h1:KrjyosZvkpJjcwMk0RNxMZewQ47v7+ZkbQDXjWsJMs8=
|
||||
github.com/aws/aws-sdk-go v1.30.12/go.mod h1:5zCpMtNQVjRREroY7sYe8lOMRSxkhG6MZveU8YkpAk0=
|
||||
github.com/aws/aws-sdk-go-v2 v0.18.0/go.mod h1:JWVYvqSMppoMJC0x5wdwiImzgXTI9FuZwxzkQq9wy+g=
|
||||
github.com/benbjohnson/immutable v0.2.1 h1:EVv7H1ju7cDg/a8HUF4hAH4DBrMJh6RWWFwq9JfoO9I=
|
||||
github.com/benbjohnson/immutable v0.2.1/go.mod h1:uc6OHo6PN2++n98KHLxW8ef4W42ylHiQSENghE1ezxI=
|
||||
github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q=
|
||||
github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8=
|
||||
github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM=
|
||||
|
@ -101,8 +129,11 @@ github.com/bmizerany/pat v0.0.0-20170815010413-6226ea591a40 h1:y4B3+GPxKlrigF1ha
|
|||
github.com/bmizerany/pat v0.0.0-20170815010413-6226ea591a40/go.mod h1:8rLXio+WjiTceGBHIoTvn60HIbs7Hm7bcHjyrSqYB9c=
|
||||
github.com/boltdb/bolt v1.3.1 h1:JQmyP4ZBrce+ZQu0dY660FMfatumYDLun9hBCUVIkF4=
|
||||
github.com/boltdb/bolt v1.3.1/go.mod h1:clJnj/oiGkjum5o1McbSZDSLxVThjynRyGBgiAx27Ps=
|
||||
github.com/bonitoo-io/go-sql-bigquery v0.3.4-1.4.0 h1:MaVh0h9+KaMnJcoDvvIGp+O3fefdWm+8MBUX6ELTJTM=
|
||||
github.com/bonitoo-io/go-sql-bigquery v0.3.4-1.4.0/go.mod h1:J4Y6YJm0qTWB9aFziB7cPeSyc6dOZFyJdteSeybVpXQ=
|
||||
github.com/c-bata/go-prompt v0.2.2 h1:uyKRz6Z6DUyj49QVijyM339UJV9yhbr70gESwbNU3e0=
|
||||
github.com/c-bata/go-prompt v0.2.2/go.mod h1:VzqtzE2ksDBcdln8G7mk2RX9QyGjH+OVqOCSiVIqS34=
|
||||
github.com/cactus/go-statsd-client/statsd v0.0.0-20191106001114-12b4e2b38748/go.mod h1:l/bIBLeOl9eX+wxJAzxS4TveKRtAqlyDpHjhkfO0MEI=
|
||||
github.com/casbin/casbin/v2 v2.1.2/go.mod h1:YcPU1XXisHhLzuxH9coDNf2FbKpjGlbCg3n9yuLkIJQ=
|
||||
github.com/cenkalti/backoff v0.0.0-20181003080854-62661b46c409/go.mod h1:90ReRw6GdpyfrHakVjL/QHaoyV4aDUVVkXQJJJ3NXXM=
|
||||
github.com/cenkalti/backoff v2.2.1+incompatible/go.mod h1:90ReRw6GdpyfrHakVjL/QHaoyV4aDUVVkXQJJJ3NXXM=
|
||||
|
@ -131,6 +162,8 @@ github.com/dave/jennifer v1.2.0/go.mod h1:fIb+770HOpJ2fmN9EPPKOqm1vMGhB+TwXKMZhr
|
|||
github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38=
|
||||
github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c=
|
||||
github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38=
|
||||
github.com/denisenkom/go-mssqldb v0.0.0-20200428022330-06a60b6afbbc h1:VRRKCwnzqk8QCaRC4os14xoKDdbHqqlJtJA0oc1ZAjg=
|
||||
github.com/denisenkom/go-mssqldb v0.0.0-20200428022330-06a60b6afbbc/go.mod h1:xbL0rPBG9cCiLr28tMa8zpbdarY27NDyej4t/EjAShU=
|
||||
github.com/dgrijalva/jwt-go v3.2.0+incompatible h1:7qlOGliEKZXTDg6OTjfoBKDXWrumCAMpl/TFQ4/5kLM=
|
||||
github.com/dgrijalva/jwt-go v3.2.0+incompatible/go.mod h1:E3ru+11k8xSBh+hMPgOLZmtrrCbhqsmaPHjLKYnJCaQ=
|
||||
github.com/dgrijalva/jwt-go/v4 v4.0.0-preview1 h1:CaO/zOnF8VvUfEbhRatPcwKVWamvbYd8tQGRWacE9kU=
|
||||
|
@ -138,6 +171,8 @@ github.com/dgrijalva/jwt-go/v4 v4.0.0-preview1/go.mod h1:+hnT3ywWDTAFrW5aE+u2Sa/
|
|||
github.com/dgryski/go-bitstream v0.0.0-20180413035011-3522498ce2c8 h1:akOQj8IVgoeFfBTzGOEQakCYshWD6RNo1M5pivFXt70=
|
||||
github.com/dgryski/go-bitstream v0.0.0-20180413035011-3522498ce2c8/go.mod h1:VMaSuZ+SZcx/wljOQKvp5srsbCiKDEb6K2wC4+PiBmQ=
|
||||
github.com/dgryski/go-sip13 v0.0.0-20190329191031-25c5027a8c7b/go.mod h1:vAd38F8PWV+bWy6jNmig1y/TA+kYO4g3RSRF0IAv0no=
|
||||
github.com/dimchansky/utfbom v1.1.0 h1:FcM3g+nofKgUteL8dm/UpdRXNC9KmADgTpLKsu0TRo4=
|
||||
github.com/dimchansky/utfbom v1.1.0/go.mod h1:rO41eb7gLfo8SF1jd9F8HplJm1Fewwi4mQvIirEdv+8=
|
||||
github.com/docker/go-units v0.3.3/go.mod h1:fgPhTUdO+D/Jk86RDLlptpiXQzgHJF7gydDDbaIK4Dk=
|
||||
github.com/docker/go-units v0.4.0/go.mod h1:fgPhTUdO+D/Jk86RDLlptpiXQzgHJF7gydDDbaIK4Dk=
|
||||
github.com/docker/spdystream v0.0.0-20160310174837-449fdfce4d96/go.mod h1:Qh8CwZgvJUkLughtfhJv5dyTYa91l1fOUCrgjqmcifM=
|
||||
|
@ -160,6 +195,8 @@ github.com/evanphx/json-patch v4.2.0+incompatible/go.mod h1:50XU6AFN0ol/bzJsmQLi
|
|||
github.com/fatih/color v1.7.0/go.mod h1:Zm6kSWBoL9eyXnKyktHP6abPY2pDugNf5KwzbycvMj4=
|
||||
github.com/fatih/color v1.9.0/go.mod h1:eQcE1qtQxscV5RaZvpXrrb8Drkc3/DdQ+uUYCNjL+zU=
|
||||
github.com/fogleman/gg v1.2.1-0.20190220221249-0403632d5b90/go.mod h1:R/bRT+9gY/C5z7JzPU0zXsXHKM4/ayA+zqcVNZzPa1k=
|
||||
github.com/foxcpp/go-mockdns v0.0.0-20201212160233-ede2f9158d15 h1:nLPjjvpUAODOR6vY/7o0hBIk8iTr19Fvmf8aFx/kC7A=
|
||||
github.com/foxcpp/go-mockdns v0.0.0-20201212160233-ede2f9158d15/go.mod h1:tPg4cp4nseejPd+UKxtCVQ2hUxNTZ7qQZJa7CLriIeo=
|
||||
github.com/franela/goblin v0.0.0-20200105215937-c9ffbefa60db/go.mod h1:7dvUGVsVBjqR7JHJk0brhHOZYGmfBYOrK0ZhYMEtBr4=
|
||||
github.com/franela/goreq v0.0.0-20171204163338-bcd34c9993f8/go.mod h1:ZhphrRTfi2rbfLwlschooIH4+wKKDR4Pdxhh+TRoA20=
|
||||
github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo=
|
||||
|
@ -268,6 +305,8 @@ github.com/gobuffalo/packd v0.1.0/go.mod h1:M2Juc+hhDXf/PnmBANFCqx4DM3wRbgDvnVWe
|
|||
github.com/gobuffalo/packr/v2 v2.0.9/go.mod h1:emmyGweYTm6Kdper+iywB6YK5YzuKchGtJQZ0Odn4pQ=
|
||||
github.com/gobuffalo/packr/v2 v2.2.0/go.mod h1:CaAwI0GPIAv+5wKLtv8Afwl+Cm78K/I/VCm/3ptBN+0=
|
||||
github.com/gobuffalo/syncx v0.0.0-20190224160051-33c29581e754/go.mod h1:HhnNqWY95UYwwW3uSASeV7vtgYkT2t16hJgV3AEPUpw=
|
||||
github.com/gofrs/uuid v3.3.0+incompatible h1:8K4tyRfvU1CYPgJsveYFQMhpFd/wXNM7iK6rR7UHz84=
|
||||
github.com/gofrs/uuid v3.3.0+incompatible/go.mod h1:b2aQJv3Z4Fp6yNu3cdSllBxTCLRxnplIgP/c0N/04lM=
|
||||
github.com/gogo/googleapis v1.1.0/go.mod h1:gf4bu3Q80BeJ6H1S1vYPm8/ELATdvryBaNFGgqEef3s=
|
||||
github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ=
|
||||
github.com/gogo/protobuf v1.2.0/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ=
|
||||
|
@ -276,9 +315,12 @@ github.com/gogo/protobuf v1.2.2-0.20190723190241-65acae22fc9d/go.mod h1:SlYgWuQ5
|
|||
github.com/gogo/protobuf v1.2.2-0.20190730201129-28a6bbf47e48/go.mod h1:SlYgWuQ5SjCEi6WLHjHCa1yvBfUnHcTbrrZtXPKa29o=
|
||||
github.com/gogo/protobuf v1.3.1 h1:DqDEcV5aeaTmdFBePNpYsp3FlcVH/2ISVVM9Qf8PSls=
|
||||
github.com/gogo/protobuf v1.3.1/go.mod h1:SlYgWuQ5SjCEi6WLHjHCa1yvBfUnHcTbrrZtXPKa29o=
|
||||
github.com/golang-sql/civil v0.0.0-20190719163853-cb61b32ac6fe h1:lXe2qZdvpiX5WZkZR4hgp4KJVfY3nMkvmwbVkpv1rVY=
|
||||
github.com/golang-sql/civil v0.0.0-20190719163853-cb61b32ac6fe/go.mod h1:8vg3r2VgvsThLBIFL93Qb5yWzgyZWhEmBwUJWevAkK0=
|
||||
github.com/golang/freetype v0.0.0-20170609003504-e2365dfdc4a0/go.mod h1:E/TSTwGwJL78qG/PmXZO1EjYhfJinVAhrmmHX6Z8B9k=
|
||||
github.com/golang/geo v0.0.0-20190916061304-5b978397cfec h1:lJwO/92dFXWeXOZdoGXgptLmNLwynMSHUmU6besqtiw=
|
||||
github.com/golang/geo v0.0.0-20190916061304-5b978397cfec/go.mod h1:QZ0nwyI2jOfgRAoBvP+ab5aRr7c9x7lhGEJrKvBwjWI=
|
||||
github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b h1:VKtxabqXZkF25pY9ekfRL6a582T4P37/31XEstQ5p58=
|
||||
github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q=
|
||||
github.com/golang/groupcache v0.0.0-20160516000752-02826c3e7903/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc=
|
||||
github.com/golang/groupcache v0.0.0-20190702054246-869f871628b6/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc=
|
||||
|
@ -327,6 +369,7 @@ github.com/google/pprof v0.0.0-20200229191704-1ebb73c60ed3/go.mod h1:ZgVRPoUq/hf
|
|||
github.com/google/pprof v0.0.0-20200417002340-c6e0a841f49a/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM=
|
||||
github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI=
|
||||
github.com/google/uuid v1.0.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo=
|
||||
github.com/google/uuid v1.1.1 h1:Gkbcsh/GbpXz7lPftLA3P6TYMwjCLYm83jiFQZF/3gY=
|
||||
github.com/google/uuid v1.1.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo=
|
||||
github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg=
|
||||
github.com/googleapis/gax-go/v2 v2.0.5 h1:sjZBwGj9Jlw33ImPtvFviGYvseOtDM7hkSKB7+Tv3SM=
|
||||
|
@ -388,15 +431,18 @@ github.com/imdario/mergo v0.3.5/go.mod h1:2EnlNZ0deacrJVfApfmtdGgDfMuh/nq6Ok1EcJ
|
|||
github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8=
|
||||
github.com/influxdata/flux v0.65.0 h1:57tk1Oo4gpGIDbV12vUAPCMtLtThhaXzub1XRIuqv6A=
|
||||
github.com/influxdata/flux v0.65.0/go.mod h1:BwN2XG2lMszOoquQaFdPET8FRQfrXiZsWmcMO9rkaVY=
|
||||
github.com/influxdata/flux v0.105.1 h1:OnSjI/KZ80+rU0tlFmpm8eTfxzdx1Dt/+HAdh0/7Lxo=
|
||||
github.com/influxdata/flux v0.105.1/go.mod h1:QEVEEaLEVtLXJ9YQzvVLVMoCmElUyqkFYSjfy1BEKiE=
|
||||
github.com/influxdata/influxdb v1.8.0/go.mod h1:SIzcnsjaHRFpmlxpJ4S3NT64qtEKYweNTUMb/vh0OMQ=
|
||||
github.com/influxdata/influxdb1-client v0.0.0-20191209144304-8bf82d3c094d/go.mod h1:qj24IKcXYK6Iy9ceXlo3Tc+vtHo9lIhSX5JddghvEPo=
|
||||
github.com/influxdata/influxql v1.1.0/go.mod h1:KpVI7okXjK6PRi3Z5B+mtKZli+R1DnZgb3N+tzevNgo=
|
||||
github.com/influxdata/influxql v1.1.1-0.20200828144457-65d3ef77d385 h1:ED4e5Cc3z5vSN2Tz2GkOHN7vs4Sxe2yds6CXvDnvZFE=
|
||||
github.com/influxdata/influxql v1.1.1-0.20200828144457-65d3ef77d385/go.mod h1:gHp9y86a/pxhjJ+zMjNXiQAA197Xk9wLxaz+fGG+kWk=
|
||||
github.com/influxdata/influxql v1.1.1-0.20210223160523-b6ab99450c93 h1:4t/8PcmLnI2vrcaHcEKeeLsGxC0WMRaOQdPX9b7DF8Y=
|
||||
github.com/influxdata/influxql v1.1.1-0.20210223160523-b6ab99450c93/go.mod h1:gHp9y86a/pxhjJ+zMjNXiQAA197Xk9wLxaz+fGG+kWk=
|
||||
github.com/influxdata/line-protocol v0.0.0-20180522152040-32c6aa80de5e h1:/o3vQtpWJhvnIbXley4/jwzzqNeigJK9z+LZcJZ9zfM=
|
||||
github.com/influxdata/line-protocol v0.0.0-20180522152040-32c6aa80de5e/go.mod h1:4kt73NQhadE3daL3WhR5EJ/J2ocX0PZzwxQ0gXJ7oFE=
|
||||
github.com/influxdata/pkg-config v0.2.5/go.mod h1:EMS7Ll0S4qkzDk53XS3Z72/egBsPInt+BeRxb0WeSwk=
|
||||
github.com/influxdata/pkg-config v0.2.6 h1:GQFKw3m3OmmPMze9n75ZVVtNu4LJ2MJolHbxvg4AAvg=
|
||||
github.com/influxdata/pkg-config v0.2.6/go.mod h1:EMS7Ll0S4qkzDk53XS3Z72/egBsPInt+BeRxb0WeSwk=
|
||||
github.com/influxdata/promql/v2 v2.12.0/go.mod h1:fxOPu+DY0bqCTCECchSRtWfc+0X19ybifQhZoQNF5D8=
|
||||
github.com/influxdata/roaring v0.4.13-0.20180809181101-fc520f41fab6 h1:UzJnB7VRL4PSkUJHwsyzseGOmrO/r4yA+AuxGJxiZmA=
|
||||
github.com/influxdata/roaring v0.4.13-0.20180809181101-fc520f41fab6/go.mod h1:bSgUQ7q5ZLSO+bKBGqJiCBGAl+9DxyW63zLTujjUlOE=
|
||||
|
@ -406,6 +452,7 @@ github.com/influxdata/usage-client v0.0.0-20160829180054-6d3895376368 h1:+TUUmaF
|
|||
github.com/influxdata/usage-client v0.0.0-20160829180054-6d3895376368/go.mod h1:Wbbw6tYNvwa5dlB6304Sd+82Z3f7PmVZHVKU637d4po=
|
||||
github.com/jessevdk/go-flags v1.4.0/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJSGM75WSRxI=
|
||||
github.com/jmespath/go-jmespath v0.0.0-20180206201540-c2b33e8439af/go.mod h1:Nht3zPeWKUH0NzdCt2Blrr5ys8VGpn0CEB0cQHVjt7k=
|
||||
github.com/jmespath/go-jmespath v0.3.0 h1:OS12ieG61fsCg5+qLJ+SsW9NicxNkg3b25OyT2yCeUc=
|
||||
github.com/jmespath/go-jmespath v0.3.0/go.mod h1:9QtRXoHjLGCJ5IBSaohpXITPlowMeeYCZ7fLUTSywik=
|
||||
github.com/joho/godotenv v1.3.0/go.mod h1:7hK45KPybAkOC6peb+G5yklZfMxEjkZhHbwpqxOKXbg=
|
||||
github.com/jonboulle/clockwork v0.1.0/go.mod h1:Ii8DK3G1RaLaWxj9trq07+26W01tbo22gdxWY5EU2bo=
|
||||
|
@ -486,10 +533,13 @@ github.com/mattn/go-tty v0.0.0-20180907095812-13ff1204f104/go.mod h1:XPvLUNfbS4f
|
|||
github.com/matttproud/golang_protobuf_extensions v1.0.1 h1:4hp9jkHxhMHkqkrB3Ix0jegS5sx/RkqARlsWZ6pIwiU=
|
||||
github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0=
|
||||
github.com/miekg/dns v1.0.14/go.mod h1:W1PPwlIAgtquWBMBEV9nkV9Cazfe8ScdGz/Lj7v3Nrg=
|
||||
github.com/miekg/dns v1.1.22/go.mod h1:bPDLeHnStXmXAq1m/Ch/hvfNHr14JKNPMBo3VZKjuso=
|
||||
github.com/miekg/dns v1.1.26/go.mod h1:bPDLeHnStXmXAq1m/Ch/hvfNHr14JKNPMBo3VZKjuso=
|
||||
github.com/miekg/dns v1.1.29 h1:xHBEhR+t5RzcFJjBLJlax2daXOrTYtr9z4WdKEfWFzg=
|
||||
github.com/miekg/dns v1.1.29/go.mod h1:KNUDUusw/aVsxyTYZM1oqvCicbwhgbNgztCETuNZ7xM=
|
||||
github.com/mitchellh/cli v1.0.0/go.mod h1:hNIlj7HEI86fIcpObd7a0FcrxTWetlwJDGcceTlRvqc=
|
||||
github.com/mitchellh/go-homedir v1.0.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0=
|
||||
github.com/mitchellh/go-homedir v1.1.0 h1:lukF9ziXFxDFPkA1vsr5zpc1XuPDn/wFntq5mG+4E0Y=
|
||||
github.com/mitchellh/go-homedir v1.1.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0=
|
||||
github.com/mitchellh/go-testing-interface v1.0.0/go.mod h1:kRemZodwjscx+RGhAo8eIhFbs2+BFgRtFPeD/KE+zxI=
|
||||
github.com/mitchellh/go-wordwrap v1.0.0/go.mod h1:ZXFpozHsX6DPmq2I0TCekCxypsnAUbP2oI0UX1GXzOo=
|
||||
|
@ -556,6 +606,8 @@ github.com/philhofer/fwd v1.0.0/go.mod h1:gk3iGcWd9+svBvR0sR+KPcfE+RNWozjowpeBVG
|
|||
github.com/pierrec/lz4 v1.0.2-0.20190131084431-473cd7ce01a1/go.mod h1:3/3N9NVKO0jef7pBehbT1qWhCMrIgbYNnFAZCqQ5LRc=
|
||||
github.com/pierrec/lz4 v2.0.5+incompatible h1:2xWsjqPFWcplujydGg4WmhC/6fZqK42wMM8aXeqhl0I=
|
||||
github.com/pierrec/lz4 v2.0.5+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY=
|
||||
github.com/pkg/browser v0.0.0-20180916011732-0a3d74bf9ce4 h1:49lOXmGaUpV9Fz3gd7TFZY106KVlPVa5jcYD1gaQf98=
|
||||
github.com/pkg/browser v0.0.0-20180916011732-0a3d74bf9ce4/go.mod h1:4OwLy04Bl9Ef3GJJCoec+30X3LQs/0/m4HFRt/2LUSA=
|
||||
github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0=
|
||||
github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0=
|
||||
github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4=
|
||||
|
@ -611,7 +663,6 @@ github.com/ryanuber/columnize v0.0.0-20160712163229-9b3edd62028f/go.mod h1:sm1tb
|
|||
github.com/ryanuber/columnize v2.1.0+incompatible/go.mod h1:sm1tb6uqfes/u+d4ooFouqFdy9/2g9QGwK3SQygK0Ts=
|
||||
github.com/samuel/go-zookeeper v0.0.0-20190923202752-2cc03de413da/go.mod h1:gi+0XIa01GRL2eRQVjQkKGqKF3SF9vZR/HnPullcV2E=
|
||||
github.com/satori/go.uuid v0.0.0-20160603004225-b111a074d5ef/go.mod h1:dA0hQrYB0VpLJoorglMZABFdXlWrHn1NEOzdhQKdks0=
|
||||
github.com/satori/go.uuid v1.2.1-0.20181028125025-b2ce2384e17b h1:gQZ0qzfKHQIybLANtM3mBXNUtOfsCFXeTsnBqCsx1KM=
|
||||
github.com/satori/go.uuid v1.2.1-0.20181028125025-b2ce2384e17b/go.mod h1:dA0hQrYB0VpLJoorglMZABFdXlWrHn1NEOzdhQKdks0=
|
||||
github.com/sean-/seed v0.0.0-20170313163322-e2103e2c3529/go.mod h1:DxrIzT+xaE7yg65j358z/aeFdxmN0P9QXhEzd20vsDc=
|
||||
github.com/segmentio/kafka-go v0.1.0/go.mod h1:X6itGqS9L4jDletMsxZ7Dz+JFWxM6JHfPOCvTvk+EJo=
|
||||
|
@ -630,6 +681,8 @@ github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d h1:zE9ykE
|
|||
github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc=
|
||||
github.com/smartystreets/goconvey v1.6.4 h1:fv0U8FUIMPNf1L9lnHLvLhgicrIVChEkdzIKYqbNC9s=
|
||||
github.com/smartystreets/goconvey v1.6.4/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA=
|
||||
github.com/snowflakedb/gosnowflake v1.3.4 h1:Gyoi6g4lMHsilEwW9+KV+bgYkJTgf5pVfvL7Utus920=
|
||||
github.com/snowflakedb/gosnowflake v1.3.4/go.mod h1:NsRq2QeiMUuoNUJhp5Q6xGC4uBrsS9g6LwZVEkTWgsE=
|
||||
github.com/soheilhy/cmux v0.1.4/go.mod h1:IM3LyeVVIOuxMH7sFAkER9+bJ4dT7Ms6E4xg4kGIyLM=
|
||||
github.com/sony/gobreaker v0.4.1/go.mod h1:ZKptC7FHNvhBz7dN2LGjPVBz2sZJmc0/PkyDJOjmxWY=
|
||||
github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72 h1:qLC7fQah7D6K1B0ujays3HV9gkFtllcxhzImRR7ArPQ=
|
||||
|
@ -641,6 +694,7 @@ github.com/spf13/cobra v0.0.3/go.mod h1:1l0Ry5zgKvJasoi3XT1TypsSe7PqH0Sj9dhYf7v3
|
|||
github.com/spf13/pflag v0.0.0-20170130214245-9ff6c6923cff/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4=
|
||||
github.com/spf13/pflag v1.0.1/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4=
|
||||
github.com/spf13/pflag v1.0.3/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4=
|
||||
github.com/spf13/pflag v1.0.5 h1:iy+VFUOCP1a+8yFto/drg2CJ5u0yRoB7fZw3DKv/JXA=
|
||||
github.com/spf13/pflag v1.0.5/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An2Bg=
|
||||
github.com/streadway/amqp v0.0.0-20190404075320-75d898a42a94/go.mod h1:AZpEONHx3DKn8O/DFsRAY58/XVQiIPMTMB1SddzLXVw=
|
||||
github.com/streadway/amqp v0.0.0-20190827072141-edfb9018d271/go.mod h1:AZpEONHx3DKn8O/DFsRAY58/XVQiIPMTMB1SddzLXVw=
|
||||
|
@ -660,6 +714,10 @@ github.com/tinylib/msgp v1.1.0 h1:9fQd+ICuRIu/ue4vxJZu6/LzxN0HwMds2nq/0cFvxHU=
|
|||
github.com/tinylib/msgp v1.1.0/go.mod h1:+d+yLhGm8mzTaHzB+wgMYrodPfmZrzkirds8fDWklFE=
|
||||
github.com/tmc/grpc-websocket-proxy v0.0.0-20170815181823-89b8d40f7ca8/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U=
|
||||
github.com/tv42/httpunix v0.0.0-20150427012821-b75d8614f926/go.mod h1:9ESjWnEqriFuLhtthL60Sar/7RFoluCcXsuvEwTV5KM=
|
||||
github.com/uber-go/tally v3.3.15+incompatible h1:9hLSgNBP28CjIaDmAuRTq9qV+UZY+9PcvAkXO4nNMwg=
|
||||
github.com/uber-go/tally v3.3.15+incompatible/go.mod h1:YDTIBxdXyOU/sCWilKB4bgyufu1cEi0jdVnRdxvjnmU=
|
||||
github.com/uber/athenadriver v1.1.4 h1:k6k0RBeXjR7oZ8NO557MsRw3eX1cc/9B0GNx+W9eHiQ=
|
||||
github.com/uber/athenadriver v1.1.4/go.mod h1:tQjho4NzXw55LGfSZEcETuYydpY1vtmixUabHkC1K/E=
|
||||
github.com/uber/jaeger-client-go v2.23.0+incompatible/go.mod h1:WVhlPFC8FDjOFMMWRy2pZqQJSXxYSwNYOkTr/Z6d3Kk=
|
||||
github.com/uber/jaeger-lib v2.2.0+incompatible/go.mod h1:ComeNDZlWwrWnDv8aPp0Ba6+uUTzImX/AauajbLI56U=
|
||||
github.com/urfave/cli v1.20.0/go.mod h1:70zkFmudgCuE/ngEzBv17Jvp/497gISqfk5gWijbERA=
|
||||
|
@ -675,6 +733,7 @@ github.com/xlab/treeprint v0.0.0-20180616005107-d6fb6747feb6 h1:YdYsPAZ2pC6Tow/n
|
|||
github.com/xlab/treeprint v0.0.0-20180616005107-d6fb6747feb6/go.mod h1:ce1O1j6UtZfjr22oyGxGLbauSBp2YVXpARAosm7dHBg=
|
||||
github.com/yuin/goldmark v1.1.25/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74=
|
||||
github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74=
|
||||
github.com/yuin/goldmark v1.1.32/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74=
|
||||
go.etcd.io/bbolt v1.3.3/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU=
|
||||
go.etcd.io/etcd v0.0.0-20191023171146-3cf2f69b5738/go.mod h1:dnLIgRNXwCJa5e+c6mIZCrds/GIG4ncV9HhK5PX7jPg=
|
||||
go.mongodb.org/mongo-driver v1.0.3/go.mod h1:u7ryQJ+DOzQmeO7zB6MHyr8jkEQvC8vH7qLUO4lqsUM=
|
||||
|
@ -691,10 +750,12 @@ go.opencensus.io v0.22.3 h1:8sGtKOrtQqkN1bp2AtX+misvLIlOmsEsNd+9NIcPEm8=
|
|||
go.opencensus.io v0.22.3/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw=
|
||||
go.uber.org/atomic v1.3.2/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE=
|
||||
go.uber.org/atomic v1.5.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ=
|
||||
go.uber.org/atomic v1.5.1/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ=
|
||||
go.uber.org/atomic v1.6.0 h1:Ezj3JGmsOnG1MoRWQkPBsKLe9DwWD9QeXzTRzzldNVk=
|
||||
go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ=
|
||||
go.uber.org/multierr v1.1.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/0=
|
||||
go.uber.org/multierr v1.3.0/go.mod h1:VgVr7evmIr6uPjLBxg28wmKNXyqE9akIJ5XnfpiKl+4=
|
||||
go.uber.org/multierr v1.4.0/go.mod h1:VgVr7evmIr6uPjLBxg28wmKNXyqE9akIJ5XnfpiKl+4=
|
||||
go.uber.org/multierr v1.5.0 h1:KCa4XfM8CWFCpxXRGok+Q0SS/0XBhMDbHHGABQLvD2A=
|
||||
go.uber.org/multierr v1.5.0/go.mod h1:FeouvMocqHpRaaGuG9EjoKcStLC43Zu/fmqdUMPcKYU=
|
||||
go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee h1:0mgffUl7nfd+FpvXMVz4IDEaUSmT1ysygQC7qYo7sG4=
|
||||
|
@ -702,6 +763,7 @@ go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee/go.mod h1:vJERXedbb3MVM5f9E
|
|||
go.uber.org/zap v1.9.1/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q=
|
||||
go.uber.org/zap v1.10.0/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q=
|
||||
go.uber.org/zap v1.13.0/go.mod h1:zwrFLgMcdUuIBviXEYEH1YKNaOBnKXsx2IPda5bBwHM=
|
||||
go.uber.org/zap v1.14.0/go.mod h1:zwrFLgMcdUuIBviXEYEH1YKNaOBnKXsx2IPda5bBwHM=
|
||||
go.uber.org/zap v1.14.1 h1:nYDKopTbvAPq/NrUVZwT15y2lpROBiLLyoRTbXOYWOo=
|
||||
go.uber.org/zap v1.14.1/go.mod h1:Mb2vm2krFEG5DV0W9qcHBYFtp/Wku1cvYaqPsS/WYfc=
|
||||
golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4=
|
||||
|
@ -709,6 +771,7 @@ golang.org/x/crypto v0.0.0-20181029021203-45a5f77698d3/go.mod h1:6SG95UA2DQfeDnf
|
|||
golang.org/x/crypto v0.0.0-20190211182817-74369b46fc67/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4=
|
||||
golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w=
|
||||
golang.org/x/crypto v0.0.0-20190320223903-b7391e95e576/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w=
|
||||
golang.org/x/crypto v0.0.0-20190325154230-a5d413f7728c/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w=
|
||||
golang.org/x/crypto v0.0.0-20190422162423-af44ce270edf/go.mod h1:WFFai1msRO1wXaEeE5yQxYXgSfI8pQAWXbQop6sCtWE=
|
||||
golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI=
|
||||
golang.org/x/crypto v0.0.0-20190530122614-20be4c3c3ed5/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI=
|
||||
|
@ -721,8 +784,9 @@ golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8U
|
|||
golang.org/x/crypto v0.0.0-20191202143827-86a70503ff7e/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto=
|
||||
golang.org/x/crypto v0.0.0-20191206172530-e9b2fee46413/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto=
|
||||
golang.org/x/crypto v0.0.0-20200220183623-bac4c82f6975/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto=
|
||||
golang.org/x/crypto v0.0.0-20200422194213-44a606286825 h1:dSChiwOTvzwbHFTMq2l6uRardHH7/E6SqEkqccinS/o=
|
||||
golang.org/x/crypto v0.0.0-20200422194213-44a606286825/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto=
|
||||
golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9 h1:psW17arqaxU48Z5kZ0CQnkZWQJsqcURM6tKiBApRjXI=
|
||||
golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto=
|
||||
golang.org/x/exp v0.0.0-20180321215751-8460e604b9de/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA=
|
||||
golang.org/x/exp v0.0.0-20180807140117-3d87b88a115f/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA=
|
||||
golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA=
|
||||
|
@ -757,8 +821,9 @@ golang.org/x/mod v0.0.0-20190513183733-4bf6d317e70e/go.mod h1:mXi4GBBbnImb6dmsKG
|
|||
golang.org/x/mod v0.1.0/go.mod h1:0QHyrYULN0/3qlju5TqG8bIK38QM8yzMo5ekMj3DlcY=
|
||||
golang.org/x/mod v0.1.1-0.20191105210325-c90efee705ee/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg=
|
||||
golang.org/x/mod v0.1.1-0.20191107180719-034126e5016b/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg=
|
||||
golang.org/x/mod v0.2.0 h1:KU7oHjnv3XNWfa5COkzUifxZmxp1TyI7ImMXqFxLwvQ=
|
||||
golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA=
|
||||
golang.org/x/mod v0.3.0 h1:RM4zey1++hCTbCVQfnWeKs9/IEsaBLA8vTkd0WVtmH4=
|
||||
golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA=
|
||||
golang.org/x/net v0.0.0-20170114055629-f2499483f923/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
|
||||
golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
|
||||
golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
|
||||
|
@ -792,8 +857,9 @@ golang.org/x/net v0.0.0-20200202094626-16171245cfb2/go.mod h1:z5CRVTTTmAJ677TzLL
|
|||
golang.org/x/net v0.0.0-20200222125558-5a598a2470a0/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
|
||||
golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
|
||||
golang.org/x/net v0.0.0-20200324143707-d3edc9973b7e/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A=
|
||||
golang.org/x/net v0.0.0-20200421231249-e086a090c8fd h1:QPwSajcTUrFriMF1nJ3XzgoqakqQEsnZf9LdXdi2nkI=
|
||||
golang.org/x/net v0.0.0-20200421231249-e086a090c8fd/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A=
|
||||
golang.org/x/net v0.0.0-20200625001655-4c5254603344 h1:vGXIOMxbNfDTk/aXCmfdLgkrSV+Z2tcbze+pEc3v5W4=
|
||||
golang.org/x/net v0.0.0-20200625001655-4c5254603344/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA=
|
||||
golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U=
|
||||
golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw=
|
||||
golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw=
|
||||
|
@ -807,8 +873,9 @@ golang.org/x/sync v0.0.0-20190227155943-e225da77a7e6/go.mod h1:RxMgew5VJxzue5/jJ
|
|||
golang.org/x/sync v0.0.0-20190412183630-56d357773e84/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
|
||||
golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
|
||||
golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
|
||||
golang.org/x/sync v0.0.0-20200317015054-43a5402ce75a h1:WXEvlFVvvGxCJLG6REjsT03iWnKLEWinaScsxF2Vm2o=
|
||||
golang.org/x/sync v0.0.0-20200317015054-43a5402ce75a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
|
||||
golang.org/x/sync v0.0.0-20200625203802-6e8e738ad208 h1:qwRHBd0NqMbJxfbotnDhm2ByMI1Shq4Y6oRJo21SGJA=
|
||||
golang.org/x/sync v0.0.0-20200625203802-6e8e738ad208/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
|
||||
golang.org/x/sys v0.0.0-20170830134202-bb24a47a89ea/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
|
||||
golang.org/x/sys v0.0.0-20180823144017-11551d06cbcc/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
|
||||
golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
|
||||
|
@ -921,9 +988,11 @@ golang.org/x/tools v0.0.0-20200204074204-1cc6d1ef6c74/go.mod h1:TB2adYChydJhpapK
|
|||
golang.org/x/tools v0.0.0-20200207183749-b753a1ba74fa/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28=
|
||||
golang.org/x/tools v0.0.0-20200212150539-ea181f53ac56/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28=
|
||||
golang.org/x/tools v0.0.0-20200224181240-023911ca70b2/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28=
|
||||
golang.org/x/tools v0.0.0-20200304024140-c4206d458c3f/go.mod h1:o4KQGtdN14AW+yjsvvwRTJJuXz8XRtIHtEnmAXLyFUw=
|
||||
golang.org/x/tools v0.0.0-20200331025713-a30bf2db82d4/go.mod h1:Sl4aGygMT6LrqrWclx+PTx3U+LnKx/seiNR+3G19Ar8=
|
||||
golang.org/x/tools v0.0.0-20200422205258-72e4a01eba43 h1:Lcsc5ErIWemp8qAbYffG5vPrqjJ0zk82RTFGifeS1Pc=
|
||||
golang.org/x/tools v0.0.0-20200422205258-72e4a01eba43/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE=
|
||||
golang.org/x/tools v0.0.0-20200721032237-77f530d86f9a h1:kVMPw4f6EVqYdfGQTedjrpw1dbE2PEMfw4jwXsNdn9s=
|
||||
golang.org/x/tools v0.0.0-20200721032237-77f530d86f9a/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA=
|
||||
golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=
|
||||
golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=
|
||||
golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543 h1:E7g+9GITq07hpfrRu66IVDexMakfv52eLZ2CXBWiKr4=
|
||||
|
|
|
@ -5,6 +5,7 @@ import (
|
|||
|
||||
"github.com/influxdata/flux"
|
||||
"github.com/influxdata/flux/memory"
|
||||
"github.com/influxdata/flux/runtime"
|
||||
"github.com/prometheus/client_golang/prometheus"
|
||||
)
|
||||
|
||||
|
@ -15,7 +16,7 @@ type FluxControllerMock struct {
|
|||
func NewFluxControllerMock() *FluxControllerMock {
|
||||
return &FluxControllerMock{
|
||||
QueryFn: func(ctx context.Context, compiler flux.Compiler) (query flux.Query, e error) {
|
||||
p, err := compiler.Compile(ctx)
|
||||
p, err := compiler.Compile(ctx, runtime.Default)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
|
|
@ -56,6 +56,10 @@ var (
|
|||
|
||||
// ErrInvalidPoint is returned when a point cannot be parsed correctly.
|
||||
ErrInvalidPoint = errors.New("point is invalid")
|
||||
|
||||
// ErrInvalidKevValuePairs is returned when the number of key, value pairs
|
||||
// is odd, indicating a missing value.
|
||||
ErrInvalidKevValuePairs = errors.New("key/value pairs is an odd length")
|
||||
)
|
||||
|
||||
const (
|
||||
|
@ -1953,6 +1957,53 @@ func NewTags(m map[string]string) Tags {
|
|||
return a
|
||||
}
|
||||
|
||||
// NewTagsKeyValues returns a new Tags from a list of key, value pairs,
|
||||
// ensuring the returned result is correctly sorted. Duplicate keys are removed,
|
||||
// however, it which duplicate that remains is undefined.
|
||||
// NewTagsKeyValues will return ErrInvalidKevValuePairs if len(kvs) is not even.
|
||||
// If the input is guaranteed to be even, the error can be safely ignored.
|
||||
// If a has enough capacity, it will be reused.
|
||||
func NewTagsKeyValues(a Tags, kv ...[]byte) (Tags, error) {
|
||||
if len(kv)%2 == 1 {
|
||||
return nil, ErrInvalidKevValuePairs
|
||||
}
|
||||
if len(kv) == 0 {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
l := len(kv) / 2
|
||||
if cap(a) < l {
|
||||
a = make(Tags, 0, l)
|
||||
} else {
|
||||
a = a[:0]
|
||||
}
|
||||
|
||||
for i := 0; i < len(kv)-1; i += 2 {
|
||||
a = append(a, NewTag(kv[i], kv[i+1]))
|
||||
}
|
||||
|
||||
if !a.sorted() {
|
||||
sort.Sort(a)
|
||||
}
|
||||
|
||||
// remove duplicates
|
||||
j := 0
|
||||
for i := 0; i < len(a)-1; i++ {
|
||||
if !bytes.Equal(a[i].Key, a[i+1].Key) {
|
||||
if j != i {
|
||||
// only copy if j has deviated from i, indicating duplicates
|
||||
a[j] = a[i]
|
||||
}
|
||||
j++
|
||||
}
|
||||
}
|
||||
|
||||
a[j] = a[len(a)-1]
|
||||
j++
|
||||
|
||||
return a[:j], nil
|
||||
}
|
||||
|
||||
// Keys returns the list of keys for a tag set.
|
||||
func (a Tags) Keys() []string {
|
||||
if len(a) == 0 {
|
||||
|
@ -2019,6 +2070,18 @@ func (a Tags) Clone() Tags {
|
|||
return others
|
||||
}
|
||||
|
||||
// sorted returns true if a is sorted and is an optimization
|
||||
// to avoid an allocation when calling sort.IsSorted, improving
|
||||
// performance as much as 50%.
|
||||
func (a Tags) sorted() bool {
|
||||
for i := len(a) - 1; i > 0; i-- {
|
||||
if bytes.Compare(a[i].Key, a[i-1].Key) == -1 {
|
||||
return false
|
||||
}
|
||||
}
|
||||
return true
|
||||
}
|
||||
|
||||
func (a Tags) Len() int { return len(a) }
|
||||
func (a Tags) Less(i, j int) bool { return bytes.Compare(a[i].Key, a[j].Key) == -1 }
|
||||
func (a Tags) Swap(i, j int) { a[i], a[j] = a[j], a[i] }
|
||||
|
|
|
@ -0,0 +1,14 @@
|
|||
#!/usr/bin/env bash
|
||||
|
||||
tmpdir=$(mktemp -d)
|
||||
trap "{ rm -rf ${tmpdir}; }" EXIT
|
||||
|
||||
# "go build" can be noisy, and when Go invokes pkg-config (by calling this script) it will merge stdout and stderr.
|
||||
# Discard any output unless "go build" terminates with an error.
|
||||
go build -o ${tmpdir}/pkg-config github.com/influxdata/pkg-config &> ${tmpdir}/go_build_output
|
||||
if [ "$?" -ne 0 ]; then
|
||||
cat ${tmpdir}/go_build_output 1>&2
|
||||
exit 1
|
||||
fi
|
||||
|
||||
${tmpdir}/pkg-config "$@"
|
|
@ -0,0 +1,120 @@
|
|||
package query
|
||||
|
||||
import (
|
||||
"io"
|
||||
"net/http"
|
||||
|
||||
"github.com/influxdata/flux"
|
||||
"github.com/influxdata/flux/csv"
|
||||
)
|
||||
|
||||
const (
|
||||
NoContentDialectType = "no-content"
|
||||
NoContentWErrDialectType = "no-content-with-error"
|
||||
)
|
||||
|
||||
// NoContentDialect is a dialect that provides an Encoder that discards query results.
|
||||
// When invoking `dialect.Encoder().Encode(writer, results)`, `results` get consumed,
|
||||
// while the `writer` is left intact.
|
||||
// It is an HTTPDialect that sets the response status code to 204 NoContent.
|
||||
type NoContentDialect struct{}
|
||||
|
||||
func NewNoContentDialect() *NoContentDialect {
|
||||
return &NoContentDialect{}
|
||||
}
|
||||
|
||||
func (d *NoContentDialect) Encoder() flux.MultiResultEncoder {
|
||||
return &NoContentEncoder{}
|
||||
}
|
||||
|
||||
func (d *NoContentDialect) DialectType() flux.DialectType {
|
||||
return NoContentDialectType
|
||||
}
|
||||
|
||||
func (d *NoContentDialect) SetHeaders(w http.ResponseWriter) {
|
||||
w.WriteHeader(http.StatusNoContent)
|
||||
}
|
||||
|
||||
type NoContentEncoder struct{}
|
||||
|
||||
func (e *NoContentEncoder) Encode(w io.Writer, results flux.ResultIterator) (int64, error) {
|
||||
defer results.Release()
|
||||
// Consume and discard results.
|
||||
for results.More() {
|
||||
if err := results.Next().Tables().Do(func(tbl flux.Table) error {
|
||||
return tbl.Do(func(cr flux.ColReader) error {
|
||||
return nil
|
||||
})
|
||||
}); err != nil {
|
||||
return 0, err
|
||||
}
|
||||
}
|
||||
// Do not write anything.
|
||||
return 0, nil
|
||||
}
|
||||
|
||||
// NoContentWithErrorDialect is a dialect that provides an Encoder that discards query results,
|
||||
// but it encodes runtime errors from the Flux query in CSV format.
|
||||
// To discover if there was any runtime error in the query, one should check the response size.
|
||||
// If it is equal to zero, then no error was present.
|
||||
// Otherwise one can decode the response body to get the error. For example:
|
||||
// ```
|
||||
// _, err = csv.NewResultDecoder(csv.ResultDecoderConfig{}).Decode(bytes.NewReader(res))
|
||||
// if err != nil {
|
||||
// // we got some runtime error
|
||||
// }
|
||||
// ```
|
||||
type NoContentWithErrorDialect struct {
|
||||
csv.ResultEncoderConfig
|
||||
}
|
||||
|
||||
func NewNoContentWithErrorDialect() *NoContentWithErrorDialect {
|
||||
return &NoContentWithErrorDialect{
|
||||
ResultEncoderConfig: csv.DefaultEncoderConfig(),
|
||||
}
|
||||
}
|
||||
|
||||
func (d *NoContentWithErrorDialect) Encoder() flux.MultiResultEncoder {
|
||||
return &NoContentWithErrorEncoder{
|
||||
errorEncoder: csv.NewResultEncoder(d.ResultEncoderConfig),
|
||||
}
|
||||
}
|
||||
|
||||
func (d *NoContentWithErrorDialect) DialectType() flux.DialectType {
|
||||
return NoContentWErrDialectType
|
||||
}
|
||||
|
||||
func (d *NoContentWithErrorDialect) SetHeaders(w http.ResponseWriter) {
|
||||
w.Header().Set("Content-Type", "text/csv; charset=utf-8")
|
||||
w.Header().Set("Transfer-Encoding", "chunked")
|
||||
}
|
||||
|
||||
type NoContentWithErrorEncoder struct {
|
||||
errorEncoder *csv.ResultEncoder
|
||||
}
|
||||
|
||||
func (e *NoContentWithErrorEncoder) Encode(w io.Writer, results flux.ResultIterator) (int64, error) {
|
||||
// Make sure we release results.
|
||||
// Remember, it is safe to call `Release` multiple times.
|
||||
defer results.Release()
|
||||
// Consume and discard results, but keep an eye on errors.
|
||||
for results.More() {
|
||||
if err := results.Next().Tables().Do(func(tbl flux.Table) error {
|
||||
return tbl.Do(func(cr flux.ColReader) error {
|
||||
return nil
|
||||
})
|
||||
}); err != nil {
|
||||
// If there is an error, then encode it in the response.
|
||||
if encErr := e.errorEncoder.EncodeError(w, err); encErr != nil {
|
||||
return 0, encErr
|
||||
}
|
||||
}
|
||||
}
|
||||
// Now Release in order to populate the error, if present.
|
||||
results.Release()
|
||||
err := results.Err()
|
||||
if err != nil {
|
||||
return 0, e.errorEncoder.EncodeError(w, err)
|
||||
}
|
||||
return 0, nil
|
||||
}
|
|
@ -0,0 +1,142 @@
|
|||
package query_test
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"fmt"
|
||||
"testing"
|
||||
|
||||
"github.com/google/go-cmp/cmp"
|
||||
"github.com/influxdata/flux/csv"
|
||||
"github.com/influxdata/influxdb/pkg/testing/assert"
|
||||
"github.com/influxdata/influxdb/query"
|
||||
|
||||
"github.com/influxdata/flux"
|
||||
"github.com/influxdata/flux/execute"
|
||||
"github.com/influxdata/flux/execute/executetest"
|
||||
)
|
||||
|
||||
func TestReturnNoContent(t *testing.T) {
|
||||
getMockResult := func() flux.Result {
|
||||
// Some random data.
|
||||
r := executetest.NewResult([]*executetest.Table{{
|
||||
KeyCols: []string{"t1"},
|
||||
ColMeta: []flux.ColMeta{
|
||||
{Label: "_time", Type: flux.TTime},
|
||||
{Label: "_value", Type: flux.TFloat},
|
||||
{Label: "t1", Type: flux.TString},
|
||||
{Label: "t2", Type: flux.TString},
|
||||
},
|
||||
Data: [][]interface{}{
|
||||
{execute.Time(0), 1.0, "a", "y"},
|
||||
{execute.Time(10), 2.0, "a", "x"},
|
||||
{execute.Time(20), 3.0, "a", "y"},
|
||||
{execute.Time(30), 4.0, "a", "x"},
|
||||
{execute.Time(40), 5.0, "a", "y"},
|
||||
},
|
||||
}})
|
||||
r.Nm = "foo"
|
||||
return r
|
||||
}
|
||||
assertNoContent := func(t *testing.T, respBody []byte, reqErr error) {
|
||||
if reqErr != nil {
|
||||
t.Fatalf("unexpected error on query: %v", reqErr)
|
||||
}
|
||||
if body := string(respBody); len(body) > 0 {
|
||||
t.Fatalf("response body should be empty, but was: %s", body)
|
||||
}
|
||||
}
|
||||
|
||||
testCases := []struct {
|
||||
name string
|
||||
query flux.Query
|
||||
dialect flux.Dialect
|
||||
assertFn func(t *testing.T, respBody []byte, reqErr error)
|
||||
}{
|
||||
{
|
||||
name: "no-content - no error",
|
||||
query: &mockQuery{Result: getMockResult()},
|
||||
dialect: query.NewNoContentDialect(),
|
||||
assertFn: assertNoContent,
|
||||
},
|
||||
{
|
||||
name: "no-content - error",
|
||||
query: &mockQuery{Result: getMockResult(), Error: fmt.Errorf("I am a runtime error")},
|
||||
dialect: query.NewNoContentDialect(),
|
||||
assertFn: assertNoContent,
|
||||
},
|
||||
{
|
||||
name: "no-content-with-error - no error",
|
||||
query: &mockQuery{Result: getMockResult()},
|
||||
dialect: query.NewNoContentWithErrorDialect(),
|
||||
assertFn: assertNoContent,
|
||||
},
|
||||
{
|
||||
name: "no-content-with-error - error",
|
||||
query: &mockQuery{Result: getMockResult(), Error: fmt.Errorf("I am a runtime error")},
|
||||
dialect: query.NewNoContentWithErrorDialect(),
|
||||
assertFn: func(t *testing.T, respBody []byte, reqErr error) {
|
||||
if reqErr != nil {
|
||||
t.Fatalf("unexpected error on query: %v", reqErr)
|
||||
}
|
||||
if len(respBody) == 0 {
|
||||
t.Fatalf("response body should not be empty, but it was")
|
||||
}
|
||||
_, err := csv.NewResultDecoder(csv.ResultDecoderConfig{}).Decode(bytes.NewReader(respBody))
|
||||
if err == nil {
|
||||
t.Fatalf("expected error got none")
|
||||
} else if diff := cmp.Diff(err.Error(), "I am a runtime error"); diff != "" {
|
||||
t.Fatalf("unexpected error, -want/+got:\n\t%s", diff)
|
||||
}
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
for _, tc := range testCases {
|
||||
tc := tc
|
||||
t.Run(tc.name, func(t *testing.T) {
|
||||
results := flux.NewResultIteratorFromQuery(tc.query)
|
||||
defer results.Release()
|
||||
w := bytes.NewBuffer([]byte{})
|
||||
encoder := tc.dialect.Encoder()
|
||||
_, err := encoder.Encode(w, results)
|
||||
assert.NoError(t, err)
|
||||
tc.assertFn(t, w.Bytes(), err)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
type mockQuery struct {
|
||||
Result flux.Result
|
||||
Error error
|
||||
c chan flux.Result
|
||||
}
|
||||
|
||||
func (m *mockQuery) Results() <-chan flux.Result {
|
||||
if m.c != nil {
|
||||
return m.c
|
||||
}
|
||||
m.c = make(chan flux.Result, 1)
|
||||
m.c <- m.Result
|
||||
close(m.c)
|
||||
return m.c
|
||||
}
|
||||
|
||||
func (m *mockQuery) Done() {
|
||||
// no-op
|
||||
}
|
||||
|
||||
func (m *mockQuery) Cancel() {
|
||||
panic("not implemented")
|
||||
}
|
||||
|
||||
func (m *mockQuery) Err() error {
|
||||
return m.Error
|
||||
}
|
||||
|
||||
func (m *mockQuery) Statistics() flux.Statistics {
|
||||
panic("not implemented")
|
||||
}
|
||||
|
||||
func (m *mockQuery) ProfilerResults() (flux.ResultIterator, error) {
|
||||
panic("not implemented")
|
||||
}
|
|
@ -28,7 +28,6 @@ import (
|
|||
"github.com/influxdata/flux"
|
||||
"github.com/influxdata/flux/lang"
|
||||
"github.com/influxdata/flux/mock"
|
||||
"github.com/influxdata/flux/repl"
|
||||
"github.com/influxdata/influxdb/flux/client"
|
||||
"github.com/influxdata/influxdb/internal"
|
||||
"github.com/influxdata/influxdb/logger"
|
||||
|
@ -1294,39 +1293,6 @@ func TestHandler_Flux_QueryJSON(t *testing.T) {
|
|||
}
|
||||
}
|
||||
|
||||
func TestHandler_Flux_REPL(t *testing.T) {
|
||||
h := NewHandlerWithConfig(NewHandlerConfig(WithFlux(), WithNoLog()))
|
||||
called := false
|
||||
h.Controller.QueryFn = func(ctx context.Context, compiler flux.Compiler) (i flux.Query, e error) {
|
||||
if exp := flux.CompilerType(repl.CompilerType); compiler.CompilerType() != exp {
|
||||
t.Fatalf("unexpected compiler type -got/+exp\n%s", cmp.Diff(compiler.CompilerType(), exp))
|
||||
}
|
||||
called = true
|
||||
|
||||
p := &mock.Program{}
|
||||
return p.Start(ctx, nil)
|
||||
}
|
||||
|
||||
q := client.QueryRequest{Spec: &flux.Spec{}}
|
||||
var body bytes.Buffer
|
||||
if err := json.NewEncoder(&body).Encode(q); err != nil {
|
||||
t.Fatalf("unexpected JSON encoding error: %q", err.Error())
|
||||
}
|
||||
|
||||
req := MustNewRequest("POST", "/api/v2/query", &body)
|
||||
req.Header.Add("content-type", "application/json")
|
||||
|
||||
w := httptest.NewRecorder()
|
||||
h.ServeHTTP(w, req)
|
||||
if got := w.Code; !cmp.Equal(got, http.StatusOK) {
|
||||
t.Fatalf("unexpected status: %d", got)
|
||||
}
|
||||
|
||||
if !called {
|
||||
t.Fatalf("expected QueryFn to be called")
|
||||
}
|
||||
}
|
||||
|
||||
func TestHandler_Flux_QueryText(t *testing.T) {
|
||||
h := NewHandlerWithConfig(NewHandlerConfig(WithFlux(), WithNoLog()))
|
||||
called := false
|
||||
|
|
|
@ -52,41 +52,23 @@ func (r *storeReader) ReadGroup(ctx context.Context, spec influxdb.ReadGroupSpec
|
|||
}
|
||||
|
||||
func (r *storeReader) ReadTagKeys(ctx context.Context, spec influxdb.ReadTagKeysSpec, alloc *memory.Allocator) (influxdb.TableIterator, error) {
|
||||
var predicate *datatypes.Predicate
|
||||
if spec.Predicate != nil {
|
||||
p, err := toStoragePredicate(spec.Predicate)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
predicate = p
|
||||
}
|
||||
|
||||
return &tagKeysIterator{
|
||||
ctx: ctx,
|
||||
bounds: spec.Bounds,
|
||||
s: r.s,
|
||||
readSpec: spec,
|
||||
predicate: predicate,
|
||||
predicate: spec.Predicate,
|
||||
alloc: alloc,
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (r *storeReader) ReadTagValues(ctx context.Context, spec influxdb.ReadTagValuesSpec, alloc *memory.Allocator) (influxdb.TableIterator, error) {
|
||||
var predicate *datatypes.Predicate
|
||||
if spec.Predicate != nil {
|
||||
p, err := toStoragePredicate(spec.Predicate)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
predicate = p
|
||||
}
|
||||
|
||||
return &tagValuesIterator{
|
||||
ctx: ctx,
|
||||
bounds: spec.Bounds,
|
||||
s: r.s,
|
||||
readSpec: spec,
|
||||
predicate: predicate,
|
||||
predicate: spec.Predicate,
|
||||
alloc: alloc,
|
||||
}, nil
|
||||
}
|
||||
|
@ -116,18 +98,9 @@ func (fi *filterIterator) Do(f func(flux.Table) error) error {
|
|||
return err
|
||||
}
|
||||
|
||||
var predicate *datatypes.Predicate
|
||||
if fi.spec.Predicate != nil {
|
||||
p, err := toStoragePredicate(fi.spec.Predicate)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
predicate = p
|
||||
}
|
||||
|
||||
var req datatypes.ReadFilterRequest
|
||||
req.ReadSource = any
|
||||
req.Predicate = predicate
|
||||
req.Predicate = fi.spec.Predicate
|
||||
req.Range.Start = int64(fi.spec.Bounds.Start)
|
||||
req.Range.End = int64(fi.spec.Bounds.Stop)
|
||||
|
||||
|
@ -240,18 +213,9 @@ func (gi *groupIterator) Do(f func(flux.Table) error) error {
|
|||
return err
|
||||
}
|
||||
|
||||
var predicate *datatypes.Predicate
|
||||
if gi.spec.Predicate != nil {
|
||||
p, err := toStoragePredicate(gi.spec.Predicate)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
predicate = p
|
||||
}
|
||||
|
||||
var req datatypes.ReadGroupRequest
|
||||
req.ReadSource = any
|
||||
req.Predicate = predicate
|
||||
req.Predicate = gi.spec.Predicate
|
||||
req.Range.Start = int64(gi.spec.Bounds.Start)
|
||||
req.Range.End = int64(gi.spec.Bounds.Stop)
|
||||
|
||||
|
|
18
test.sh
18
test.sh
|
@ -21,6 +21,7 @@ DIR=$(cd $(dirname "${BASH_SOURCE[0]}") && pwd)
|
|||
cd $DIR
|
||||
|
||||
ENVIRONMENT_INDEX=$1
|
||||
|
||||
# Set the default OUTPUT_DIR
|
||||
OUTPUT_DIR=${OUTPUT_DIR-./test-logs}
|
||||
# Set default parallelism
|
||||
|
@ -32,7 +33,7 @@ TIMEOUT=${TIMEOUT-1500s}
|
|||
DOCKER_RM=${DOCKER_RM-true}
|
||||
|
||||
# Update this value if you add a new test environment.
|
||||
ENV_COUNT=5
|
||||
ENV_COUNT=4
|
||||
|
||||
# Default return code 0
|
||||
rc=0
|
||||
|
@ -102,29 +103,24 @@ fi
|
|||
# Run the tests.
|
||||
case $ENVIRONMENT_INDEX in
|
||||
0)
|
||||
# 64 bit tests
|
||||
>&2 echo '64 bit tests'
|
||||
run_test_docker Dockerfile_build_ubuntu64 test_64bit --test --junit-report
|
||||
rc=$?
|
||||
;;
|
||||
1)
|
||||
# 64 bit race tests
|
||||
>&2 echo '64 bit race tests'
|
||||
GORACE="halt_on_error=1"
|
||||
run_test_docker Dockerfile_build_ubuntu64 test_64bit_race --test --junit-report --race
|
||||
rc=$?
|
||||
;;
|
||||
2)
|
||||
# 32 bit tests
|
||||
run_test_docker Dockerfile_build_ubuntu32 test_32bit --test --junit-report --arch=i386
|
||||
rc=$?
|
||||
;;
|
||||
3)
|
||||
# tsi
|
||||
>&2 echo 'tsi tests'
|
||||
INFLUXDB_DATA_INDEX_VERSION="tsi1"
|
||||
run_test_docker Dockerfile_build_ubuntu64 test_64bit --test --junit-report
|
||||
rc=$?
|
||||
;;
|
||||
4)
|
||||
# go1.13
|
||||
3)
|
||||
>&2 echo 'go1.13 tests'
|
||||
run_test_docker Dockerfile_build_ubuntu64_go1.13 test_64bit --test --junit-report
|
||||
rc=$?
|
||||
;;
|
||||
|
|
|
@ -3,8 +3,10 @@ package tests
|
|||
import (
|
||||
"bytes"
|
||||
"encoding/json"
|
||||
"errors"
|
||||
"flag"
|
||||
"fmt"
|
||||
"io/ioutil"
|
||||
"math/rand"
|
||||
"net/http"
|
||||
"net/url"
|
||||
|
@ -19,7 +21,14 @@ import (
|
|||
"github.com/gogo/protobuf/proto"
|
||||
"github.com/golang/snappy"
|
||||
"github.com/google/go-cmp/cmp"
|
||||
"github.com/influxdata/flux"
|
||||
"github.com/influxdata/flux/ast"
|
||||
"github.com/influxdata/flux/csv"
|
||||
"github.com/influxdata/flux/execute/table"
|
||||
"github.com/influxdata/flux/parser"
|
||||
"github.com/influxdata/flux/stdlib"
|
||||
"github.com/influxdata/influxdb/coordinator"
|
||||
fluxClient "github.com/influxdata/influxdb/flux/client"
|
||||
"github.com/influxdata/influxdb/models"
|
||||
"github.com/influxdata/influxdb/tsdb"
|
||||
"github.com/prometheus/prometheus/prompb"
|
||||
|
@ -9848,7 +9857,478 @@ func TestServer_Prometheus_Write(t *testing.T) {
|
|||
}
|
||||
}
|
||||
|
||||
func TestFluxBasicEndToEnd(t *testing.T) {
|
||||
config := NewConfig()
|
||||
config.HTTPD.FluxEnabled = true
|
||||
s := OpenServer(config)
|
||||
defer s.Close()
|
||||
|
||||
s.CreateDatabase(t.Name())
|
||||
defer s.DropDatabase(t.Name())
|
||||
u, err := url.Parse(s.URL())
|
||||
assert.NoError(t, err)
|
||||
u.Path = "/api/v2/query"
|
||||
httpClient := &http.Client{}
|
||||
|
||||
{
|
||||
// Query with json body
|
||||
query := fluxClient.QueryRequest{}.WithDefaults()
|
||||
query.Query = `import "influxdata/influxdb/v1" v1.databases()`
|
||||
j, err := json.Marshal(query)
|
||||
assert.NoError(t, err)
|
||||
req, err := http.NewRequest("POST", u.String(), bytes.NewBuffer(j))
|
||||
req.Header.Set("Content-Type", "application/json")
|
||||
assert.NoError(t, err)
|
||||
resp, err := httpClient.Do(req)
|
||||
assert.NoError(t, err)
|
||||
defer resp.Body.Close()
|
||||
b, err := ioutil.ReadAll(resp.Body)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t,
|
||||
strings.ReplaceAll(`,result,table,organizationID,databaseName,retentionPolicy,retentionPeriod,default,bucketId
|
||||
,_result,0,,TestFluxBasicEndToEnd,autogen,0,true,
|
||||
|
||||
`, "\n", "\r\n"),
|
||||
string(b))
|
||||
}
|
||||
{
|
||||
// Query with json body, with annotations
|
||||
query := fluxClient.QueryRequest{}.WithDefaults()
|
||||
query.Query = `import "influxdata/influxdb/v1" v1.databases()`
|
||||
query.Dialect.Annotations = csv.DefaultDialect().Annotations
|
||||
j, err := json.Marshal(query)
|
||||
assert.NoError(t, err)
|
||||
req, err := http.NewRequest("POST", u.String(), bytes.NewBuffer(j))
|
||||
req.Header.Set("Content-Type", "application/json")
|
||||
assert.NoError(t, err)
|
||||
resp, err := httpClient.Do(req)
|
||||
assert.NoError(t, err)
|
||||
defer resp.Body.Close()
|
||||
b, err := ioutil.ReadAll(resp.Body)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t,
|
||||
strings.ReplaceAll(`#datatype,string,long,string,string,string,long,boolean,string
|
||||
#group,false,false,true,false,false,false,false,false
|
||||
#default,_result,,,,,,,
|
||||
,result,table,organizationID,databaseName,retentionPolicy,retentionPeriod,default,bucketId
|
||||
,,0,,TestFluxBasicEndToEnd,autogen,0,true,
|
||||
|
||||
`, "\n", "\r\n"),
|
||||
string(b))
|
||||
assert.NoError(t, err)
|
||||
}
|
||||
{
|
||||
// Query with raw flux
|
||||
assert.NoError(t, err)
|
||||
req, err := http.NewRequest("POST", u.String(), bytes.NewBuffer([]byte(`import "influxdata/influxdb/v1" v1.databases()`)))
|
||||
req.Header.Set("Content-Type", "application/vnd.flux")
|
||||
assert.NoError(t, err)
|
||||
resp, err := httpClient.Do(req)
|
||||
assert.NoError(t, err)
|
||||
defer resp.Body.Close()
|
||||
b, err := ioutil.ReadAll(resp.Body)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t,
|
||||
strings.ReplaceAll(`,result,table,organizationID,databaseName,retentionPolicy,retentionPeriod,default,bucketId
|
||||
,_result,0,,TestFluxBasicEndToEnd,autogen,0,true,
|
||||
|
||||
`, "\n", "\r\n"),
|
||||
string(b))
|
||||
}
|
||||
{
|
||||
// Make sure runFluxBuiltinTest complains when it finds a diff
|
||||
testFluxTmpl := `package universe_test
|
||||
import "testing"
|
||||
option now = () => (2030-01-01T00:00:00Z)
|
||||
|
||||
inData = "#datatype,string,long,string,string,dateTime:RFC3339,unsignedLong
|
||||
#group,false,false,true,true,false,false
|
||||
#default,_result,,,,,
|
||||
,result,table,_measurement,_field,_time,_value
|
||||
,,0,Sgf,DlXwgrw,2018-12-18T22:11:05Z,70
|
||||
,,0,Sgf,DlXwgrw,2018-12-18T22:11:15Z,50"
|
||||
|
||||
outData = "#datatype,string,long,dateTime:RFC3339,dateTime:RFC3339,string,string,unsignedLong
|
||||
#group,false,false,true,true,true,true,false
|
||||
#default,_result,,,,,,
|
||||
,result,table,_start,_stop,_measurement,_field,_value
|
||||
,,0,2018-12-01T00:00:00Z,2030-01-01T00:00:00Z,Sgf,DlXwgrw,%d"
|
||||
t_sum = (table=<-) => (table |> range(start: 2018-12-01T00:00:00Z) |> sum())
|
||||
test _sum = () => ({input: testing.loadStorage(csv: inData), want: testing.loadMem(csv: outData), fn: t_sum})
|
||||
`
|
||||
// This test passes: 70+50=120
|
||||
databasePass := t.Name() + "_pass"
|
||||
s.CreateDatabase(databasePass)
|
||||
defer s.DropDatabase(databasePass)
|
||||
file := mustParse(fmt.Sprintf(testFluxTmpl, 120))
|
||||
bucket := databasePass + "/autogen"
|
||||
runFluxBuiltinTest(t, file, u, bucket, false)
|
||||
err := runFluxBuiltinTest(t, file, u, bucket, true)
|
||||
assert.NoError(t, err)
|
||||
|
||||
// We want to make sure the end to end tests are doing something. We assert that the test runner returns
|
||||
// an error on diffs
|
||||
databaseFail := t.Name() + "_fail"
|
||||
s.CreateDatabase(databaseFail)
|
||||
defer s.DropDatabase(databaseFail)
|
||||
file = mustParse(fmt.Sprintf(testFluxTmpl, 121))
|
||||
bucket = databaseFail + "/autogen"
|
||||
runFluxBuiltinTest(t, file, u, bucket, false)
|
||||
err = runFluxBuiltinTest(t, file, u, bucket, true)
|
||||
assert.EqualError(t, err, "test failed - diff table in output")
|
||||
}
|
||||
}
|
||||
|
||||
var FluxEndToEndSkipList = map[string]map[string]string{
|
||||
"universe": {
|
||||
// TODO(adam) determine the reason for these test failures.
|
||||
"cov": "Reason TBD",
|
||||
"covariance": "Reason TBD",
|
||||
"cumulative_sum": "Reason TBD",
|
||||
"cumulative_sum_default": "Reason TBD",
|
||||
"cumulative_sum_noop": "Reason TBD",
|
||||
"drop_non_existent": "Reason TBD",
|
||||
"first": "Reason TBD",
|
||||
"highestAverage": "Reason TBD",
|
||||
"highestMax": "Reason TBD",
|
||||
"histogram": "Reason TBD",
|
||||
"histogram_normalize": "Reason TBD",
|
||||
"histogram_quantile": "Reason TBD",
|
||||
"join": "Reason TBD",
|
||||
"join_across_measurements": "Reason TBD",
|
||||
"join_agg": "Reason TBD",
|
||||
"keep_non_existent": "Reason TBD",
|
||||
"key_values": "Reason TBD",
|
||||
"key_values_host_name": "Reason TBD",
|
||||
"last": "Reason TBD",
|
||||
"lowestAverage": "Reason TBD",
|
||||
"max": "Reason TBD",
|
||||
"min": "Reason TBD",
|
||||
"sample": "Reason TBD",
|
||||
"selector_preserve_time": "Reason TBD",
|
||||
"shift": "Reason TBD",
|
||||
"shift_negative_duration": "Reason TBD",
|
||||
"task_per_line": "Reason TBD",
|
||||
"top": "Reason TBD",
|
||||
"union": "Reason TBD",
|
||||
"union_heterogeneous": "Reason TBD",
|
||||
"unique": "Reason TBD",
|
||||
"distinct": "Reason TBD",
|
||||
|
||||
// it appears these occur when writing the input data. `to` may not be null safe.
|
||||
"fill_bool": "failed to read meta data: panic: interface conversion: interface {} is nil, not uint64",
|
||||
"fill_float": "failed to read meta data: panic: interface conversion: interface {} is nil, not uint64",
|
||||
"fill_int": "failed to read meta data: panic: interface conversion: interface {} is nil, not uint64",
|
||||
"fill_string": "failed to read meta data: panic: interface conversion: interface {} is nil, not uint64",
|
||||
"fill_time": "failed to read meta data: panic: interface conversion: interface {} is nil, not uint64",
|
||||
"fill_uint": "failed to read meta data: panic: interface conversion: interface {} is nil, not uint64",
|
||||
"window_null": "failed to read meta data: panic: interface conversion: interface {} is nil, not float64",
|
||||
|
||||
// these may just be missing calls to range() in the tests. easy to fix in a new PR.
|
||||
"group_nulls": "unbounded test",
|
||||
"integral": "unbounded test",
|
||||
"integral_columns": "unbounded test",
|
||||
"map": "unbounded test",
|
||||
"join_missing_on_col": "unbounded test",
|
||||
"join_use_previous": "unbounded test (https://github.com/influxdata/flux/issues/2996)",
|
||||
"join_panic": "unbounded test (https://github.com/influxdata/flux/issues/3465)",
|
||||
"rowfn_with_import": "unbounded test",
|
||||
|
||||
// the following tests have a difference between the CSV-decoded input table, and the storage-retrieved version of that table
|
||||
"columns": "group key mismatch",
|
||||
"set": "column order mismatch",
|
||||
"simple_max": "_stop missing from expected output",
|
||||
"derivative": "time bounds mismatch (engine uses now() instead of bounds on input table)",
|
||||
"difference_columns": "data write/read path loses columns x and y",
|
||||
"keys": "group key mismatch",
|
||||
|
||||
// failed to read meta data errors: the CSV encoding is incomplete probably due to data schema errors. needs more detailed investigation to find root cause of error
|
||||
// "filter_by_regex": "failed to read metadata",
|
||||
// "filter_by_tags": "failed to read metadata",
|
||||
"group": "failed to read metadata",
|
||||
"group_except": "failed to read metadata",
|
||||
"group_ungroup": "failed to read metadata",
|
||||
"pivot_mean": "failed to read metadata",
|
||||
"histogram_quantile_minvalue": "failed to read meta data: no column with label _measurement exists",
|
||||
"increase": "failed to read meta data: table has no _value column",
|
||||
|
||||
"string_max": "error: invalid use of function: *functions.MaxSelector has no implementation for type string (https://github.com/influxdata/platform/issues/224)",
|
||||
"null_as_value": "null not supported as value in influxql (https://github.com/influxdata/platform/issues/353)",
|
||||
"string_interp": "string interpolation not working as expected in flux (https://github.com/influxdata/platform/issues/404)",
|
||||
"to": "to functions are not supported in the testing framework (https://github.com/influxdata/flux/issues/77)",
|
||||
"covariance_missing_column_1": "need to support known errors in new test framework (https://github.com/influxdata/flux/issues/536)",
|
||||
"covariance_missing_column_2": "need to support known errors in new test framework (https://github.com/influxdata/flux/issues/536)",
|
||||
"drop_before_rename": "need to support known errors in new test framework (https://github.com/influxdata/flux/issues/536)",
|
||||
"drop_referenced": "need to support known errors in new test framework (https://github.com/influxdata/flux/issues/536)",
|
||||
"yield": "yield requires special test case (https://github.com/influxdata/flux/issues/535)",
|
||||
|
||||
"window_group_mean_ungroup": "window trigger optimization modifies sort order of its output tables (https://github.com/influxdata/flux/issues/1067)",
|
||||
|
||||
"median_column": "failing in different ways (https://github.com/influxdata/influxdb/issues/13909)",
|
||||
"dynamic_query": "tableFind does not work in e2e tests: https://github.com/influxdata/influxdb/issues/13975",
|
||||
|
||||
"to_int": "dateTime conversion issue: https://github.com/influxdata/influxdb/issues/14575",
|
||||
"to_uint": "dateTime conversion issue: https://github.com/influxdata/influxdb/issues/14575",
|
||||
|
||||
"holt_winters_panic": "Expected output is an empty table which breaks the testing framework (https://github.com/influxdata/influxdb/issues/14749)",
|
||||
"map_nulls": "to cannot write null values",
|
||||
},
|
||||
"array": {
|
||||
"from": "test not meant to be consumed by influxdb",
|
||||
"from_group": "test not meant to be consumed by influxdb",
|
||||
},
|
||||
"experimental": {
|
||||
"set": "Reason TBD",
|
||||
"join": "unbounded test",
|
||||
"alignTime": "unbounded test",
|
||||
},
|
||||
"experimental/geo": {
|
||||
"filterRowsNotStrict": "tableFind does not work in e2e tests: https://github.com/influxdata/influxdb/issues/13975",
|
||||
"filterRowsStrict": "tableFind does not work in e2e tests: https://github.com/influxdata/influxdb/issues/13975",
|
||||
"gridFilterLevel": "tableFind does not work in e2e tests: https://github.com/influxdata/influxdb/issues/13975",
|
||||
"gridFilter": "tableFind does not work in e2e tests: https://github.com/influxdata/influxdb/issues/13975",
|
||||
"groupByArea": "tableFind does not work in e2e tests: https://github.com/influxdata/influxdb/issues/13975",
|
||||
"filterRowsPivoted": "tableFind does not work in e2e tests: https://github.com/influxdata/influxdb/issues/13975",
|
||||
"shapeDataWithFilter": "tableFind does not work in e2e tests: https://github.com/influxdata/influxdb/issues/13975",
|
||||
"shapeData": "test run before to() is finished: https://github.com/influxdata/influxdb/issues/13975",
|
||||
},
|
||||
"regexp": {
|
||||
"replaceAllString": "Reason TBD",
|
||||
},
|
||||
"http": {
|
||||
"http_endpoint": "need ability to test side effects in e2e tests: (https://github.com/influxdata/flux/issues/1723)",
|
||||
},
|
||||
"influxdata/influxdb/schema": {
|
||||
"show_tag_keys": "failing due to bug in test, unskip this after upgrading from Flux v0.91.0",
|
||||
},
|
||||
"influxdata/influxdb/monitor": {
|
||||
"state_changes_big_any_to_any": "unbounded test",
|
||||
"state_changes_big_info_to_ok": "unbounded test",
|
||||
"state_changes_big_ok_to_info": "unbounded test",
|
||||
"state_changes_any_to_any": "test run before to() is finished: https://github.com/influxdata/influxdb/issues/13975",
|
||||
"state_changes_info_to_any": "test run before to() is finished: https://github.com/influxdata/influxdb/issues/13975",
|
||||
"state_changes_invalid_any_to_any": "test run before to() is finished: https://github.com/influxdata/influxdb/issues/13975",
|
||||
"state_changes": "test run before to() is finished: https://github.com/influxdata/influxdb/issues/13975",
|
||||
},
|
||||
"influxdata/influxdb/secrets": {
|
||||
"secrets": "Cannot inject custom deps into the test framework so the secrets don't lookup correctly",
|
||||
},
|
||||
"internal/promql": {
|
||||
"join": "unbounded test",
|
||||
},
|
||||
"testing/chronograf": {
|
||||
"buckets": "unbounded test",
|
||||
"aggregate_window_count": "flakey test: https://github.com/influxdata/influxdb/issues/18463",
|
||||
},
|
||||
"testing/kapacitor": {
|
||||
"fill_default": "unknown field type for f1",
|
||||
},
|
||||
"testing/pandas": {
|
||||
"extract_regexp_findStringIndex": "pandas. map does not correctly handled returned arrays (https://github.com/influxdata/flux/issues/1387)",
|
||||
"partition_strings_splitN": "pandas. map does not correctly handled returned arrays (https://github.com/influxdata/flux/issues/1387)",
|
||||
},
|
||||
"testing/promql": {
|
||||
"emptyTable": "tests a source",
|
||||
"year": "flakey test: https://github.com/influxdata/influxdb/issues/15667",
|
||||
"extrapolatedRate_counter_rate": "option \"testing.loadStorage\" reassigned: https://github.com/influxdata/flux/issues/3155",
|
||||
"extrapolatedRate_nocounter": "option \"testing.loadStorage\" reassigned: https://github.com/influxdata/flux/issues/3155",
|
||||
"extrapolatedRate_norate": "option \"testing.loadStorage\" reassigned: https://github.com/influxdata/flux/issues/3155",
|
||||
"linearRegression_nopredict": "option \"testing.loadStorage\" reassigned: https://github.com/influxdata/flux/issues/3155",
|
||||
"linearRegression_predict": "option \"testing.loadStorage\" reassigned: https://github.com/influxdata/flux/issues/3155",
|
||||
},
|
||||
"testing/influxql": {
|
||||
"cumulative_sum": "invalid test data requires loadStorage to be overridden. See https://github.com/influxdata/flux/issues/3145",
|
||||
"elapsed": "failing since split with Flux upgrade: https://github.com/influxdata/influxdb/issues/19568",
|
||||
},
|
||||
"contrib/RohanSreerama5/naiveBayesClassifier": {
|
||||
"bayes": "error calling tableFind: ",
|
||||
},
|
||||
}
|
||||
|
||||
func TestFluxEndToEnd(t *testing.T) {
|
||||
runEndToEnd(t, stdlib.FluxTestPackages)
|
||||
}
|
||||
|
||||
func runEndToEnd(t *testing.T, pkgs []*ast.Package) {
|
||||
config := NewConfig()
|
||||
config.HTTPD.FluxEnabled = true
|
||||
s := OpenServer(config)
|
||||
defer s.Close()
|
||||
|
||||
for _, pkg := range pkgs {
|
||||
test := func(t *testing.T, f func(t *testing.T)) {
|
||||
t.Run(pkg.Path, f)
|
||||
}
|
||||
if pkg.Path == "universe" {
|
||||
test = func(t *testing.T, f func(t *testing.T)) {
|
||||
f(t)
|
||||
}
|
||||
}
|
||||
|
||||
test(t, func(t *testing.T) {
|
||||
for _, file := range pkg.Files {
|
||||
name := strings.TrimSuffix(file.Name, "_test.flux")
|
||||
t.Run(name, func(t *testing.T) {
|
||||
if reason, ok := FluxEndToEndSkipList[pkg.Path][name]; ok {
|
||||
t.Skip(reason)
|
||||
}
|
||||
// Set up the database & URL
|
||||
// We don't properly support slashes in database names for flux queries
|
||||
databaseName := strings.ReplaceAll(t.Name(), "/", "_")
|
||||
s.CreateDatabase(databaseName)
|
||||
defer s.DropDatabase(databaseName)
|
||||
u, err := url.Parse(s.URL())
|
||||
assert.NoError(t, err)
|
||||
u.Path = "/api/v2/query"
|
||||
bucket := databaseName + "/autogen"
|
||||
|
||||
// Run the end to end test. The first time we ignore the results, but as a side
|
||||
// effect the data is loaded into the TSDB store. The second test runs with `from`
|
||||
// gathering data from TSDB.
|
||||
runFluxBuiltinTest(t, file, u, bucket, false)
|
||||
err = runFluxBuiltinTest(t, file, u, bucket, true)
|
||||
})
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func makeTestPackage(file *ast.File) *ast.Package {
|
||||
file = file.Copy().(*ast.File)
|
||||
file.Package.Name.Name = "main"
|
||||
pkg := &ast.Package{
|
||||
Package: "main",
|
||||
Files: []*ast.File{file},
|
||||
}
|
||||
return pkg
|
||||
}
|
||||
|
||||
// This options definition puts to() in the path of the CSV input. The tests
|
||||
// get run in this case and they would normally pass, if we checked the
|
||||
// results, but don't look at them.
|
||||
var writeOptSource = `
|
||||
import "testing"
|
||||
import c "csv"
|
||||
|
||||
option testing.loadStorage = (csv) => {
|
||||
return c.from(csv: csv) |> to(bucket: bucket)
|
||||
}
|
||||
`
|
||||
|
||||
// This options definition is for the second run, the test run. It loads the
|
||||
// data from previously written bucket. We check the results after running this
|
||||
// second pass and report on them.
|
||||
var readOptSource = `
|
||||
import "testing"
|
||||
import c "csv"
|
||||
|
||||
option testing.loadStorage = (csv) => {
|
||||
return from(bucket: bucket)
|
||||
}
|
||||
`
|
||||
|
||||
var writeOptAST *ast.File
|
||||
var readOptAST *ast.File
|
||||
|
||||
func mustParse(flux string) *ast.File {
|
||||
pkg := parser.ParseSource(flux)
|
||||
if ast.Check(pkg) > 0 {
|
||||
panic(ast.GetError(pkg))
|
||||
}
|
||||
return pkg.Files[0]
|
||||
}
|
||||
|
||||
func runFluxBuiltinTest(t *testing.T, file *ast.File, u *url.URL, bucket string, readTest bool) error {
|
||||
var options *ast.File
|
||||
if readTest {
|
||||
// load input data from database
|
||||
options = readOptAST.Copy().(*ast.File)
|
||||
} else {
|
||||
// load input data from csv and store it to the database
|
||||
options = writeOptAST.Copy().(*ast.File)
|
||||
}
|
||||
|
||||
// Allow the script to refer to a bucket as a variable
|
||||
bucketOpt := &ast.OptionStatement{
|
||||
Assignment: &ast.VariableAssignment{
|
||||
ID: &ast.Identifier{Name: "bucket"},
|
||||
Init: &ast.StringLiteral{Value: bucket},
|
||||
},
|
||||
}
|
||||
options.Body = append([]ast.Statement{bucketOpt}, options.Body...)
|
||||
|
||||
pkg := makeTestPackage(file)
|
||||
pkg.Files = append(pkg.Files, options)
|
||||
|
||||
inspectCalls := stdlib.TestingInspectCalls(pkg)
|
||||
if len(inspectCalls.Body) == 0 {
|
||||
t.Skip("No tests in builtin test package")
|
||||
}
|
||||
pkg.Files = append(pkg.Files, inspectCalls)
|
||||
|
||||
parsedQuery, err := json.Marshal(pkg)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
httpClient := &http.Client{}
|
||||
// Query with json body
|
||||
query := fluxClient.QueryRequest{}.WithDefaults()
|
||||
query.AST = parsedQuery
|
||||
query.Dialect.Annotations = csv.DefaultDialect().Annotations
|
||||
j, err := json.Marshal(query)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
req, err := http.NewRequest("POST", u.String(), bytes.NewBuffer(j))
|
||||
req.Header.Set("Content-Type", "application/json")
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
resp, err := httpClient.Do(req)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if resp.StatusCode/100 != 2 {
|
||||
b, _ := ioutil.ReadAll(resp.Body)
|
||||
t.Log("Bad response from flux:", string(b))
|
||||
return fmt.Errorf("Bad status code %d from flux query", resp.StatusCode)
|
||||
}
|
||||
|
||||
decoder := csv.NewMultiResultDecoder(csv.ResultDecoderConfig{})
|
||||
itr, err := decoder.Decode(resp.Body)
|
||||
defer itr.Release() // will close body
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
wasDiff := false
|
||||
for itr.More() {
|
||||
v := itr.Next()
|
||||
if v.Name() == "diff" {
|
||||
wasDiff = true
|
||||
}
|
||||
if err := v.Tables().Do(func(tbl flux.Table) error {
|
||||
if readTest {
|
||||
t.Log(table.Stringify(tbl))
|
||||
}
|
||||
return nil
|
||||
}); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
assert.NoError(t, itr.Err())
|
||||
if wasDiff && readTest {
|
||||
return errors.New("test failed - diff table in output")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func init() {
|
||||
// Force uint support to be enabled for testing.
|
||||
models.EnableUintSupport()
|
||||
|
||||
writeOptAST = mustParse(writeOptSource)
|
||||
readOptAST = mustParse(readOptSource)
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue