From 33f8274c65d944c79302af275a7ccde3a0e3f9be Mon Sep 17 00:00:00 2001 From: Nathaniel Cook Date: Wed, 9 Dec 2015 16:08:46 -0700 Subject: [PATCH] User Defined Functions (UDFs) --- .dockerignore | 1 + .gitignore | 1 + CHANGELOG.md | 15 +- Dockerfile | 34 +- alert.go | 7 +- batch.go | 13 +- build.py | 24 +- circle.yml | 6 +- cmd/kapacitord/run/command.go | 9 +- cmd/kapacitord/run/config.go | 7 + cmd/kapacitord/run/server.go | 15 + cmd/kapacitord/run/server_test.go | 150 ++ command/command.go | 45 + command/test/command_helper.go | 141 ++ derivative.go | 7 +- edge.go | 51 +- etc/kapacitor/kapacitor.conf | 37 + eval.go | 11 +- group_by.go | 7 +- http_out.go | 7 +- influxdb_out.go | 8 +- integrations/batcher_test.go | 14 +- .../data/TestStream_CustomFunctions.srpl | 81 + integrations/helpers_test.go | 22 + integrations/streamer_test.go | 243 ++- join.go | 8 +- map_reduce.go | 13 +- node.go | 23 +- pipeline/udf.go | 115 ++ sample.go | 7 +- services/reporting/service.go | 8 +- services/task_store/config.go | 9 +- services/task_store/service.go | 134 +- services/udf/config.go | 32 + services/udf/service.go | 93 ++ stream.go | 8 +- task.go | 187 ++- task_master.go | 75 +- tick/eval.go | 217 ++- tick/eval_test.go | 68 +- tick/example_scope_test.go | 8 +- tick/scope.go | 15 +- udf.go | 149 ++ udf/agent/README.md | 50 + udf/agent/agent.go | 235 +++ udf/agent/examples/moving_avg.go | 169 ++ udf/agent/examples/moving_avg.py | 155 ++ udf/agent/py/agent.py | 168 ++ udf/agent/py/udf_pb2.py | 1470 +++++++++++++++++ udf/io.go | 67 + udf/io_test.go | 69 + udf/udf.pb.go | 1039 ++++++++++++ udf/udf.proto | 217 +++ udf_process.go | 771 +++++++++ udf_process_test.go | 337 ++++ union.go | 8 +- where.go | 7 +- window.go | 6 +- 58 files changed, 6569 insertions(+), 324 deletions(-) create mode 100644 command/command.go create mode 100644 command/test/command_helper.go create mode 100644 integrations/data/TestStream_CustomFunctions.srpl create mode 100644 pipeline/udf.go create mode 100644 services/udf/config.go create mode 100644 services/udf/service.go create mode 100644 udf.go create mode 100644 udf/agent/README.md create mode 100644 udf/agent/agent.go create mode 100644 udf/agent/examples/moving_avg.go create mode 100644 udf/agent/examples/moving_avg.py create mode 100644 udf/agent/py/agent.py create mode 100644 udf/agent/py/udf_pb2.py create mode 100644 udf/io.go create mode 100644 udf/io_test.go create mode 100644 udf/udf.pb.go create mode 100644 udf/udf.proto create mode 100644 udf_process.go create mode 100644 udf_process_test.go diff --git a/.dockerignore b/.dockerignore index 6b8710a71..04c6da65b 100644 --- a/.dockerignore +++ b/.dockerignore @@ -1 +1,2 @@ .git +build diff --git a/.gitignore b/.gitignore index d3130bd4d..f02cd7529 100644 --- a/.gitignore +++ b/.gitignore @@ -11,3 +11,4 @@ kapacitor*.rpm kapacitor*.deb kapacitor*.tar kapacitor*.zip +*.pyc diff --git a/CHANGELOG.md b/CHANGELOG.md index ac24f9bc1..64dc46275 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,10 +1,23 @@ # Changelog -## v0.2.5 [unreleased] +## v0.10.0 [unreleased] ### Release Notes +This release marks the next release of Kapacitor. +With this release you can now run your own custom code for processing data within Kapacitor. +See [udf/agent/README.md](https://github.com/influxdata/kapacitor/blob/master/udf/agent/README.md) for more details. + +With the addition of UDFs it is now possible to run custom anomaly detection alogrithms suited to your needs. +There are simple examples of how to use UDFs in [udf/agent/examples](https://github.com/influxdata/kapacitor/tree/master/udf/agent/examples/). + + +The version has jumped significantly so that it is inline with other projects in the TICK stack. +This way you can easily tell which versions of Telegraf, InfluxDB, Chronograf and Kapacitor work together. + + ### Features +- [#72](https://github.com/influxdata/kapacitor/issues/72): Add support for User Defined Functions (UDFs). - [#138](https://github.com/influxdata/kapacitor/issues/138): Change over to influxdata github org. - [#139](https://github.com/influxdata/kapacitor/issues/139): Alerta.io support thanks! @md14454 diff --git a/Dockerfile b/Dockerfile index 5240c6ce1..70f5649d8 100644 --- a/Dockerfile +++ b/Dockerfile @@ -19,14 +19,42 @@ RUN apt-get install -y \ RUN gem install fpm +# Install protobuf3 +RUN apt-get install -y \ + build-essential \ + autoconf \ + automake \ + libtool \ + python-setuptools \ + curl + +ENV PROTO_VERSION 3.0.0-beta-2 +# Download and compile protoc +RUN wget https://github.com/google/protobuf/archive/v${PROTO_VERSION}.tar.gz && \ + tar xf v${PROTO_VERSION}.tar.gz && \ + rm -f v${PROTO_VERSION}.tar.gz && \ + cd protobuf-${PROTO_VERSION} && \ + ./autogen.sh && \ + ./configure --prefix=/usr && \ + make -j $(nproc) && \ + make check && \ + make install + +# Install Python Protobuf3 +RUN cd protobuf-${PROTO_VERSION}/python && \ + python setup.py install; + + + # Install go -ENV GO_VERSION 1.5.2 +ENV GO_VERSION 1.5.3 ENV GO_ARCH amd64 RUN wget 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 ; \ rm /go${GO_VERSION}.linux-${GO_ARCH}.tar.gz ENV PATH /usr/local/go/bin:$PATH ENV GOPATH /gopath +ENV PATH $GOPATH/bin:$PATH ENV PROJECT_PATH $GOPATH/src/github.com/influxdata/kapacitor RUN mkdir -p $PROJECT_PATH @@ -34,4 +62,8 @@ WORKDIR $PROJECT_PATH ENTRYPOINT ["/usr/local/bin/build"] CMD [] +# Get gogo for golang protobuf +RUN go get github.com/gogo/protobuf/protoc-gen-gogo + ADD ./build.py /usr/local/bin/build + diff --git a/alert.go b/alert.go index a66bd1209..3e3b628b3 100644 --- a/alert.go +++ b/alert.go @@ -5,6 +5,7 @@ import ( "encoding/json" "errors" "fmt" + "log" "net/http" "os" "os/exec" @@ -75,9 +76,9 @@ type AlertNode struct { } // Create a new AlertNode which caches the most recent item and exposes it over the HTTP API. -func newAlertNode(et *ExecutingTask, n *pipeline.AlertNode) (an *AlertNode, err error) { +func newAlertNode(et *ExecutingTask, n *pipeline.AlertNode, l *log.Logger) (an *AlertNode, err error) { an = &AlertNode{ - node: node{Node: n, et: et}, + node: node{Node: n, et: et, logger: l}, a: n, } an.node.runF = an.runAlert @@ -207,7 +208,7 @@ func newAlertNode(et *ExecutingTask, n *pipeline.AlertNode) (an *AlertNode, err return } -func (a *AlertNode) runAlert() error { +func (a *AlertNode) runAlert([]byte) error { switch a.Wants() { case pipeline.StreamEdge: for p, ok := a.ins[0].NextPoint(); ok; p, ok = a.ins[0].NextPoint() { diff --git a/batch.go b/batch.go index 392baab93..493931054 100644 --- a/batch.go +++ b/batch.go @@ -3,6 +3,7 @@ package kapacitor import ( "errors" "fmt" + "log" "sync" "time" @@ -19,9 +20,9 @@ type SourceBatchNode struct { idx int } -func newSourceBatchNode(et *ExecutingTask, n *pipeline.SourceBatchNode) (*SourceBatchNode, error) { +func newSourceBatchNode(et *ExecutingTask, n *pipeline.SourceBatchNode, l *log.Logger) (*SourceBatchNode, error) { sn := &SourceBatchNode{ - node: node{Node: n, et: et}, + node: node{Node: n, et: et, logger: l}, s: n, } return sn, nil @@ -47,7 +48,7 @@ func (s *SourceBatchNode) addParentEdge(in *Edge) { s.idx++ } -func (s *SourceBatchNode) start() { +func (s *SourceBatchNode) start([]byte) { } func (s *SourceBatchNode) Err() error { @@ -96,9 +97,9 @@ type BatchNode struct { closing chan struct{} } -func newBatchNode(et *ExecutingTask, n *pipeline.BatchNode) (*BatchNode, error) { +func newBatchNode(et *ExecutingTask, n *pipeline.BatchNode, l *log.Logger) (*BatchNode, error) { bn := &BatchNode{ - node: node{Node: n, et: et}, + node: node{Node: n, et: et, logger: l}, b: n, closing: make(chan struct{}), } @@ -264,7 +265,7 @@ func (b *BatchNode) doQuery() error { } } -func (b *BatchNode) runBatch() error { +func (b *BatchNode) runBatch([]byte) error { errC := make(chan error, 1) go func() { defer func() { diff --git a/build.py b/build.py index a365fc656..2142b5387 100755 --- a/build.py +++ b/build.py @@ -1,4 +1,4 @@ -#!/usr/bin/env python2.7 +#!/usr/bin/python2.7 -u import sys import os @@ -204,7 +204,7 @@ def run_tests(race): run(get_command) print "done." print "Running tests..." - test_command = "go test ./..." + test_command = "go test -v ./..." if race: test_command = "go test -race ./..." code = os.system(test_command) @@ -215,6 +215,18 @@ def run_tests(race): print "Tests Passed" return True +def run_generate(): + print "Running generate..." + command = "go generate ./..." + code = os.system(command) + if code != 0: + print "Generate Failed" + return False + else: + print "Generate Succeeded" + return True + + def build(version=None, branch=None, commit=None, @@ -407,6 +419,7 @@ def main(): update = False upload = False test = False + generate = False for arg in sys.argv[1:]: if '--outdir' in arg: @@ -451,6 +464,9 @@ def main(): elif '--test' in arg: # Run tests and exit test = True + elif '--generate' in arg: + # Run go generate ./... + generate = True else: print "!! Unknown argument: {}".format(arg) sys.exit(1) @@ -474,6 +490,10 @@ def main(): # TODO(rossmcdonald): Prepare git repo for build (checking out correct branch/commit, etc.) # prepare(branch=branch, commit=commit) + if generate: + if not run_generate(): + return 1 + if test: if not run_tests(race): return 1 diff --git a/circle.yml b/circle.yml index a8fe78224..71767307d 100644 --- a/circle.yml +++ b/circle.yml @@ -21,12 +21,12 @@ dependencies: test: override: - - ./build.sh --test - - ./build.sh --test --race + - ./build.sh --test --generate + - ./build.sh --test --generate --race deployment: release: tag: /v[0-9]+(\.[0-9]+){2}(-rc[0-9]+)?/ commands: - - ./build.sh --clean --packages --upload --platform=all --arch=all + - ./build.sh --clean --generate --packages --upload --platform=all --arch=all diff --git a/cmd/kapacitord/run/command.go b/cmd/kapacitord/run/command.go index b68d823d8..8644c4d1f 100644 --- a/cmd/kapacitord/run/command.go +++ b/cmd/kapacitord/run/command.go @@ -93,11 +93,6 @@ func (cmd *Command) Run(args ...string) error { config.Logging.Level = options.LogLevel } - // Validate the configuration. - if err := config.Validate(); err != nil { - return fmt.Errorf("%s. To generate a valid configuration file run `kapacitord config > kapacitor.generated.conf`.", err) - } - // Initialize Logging Services cmd.logService = logging.NewService(config.Logging, cmd.Stdout, cmd.Stderr) err = cmd.logService.Open() @@ -151,7 +146,9 @@ func (cmd *Command) monitorServerErrors() { for { select { case err := <-cmd.Server.Err(): - cmd.Logger.Println("E! " + err.Error()) + if err != nil { + cmd.Logger.Println("E! " + err.Error()) + } case <-cmd.closing: return } diff --git a/cmd/kapacitord/run/config.go b/cmd/kapacitord/run/config.go index 5ec7a5d2a..67ea806a1 100644 --- a/cmd/kapacitord/run/config.go +++ b/cmd/kapacitord/run/config.go @@ -23,6 +23,7 @@ import ( "github.com/influxdata/kapacitor/services/smtp" "github.com/influxdata/kapacitor/services/stats" "github.com/influxdata/kapacitor/services/task_store" + "github.com/influxdata/kapacitor/services/udf" "github.com/influxdata/kapacitor/services/udp" "github.com/influxdata/kapacitor/services/victorops" @@ -52,6 +53,7 @@ type Config struct { Alerta alerta.Config `toml:"alerta"` Reporting reporting.Config `toml:"reporting"` Stats stats.Config `toml:"stats"` + UDF udf.Config `toml:"udf"` Hostname string `toml:"hostname"` DataDir string `toml:"data_dir"` @@ -79,6 +81,7 @@ func NewConfig() *Config { c.Alerta = alerta.NewConfig() c.Reporting = reporting.NewConfig() c.Stats = stats.NewConfig() + c.UDF = udf.NewConfig() return c } @@ -125,6 +128,10 @@ func (c *Config) Validate() error { if err != nil { return err } + err = c.UDF.Validate() + if err != nil { + return err + } for _, g := range c.Graphites { if err := g.Validate(); err != nil { return fmt.Errorf("invalid graphite config: %v", err) diff --git a/cmd/kapacitord/run/server.go b/cmd/kapacitord/run/server.go index 9f7c7ac76..79704d811 100644 --- a/cmd/kapacitord/run/server.go +++ b/cmd/kapacitord/run/server.go @@ -27,6 +27,7 @@ import ( "github.com/influxdata/kapacitor/services/smtp" "github.com/influxdata/kapacitor/services/stats" "github.com/influxdata/kapacitor/services/task_store" + "github.com/influxdata/kapacitor/services/udf" "github.com/influxdata/kapacitor/services/udp" "github.com/influxdata/kapacitor/services/victorops" "github.com/influxdata/kapacitor/wlog" @@ -104,6 +105,10 @@ type Server struct { // NewServer returns a new instance of Server built from a config. func NewServer(c *Config, buildInfo *BuildInfo, logService logging.Interface) (*Server, error) { + err := c.Validate() + if err != nil { + return nil, fmt.Errorf("%s. To generate a valid configuration file run `kapacitord config > kapacitor.generated.conf`.", err) + } l := logService.NewLogger("[srv] ", log.LstdFlags) s := &Server{ buildInfo: *buildInfo, @@ -124,6 +129,7 @@ func NewServer(c *Config, buildInfo *BuildInfo, logService logging.Interface) (* } // Append Kapacitor services. + s.appendUDFService(c.UDF) s.appendSMTPService(c.SMTP) s.appendHTTPDService(c.HTTP) s.appendInfluxDBService(c.InfluxDB, c.Hostname) @@ -201,6 +207,7 @@ func (s *Server) appendTaskStoreService(c task_store.Config) { srv.TaskMaster = s.TaskMaster s.TaskStore = srv + s.TaskMaster.TaskStore = srv s.Services = append(s.Services, srv) } @@ -216,6 +223,14 @@ func (s *Server) appendReplayStoreService(c replay.Config) { s.Services = append(s.Services, srv) } +func (s *Server) appendUDFService(c udf.Config) { + l := s.LogService.NewLogger("[udf] ", log.LstdFlags) + srv := udf.NewService(c, l) + + s.TaskMaster.UDFService = srv + s.Services = append(s.Services, srv) +} + func (s *Server) appendOpsGenieService(c opsgenie.Config) { if c.Enabled { l := s.LogService.NewLogger("[opsgenie] ", log.LstdFlags) diff --git a/cmd/kapacitord/run/server_test.go b/cmd/kapacitord/run/server_test.go index 52d509b4b..6bec48b2b 100644 --- a/cmd/kapacitord/run/server_test.go +++ b/cmd/kapacitord/run/server_test.go @@ -8,15 +8,21 @@ import ( "net/http" "net/url" "os" + "os/exec" "path" + "path/filepath" "reflect" + "strings" "testing" "time" "github.com/influxdata/kapacitor" + "github.com/influxdata/kapacitor/cmd/kapacitord/run" + "github.com/influxdata/kapacitor/services/udf" "github.com/influxdb/influxdb/client" "github.com/influxdb/influxdb/influxql" "github.com/influxdb/influxdb/models" + "github.com/influxdb/influxdb/toml" ) func TestServer_Ping(t *testing.T) { @@ -692,3 +698,147 @@ batch t.Errorf("unexpected alert log:\ngot %v\nexp %v", got[1].Data.Series[0], exp[1].Data.Series[0]) } } + +func TestServer_UDFAgents(t *testing.T) { + + dir, err := os.Getwd() + udfDir := filepath.Clean(filepath.Join(dir, "../../../udf")) + if err != nil { + t.Fatal(err) + } + + tmpDir, err := ioutil.TempDir("", "testStreamTaskRecording") + if err != nil { + t.Fatal(err) + } + defer os.RemoveAll(tmpDir) + agents := []struct { + buildFunc func() error + config udf.FunctionConfig + }{ + // Go + { + buildFunc: func() error { + cmd := exec.Command( + "go", + "build", + "-o", + filepath.Join(tmpDir, "go-moving_avg"), + filepath.Join(udfDir, "agent/examples/moving_avg.go"), + ) + out, err := cmd.CombinedOutput() + if err != nil { + return fmt.Errorf("go build failed: %v: %s", err, string(out)) + } + return nil + }, + config: udf.FunctionConfig{ + Prog: filepath.Join(tmpDir, "go-moving_avg"), + Timeout: toml.Duration(time.Minute), + }, + }, + // Python + { + buildFunc: func() error { return nil }, + config: udf.FunctionConfig{ + Prog: "python2", + Args: []string{"-u", filepath.Join(udfDir, "agent/examples/moving_avg.py")}, + Timeout: toml.Duration(time.Minute), + Env: map[string]string{ + "PYTHONPATH": strings.Join( + []string{filepath.Join(udfDir, "agent/py"), os.Getenv("PYTHONPATH")}, + string(filepath.ListSeparator), + ), + }, + }, + }, + } + for _, agent := range agents { + err := agent.buildFunc() + if err != nil { + t.Fatal(err) + } + c := NewConfig() + c.UDF.Functions = map[string]udf.FunctionConfig{ + "movingAvg": agent.config, + } + testAgent(t, c) + } +} + +func testAgent(t *testing.T, c *run.Config) { + s := NewServer(c) + err := s.Open() + if err != nil { + t.Fatal(err) + } + defer s.Close() + + name := "testUDFTask" + ttype := "stream" + dbrps := []kapacitor.DBRP{{ + Database: "mydb", + RetentionPolicy: "myrp", + }} + tick := ` +stream + .from().measurement('test') + .movingAvg() + .field('value') + .size(10) + .as('mean') + .window() + .period(11s) + .every(11s) + .mapReduce(influxql.last('mean')).as('mean') + .httpOut('moving_avg') +` + + r, err := s.DefineTask(name, ttype, tick, dbrps) + if err != nil { + t.Fatal(err) + } + if r != "" { + t.Fatal("unexpected result", r) + } + + r, err = s.EnableTask(name) + if err != nil { + t.Fatal(err) + } + if r != "" { + t.Fatal("unexpected result", r) + } + + endpoint := fmt.Sprintf("%s/api/v1/%s/moving_avg", s.URL(), name) + + // Request data before any writes and expect null responses + nullResponse := `{"Series":null,"Err":null}` + err = s.HTTPGetRetry(endpoint, "", nullResponse, 1000, time.Millisecond*5) + if err != nil { + t.Error(err) + } + + points := `test value=1 0000000000 +test value=1 0000000001 +test value=1 0000000002 +test value=1 0000000003 +test value=1 0000000004 +test value=1 0000000005 +test value=1 0000000006 +test value=1 0000000007 +test value=1 0000000008 +test value=1 0000000009 +test value=0 0000000010 +test value=0 0000000011 +` + v := url.Values{} + v.Add("precision", "s") + s.MustWrite("mydb", "myrp", points, v) + + exp := `{"Series":[{"name":"test","columns":["time","mean"],"values":[["1970-01-01T00:00:11Z",0.9]]}],"Err":null}` + err = s.HTTPGetRetry(endpoint, nullResponse, exp, 100, time.Millisecond*5) + if err != nil { + t.Error(err) + } +} diff --git a/command/command.go b/command/command.go new file mode 100644 index 000000000..95b178fb3 --- /dev/null +++ b/command/command.go @@ -0,0 +1,45 @@ +package command + +import ( + "io" + "os/exec" +) + +type Command interface { + Start() error + Wait() error + + StdinPipe() (io.WriteCloser, error) + StdoutPipe() (io.ReadCloser, error) + StderrPipe() (io.ReadCloser, error) + + Kill() +} + +type Commander interface { + NewCommand() Command +} + +// Necessary information to create a new command +type CommandInfo struct { + Prog string + Args []string + Env []string +} + +type killCmd struct { + *exec.Cmd +} + +func (k killCmd) Kill() { + if k.Process != nil { + k.Process.Kill() + } +} + +// Create a new Command using golang exec package and the information. +func (ci CommandInfo) NewCommand() Command { + cmd := exec.Command(ci.Prog, ci.Args...) + cmd.Env = ci.Env + return killCmd{Cmd: cmd} +} diff --git a/command/test/command_helper.go b/command/test/command_helper.go new file mode 100644 index 000000000..416b516bf --- /dev/null +++ b/command/test/command_helper.go @@ -0,0 +1,141 @@ +package test + +import ( + "bufio" + "io" + + "github.com/influxdata/kapacitor/command" + "github.com/influxdata/kapacitor/udf" +) + +type CommandHelper struct { + inr *io.PipeReader + inw *io.PipeWriter + + outr *io.PipeReader + outw *io.PipeWriter + + errr *io.PipeReader + errw *io.PipeWriter + + Requests chan *udf.Request + Responses chan *udf.Response + ErrC chan error +} + +func NewCommandHelper() *CommandHelper { + cmd := &CommandHelper{ + Requests: make(chan *udf.Request), + Responses: make(chan *udf.Response), + ErrC: make(chan error, 1), + } + return cmd +} + +func (c *CommandHelper) NewCommand() command.Command { + return c +} + +// Forcefully kill the command. +// This will likely cause a panic. +func (c *CommandHelper) Kill() { + close(c.Requests) + close(c.Responses) + close(c.ErrC) + c.inr.Close() + c.inw.Close() + c.outr.Close() + c.outw.Close() + c.errr.Close() + c.errw.Close() +} + +func (c *CommandHelper) readRequests() error { + defer c.inr.Close() + defer close(c.Requests) + buf := bufio.NewReader(c.inr) + var b []byte + for { + req := &udf.Request{} + err := udf.ReadMessage(&b, buf, req) + if err == io.EOF { + return nil + } + if err != nil { + return err + } + c.Requests <- req + } +} + +func (c *CommandHelper) writeResponses() error { + defer c.outw.Close() + for response := range c.Responses { + udf.WriteMessage(response, c.outw) + } + return nil +} + +func (c *CommandHelper) Start() error { + go func() { + readErrC := make(chan error, 1) + writeErrC := make(chan error, 1) + go func() { + readErrC <- c.readRequests() + }() + go func() { + writeErrC <- c.writeResponses() + }() + var readErr, writeErr error + for readErrC != nil || writeErrC != nil { + select { + case readErr = <-readErrC: + readErrC = nil + case writeErr = <-writeErrC: + writeErrC = nil + } + } + + if readErr != nil { + c.ErrC <- readErr + } else { + c.ErrC <- writeErr + } + }() + return nil +} + +func (c *CommandHelper) Wait() error { + return nil +} + +// Wrapps the STDIN pipe and when it is closed +// closes the STDOUT and STDERR pipes of the command. +type cmdCloser struct { + *io.PipeWriter + cmd *CommandHelper +} + +func (cc *cmdCloser) Close() error { + cc.cmd.errw.Close() + return cc.PipeWriter.Close() +} + +func (c *CommandHelper) StdinPipe() (io.WriteCloser, error) { + c.inr, c.inw = io.Pipe() + closer := &cmdCloser{ + PipeWriter: c.inw, + cmd: c, + } + return closer, nil +} + +func (c *CommandHelper) StdoutPipe() (io.ReadCloser, error) { + c.outr, c.outw = io.Pipe() + return c.outr, nil +} + +func (c *CommandHelper) StderrPipe() (io.ReadCloser, error) { + c.errr, c.errw = io.Pipe() + return c.errr, nil +} diff --git a/derivative.go b/derivative.go index 0f32e320f..e4f5e35bd 100644 --- a/derivative.go +++ b/derivative.go @@ -1,6 +1,7 @@ package kapacitor import ( + "log" "time" "github.com/influxdata/kapacitor/models" @@ -13,9 +14,9 @@ type DerivativeNode struct { } // Create a new derivative node. -func newDerivativeNode(et *ExecutingTask, n *pipeline.DerivativeNode) (*DerivativeNode, error) { +func newDerivativeNode(et *ExecutingTask, n *pipeline.DerivativeNode, l *log.Logger) (*DerivativeNode, error) { dn := &DerivativeNode{ - node: node{Node: n, et: et}, + node: node{Node: n, et: et, logger: l}, d: n, } // Create stateful expressions @@ -23,7 +24,7 @@ func newDerivativeNode(et *ExecutingTask, n *pipeline.DerivativeNode) (*Derivati return dn, nil } -func (d *DerivativeNode) runDerivative() error { +func (d *DerivativeNode) runDerivative([]byte) error { switch d.Provides() { case pipeline.StreamEdge: previous := make(map[models.GroupID]models.Point) diff --git a/edge.go b/edge.go index d24d5f266..36f67aade 100644 --- a/edge.go +++ b/edge.go @@ -8,7 +8,6 @@ import ( "github.com/influxdata/kapacitor/models" "github.com/influxdata/kapacitor/pipeline" - "github.com/influxdata/kapacitor/wlog" ) const ( @@ -70,7 +69,7 @@ func (e *Edge) collectedCount() string { // collect calls to the edge have finished. func (e *Edge) Close() { e.logger.Printf( - "I! closing c: %s e: %s\n", + "D! closing c: %s e: %s\n", e.statMap.Get(statCollected), e.statMap.Get(statEmitted), ) @@ -104,14 +103,6 @@ func (e *Edge) Next() (p models.PointInterface, ok bool) { } func (e *Edge) NextPoint() (p models.Point, ok bool) { - if wlog.LogLevel() == wlog.DEBUG { - // Explicitly check log level since this is expensive and frequent - e.logger.Printf( - "D! next point c: %s e: %s\n", - e.statMap.Get(statCollected), - e.statMap.Get(statEmitted), - ) - } select { case <-e.aborted: case p, ok = <-e.stream: @@ -123,14 +114,6 @@ func (e *Edge) NextPoint() (p models.Point, ok bool) { } func (e *Edge) NextBatch() (b models.Batch, ok bool) { - if wlog.LogLevel() == wlog.DEBUG { - // Explicitly check log level since this is expensive and frequent - e.logger.Printf( - "D! next batch c: %s e: %s\n", - e.statMap.Get(statCollected), - e.statMap.Get(statEmitted), - ) - } select { case <-e.aborted: case b, ok = <-e.batch: @@ -142,14 +125,6 @@ func (e *Edge) NextBatch() (b models.Batch, ok bool) { } func (e *Edge) NextMaps() (m *MapResult, ok bool) { - if wlog.LogLevel() == wlog.DEBUG { - // Explicitly check log level since this is expensive and frequent - e.logger.Printf( - "D! next maps c: %s e: %s\n", - e.statMap.Get(statCollected), - e.statMap.Get(statEmitted), - ) - } select { case <-e.aborted: case m, ok = <-e.reduce: @@ -161,14 +136,6 @@ func (e *Edge) NextMaps() (m *MapResult, ok bool) { } func (e *Edge) CollectPoint(p models.Point) error { - if wlog.LogLevel() == wlog.DEBUG { - // Explicitly check log level since this is expensive and frequent - e.logger.Printf( - "D! collect point c: %s e: %s\n", - e.statMap.Get(statCollected), - e.statMap.Get(statEmitted), - ) - } e.statMap.Add(statCollected, 1) select { case <-e.aborted: @@ -179,14 +146,6 @@ func (e *Edge) CollectPoint(p models.Point) error { } func (e *Edge) CollectBatch(b models.Batch) error { - if wlog.LogLevel() == wlog.DEBUG { - // Explicitly check log level since this is expensive and frequent - e.logger.Printf( - "D! collect batch c: %s e: %s\n", - e.statMap.Get(statCollected), - e.statMap.Get(statEmitted), - ) - } e.statMap.Add(statCollected, 1) select { case <-e.aborted: @@ -197,14 +156,6 @@ func (e *Edge) CollectBatch(b models.Batch) error { } func (e *Edge) CollectMaps(m *MapResult) error { - if wlog.LogLevel() == wlog.DEBUG { - // Explicitly check log level since this is expensive and frequent - e.logger.Printf( - "D! collect maps c: %s e: %s\n", - e.statMap.Get(statCollected), - e.statMap.Get(statEmitted), - ) - } e.statMap.Add(statCollected, 1) select { case <-e.aborted: diff --git a/etc/kapacitor/kapacitor.conf b/etc/kapacitor/kapacitor.conf index 0765a5cb0..7d00492b8 100644 --- a/etc/kapacitor/kapacitor.conf +++ b/etc/kapacitor/kapacitor.conf @@ -33,6 +33,8 @@ data_dir = "/var/lib/kapacitor" [task] # Where to store the tasks database dir = "/var/lib/kapacitor/tasks" + # How often to snapshot running task state. + snapshot-interval = 60s [influxdb] # Connect to an InfluxDB cluster @@ -185,6 +187,41 @@ data_dir = "/var/lib/kapacitor" enabled = true url = "https://usage.influxdata.com" +[udf] +# Configuration for UDFs (User Defined Functions) +[udf.functions] + # Example go UDF. + # First compile example: + # go build -o avg_udf ./udf/agent/examples/moving_avg.go + # + # Use in TICKscript like: + # stream.goavg() + # .field('value') + # .size(10) + # .as('m_average') + # + # uncomment to enable + #[udf.functions.goavg] + # prog = "./avg_udf" + # args = [] + # timeout = "10s" + + # Example python UDF. + # Use in TICKscript like: + # stream.pyavg() + # .field('value') + # .size(10) + # .as('m_average') + # + # uncomment to enable + #[udf.functions.pyavg] + # prog = "/usr/bin/python2" + # args = ["-u", "./udf/agent/examples/moving_avg.py"] + # timeout = "10s" + # [udf.functions.pyavg.env] + # PYTHONPATH = "./udf/agent/py" + + ################################## # Input Methods, same as InfluxDB # diff --git a/eval.go b/eval.go index dc1fa8648..e342da93e 100644 --- a/eval.go +++ b/eval.go @@ -2,6 +2,7 @@ package kapacitor import ( "errors" + "log" "github.com/influxdata/kapacitor/models" "github.com/influxdata/kapacitor/pipeline" @@ -14,13 +15,13 @@ type EvalNode struct { expressions []*tick.StatefulExpr } -// Create a new ApplyNode which applies a transformation func to each point in a stream and returns a single point. -func newApplyNode(et *ExecutingTask, n *pipeline.EvalNode) (*EvalNode, error) { +// Create a new EvalNode which applies a transformation func to each point in a stream and returns a single point. +func newEvalNode(et *ExecutingTask, n *pipeline.EvalNode, l *log.Logger) (*EvalNode, error) { if len(n.AsList) != len(n.Expressions) { return nil, errors.New("must provide one name per expression via the 'As' property") } en := &EvalNode{ - node: node{Node: n, et: et}, + node: node{Node: n, et: et, logger: l}, e: n, } // Create stateful expressions @@ -29,11 +30,11 @@ func newApplyNode(et *ExecutingTask, n *pipeline.EvalNode) (*EvalNode, error) { en.expressions[i] = tick.NewStatefulExpr(expr) } - en.node.runF = en.runApply + en.node.runF = en.runEval return en, nil } -func (e *EvalNode) runApply() error { +func (e *EvalNode) runEval(snapshot []byte) error { switch e.Provides() { case pipeline.StreamEdge: for p, ok := e.ins[0].NextPoint(); ok; p, ok = e.ins[0].NextPoint() { diff --git a/group_by.go b/group_by.go index a50e6b624..aadc476eb 100644 --- a/group_by.go +++ b/group_by.go @@ -1,6 +1,7 @@ package kapacitor import ( + "log" "sort" "github.com/influxdata/kapacitor/models" @@ -16,9 +17,9 @@ type GroupByNode struct { } // Create a new GroupByNode which splits the stream dynamically based on the specified dimensions. -func newGroupByNode(et *ExecutingTask, n *pipeline.GroupByNode) (*GroupByNode, error) { +func newGroupByNode(et *ExecutingTask, n *pipeline.GroupByNode, l *log.Logger) (*GroupByNode, error) { gn := &GroupByNode{ - node: node{Node: n, et: et}, + node: node{Node: n, et: et, logger: l}, g: n, } gn.node.runF = gn.runGroupBy @@ -27,7 +28,7 @@ func newGroupByNode(et *ExecutingTask, n *pipeline.GroupByNode) (*GroupByNode, e return gn, nil } -func (g *GroupByNode) runGroupBy() error { +func (g *GroupByNode) runGroupBy([]byte) error { switch g.Wants() { case pipeline.StreamEdge: for pt, ok := g.ins[0].NextPoint(); ok; pt, ok = g.ins[0].NextPoint() { diff --git a/http_out.go b/http_out.go index f6c5dcd2e..5dd3974a4 100644 --- a/http_out.go +++ b/http_out.go @@ -2,6 +2,7 @@ package kapacitor import ( "encoding/json" + "log" "net/http" "path" "sync" @@ -24,9 +25,9 @@ type HTTPOutNode struct { } // Create a new HTTPOutNode which caches the most recent item and exposes it over the HTTP API. -func newHTTPOutNode(et *ExecutingTask, n *pipeline.HTTPOutNode) (*HTTPOutNode, error) { +func newHTTPOutNode(et *ExecutingTask, n *pipeline.HTTPOutNode, l *log.Logger) (*HTTPOutNode, error) { hn := &HTTPOutNode{ - node: node{Node: n, et: et}, + node: node{Node: n, et: et, logger: l}, c: n, groupSeriesIdx: make(map[models.GroupID]int), } @@ -40,7 +41,7 @@ func (h *HTTPOutNode) Endpoint() string { return h.endpoint } -func (h *HTTPOutNode) runOut() error { +func (h *HTTPOutNode) runOut([]byte) error { hndl := func(w http.ResponseWriter, req *http.Request) { h.mu.RLock() diff --git a/influxdb_out.go b/influxdb_out.go index 81c2560c1..f7fbf2a99 100644 --- a/influxdb_out.go +++ b/influxdb_out.go @@ -1,6 +1,8 @@ package kapacitor import ( + "log" + "github.com/influxdata/kapacitor/models" "github.com/influxdata/kapacitor/pipeline" "github.com/influxdb/influxdb/client" @@ -12,16 +14,16 @@ type InfluxDBOutNode struct { conn *client.Client } -func newInfluxDBOutNode(et *ExecutingTask, n *pipeline.InfluxDBOutNode) (*InfluxDBOutNode, error) { +func newInfluxDBOutNode(et *ExecutingTask, n *pipeline.InfluxDBOutNode, l *log.Logger) (*InfluxDBOutNode, error) { in := &InfluxDBOutNode{ - node: node{Node: n, et: et}, + node: node{Node: n, et: et, logger: l}, i: n, } in.node.runF = in.runOut return in, nil } -func (i *InfluxDBOutNode) runOut() error { +func (i *InfluxDBOutNode) runOut([]byte) error { switch i.Wants() { case pipeline.StreamEdge: for p, ok := i.ins[0].NextPoint(); ok; p, ok = i.ins[0].NextPoint() { diff --git a/integrations/batcher_test.go b/integrations/batcher_test.go index 89bd7f180..2134b0d40 100644 --- a/integrations/batcher_test.go +++ b/integrations/batcher_test.go @@ -309,8 +309,15 @@ func testBatcher(t *testing.T, name, script string) (clock.Setter, *kapacitor.Ex wlog.SetLevel(wlog.OFF) } + // Create a new execution env + tm := kapacitor.NewTaskMaster(logService) + tm.HTTPDService = httpService + tm.TaskStore = taskStore{} + tm.Open() + scope := tm.CreateTICKScope() + // Create task - task, err := kapacitor.NewBatcher(name, script, dbrps) + task, err := kapacitor.NewTask(name, script, kapacitor.BatchTask, dbrps, 0, scope) if err != nil { t.Fatal(err) } @@ -334,11 +341,6 @@ func testBatcher(t *testing.T, name, script string) (clock.Setter, *kapacitor.Ex c := clock.New(time.Date(1971, 1, 1, 0, 0, 0, 0, time.UTC)) r := kapacitor.NewReplay(c) - // Create a new execution env - tm := kapacitor.NewTaskMaster(logService) - tm.HTTPDService = httpService - tm.Open() - //Start the task et, err := tm.StartTask(task) if err != nil { diff --git a/integrations/data/TestStream_CustomFunctions.srpl b/integrations/data/TestStream_CustomFunctions.srpl new file mode 100644 index 000000000..d327ab258 --- /dev/null +++ b/integrations/data/TestStream_CustomFunctions.srpl @@ -0,0 +1,81 @@ +dbname +rpname +cpu,type=idle,host=serverA value=97.1 0000000001 +dbname +rpname +cpu,type=idle,host=serverB value=97.1 0000000001 +dbname +rpname +disk,type=sda,host=serverB value=39 0000000001 +dbname +rpname +cpu,type=idle,host=serverA value=92.6 0000000002 +dbname +rpname +cpu,type=idle,host=serverB value=92.6 0000000002 +dbname +rpname +cpu,type=idle,host=serverA value=95.6 0000000003 +dbname +rpname +cpu,type=idle,host=serverB value=95.6 0000000003 +dbname +rpname +cpu,type=idle,host=serverA value=93.1 0000000004 +dbname +rpname +cpu,type=idle,host=serverB value=93.1 0000000004 +dbname +rpname +cpu,type=idle,host=serverA value=92.6 0000000005 +dbname +rpname +cpu,type=idle,host=serverB value=92.6 0000000005 +dbname +rpname +cpu,type=idle,host=serverA value=95.8 0000000006 +dbname +rpname +cpu,type=idle,host=serverB value=95.8 0000000006 +dbname +rpname +cpu,type=idle,host=serverC value=95.8 0000000006 +dbname +rpname +cpu,type=idle,host=serverA value=92.7 0000000007 +dbname +rpname +cpu,type=idle,host=serverB value=92.7 0000000007 +dbname +rpname +cpu,type=idle,host=serverA value=96.0 0000000008 +dbname +rpname +cpu,type=idle,host=serverB value=96.0 0000000008 +dbname +rpname +cpu,type=idle,host=serverA value=93.4 0000000009 +dbname +rpname +cpu,type=idle,host=serverB value=93.4 0000000009 +dbname +rpname +disk,type=sda,host=serverB value=423 0000000009 +dbname +rpname +cpu,type=idle,host=serverA value=95.3 0000000010 +dbname +rpname +cpu,type=idle,host=serverB value=95.3 0000000010 +dbname +rpname +cpu,type=idle,host=serverA value=96.4 0000000011 +dbname +rpname +cpu,type=idle,host=serverB value=96.4 0000000011 +dbname +rpname +cpu,type=idle,host=serverA value=95.1 0000000012 +dbname +rpname +cpu,type=idle,host=serverB value=95.1 0000000012 diff --git a/integrations/helpers_test.go b/integrations/helpers_test.go index 933627997..f052e2f1e 100644 --- a/integrations/helpers_test.go +++ b/integrations/helpers_test.go @@ -1,6 +1,7 @@ package integrations import ( + "errors" "fmt" "log" "net/http" @@ -104,3 +105,24 @@ type LogService struct{} func (l *LogService) NewLogger(prefix string, flag int) *log.Logger { return wlog.New(os.Stderr, prefix, flag) } + +type UDFService struct { + FunctionListFunc func() []string + FunctionInfoFunc func(name string) (kapacitor.UDFProcessInfo, bool) +} + +func (u UDFService) FunctionList() []string { + return u.FunctionListFunc() +} + +func (u UDFService) FunctionInfo(name string) (kapacitor.UDFProcessInfo, bool) { + return u.FunctionInfoFunc(name) +} + +type taskStore struct{} + +func (ts taskStore) SaveSnapshot(name string, snapshot *kapacitor.TaskSnapshot) error { return nil } +func (ts taskStore) HasSnapshot(name string) bool { return false } +func (ts taskStore) LoadSnapshot(name string) (*kapacitor.TaskSnapshot, error) { + return nil, errors.New("not implemented") +} diff --git a/integrations/streamer_test.go b/integrations/streamer_test.go index 5ac32f1e3..ea6cd7fe1 100644 --- a/integrations/streamer_test.go +++ b/integrations/streamer_test.go @@ -9,12 +9,15 @@ import ( "net/http/httptest" "os" "path" + "reflect" "testing" "text/template" "time" "github.com/influxdata/kapacitor" "github.com/influxdata/kapacitor/clock" + cmd_test "github.com/influxdata/kapacitor/command/test" + "github.com/influxdata/kapacitor/pipeline" "github.com/influxdata/kapacitor/services/alerta" "github.com/influxdata/kapacitor/services/hipchat" "github.com/influxdata/kapacitor/services/httpd" @@ -22,6 +25,7 @@ import ( "github.com/influxdata/kapacitor/services/pagerduty" "github.com/influxdata/kapacitor/services/slack" "github.com/influxdata/kapacitor/services/victorops" + "github.com/influxdata/kapacitor/udf" "github.com/influxdata/kapacitor/wlog" "github.com/influxdb/influxdb/client" imodels "github.com/influxdb/influxdb/models" @@ -75,7 +79,7 @@ stream }, } - testStreamerWithOutput(t, "TestStream_Derivative", script, 15*time.Second, er) + testStreamerWithOutput(t, "TestStream_Derivative", script, 15*time.Second, er, nil, false) } func TestStream_DerivativeUnit(t *testing.T) { @@ -105,7 +109,7 @@ stream }, } - testStreamerWithOutput(t, "TestStream_Derivative", script, 15*time.Second, er) + testStreamerWithOutput(t, "TestStream_Derivative", script, 15*time.Second, er, nil, false) } func TestStream_DerivativeNN(t *testing.T) { @@ -135,7 +139,7 @@ stream }, } - testStreamerWithOutput(t, "TestStream_DerivativeNN", script, 15*time.Second, er) + testStreamerWithOutput(t, "TestStream_DerivativeNN", script, 15*time.Second, er, nil, false) } func TestStream_DerivativeN(t *testing.T) { @@ -164,7 +168,7 @@ stream }, } - testStreamerWithOutput(t, "TestStream_DerivativeNN", script, 15*time.Second, er) + testStreamerWithOutput(t, "TestStream_DerivativeNN", script, 15*time.Second, er, nil, false) } func TestStream_Window(t *testing.T) { @@ -218,7 +222,7 @@ stream }, } - testStreamerWithOutput(t, "TestStream_Window", script, 13*time.Second, er) + testStreamerWithOutput(t, "TestStream_Window", script, 13*time.Second, er, nil, false) } func TestStream_SimpleMR(t *testing.T) { @@ -247,7 +251,7 @@ stream }, } - testStreamerWithOutput(t, "TestStream_SimpleMR", script, 15*time.Second, er) + testStreamerWithOutput(t, "TestStream_SimpleMR", script, 15*time.Second, er, nil, false) } func TestStream_SimpleWhere(t *testing.T) { @@ -277,7 +281,7 @@ stream }, } - testStreamerWithOutput(t, "TestStream_SimpleMR", script, 15*time.Second, er) + testStreamerWithOutput(t, "TestStream_SimpleMR", script, 15*time.Second, er, nil, false) } func TestStream_VarWhereString(t *testing.T) { @@ -307,7 +311,7 @@ stream }, } - testStreamerWithOutput(t, "TestStream_SimpleMR", script, 15*time.Second, er) + testStreamerWithOutput(t, "TestStream_SimpleMR", script, 15*time.Second, er, nil, false) } func TestStream_VarWhereRegex(t *testing.T) { @@ -337,7 +341,7 @@ stream }, } - testStreamerWithOutput(t, "TestStream_SimpleMR", script, 15*time.Second, er) + testStreamerWithOutput(t, "TestStream_SimpleMR", script, 15*time.Second, er, nil, false) } func TestStream_GroupBy(t *testing.T) { @@ -385,7 +389,7 @@ stream }, } - testStreamerWithOutput(t, "TestStream_GroupBy", script, 13*time.Second, er) + testStreamerWithOutput(t, "TestStream_GroupBy", script, 13*time.Second, er, nil, false) } func TestStream_Join(t *testing.T) { @@ -456,7 +460,7 @@ errorCounts.join(viewCounts) }, } - testStreamerWithOutput(t, "TestStream_Join", script, 13*time.Second, er, true) + testStreamerWithOutput(t, "TestStream_Join", script, 13*time.Second, er, nil, true) } func TestStream_JoinTolerance(t *testing.T) { @@ -516,7 +520,7 @@ errorCounts.join(viewCounts) }, } - testStreamerWithOutput(t, "TestStream_JoinTolerance", script, 13*time.Second, er, true) + testStreamerWithOutput(t, "TestStream_JoinTolerance", script, 13*time.Second, er, nil, true) } func TestStream_JoinFill(t *testing.T) { @@ -574,7 +578,7 @@ errorCounts.join(viewCounts) }, } - testStreamerWithOutput(t, "TestStream_JoinFill", script, 13*time.Second, er, true) + testStreamerWithOutput(t, "TestStream_JoinFill", script, 13*time.Second, er, nil, true) } func TestStream_JoinN(t *testing.T) { @@ -615,7 +619,7 @@ cpu.join(mem, disk) }, } - testStreamerWithOutput(t, "TestStream_JoinN", script, 15*time.Second, er) + testStreamerWithOutput(t, "TestStream_JoinN", script, 15*time.Second, er, nil, false) } func TestStream_Union(t *testing.T) { @@ -654,7 +658,7 @@ cpu.union(mem, disk) }, } - testStreamerWithOutput(t, "TestStream_Union", script, 15*time.Second, er) + testStreamerWithOutput(t, "TestStream_Union", script, 15*time.Second, er, nil, false) } func TestStream_Aggregations(t *testing.T) { @@ -1104,48 +1108,162 @@ stream string(script.Bytes()), 13*time.Second, tc.ER, + nil, + false, ) } } func TestStream_CustomFunctions(t *testing.T) { - t.Skip() var script = ` -var fMap = loadMapFunc('./TestCustomMapFunction.py') -var fReduce = loadReduceFunc('./TestCustomReduceFunction.py') stream .from().measurement('cpu') .where(lambda: "host" == 'serverA') .window() - .period(1s) - .every(1s) - .map(fMap, 'idle') - .reduce(fReduce) - .cache() + .period(10s) + .every(10s) + .mapReduce(influxql.count('value')) + .customFunc() + .opt1('count') + .opt2(FALSE, 1, 1.0, '1.0', 1s) + .httpOut('TestStream_CustomFunctions') ` - //er := kapacitor.Result{} + cmd := cmd_test.NewCommandHelper() + udfService := UDFService{} + udfService.FunctionListFunc = func() []string { + return []string{"customFunc"} + } + udfService.FunctionInfoFunc = func(name string) (info kapacitor.UDFProcessInfo, ok bool) { + if name != "customFunc" { + return + } + info.Commander = cmd + info.Wants = pipeline.StreamEdge + info.Provides = pipeline.StreamEdge + info.Options = map[string]*udf.OptionInfo{ + "opt1": { + ValueTypes: []udf.ValueType{udf.ValueType_STRING}, + }, + "opt2": { + ValueTypes: []udf.ValueType{ + udf.ValueType_BOOL, + udf.ValueType_INT, + udf.ValueType_DOUBLE, + udf.ValueType_STRING, + udf.ValueType_DURATION, + }, + }, + } + return + } - testStreamer(t, "TestStream_CustomFunctions", script) -} + done := make(chan struct{}) + go func() { + defer close(done) + req := <-cmd.Requests + i, ok := req.Message.(*udf.Request_Init) + if !ok { + t.Error("expected init message") + } + init := i.Init -func TestStream_CustomMRFunction(t *testing.T) { - t.Skip() - var script = ` -var fMapReduce = loadMapReduceFunc('./TestCustomMapReduceFunction.py') -stream - .from().measurement('cpu') - .where(lambda: "host" = 'serverA') - .window() - .period(1s) - .every(1s) - .mapReduce(fMap, 'idle') - .cache() -` + if got, exp := len(init.Options), 2; got != exp { + t.Fatalf("unexpected number of options in init request, got %d exp %d", got, exp) + } + for i, opt := range init.Options { + exp := &udf.Option{} + switch i { + case 0: + exp.Name = "opt1" + exp.Values = []*udf.OptionValue{ + { + Type: udf.ValueType_STRING, + Value: &udf.OptionValue_StringValue{"count"}, + }, + } + case 1: + exp.Name = "opt2" + exp.Values = []*udf.OptionValue{ + { + Type: udf.ValueType_BOOL, + Value: &udf.OptionValue_BoolValue{false}, + }, + { + Type: udf.ValueType_INT, + Value: &udf.OptionValue_IntValue{1}, + }, + { + Type: udf.ValueType_DOUBLE, + Value: &udf.OptionValue_DoubleValue{1.0}, + }, + { + Type: udf.ValueType_STRING, + Value: &udf.OptionValue_StringValue{"1.0"}, + }, + { + Type: udf.ValueType_DURATION, + Value: &udf.OptionValue_DurationValue{int64(time.Second)}, + }, + } + } + if !reflect.DeepEqual(exp, opt) { + t.Errorf("unexpected init option %d\ngot %v\nexp %v", i, opt, exp) + } + } + + resp := &udf.Response{ + Message: &udf.Response_Init{ + Init: &udf.InitResponse{ + Success: true, + }, + }, + } + cmd.Responses <- resp + + // read all requests and wait till the chan is closed + for req := range cmd.Requests { + p, ok := req.Message.(*udf.Request_Point) + if ok { + pt := p.Point + resp := &udf.Response{ + Message: &udf.Response_Point{ + Point: &udf.Point{ + Name: pt.Name, + Time: pt.Time, + Group: pt.Group, + Tags: pt.Tags, + FieldsDouble: map[string]float64{"customField": 42.0}, + }, + }, + } + cmd.Responses <- resp + } + } - //er := kapacitor.Result{} + close(cmd.Responses) - testStreamer(t, "TestStream_CustomMRFunction", script) + if err := <-cmd.ErrC; err != nil { + t.Error(err) + } + }() + + er := kapacitor.Result{ + Series: imodels.Rows{ + { + Name: "cpu", + Tags: nil, + Columns: []string{"time", "customField"}, + Values: [][]interface{}{[]interface{}{ + time.Date(1971, 1, 1, 0, 0, 10, 0, time.UTC), + 42.0, + }}, + }, + }, + } + + testStreamerWithOutput(t, "TestStream_CustomFunctions", script, 15*time.Second, er, udfService, false) + <-done } func TestStream_Alert(t *testing.T) { @@ -1264,7 +1382,7 @@ stream .channel('@jim') ` - clock, et, replayErr, tm := testStreamer(t, "TestStream_Alert", script) + clock, et, replayErr, tm := testStreamer(t, "TestStream_Alert", script, nil) defer tm.Close() c := slack.NewConfig() @@ -1343,7 +1461,7 @@ stream .token('testtokenTestRoom') ` - clock, et, replayErr, tm := testStreamer(t, "TestStream_Alert", script) + clock, et, replayErr, tm := testStreamer(t, "TestStream_Alert", script, nil) defer tm.Close() c := hipchat.NewConfig() @@ -1439,7 +1557,7 @@ stream .origin('override') ` - clock, et, replayErr, tm := testStreamer(t, "TestStream_Alert", script) + clock, et, replayErr, tm := testStreamer(t, "TestStream_Alert", script, nil) defer tm.Close() c := alerta.NewConfig() @@ -1558,7 +1676,7 @@ stream .recipients('test_recipient2', 'another_recipient') ` - clock, et, replayErr, tm := testStreamer(t, "TestStream_Alert", script) + clock, et, replayErr, tm := testStreamer(t, "TestStream_Alert", script, nil) defer tm.Close() c := opsgenie.NewConfig() c.URL = ts.URL @@ -1631,7 +1749,7 @@ stream .pagerDuty() ` - clock, et, replayErr, tm := testStreamer(t, "TestStream_Alert", script) + clock, et, replayErr, tm := testStreamer(t, "TestStream_Alert", script, nil) defer tm.Close() c := pagerduty.NewConfig() c.URL = ts.URL @@ -1715,7 +1833,7 @@ stream .routingKey('test_key2') ` - clock, et, replayErr, tm := testStreamer(t, "TestStream_Alert", script) + clock, et, replayErr, tm := testStreamer(t, "TestStream_Alert", script, nil) defer tm.Close() c := victorops.NewConfig() c.URL = ts.URL @@ -1770,7 +1888,7 @@ stream .post('` + ts.URL + `') ` - clock, et, replayErr, tm := testStreamer(t, "TestStream_AlertSigma", script) + clock, et, replayErr, tm := testStreamer(t, "TestStream_AlertSigma", script, nil) defer tm.Close() // Move time forward @@ -1917,7 +2035,7 @@ stream done <- err })) - clock, et, replayErr, tm := testStreamer(t, "TestStream_InfluxDBOut", script) + clock, et, replayErr, tm := testStreamer(t, "TestStream_InfluxDBOut", script, nil) tm.InfluxDBService = influxdb defer tm.Close() @@ -2040,7 +2158,7 @@ topScores.sample(4s) }, } - clock, et, replayErr, tm := testStreamer(t, "TestStream_TopSelector", script) + clock, et, replayErr, tm := testStreamer(t, "TestStream_TopSelector", script, nil) defer tm.Close() err := fastForwardTask(clock, et, replayErr, tm, 10*time.Second) @@ -2088,6 +2206,7 @@ func testStreamer( t *testing.T, name, script string, + udfService kapacitor.UDFService, ) ( clock.Setter, *kapacitor.ExecutingTask, @@ -2100,8 +2219,16 @@ func testStreamer( wlog.SetLevel(wlog.OFF) } + // Create a new execution env + tm := kapacitor.NewTaskMaster(logService) + tm.HTTPDService = httpService + tm.UDFService = udfService + tm.TaskStore = taskStore{} + tm.Open() + scope := tm.CreateTICKScope() + //Create the task - task, err := kapacitor.NewStreamer(name, script, dbrps) + task, err := kapacitor.NewTask(name, script, kapacitor.StreamTask, dbrps, 0, scope) if err != nil { t.Fatal(err) } @@ -2119,11 +2246,6 @@ func testStreamer( c := clock.New(time.Date(1971, 1, 1, 0, 0, 0, 0, time.UTC)) r := kapacitor.NewReplay(c) - // Create a new execution env - tm := kapacitor.NewTaskMaster(logService) - tm.HTTPDService = httpService - tm.Open() - //Start the task et, err := tm.StartTask(task) if err != nil { @@ -2168,7 +2290,7 @@ func testStreamerNoOutput( script string, duration time.Duration, ) { - clock, et, replayErr, tm := testStreamer(t, name, script) + clock, et, replayErr, tm := testStreamer(t, name, script, nil) err := fastForwardTask(clock, et, replayErr, tm, duration) if err != nil { t.Error(err) @@ -2182,9 +2304,10 @@ func testStreamerWithOutput( script string, duration time.Duration, er kapacitor.Result, - ignoreOrder ...bool, + udfService kapacitor.UDFService, + ignoreOrder bool, ) { - clock, et, replayErr, tm := testStreamer(t, name, script) + clock, et, replayErr, tm := testStreamer(t, name, script, udfService) err := fastForwardTask(clock, et, replayErr, tm, duration) if err != nil { t.Error(err) @@ -2204,7 +2327,7 @@ func testStreamerWithOutput( // Assert we got the expected result result := kapacitor.ResultFromJSON(resp.Body) - if len(ignoreOrder) > 0 && ignoreOrder[0] { + if ignoreOrder { if eq, msg := compareResultsIgnoreSeriesOrder(er, result); !eq { t.Error(msg) } diff --git a/join.go b/join.go index f67b70767..f8b853852 100644 --- a/join.go +++ b/join.go @@ -23,7 +23,7 @@ type JoinNode struct { } // Create a new JoinNode, which takes pairs from parent streams combines them into a single point. -func newJoinNode(et *ExecutingTask, n *pipeline.JoinNode) (*JoinNode, error) { +func newJoinNode(et *ExecutingTask, n *pipeline.JoinNode, l *log.Logger) (*JoinNode, error) { for _, name := range n.Names { if len(name) == 0 { return nil, fmt.Errorf("must provide a prefix name for the join node, see .as() property method") @@ -42,7 +42,7 @@ func newJoinNode(et *ExecutingTask, n *pipeline.JoinNode) (*JoinNode, error) { jn := &JoinNode{ j: n, - node: node{Node: n, et: et}, + node: node{Node: n, et: et, logger: l}, } // Set fill switch fill := n.Fill.(type) { @@ -65,7 +65,7 @@ func newJoinNode(et *ExecutingTask, n *pipeline.JoinNode) (*JoinNode, error) { return jn, nil } -func (j *JoinNode) runJoin() error { +func (j *JoinNode) runJoin([]byte) error { j.groups = make(map[models.GroupID]*group) wg := sync.WaitGroup{} for i := range j.ins { @@ -107,7 +107,6 @@ func (j *JoinNode) getGroup(p models.PointInterface) *group { group = newGroup(len(j.ins), j) j.groups[p.PointGroup()] = group j.runningGroups.Add(1) - j.logger.Println("D! group started ") go group.run() } return group @@ -173,7 +172,6 @@ func (g *group) run() { for sp := range g.points { g.collect(sp.src, sp.p) } - g.j.logger.Println("D! group done ") } // collect a point from a given parent. diff --git a/map_reduce.go b/map_reduce.go index df002183b..7396726d9 100644 --- a/map_reduce.go +++ b/map_reduce.go @@ -2,6 +2,7 @@ package kapacitor import ( "fmt" + "log" "time" "github.com/influxdata/kapacitor/models" @@ -34,13 +35,13 @@ type MapNode struct { parallel int } -func newMapNode(et *ExecutingTask, n *pipeline.MapNode) (*MapNode, error) { +func newMapNode(et *ExecutingTask, n *pipeline.MapNode, l *log.Logger) (*MapNode, error) { f, ok := n.Map.(MapInfo) if !ok { return nil, fmt.Errorf("invalid map given to map node %T", n.Map) } m := &MapNode{ - node: node{Node: n, et: et}, + node: node{Node: n, et: et, logger: l}, mr: n, f: f.Func, field: f.Field, @@ -50,7 +51,7 @@ func newMapNode(et *ExecutingTask, n *pipeline.MapNode) (*MapNode, error) { return m, nil } -func (m *MapNode) runMaps() error { +func (m *MapNode) runMaps([]byte) error { switch m.mr.Wants() { case pipeline.StreamEdge: return m.runStreamMap() @@ -160,13 +161,13 @@ type ReduceNode struct { f ReduceFunc } -func newReduceNode(et *ExecutingTask, n *pipeline.ReduceNode) (*ReduceNode, error) { +func newReduceNode(et *ExecutingTask, n *pipeline.ReduceNode, l *log.Logger) (*ReduceNode, error) { f, ok := n.Reduce.(ReduceFunc) if !ok { return nil, fmt.Errorf("invalid func given to batch reduce node %T", n.Reduce) } b := &ReduceNode{ - node: node{Node: n, et: et}, + node: node{Node: n, et: et, logger: l}, r: n, f: f, } @@ -174,7 +175,7 @@ func newReduceNode(et *ExecutingTask, n *pipeline.ReduceNode) (*ReduceNode, erro return b, nil } -func (r *ReduceNode) runReduce() error { +func (r *ReduceNode) runReduce([]byte) error { for m, ok := r.ins[0].NextMaps(); ok; m, ok = r.ins[0].NextMaps() { rr := r.f(m.Outs, m.TMax, r.r.PointTimes, r.r.As) switch result := rr.(type) { diff --git a/node.go b/node.go index c803899b2..9f56d01c2 100644 --- a/node.go +++ b/node.go @@ -17,11 +17,12 @@ type Node interface { addParentEdge(*Edge) // start the node and its children - start() + start(snapshot []byte) stop() - // set the logger - setLogger(logger *log.Logger) + // snapshot running state + snapshot() ([]byte, error) + restore(snapshot []byte) error // wait for the node to finish processing and return any errors Err() error @@ -45,7 +46,7 @@ type node struct { et *ExecutingTask parents []Node children []Node - runF func() error + runF func(snapshot []byte) error stopF func() errCh chan error err error @@ -66,11 +67,7 @@ func (n *node) abortParentEdges() { } } -func (n *node) setLogger(l *log.Logger) { - n.logger = l -} - -func (n *node) start() { +func (n *node) start(snapshot []byte) { n.errCh = make(chan error, 1) go func() { var err error @@ -92,7 +89,7 @@ func (n *node) start() { n.errCh <- err }() // Run node - err = n.runF() + err = n.runF(snapshot) }() } @@ -103,6 +100,12 @@ func (n *node) stop() { } +// no-op snapshot +func (n *node) snapshot() (b []byte, err error) { return } + +// no-op restore +func (n *node) restore([]byte) error { return nil } + func (n *node) Err() error { n.finishedMu.Lock() defer n.finishedMu.Unlock() diff --git a/pipeline/udf.go b/pipeline/udf.go new file mode 100644 index 000000000..08241fa83 --- /dev/null +++ b/pipeline/udf.go @@ -0,0 +1,115 @@ +package pipeline + +import ( + "fmt" + "time" + + "github.com/influxdata/kapacitor/command" + "github.com/influxdata/kapacitor/tick" + "github.com/influxdata/kapacitor/udf" +) + +// A UDFNode is a User Defined Function. +// UDFs can be defined in the configuration file. in the [udf] section. +// +type UDFNode struct { + chainnode + + desc string + //tick:ignore + Commander command.Commander + // tick:ignore + Timeout time.Duration + + options map[string]*udf.OptionInfo + + // Options that were set on the node + // tick:ignore + Options []*udf.Option + + describer *tick.ReflectionDescriber +} + +func NewUDF( + parent Node, + name string, + commander command.Commander, + timeout time.Duration, + wants, + provides EdgeType, + options map[string]*udf.OptionInfo, +) *UDFNode { + udf := &UDFNode{ + chainnode: newBasicChainNode(name, wants, provides), + desc: name, + Commander: commander, + Timeout: timeout, + options: options, + } + udf.describer = tick.NewReflectionDescriber(udf) + parent.linkChild(udf) + return udf +} + +func (u *UDFNode) Desc() string { + return u.desc +} + +func (u *UDFNode) HasMethod(name string) bool { + _, ok := u.options[name] + if ok { + return ok + } + return u.describer.HasMethod(name) +} + +func (u *UDFNode) CallMethod(name string, args ...interface{}) (interface{}, error) { + opt, ok := u.options[name] + if ok { + if got, exp := len(args), len(opt.ValueTypes); got != exp { + return nil, fmt.Errorf("unexpected number of args to %s, got %d expected %d", name, got, exp) + } + values := make([]*udf.OptionValue, len(args)) + for i, arg := range args { + values[i] = &udf.OptionValue{} + switch v := arg.(type) { + case bool: + values[i].Type = udf.ValueType_BOOL + values[i].Value = &udf.OptionValue_BoolValue{v} + case int64: + values[i].Type = udf.ValueType_INT + values[i].Value = &udf.OptionValue_IntValue{v} + case float64: + values[i].Type = udf.ValueType_DOUBLE + values[i].Value = &udf.OptionValue_DoubleValue{v} + case string: + values[i].Type = udf.ValueType_STRING + values[i].Value = &udf.OptionValue_StringValue{v} + case time.Duration: + values[i].Type = udf.ValueType_DURATION + values[i].Value = &udf.OptionValue_DurationValue{int64(v)} + } + if values[i].Type != opt.ValueTypes[i] { + return nil, fmt.Errorf("unexpected arg to %s, got %v expected %v", name, values[i].Type, opt.ValueTypes[i]) + } + } + u.Options = append(u.Options, &udf.Option{ + Name: name, + Values: values, + }) + return u, nil + } + return u.describer.CallMethod(name, args...) +} + +func (u *UDFNode) HasProperty(name string) bool { + return u.describer.HasProperty(name) +} + +func (u *UDFNode) Property(name string) interface{} { + return u.describer.Property(name) +} + +func (u *UDFNode) SetProperty(name string, value interface{}) error { + return u.describer.SetProperty(name, value) +} diff --git a/sample.go b/sample.go index 60cf79efe..b4f1ac198 100644 --- a/sample.go +++ b/sample.go @@ -2,6 +2,7 @@ package kapacitor import ( "errors" + "log" "time" "github.com/influxdata/kapacitor/models" @@ -17,9 +18,9 @@ type SampleNode struct { } // Create a new SampleNode which filters data from a source. -func newSampleNode(et *ExecutingTask, n *pipeline.SampleNode) (*SampleNode, error) { +func newSampleNode(et *ExecutingTask, n *pipeline.SampleNode, l *log.Logger) (*SampleNode, error) { sn := &SampleNode{ - node: node{Node: n, et: et}, + node: node{Node: n, et: et, logger: l}, s: n, counts: make(map[models.GroupID]int64), duration: n.Duration, @@ -31,7 +32,7 @@ func newSampleNode(et *ExecutingTask, n *pipeline.SampleNode) (*SampleNode, erro return sn, nil } -func (s *SampleNode) runSample() error { +func (s *SampleNode) runSample([]byte) error { switch s.Wants() { case pipeline.StreamEdge: for p, ok := s.ins[0].NextPoint(); ok; p, ok = s.ins[0].NextPoint() { diff --git a/services/reporting/service.go b/services/reporting/service.go index e2b1f7031..dcdb72768 100644 --- a/services/reporting/service.go +++ b/services/reporting/service.go @@ -59,14 +59,8 @@ func (s *Service) Open() error { s.tags["arch"] = runtime.GOARCH s.tags["os"] = runtime.GOOS - // Send anonymous usage stats on startup - s.usageTicker = time.NewTicker(reportingInterval) - err := s.sendUsageReport() - if err != nil { - s.logger.Println("E! error sending usage stats:", err) - } - // Send periodic anonymous usage stats + s.usageTicker = time.NewTicker(reportingInterval) s.wg.Add(1) go s.usage() return nil diff --git a/services/task_store/config.go b/services/task_store/config.go index f54cc91d4..39f5f7b2b 100644 --- a/services/task_store/config.go +++ b/services/task_store/config.go @@ -2,15 +2,20 @@ package task_store import ( "fmt" + "time" + + "github.com/influxdb/influxdb/toml" ) type Config struct { - Dir string `toml:"dir"` + Dir string `toml:"dir"` + SnapshotInterval toml.Duration `toml:"snapshot-interval"` } func NewConfig() Config { return Config{ - Dir: "./tasks", + Dir: "./tasks", + SnapshotInterval: toml.Duration(time.Minute), } } diff --git a/services/task_store/service.go b/services/task_store/service.go index b743eec5f..948fb613b 100644 --- a/services/task_store/service.go +++ b/services/task_store/service.go @@ -12,24 +12,29 @@ import ( "os" "path" "strings" + "time" "github.com/boltdb/bolt" "github.com/influxdata/kapacitor" "github.com/influxdata/kapacitor/services/httpd" + "github.com/influxdata/kapacitor/tick" + "github.com/influxdb/influxdb/influxql" ) const taskDB = "task.db" var ( - tasksBucket = []byte("tasks") - enabledBucket = []byte("enabled") + tasksBucket = []byte("tasks") + enabledBucket = []byte("enabled") + snapshotBucket = []byte("snapshots") ) type Service struct { - dbpath string - db *bolt.DB - routes []httpd.Route - HTTPDService interface { + dbpath string + db *bolt.DB + routes []httpd.Route + snapshotInterval time.Duration + HTTPDService interface { AddRoutes([]httpd.Route) error DelRoutes([]httpd.Route) } @@ -38,7 +43,9 @@ type Service struct { StopTask(name string) error IsExecuting(name string) bool ExecutingDot(name string) string + CreateTICKScope() *tick.Scope } + logger *log.Logger } @@ -50,8 +57,9 @@ type taskStore struct { func NewService(conf Config, l *log.Logger) *Service { return &Service{ - dbpath: path.Join(conf.Dir, taskDB), - logger: l, + dbpath: path.Join(conf.Dir, taskDB), + snapshotInterval: time.Duration(conf.SnapshotInterval), + logger: l, } } @@ -142,7 +150,7 @@ func (ts *Service) Open() error { return err } - // Start enabled tasks + // Get enabled tasks enabledTasks := make([]string, 0) err = ts.db.View(func(tx *bolt.Tx) error { b := tx.Bucket([]byte(enabledBucket)) @@ -191,6 +199,70 @@ func (ts *Service) Close() error { return nil } +func (ts *Service) SaveSnapshot(name string, snapshot *kapacitor.TaskSnapshot) error { + var buf bytes.Buffer + enc := gob.NewEncoder(&buf) + err := enc.Encode(snapshot) + if err != nil { + return fmt.Errorf("failed to encode task snapshot %s %v", name, err) + } + + err = ts.db.Update(func(tx *bolt.Tx) error { + b, err := tx.CreateBucketIfNotExists(snapshotBucket) + if err != nil { + return err + } + + return b.Put([]byte(name), buf.Bytes()) + }) + if err != nil { + return err + } + return nil +} +func (ts *Service) HasSnapshot(name string) bool { + err := ts.db.View(func(tx *bolt.Tx) error { + b := tx.Bucket(snapshotBucket) + if b == nil { + return fmt.Errorf("no snapshot found for task %s", name) + } + + data := b.Get([]byte(name)) + if data == nil { + return fmt.Errorf("no snapshot found for task %s", name) + } + return nil + }) + return err == nil +} + +func (ts *Service) LoadSnapshot(name string) (*kapacitor.TaskSnapshot, error) { + var data []byte + err := ts.db.View(func(tx *bolt.Tx) error { + b := tx.Bucket(snapshotBucket) + if b == nil { + return fmt.Errorf("no snapshot found for task %s", name) + } + + data = b.Get([]byte(name)) + if data == nil { + return fmt.Errorf("no snapshot found for task %s", name) + } + return nil + }) + if err != nil { + return nil, err + } + buf := bytes.NewBuffer(data) + dec := gob.NewDecoder(buf) + snapshot := &kapacitor.TaskSnapshot{} + err = dec.Decode(snapshot) + if err != nil { + return nil, err + } + return snapshot, nil +} + type TaskInfo struct { Name string Type kapacitor.TaskType @@ -273,13 +345,15 @@ type rawTask struct { // The task type (stream|batch). Type kapacitor.TaskType // The DBs and RPs the task is allowed to access. - DBRPs []kapacitor.DBRP + DBRPs []kapacitor.DBRP + SnapshotInterval time.Duration } func (ts *Service) handleSave(w http.ResponseWriter, r *http.Request) { name := r.URL.Query().Get("name") newTask := &rawTask{ - Name: name, + Name: name, + SnapshotInterval: ts.snapshotInterval, } // Check for existing task @@ -335,6 +409,17 @@ func (ts *Service) handleSave(w http.ResponseWriter, r *http.Request) { return } + // Get snapshot interval + snapshotIntervalStr := r.URL.Query().Get("snapshot") + if snapshotIntervalStr != "" { + snapshotInterval, err := influxql.ParseDuration(snapshotIntervalStr) + if err != nil { + httpd.HttpError(w, err.Error(), true, http.StatusBadRequest) + return + } + newTask.SnapshotInterval = snapshotInterval + } + err = ts.Save(newTask) if err != nil { httpd.HttpError(w, err.Error(), true, http.StatusInternalServerError) @@ -373,11 +458,23 @@ func (ts *Service) handleDisable(w http.ResponseWriter, r *http.Request) { func (ts *Service) Save(task *rawTask) error { // Validate task - _, err := kapacitor.NewTask(task.Name, task.TICKscript, task.Type, task.DBRPs) + _, err := kapacitor.NewTask(task.Name, + task.TICKscript, + task.Type, + task.DBRPs, + task.SnapshotInterval, + ts.TaskMaster.CreateTICKScope(), + ) if err != nil { return fmt.Errorf("invalid task: %s", err) } + // Write 0 snapshot interval if it is the default. + // This way if the default changes the task will change too. + if task.SnapshotInterval == ts.snapshotInterval { + task.SnapshotInterval = 0 + } + var buf bytes.Buffer enc := gob.NewEncoder(&buf) @@ -443,7 +540,10 @@ func (ts *Service) LoadRaw(name string) (*rawTask, error) { buf := bytes.NewBuffer(data) dec := gob.NewDecoder(buf) task := &rawTask{} - err = dec.Decode(&task) + err = dec.Decode(task) + if task.SnapshotInterval == 0 { + task.SnapshotInterval = ts.snapshotInterval + } if err != nil { return nil, err } @@ -455,7 +555,13 @@ func (ts *Service) Load(name string) (*kapacitor.Task, error) { if err != nil { return nil, err } - return kapacitor.NewTask(task.Name, task.TICKscript, task.Type, task.DBRPs) + return kapacitor.NewTask(task.Name, + task.TICKscript, + task.Type, + task.DBRPs, + task.SnapshotInterval, + ts.TaskMaster.CreateTICKScope(), + ) } func (ts *Service) Enable(name string) error { diff --git a/services/udf/config.go b/services/udf/config.go new file mode 100644 index 000000000..ccf0a00f7 --- /dev/null +++ b/services/udf/config.go @@ -0,0 +1,32 @@ +package udf + +import ( + "fmt" + "time" + + "github.com/influxdb/influxdb/toml" +) + +type Config struct { + Functions map[string]FunctionConfig `toml:"functions"` +} + +type FunctionConfig struct { + Prog string `toml:"prog"` + Args []string `toml:"args"` + Timeout toml.Duration `toml:"timeout"` + Env map[string]string `toml:"env"` +} + +func NewConfig() Config { + return Config{} +} + +func (c Config) Validate() error { + for name, fc := range c.Functions { + if time.Duration(fc.Timeout) <= time.Millisecond { + return fmt.Errorf("timeout for %s is too small: %s", name, fc.Timeout) + } + } + return nil +} diff --git a/services/udf/service.go b/services/udf/service.go new file mode 100644 index 000000000..de6ffe931 --- /dev/null +++ b/services/udf/service.go @@ -0,0 +1,93 @@ +package udf + +import ( + "fmt" + "log" + "os" + "sync" + "time" + + "github.com/influxdata/kapacitor" + "github.com/influxdata/kapacitor/command" +) + +type Service struct { + functionConfig map[string]FunctionConfig + functions map[string]kapacitor.UDFProcessInfo + logger *log.Logger + mu sync.RWMutex +} + +func NewService(c Config, l *log.Logger) *Service { + return &Service{ + functionConfig: c.Functions, + functions: make(map[string]kapacitor.UDFProcessInfo), + logger: l, + } +} + +func (s *Service) Open() error { + for name := range s.functionConfig { + err := s.RefreshFunction(name) + if err != nil { + return err + } + } + return nil +} + +func (s *Service) Close() error { + return nil +} + +func (s *Service) FunctionList() []string { + s.mu.RLock() + defer s.mu.RUnlock() + functions := make([]string, 0, len(s.functions)) + for name := range s.functions { + functions = append(functions, name) + } + return functions +} + +func (s *Service) FunctionInfo(name string) (kapacitor.UDFProcessInfo, bool) { + s.mu.RLock() + defer s.mu.RUnlock() + info, ok := s.functions[name] + return info, ok +} + +func (s *Service) RefreshFunction(name string) error { + s.mu.Lock() + defer s.mu.Unlock() + fc, ok := s.functionConfig[name] + if ok { + info, err := s.loadProcessInfo(fc) + if err != nil { + return fmt.Errorf("failed to load process info for %q: %v", name, err) + } + s.functions[name] = info + s.logger.Printf("D! loaded UDF %q", name) + return nil + } + return fmt.Errorf("no function %s configured", name) +} + +func (s *Service) loadProcessInfo(f FunctionConfig) (kapacitor.UDFProcessInfo, error) { + env := os.Environ() + for k, v := range f.Env { + env = append(env, fmt.Sprintf("%s=%s", k, v)) + } + commander := command.CommandInfo{ + Prog: f.Prog, + Args: f.Args, + Env: env, + } + p := kapacitor.NewUDFProcess(commander, s.logger, time.Duration(f.Timeout), nil) + err := p.Start() + if err != nil { + return kapacitor.UDFProcessInfo{}, err + } + defer p.Stop() + return p.Info() +} diff --git a/stream.go b/stream.go index 4f9b81b64..4a9299372 100644 --- a/stream.go +++ b/stream.go @@ -1,6 +1,8 @@ package kapacitor import ( + "log" + "github.com/influxdata/kapacitor/models" "github.com/influxdata/kapacitor/pipeline" "github.com/influxdata/kapacitor/tick" @@ -18,9 +20,9 @@ type StreamNode struct { } // Create a new StreamNode which filters data from a source. -func newStreamNode(et *ExecutingTask, n *pipeline.StreamNode) (*StreamNode, error) { +func newStreamNode(et *ExecutingTask, n *pipeline.StreamNode, l *log.Logger) (*StreamNode, error) { sn := &StreamNode{ - node: node{Node: n, et: et}, + node: node{Node: n, et: et, logger: l}, s: n, db: n.Database, rp: n.RetentionPolicy, @@ -36,7 +38,7 @@ func newStreamNode(et *ExecutingTask, n *pipeline.StreamNode) (*StreamNode, erro return sn, nil } -func (s *StreamNode) runStream() error { +func (s *StreamNode) runStream([]byte) error { for pt, ok := s.ins[0].NextPoint(); ok; pt, ok = s.ins[0].NextPoint() { if s.matches(pt) { diff --git a/task.go b/task.go index d9a22b9b9..94dde2e5f 100644 --- a/task.go +++ b/task.go @@ -5,6 +5,8 @@ import ( "errors" "fmt" "log" + "math/rand" + "sync" "time" "github.com/influxdata/kapacitor/pipeline" @@ -49,17 +51,26 @@ func (d DBRP) String() string { // The complete definition of a task, its name, pipeline and type. type Task struct { - Name string - Pipeline *pipeline.Pipeline - Type TaskType - DBRPs []DBRP + Name string + Pipeline *pipeline.Pipeline + Type TaskType + DBRPs []DBRP + SnapshotInterval time.Duration } -func NewTask(name, script string, tt TaskType, dbrps []DBRP) (*Task, error) { +func NewTask( + name, + script string, + tt TaskType, + dbrps []DBRP, + snapshotInterval time.Duration, + scope *tick.Scope, +) (*Task, error) { t := &Task{ - Name: name, - Type: tt, - DBRPs: dbrps, + Name: name, + Type: tt, + DBRPs: dbrps, + SnapshotInterval: snapshotInterval, } var srcEdge pipeline.EdgeType @@ -70,10 +81,6 @@ func NewTask(name, script string, tt TaskType, dbrps []DBRP) (*Task, error) { srcEdge = pipeline.BatchEdge } - scope := tick.NewScope() - scope.Set("influxql", newInfluxQL()) - scope.Set("time", func(d time.Duration) time.Duration { return d }) - p, err := pipeline.CreatePipeline(script, srcEdge, scope) if err != nil { return nil, err @@ -86,16 +93,6 @@ func (t *Task) Dot() []byte { return t.Pipeline.Dot(t.Name) } -// Create a new streamer task from a script. -func NewStreamer(name, script string, dbrps []DBRP) (*Task, error) { - return NewTask(name, script, StreamTask, dbrps) -} - -// Create a new batcher task from a script. -func NewBatcher(name, script string, dbrps []DBRP) (*Task, error) { - return NewTask(name, script, BatchTask, dbrps) -} - // ---------------------------------- // ExecutingTask @@ -106,17 +103,22 @@ type ExecutingTask struct { source Node outputs map[string]Output // node lookup from pipeline.ID -> Node - lookup map[pipeline.ID]Node - nodes []Node + lookup map[pipeline.ID]Node + nodes []Node + stopSnapshotter chan struct{} + wg sync.WaitGroup + logger *log.Logger } // Create a new task from a defined kapacitor. func NewExecutingTask(tm *TaskMaster, t *Task) (*ExecutingTask, error) { + l := tm.LogService.NewLogger(fmt.Sprintf("[task:%s] ", t.Name), log.LstdFlags) et := &ExecutingTask{ tm: tm, Task: t, outputs: make(map[string]Output), lookup: make(map[pipeline.ID]Node), + logger: l, } err := et.link() if err != nil { @@ -152,14 +154,14 @@ func (et *ExecutingTask) link() error { // Walk Pipeline and create equivalent executing nodes err := et.Task.Pipeline.Walk(func(n pipeline.Node) error { - en, err := et.createNode(n) + l := et.tm.LogService.NewLogger( + fmt.Sprintf("[%s:%s] ", et.Task.Name, n.Name()), + log.LstdFlags, + ) + en, err := et.createNode(n, l) if err != nil { return err } - en.setLogger(et.tm.LogService.NewLogger( - fmt.Sprintf("[%s:%s] ", et.Task.Name, en.Name()), - log.LstdFlags, - )) et.lookup[n.ID()] = en // Save the walk order et.nodes = append(et.nodes, en) @@ -180,19 +182,46 @@ func (et *ExecutingTask) link() error { } // Start the task. -func (et *ExecutingTask) start(ins []*Edge) error { +func (et *ExecutingTask) start(ins []*Edge, snapshot *TaskSnapshot) error { for _, in := range ins { et.source.addParentEdge(in) } + validSnapshot := false + if snapshot != nil { + err := et.walk(func(n Node) error { + _, ok := snapshot.NodeSnapshots[n.Name()] + if !ok { + return fmt.Errorf("task pipeline changed not using snapshot") + } + return nil + }) + validSnapshot = err == nil + } - return et.walk(func(n Node) error { - n.start() + err := et.walk(func(n Node) error { + if validSnapshot { + n.start(snapshot.NodeSnapshots[n.Name()]) + } else { + n.start(nil) + } return nil }) + if err != nil { + return err + } + if et.Task.SnapshotInterval > 0 { + et.wg.Add(1) + et.stopSnapshotter = make(chan struct{}) + go et.runSnapshotter() + } + return nil } func (et *ExecutingTask) stop() (err error) { + if et.Task.SnapshotInterval > 0 { + close(et.stopSnapshotter) + } et.walk(func(n Node) error { n.stop() e := n.Err() @@ -201,6 +230,7 @@ func (et *ExecutingTask) stop() (err error) { } return nil }) + et.wg.Wait() return } @@ -303,41 +333,100 @@ func (et *ExecutingTask) EDot() []byte { } // Create a node from a given pipeline node. -func (et *ExecutingTask) createNode(p pipeline.Node) (Node, error) { +func (et *ExecutingTask) createNode(p pipeline.Node, l *log.Logger) (Node, error) { switch t := p.(type) { case *pipeline.StreamNode: - return newStreamNode(et, t) + return newStreamNode(et, t, l) case *pipeline.SourceBatchNode: - return newSourceBatchNode(et, t) + return newSourceBatchNode(et, t, l) case *pipeline.BatchNode: - return newBatchNode(et, t) + return newBatchNode(et, t, l) case *pipeline.WindowNode: - return newWindowNode(et, t) + return newWindowNode(et, t, l) case *pipeline.HTTPOutNode: - return newHTTPOutNode(et, t) + return newHTTPOutNode(et, t, l) case *pipeline.InfluxDBOutNode: - return newInfluxDBOutNode(et, t) + return newInfluxDBOutNode(et, t, l) case *pipeline.MapNode: - return newMapNode(et, t) + return newMapNode(et, t, l) case *pipeline.ReduceNode: - return newReduceNode(et, t) + return newReduceNode(et, t, l) case *pipeline.AlertNode: - return newAlertNode(et, t) + return newAlertNode(et, t, l) case *pipeline.GroupByNode: - return newGroupByNode(et, t) + return newGroupByNode(et, t, l) case *pipeline.UnionNode: - return newUnionNode(et, t) + return newUnionNode(et, t, l) case *pipeline.JoinNode: - return newJoinNode(et, t) + return newJoinNode(et, t, l) case *pipeline.EvalNode: - return newApplyNode(et, t) + return newEvalNode(et, t, l) case *pipeline.WhereNode: - return newWhereNode(et, t) + return newWhereNode(et, t, l) case *pipeline.SampleNode: - return newSampleNode(et, t) + return newSampleNode(et, t, l) case *pipeline.DerivativeNode: - return newDerivativeNode(et, t) + return newDerivativeNode(et, t, l) + case *pipeline.UDFNode: + return newUDFNode(et, t, l) default: return nil, fmt.Errorf("unknown pipeline node type %T", p) } } + +type TaskSnapshot struct { + NodeSnapshots map[string][]byte +} + +func (et *ExecutingTask) Snapshot() (*TaskSnapshot, error) { + snapshot := &TaskSnapshot{ + NodeSnapshots: make(map[string][]byte), + } + err := et.walk(func(n Node) error { + data, err := n.snapshot() + if err != nil { + return err + } + snapshot.NodeSnapshots[n.Name()] = data + return nil + }) + if err != nil { + return nil, err + } + return snapshot, nil +} + +func (et *ExecutingTask) runSnapshotter() { + defer et.wg.Done() + // Wait random duration to splay snapshot events across interval + select { + case <-time.After(time.Duration(rand.Float64() * float64(et.Task.SnapshotInterval))): + case <-et.stopSnapshotter: + return + } + ticker := time.NewTicker(et.Task.SnapshotInterval) + defer ticker.Stop() + for { + select { + case <-ticker.C: + snapshot, err := et.Snapshot() + if err != nil { + et.logger.Println("E! failed to snapshot task", et.Task.Name, err) + break + } + size := 0 + for _, data := range snapshot.NodeSnapshots { + size += len(data) + } + // Only save the snapshot if it has content + if size > 0 { + err = et.tm.TaskStore.SaveSnapshot(et.Task.Name, snapshot) + if err != nil { + et.logger.Println("E! failed to save task snapshot", et.Task.Name, err) + } + } + case <-et.stopSnapshotter: + return + } + } +} diff --git a/task_master.go b/task_master.go index 9e64958fb..689783ab5 100644 --- a/task_master.go +++ b/task_master.go @@ -10,6 +10,7 @@ import ( "github.com/influxdata/kapacitor/models" "github.com/influxdata/kapacitor/pipeline" "github.com/influxdata/kapacitor/services/httpd" + "github.com/influxdata/kapacitor/tick" "github.com/influxdb/influxdb/client" "github.com/influxdb/influxdb/cluster" ) @@ -17,6 +18,10 @@ import ( type LogService interface { NewLogger(prefix string, flag int) *log.Logger } +type UDFService interface { + FunctionList() []string + FunctionInfo(name string) (UDFProcessInfo, bool) +} var ErrTaskMasterClosed = errors.New("TaskMaster is closed") var ErrTaskMasterOpen = errors.New("TaskMaster is open") @@ -28,6 +33,14 @@ type TaskMaster struct { DelRoutes([]httpd.Route) URL() string } + TaskStore interface { + SaveSnapshot(name string, snapshot *TaskSnapshot) error + HasSnapshot(name string) bool + LoadSnapshot(name string) (*TaskSnapshot, error) + } + + UDFService UDFService + InfluxDBService interface { NewClient() (*client.Client, error) } @@ -101,8 +114,16 @@ func NewTaskMaster(l LogService) *TaskMaster { func (tm *TaskMaster) New() *TaskMaster { n := NewTaskMaster(tm.LogService) n.HTTPDService = tm.HTTPDService + n.UDFService = tm.UDFService + n.TaskStore = tm.TaskStore n.InfluxDBService = tm.InfluxDBService n.SMTPService = tm.SMTPService + n.OpsGenieService = tm.OpsGenieService + n.VictorOpsService = tm.VictorOpsService + n.PagerDutyService = tm.PagerDutyService + n.SlackService = tm.SlackService + n.HipChatService = tm.HipChatService + n.AlertaService = tm.AlertaService return n } @@ -156,6 +177,39 @@ func (tm *TaskMaster) waitForForks() { tm.wg.Wait() } +func (tm *TaskMaster) CreateTICKScope() *tick.Scope { + scope := tick.NewScope() + scope.Set("influxql", newInfluxQL()) + scope.Set("time", func(d time.Duration) time.Duration { return d }) + // Add dynamic methods to the scope for UDFs + if tm.UDFService != nil { + for _, f := range tm.UDFService.FunctionList() { + f := f + info, _ := tm.UDFService.FunctionInfo(f) + scope.SetDynamicMethod( + f, + tick.DynamicMethod(func(self interface{}, args ...interface{}) (interface{}, error) { + parent, ok := self.(pipeline.Node) + if !ok { + return nil, fmt.Errorf("cannot call %s on %T", f, self) + } + udf := pipeline.NewUDF( + parent, + f, + info.Commander, + info.Timeout, + info.Wants, + info.Provides, + info.Options, + ) + return udf, nil + }), + ) + } + } + return scope +} + func (tm *TaskMaster) StartTask(t *Task) (*ExecutingTask, error) { tm.mu.Lock() defer tm.mu.Unlock() @@ -189,7 +243,15 @@ func (tm *TaskMaster) StartTask(t *Task) (*ExecutingTask, error) { } } - err = et.start(ins) + var snapshot *TaskSnapshot + if tm.TaskStore.HasSnapshot(t.Name) { + snapshot, err = tm.TaskStore.LoadSnapshot(t.Name) + if err != nil { + return nil, err + } + } + + err = et.start(ins, snapshot) if err != nil { return nil, err } @@ -339,3 +401,14 @@ func (tm *TaskMaster) delFork(name string) { fork.Edge.Close() } } + +func (tm *TaskMaster) SnapshotTask(name string) (*TaskSnapshot, error) { + tm.mu.RLock() + et, ok := tm.tasks[name] + tm.mu.RUnlock() + + if ok { + return et.Snapshot() + } + return nil, fmt.Errorf("task %s is not running or does not exist", name) +} diff --git a/tick/eval.go b/tick/eval.go index 9a8c485d8..7f2410147 100644 --- a/tick/eval.go +++ b/tick/eval.go @@ -12,6 +12,21 @@ import ( "unicode/utf8" ) +// Interface for interacting with objects. +// If an object does not self describe via this interface +// than a reflection based implemenation will be used. +type SelfDescriber interface { + //A description the object + Desc() string + + HasMethod(name string) bool + CallMethod(name string, args ...interface{}) (interface{}, error) + + HasProperty(name string) bool + Property(name string) interface{} + SetProperty(name string, arg interface{}) error +} + // Parse and evaluate a given script for the scope. // This evaluation method uses reflection to call // methods on objects within the scope. @@ -91,7 +106,7 @@ func eval(n Node, scope *Scope, stck *stack) (err error) { return } case *FunctionNode: - args := make([]reflect.Value, len(node.Args)) + args := make([]interface{}, len(node.Args)) for i, arg := range node.Args { err = eval(arg, scope, stck) if err != nil { @@ -113,7 +128,7 @@ func eval(n Node, scope *Scope, stck *stack) (err error) { } } - args[i] = reflect.ValueOf(a) + args[i] = a } err = evalFunc(node, scope, stck, args) if err != nil { @@ -182,22 +197,20 @@ func evalBinary(op tokenType, scope *Scope, stck *stack) error { } stck.Push(ret) case *IdentifierNode: - name := capilatizeFirst(right.Ident) + name := right.Ident //Lookup field by name of left object - v := reflect.ValueOf(l) - if !v.IsValid() { - return fmt.Errorf("object is not valid, cannot get field %s of %v", name, l) + var describer SelfDescriber + if d, ok := l.(SelfDescriber); ok { + describer = d + } else { + describer = NewReflectionDescriber(l) } - v = reflect.Indirect(v) - if v.Kind() == reflect.Struct { - field := v.FieldByName(name) - if field.IsValid() { - stck.Push(field.Interface()) - break - } + if describer.HasProperty(name) { + stck.Push(describer.Property(name)) + } else { + return fmt.Errorf("object %T has no property %s", l, name) } - return fmt.Errorf("unknown field %s of obj %T", name, l) default: return fmt.Errorf("invalid right operand of type %T to '.' operator", r) } @@ -205,7 +218,7 @@ func evalBinary(op tokenType, scope *Scope, stck *stack) error { return nil } -func evalFunc(f *FunctionNode, scope *Scope, stck *stack, args []reflect.Value) error { +func evalFunc(f *FunctionNode, scope *Scope, stck *stack, args []interface{}) error { rec := func(obj interface{}, errp *error) { e := recover() if e != nil { @@ -217,59 +230,161 @@ func evalFunc(f *FunctionNode, scope *Scope, stck *stack, args []reflect.Value) } } fnc := unboundFunc(func(obj interface{}) (_ interface{}, err error) { - //Setup recover method if there is a panic during reflection + //Setup recover method if there is a panic during the method call defer rec(obj, &err) - name := capilatizeFirst(f.Func) - // Check for method - var method reflect.Value + if obj == nil { // Object is nil, check for func in scope fnc, _ := scope.Get(f.Func) if fnc == nil { return nil, fmt.Errorf("no global function %q defined", f.Func) } - method = reflect.ValueOf(fnc) + method := reflect.ValueOf(fnc) + return callMethodReflection(method, args) + } + + // Get SelfDescriber + name := f.Func + var describer SelfDescriber + if d, ok := obj.(SelfDescriber); ok { + describer = d } else { - v := reflect.ValueOf(obj) - if !v.IsValid() { - return nil, fmt.Errorf("error calling %q on object %T", f.Func, obj) - } - method = v.MethodByName(name) - } - if method.IsValid() { - ret := method.Call(args) - if l := len(ret); l == 1 { - return ret[0].Interface(), nil - } else if l == 2 { - if i := ret[1].Interface(); i != nil { - if err, ok := i.(error); !ok { - return nil, fmt.Errorf("second return value form function must be an 'error', got %T", i) - } else { - return nil, err - } - } else { - return ret[0].Interface(), nil - } - } else { - return nil, fmt.Errorf("functions must return a single value or (interface{}, error)") - } + describer = NewReflectionDescriber(obj) } - // Check for settable field - v := reflect.Indirect(reflect.ValueOf(obj)) - if len(f.Args) == 1 && v.Kind() == reflect.Struct { - field := v.FieldByName(name) - if field.IsValid() && field.CanSet() { - field.Set(args[0]) - return obj, nil + // Check for Method + if describer.HasMethod(name) { + return describer.CallMethod(name, args...) + } + + // Check for dynamic method. + dm := scope.DynamicMethod(name) + if dm != nil { + ret, err := dm(obj, args...) + if err != nil { + return nil, err } + return ret, nil } - return nil, fmt.Errorf("No method or field %q on %T", name, obj) + + // Ran out of options... + return nil, fmt.Errorf("No method or property %q on %s", name, describer.Desc()) }) stck.Push(fnc) return nil } +// Wraps any object as a SelfDescriber using reflection. +type ReflectionDescriber struct { + obj interface{} +} + +func NewReflectionDescriber(obj interface{}) *ReflectionDescriber { + return &ReflectionDescriber{obj: obj} +} + +func (r *ReflectionDescriber) Desc() string { + return reflect.TypeOf(r.obj).Name() +} + +// Using reflection check if the object has the method or field. +// A field is a valid method because we can set it via reflection too. +func (r *ReflectionDescriber) HasMethod(name string) bool { + name = capilatizeFirst(name) + v := reflect.ValueOf(r.obj) + if !v.IsValid() { + return false + } + if v.MethodByName(name).IsValid() { + return true + } + // Check for a field of the same name, + // we can wrap setting it in a method. + return r.HasProperty(name) +} + +func (r *ReflectionDescriber) CallMethod(name string, args ...interface{}) (interface{}, error) { + name = capilatizeFirst(name) + v := reflect.ValueOf(r.obj) + if !v.IsValid() { + return nil, fmt.Errorf("cannot get reflect.ValueOf %T", r.obj) + } + + // Check for a method and call it + if method := v.MethodByName(name); method.IsValid() { + return callMethodReflection(method, args) + } + + // Check for a field and set it + if len(args) == 1 && r.HasProperty(name) { + err := r.SetProperty(name, args[0]) + if err != nil { + return nil, err + } + return r.obj, nil + } + return nil, fmt.Errorf("unknown method or field %s on %T", name, r.obj) +} + +// Using reflection check if the object has a field with the property name. +func (r *ReflectionDescriber) HasProperty(name string) bool { + name = capilatizeFirst(name) + v := reflect.Indirect(reflect.ValueOf(r.obj)) + if v.Kind() == reflect.Struct { + field := v.FieldByName(name) + return field.IsValid() && field.CanSet() + } + return false +} + +func (r *ReflectionDescriber) Property(name string) interface{} { + name = capilatizeFirst(name) + v := reflect.Indirect(reflect.ValueOf(r.obj)) + if v.Kind() == reflect.Struct { + field := v.FieldByName(name) + if field.IsValid() { + return field.Interface() + } + } + return nil +} + +func (r *ReflectionDescriber) SetProperty(name string, value interface{}) error { + v := reflect.Indirect(reflect.ValueOf(r.obj)) + if v.Kind() == reflect.Struct { + field := v.FieldByName(name) + if field.IsValid() && field.CanSet() { + field.Set(reflect.ValueOf(value)) + return nil + } + } + return fmt.Errorf("no field %s on %T", name, r.obj) +} + +func callMethodReflection(method reflect.Value, args []interface{}) (interface{}, error) { + rargs := make([]reflect.Value, len(args)) + for i, arg := range args { + rargs[i] = reflect.ValueOf(arg) + } + ret := method.Call(rargs) + if l := len(ret); l == 1 { + return ret[0].Interface(), nil + } else if l == 2 { + if i := ret[1].Interface(); i != nil { + if err, ok := i.(error); !ok { + return nil, fmt.Errorf("second return value form function must be an 'error', got %T", i) + } else { + return nil, err + } + } else { + return ret[0].Interface(), nil + } + } else { + return nil, fmt.Errorf("functions must return a single value or (interface{}, error)") + } + +} + // Capilatizes the first rune in the string func capilatizeFirst(s string) string { r, n := utf8.DecodeRuneInString(s) diff --git a/tick/eval_test.go b/tick/eval_test.go index 53210a80b..5a798f055 100644 --- a/tick/eval_test.go +++ b/tick/eval_test.go @@ -1,6 +1,7 @@ package tick_test import ( + "fmt" "testing" "time" @@ -28,6 +29,12 @@ type structC struct { AggFunc aggFunc } +type orphan struct { + parent *structA + Sad bool + args []interface{} +} + func (s *structA) StructB() *structB { return &structB{} } @@ -47,7 +54,7 @@ func (s *structC) Options(str string, f float64, d time.Duration) *structC { type aggFunc func(values []float64) []float64 type influxql struct { - Agg agg + Agg *agg } type agg struct { @@ -82,8 +89,11 @@ s2.structC() a := &structA{} scope.Set("a", a) - i := &influxql{} - i.Agg.Sum = aggSum + i := &influxql{ + Agg: &agg{ + Sum: aggSum, + }, + } scope.Set("influxql", i) err := tick.Evaluate(script, scope) @@ -111,3 +121,55 @@ s2.structC() } } } + +func TestEvaluate_DynamicMethod(t *testing.T) { + script := `var x = a.dynamicMethod(1,'str', 10s).sad(FALSE)` + + scope := tick.NewScope() + a := &structA{} + scope.Set("a", a) + + dm := func(self interface{}, args ...interface{}) (interface{}, error) { + a, ok := self.(*structA) + if !ok { + return nil, fmt.Errorf("cannot call dynamicMethod on %T", self) + } + o := &orphan{ + parent: a, + Sad: true, + args: args, + } + return o, nil + } + scope.SetDynamicMethod("dynamicMethod", dm) + + err := tick.Evaluate(script, scope) + if err != nil { + t.Fatal(err) + } + + xI, err := scope.Get("x") + if err != nil { + t.Fatal(err) + } + x, ok := xI.(*orphan) + if !ok { + t.Fatalf("expected x to be an *orphan, got %T", xI) + } + if x.Sad { + t.Errorf("expected x to not be sad") + } + + if got, exp := len(x.args), 3; exp != got { + t.Fatalf("unexpected number of args: got %d exp %d", got, exp) + } + if got, exp := x.args[0], int64(1); exp != got { + t.Errorf("unexpected x.args[0]: got %v exp %d", got, exp) + } + if got, exp := x.args[1], "str"; exp != got { + t.Errorf("unexpected x.args[1]: got %v exp %s", got, exp) + } + if got, exp := x.args[2], time.Second*10; exp != got { + t.Errorf("unexpected x.args[1]: got %v exp %v", got, exp) + } +} diff --git a/tick/example_scope_test.go b/tick/example_scope_test.go index 7ad9c4738..e175cf372 100644 --- a/tick/example_scope_test.go +++ b/tick/example_scope_test.go @@ -1,9 +1,7 @@ -package tick_test +package tick import ( "fmt" - - "github.com/influxdata/kapacitor/tick" ) type Process struct { @@ -41,11 +39,11 @@ child1.spawn().name('grandchild') parent.spawn().name('child2') ` - scope := tick.NewScope() + scope := NewScope() parent := &Process{} scope.Set("parent", parent) - err := tick.Evaluate(script, scope) + err := Evaluate(script, scope) if err != nil { fmt.Println(err) } diff --git a/tick/scope.go b/tick/scope.go index 53521fc55..3fa7bf582 100644 --- a/tick/scope.go +++ b/tick/scope.go @@ -5,15 +5,20 @@ import ( "strings" ) +type DynamicMethod func(self interface{}, args ...interface{}) (interface{}, error) + // Contains a set of variables references and their values. type Scope struct { variables map[string]interface{} + + dynamicMethods map[string]DynamicMethod } //Initialize a new Scope object. func NewScope() *Scope { return &Scope{ - variables: make(map[string]interface{}), + variables: make(map[string]interface{}), + dynamicMethods: make(map[string]DynamicMethod), } } @@ -33,3 +38,11 @@ func (s *Scope) Get(name string) (interface{}, error) { } return nil, fmt.Errorf("name %q is undefined. Names in scope: %s", name, strings.Join(possible, ",")) } + +func (s *Scope) SetDynamicMethod(name string, m DynamicMethod) { + s.dynamicMethods[name] = m +} + +func (s *Scope) DynamicMethod(name string) DynamicMethod { + return s.dynamicMethods[name] +} diff --git a/udf.go b/udf.go new file mode 100644 index 000000000..84e26eb6a --- /dev/null +++ b/udf.go @@ -0,0 +1,149 @@ +package kapacitor + +import ( + "errors" + "log" + "sync" + + "github.com/influxdata/kapacitor/pipeline" +) + +// User defined function +type UDFNode struct { + node + u *pipeline.UDFNode + process *UDFProcess + aborted chan struct{} + + wg sync.WaitGroup + mu sync.Mutex + stopped bool +} + +// Create a new UDFNode that sends incoming data to child process +func newUDFNode(et *ExecutingTask, n *pipeline.UDFNode, l *log.Logger) (*UDFNode, error) { + un := &UDFNode{ + node: node{Node: n, et: et, logger: l}, + u: n, + aborted: make(chan struct{}), + } + un.process = NewUDFProcess( + n.Commander, + l, + n.Timeout, + un.abortedCallback, + ) + + un.node.runF = un.runUDF + un.node.stopF = un.stopUDF + return un, nil +} + +var errNodeAborted = errors.New("node aborted") + +func (u *UDFNode) stopUDF() { + u.mu.Lock() + defer u.mu.Unlock() + if !u.stopped { + u.stopped = true + u.process.Abort(errNodeAborted) + } +} + +func (u *UDFNode) runUDF(snapshot []byte) (err error) { + defer func() { + u.mu.Lock() + defer u.mu.Unlock() + //Ignore stopped errors if the process was stopped externally + if u.stopped && (err == ErrUDFProcessStopped || err == errNodeAborted) { + err = nil + } + u.stopped = true + }() + err = u.process.Start() + if err != nil { + return + } + err = u.process.Init(u.u.Options) + if err != nil { + return + } + if snapshot != nil { + err = u.process.Restore(snapshot) + if err != nil { + return + } + } + forwardErr := make(chan error, 1) + go func() { + switch u.Provides() { + case pipeline.StreamEdge: + for p := range u.process.PointOut { + for _, out := range u.outs { + err := out.CollectPoint(p) + if err != nil { + forwardErr <- err + return + } + } + } + case pipeline.BatchEdge: + for b := range u.process.BatchOut { + for _, out := range u.outs { + err := out.CollectBatch(b) + if err != nil { + forwardErr <- err + return + } + } + } + } + forwardErr <- nil + }() + + // The abort callback needs to know when we are done writing + // so we wrap in a wait group. + u.wg.Add(1) + go func() { + defer u.wg.Done() + switch u.Wants() { + case pipeline.StreamEdge: + for p, ok := u.ins[0].NextPoint(); ok; p, ok = u.ins[0].NextPoint() { + select { + case u.process.PointIn <- p: + case <-u.aborted: + return + } + } + case pipeline.BatchEdge: + for b, ok := u.ins[0].NextBatch(); ok; b, ok = u.ins[0].NextBatch() { + select { + case u.process.BatchIn <- b: + case <-u.aborted: + return + } + } + } + }() + // wait till we are done writing + u.wg.Wait() + + // Stop the process + err = u.process.Stop() + if err != nil { + return + } + // Wait/Return any error from the forwarding goroutine + err = <-forwardErr + return +} + +func (u *UDFNode) abortedCallback() { + close(u.aborted) + // wait till we are done writing + u.wg.Wait() +} + +func (u *UDFNode) snapshot() ([]byte, error) { + return u.process.Snapshot() +} diff --git a/udf/agent/README.md b/udf/agent/README.md new file mode 100644 index 000000000..29914b1e3 --- /dev/null +++ b/udf/agent/README.md @@ -0,0 +1,50 @@ +# UDF Agents + +A UDF is a User Defined Function, meaning that you can write your own functions/algorithms and plug them into Kapacitor. +You custom function runs in its own process and Kapacitor communicates with it via a defined protocol, see [udf.proto](https://github.com/influxdata/kapacitor/blob/master/udf/udf.proto). +To facilitate working with the protocol several `agents` have been written in various lanuages that abstract the protocol through an interface in the respective languages. +You can find those agent implementations in this directory and subdirectories based on language name. + + +Example uses of the agents can be found in the `examples` directory. +These examples are working examples and are executed as part of the testing suite, +see [server_test.go](https://github.com/influxdata/kapacitor/blob/master/cmd/kapacitord/run/server_test.go). + + +## Agent Design + +The protocol for communicating with Kapacitor consists of Request and Response messages that are sent over STDIN and STDOUT of the process respectively. +The agents wrap the communication and serialization over the sockets and expose and interface that needs to be implemented to handle each request/response. +In addition to the request/response paradigm agents provide a way to stream data back to Kapacitor. +This is not part of the request response system because your custom function is in control of when a new point or batch is sent if at all. + + +## Writing an Agent for a new Language + +The UDF protocol is designed to be simple and consists of reading and writing protocol buffer messages to STDIN and STDOUT. + +In order to write a UDF in the language of your choice your language must have protocol buffer support and be able to read and write to a socket. + +The basic steps are: + +0. Add the language to the `udf/io.go` generate comment so the udf.proto code exists for your language. +1. Implement a Varint encoder/decoder, this is trivial see the python implementation. +2. Implement a method for reading and writing streamed protobuf messages. See `udf.proto` for more details. +3. Create an interface for handling each of the request/responses. +4. Write a loop for reading from STDIN and calling the handler interface, and write responses to STDOUT. +5. Provide an thread safe mechanism for writing points and batches to STDOUT independent of the handler interface. + This is easily accomplished with a synchronized write method, see the python implementation. +6. Implement the moving average example using your new agent. +7. Add your example to the test suite in `cmd/kapacitord/run/server_test.go`. + +It is expected that the process terminate after STDIN is closed and the remaining requests processed. +After STDIN is closed, the agent process can continue to send Responses to Kapacitor as long as a keepalive timeout does not occur. +Once a keepalive timeout is reached and after a 2*keepalive_time grace period, if the process has not terminated then it will be forcefully terminated. + + +## Docker + +It is expected that the example can run inside the test suite. +Since generating different protocol buffer code requires different plugins and libraries to run we make use of Docker to provide the necessary environment. +This makes testing the code easier as the developer does not have to install each supported language locally. + diff --git a/udf/agent/agent.go b/udf/agent/agent.go new file mode 100644 index 000000000..2fc66ca79 --- /dev/null +++ b/udf/agent/agent.go @@ -0,0 +1,235 @@ +package agent + +import ( + "bufio" + "errors" + "fmt" + "io" + "os" + "sync" + + "github.com/influxdata/kapacitor/udf" +) + +// The Agent calls the appropriate methods on the Handler as it receives requests over STDIN. +// +// Returning an error from any method will cause the Agent to stop and an ErrorResponse to be sent. +// Some *Response objects (like SnapshotResponse) allow for returning their own error within the object itself. +// These types of errors will not stop the Agent and Kapacitor will deal with them appropriately. +// +// The Handler is called from a single goroutine, meaning methods will not be called concurrently. +// +// To write Points/Batches back to the Agent/Kapacitor use the Agent.Responses channel. +type Handler interface { + // Return the InfoResponse. Describing the properties of this Handler + Info() (*udf.InfoResponse, error) + // Initialize the Handler with the provided options. + Init(*udf.InitRequest) (*udf.InitResponse, error) + // Create a snapshot of the running state of the handler. + Snaphost() (*udf.SnapshotResponse, error) + // Restore a previous snapshot. + Restore(*udf.RestoreRequest) (*udf.RestoreResponse, error) + + // A batch has begun. + BeginBatch() error + // A point has arrived. + Point(*udf.Point) error + // The batch is complete. + EndBatch(*udf.EndBatch) error + + // Gracefully stop the Handler. + // No other methods will be called. + Stop() +} + +// Go implementation of a Kapacitor UDF agent. +// This agent is responsible for reading and writing +// messages over STDIN and STDOUT. +// +// The Agent requires a Handler object in order to fulfill requests. +type Agent struct { + in io.ReadCloser + out io.WriteCloser + + outGroup sync.WaitGroup + outResponses chan *udf.Response + + responses chan *udf.Response + // A channel for writing Responses, specifically Batch and Point responses. + Responses chan<- *udf.Response + + writeErrC chan error + readErrC chan error + + // The handler for requests. + Handler Handler +} + +func New() *Agent { + s := &Agent{ + in: os.Stdin, + out: os.Stdout, + outResponses: make(chan *udf.Response), + responses: make(chan *udf.Response), + } + s.Responses = s.responses + return s +} + +// Start the Agent, you must set an Handler on the agent before starting. +func (a *Agent) Start() error { + if a.Handler == nil { + return errors.New("must set a Handler on the agent before starting") + } + + a.readErrC = make(chan error, 1) + a.writeErrC = make(chan error, 1) + a.outGroup.Add(1) + go func() { + defer a.outGroup.Done() + err := a.readLoop() + if err != nil { + a.outResponses <- &udf.Response{ + Message: &udf.Response_Error{ + Error: &udf.ErrorResponse{err.Error()}, + }, + } + } + a.readErrC <- err + }() + go func() { + a.writeErrC <- a.writeLoop() + }() + + a.outGroup.Add(1) + go a.forwardResponses() + + return nil +} + +// Wait for the Agent to terminate. +// The Agent will not terminate till the Responses channel is closed. +// You will need to close this channel externally, typically in the Stop method for the Handler. +// The Agent will terminate if STDIN is closed or an error occurs. +func (a *Agent) Wait() error { + a.outGroup.Wait() + close(a.outResponses) + for a.readErrC != nil || a.writeErrC != nil { + select { + case err := <-a.readErrC: + a.readErrC = nil + if err != nil { + return fmt.Errorf("read error: %s", err) + } + case err := <-a.writeErrC: + a.writeErrC = nil + if err != nil { + return fmt.Errorf("write error: %s", err) + } + } + } + return nil +} + +func (a *Agent) readLoop() error { + defer a.Handler.Stop() + defer a.in.Close() + in := bufio.NewReader(a.in) + var buf []byte + request := &udf.Request{} + for { + err := udf.ReadMessage(&buf, in, request) + if err == io.EOF { + break + } + if err != nil { + return err + } + + // Hand message to handler + var res *udf.Response + switch msg := request.Message.(type) { + case *udf.Request_Info: + info, err := a.Handler.Info() + if err != nil { + return err + } + res = &udf.Response{} + res.Message = &udf.Response_Info{ + Info: info, + } + case *udf.Request_Init: + init, err := a.Handler.Init(msg.Init) + if err != nil { + return err + } + res = &udf.Response{} + res.Message = &udf.Response_Init{ + Init: init, + } + case *udf.Request_Keepalive: + res = &udf.Response{ + Message: &udf.Response_Keepalive{ + Keepalive: &udf.KeepaliveResponse{ + Time: msg.Keepalive.Time, + }, + }, + } + case *udf.Request_Snapshot: + snapshot, err := a.Handler.Snaphost() + if err != nil { + return err + } + res = &udf.Response{} + res.Message = &udf.Response_Snapshot{ + Snapshot: snapshot, + } + case *udf.Request_Restore: + restore, err := a.Handler.Restore(msg.Restore) + if err != nil { + return err + } + res = &udf.Response{} + res.Message = &udf.Response_Restore{ + Restore: restore, + } + case *udf.Request_Begin: + err := a.Handler.BeginBatch() + if err != nil { + return err + } + case *udf.Request_Point: + err := a.Handler.Point(msg.Point) + if err != nil { + return err + } + case *udf.Request_End: + err := a.Handler.EndBatch(msg.End) + if err != nil { + return err + } + } + if res != nil { + a.outResponses <- res + } + } + return nil +} + +func (a *Agent) writeLoop() error { + defer a.out.Close() + for response := range a.outResponses { + err := udf.WriteMessage(response, a.out) + if err != nil { + return err + } + } + return nil +} + +func (a *Agent) forwardResponses() { + defer a.outGroup.Done() + for r := range a.responses { + a.outResponses <- r + } +} diff --git a/udf/agent/examples/moving_avg.go b/udf/agent/examples/moving_avg.go new file mode 100644 index 000000000..2cafa1769 --- /dev/null +++ b/udf/agent/examples/moving_avg.go @@ -0,0 +1,169 @@ +package main + +import ( + "bytes" + "encoding/gob" + "errors" + "log" + + "github.com/influxdata/kapacitor/udf" + "github.com/influxdata/kapacitor/udf/agent" +) + +// An Agent.Handler that computes a moving average of the data it receives. +type avgHandler struct { + field string + as string + state *avgState + + agent *agent.Agent +} + +// The state required to compute the moving average. +type avgState struct { + Size int + Window []float64 + Avg float64 +} + +// Update the moving average with the next data point. +func (a *avgState) update(value float64) float64 { + l := len(a.Window) + if a.Size == l { + a.Avg += value/float64(l) - a.Window[0]/float64(l) + a.Window = a.Window[1:] + } else { + a.Avg = (value + float64(l)*a.Avg) / float64(l+1) + } + a.Window = append(a.Window, value) + return a.Avg +} + +func newMovingAvgHandler(a *agent.Agent) *avgHandler { + return &avgHandler{ + state: &avgState{}, + as: "avg", + agent: a, + } +} + +// Return the InfoResponse. Describing the properties of this UDF agent. +func (a *avgHandler) Info() (*udf.InfoResponse, error) { + info := &udf.InfoResponse{ + Wants: udf.EdgeType_STREAM, + Provides: udf.EdgeType_STREAM, + Options: map[string]*udf.OptionInfo{ + "field": {ValueTypes: []udf.ValueType{udf.ValueType_STRING}}, + "size": {ValueTypes: []udf.ValueType{udf.ValueType_INT}}, + "as": {ValueTypes: []udf.ValueType{udf.ValueType_STRING}}, + }, + } + return info, nil +} + +// Initialze the handler based of the provided options. +func (a *avgHandler) Init(r *udf.InitRequest) (*udf.InitResponse, error) { + init := &udf.InitResponse{ + Success: true, + Error: "", + } + for _, opt := range r.Options { + switch opt.Name { + case "field": + a.field = opt.Values[0].Value.(*udf.OptionValue_StringValue).StringValue + case "size": + a.state.Size = int(opt.Values[0].Value.(*udf.OptionValue_IntValue).IntValue) + case "as": + a.as = opt.Values[0].Value.(*udf.OptionValue_StringValue).StringValue + } + } + + if a.field == "" { + init.Success = false + init.Error += " must supply field" + } + if a.state.Size == 0 { + init.Success = false + init.Error += " must supply window size" + } + if a.as == "" { + init.Success = false + init.Error += " invalid as name provided" + } + + return init, nil +} + +// Create a snapshot of the running state of the process. +func (a *avgHandler) Snaphost() (*udf.SnapshotResponse, error) { + var buf bytes.Buffer + enc := gob.NewEncoder(&buf) + enc.Encode(a.state) + + return &udf.SnapshotResponse{ + Snapshot: buf.Bytes(), + }, nil +} + +// Restore a previous snapshot. +func (a *avgHandler) Restore(req *udf.RestoreRequest) (*udf.RestoreResponse, error) { + buf := bytes.NewReader(req.Snapshot) + dec := gob.NewDecoder(buf) + err := dec.Decode(a.state) + msg := "" + if err != nil { + msg = err.Error() + } + return &udf.RestoreResponse{ + Success: err == nil, + Error: msg, + }, nil +} + +// This handler does not do batching +func (a *avgHandler) BeginBatch() error { + return errors.New("batching not supported") +} + +// Receive a point and compute the average. +// Send a response with the average value. +func (a *avgHandler) Point(p *udf.Point) error { + // Update the moving average. + value := p.FieldsDouble[a.field] + avg := a.state.update(value) + + // Re-use the existing point so we keep the same tags etc. + p.FieldsDouble = map[string]float64{a.as: avg} + p.FieldsInt = nil + p.FieldsString = nil + // Send point with average value. + a.agent.Responses <- &udf.Response{ + Message: &udf.Response_Point{ + Point: p, + }, + } + return nil +} + +// This handler does not do batching +func (a *avgHandler) EndBatch(*udf.EndBatch) error { + return errors.New("batching not supported") +} + +// Stop the handler gracefully. +func (a *avgHandler) Stop() { + close(a.agent.Responses) +} + +func main() { + a := agent.New() + h := newMovingAvgHandler(a) + a.Handler = h + + log.Println("Starting agent") + a.Start() + err := a.Wait() + if err != nil { + log.Fatal(err) + } +} diff --git a/udf/agent/examples/moving_avg.py b/udf/agent/examples/moving_avg.py new file mode 100644 index 000000000..c76311091 --- /dev/null +++ b/udf/agent/examples/moving_avg.py @@ -0,0 +1,155 @@ + +import sys +import json +from agent import Agent, Handler +import udf_pb2 + +import logging +logging.basicConfig(level=logging.DEBUG, format='%(asctime)s %(levelname)s:%(name)s: %(message)s') +logger = logging.getLogger() + + +# Computes the moving average of the data it receives. +# The options it has are: +# size - the number of data points to keep in the window +# field - the field to operate on +# as - the name of the average field, default 'avg' +# +class AvgHandler(Handler): + + class state(object): + def __init__(self): + self.size = 0 + self._window = [] + self._avg = 0.0 + + def update(self, value): + l = float(len(self._window)) + + if l == self.size: + # Window is full, just iteratively update the avg + self._avg += value/l - self._window[0]/l + self._window.pop(0) + else: + # Window is not full compute the cumulative avg + self._avg = (value + l*self._avg) / (l + 1) + + self._window.append(value) + return self._avg + + + + def snapshot(self): + data = { + 'size' : self.size, + 'window' : self._window, + 'avg' : self._avg, + } + return json.dumps(data) + + def restore(self, snapshot): + data = json.loads(snapshot) + self.size = int(data['size']) + self._window = [float(d) for d in data['window']] + self._avg = float(data['avg']) + + def __init__(self, agent): + self._agent = agent + self._field = None + self._as = 'avg' + self._state = AvgHandler.state() + + + def info(self): + response = udf_pb2.Response() + response.info.wants = udf_pb2.STREAM + response.info.provides = udf_pb2.STREAM + response.info.options['field'].valueTypes.append(udf_pb2.STRING) + response.info.options['size'].valueTypes.append(udf_pb2.INT) + response.info.options['as'].valueTypes.append(udf_pb2.STRING) + + logger.info("info") + return response + + def init(self, init_req): + success = True + msg = '' + for opt in init_req.options: + if opt.name == 'field': + self._field = opt.values[0].stringValue + elif opt.name == 'size': + self._state.size = opt.values[0].intValue + elif opt.name == 'as': + self._as = opt.values[0].stringValue + + if self._field is None: + success = False + msg += ' must supply field name' + if self._state.size == 0: + success = False + msg += ' must supply window size' + if self._as == '': + success = False + msg += ' invalid as name' + + response = udf_pb2.Response() + response.init.success = success + response.init.error = msg[1:] + + return response + + def snapshot(self): + data = self._state.snapshot() + response = udf_pb2.Response() + response.snapshot.snapshot = data + + return response + + def restore(self, restore_req): + success = False + msg = '' + try: + self._state.restore(restore_req.snapshot) + success = True + except Exception as e: + success = False + msg = str(e) + + response = udf_pb2.Response() + response.restore.success = success + response.restore.error = msg + + return response + + def begin_batch(self): + raise Exception("not supported") + + def point(self, point): + response = udf_pb2.Response() + response.point.time = point.time + response.point.name = point.name + response.point.database = point.database + response.point.retentionPolicy = point.retentionPolicy + response.point.group = point.group + response.point.dimensions.extend(point.dimensions) + response.point.tags.update(point.tags) + + value = point.fieldsDouble[self._field] + avg = self._state.update(value) + response.point.fieldsDouble[self._as] = avg + self._agent.write_response(response) + + def end_batch(self, end_req): + raise Exception("not supported") + + +if __name__ == '__main__': + a = Agent() + h = AvgHandler(a) + a.handler = h + + logger.info("Starting Agent") + a.start() + a.wait() + logger.info("Agent finished") + diff --git a/udf/agent/py/agent.py b/udf/agent/py/agent.py new file mode 100644 index 000000000..a56e1aca7 --- /dev/null +++ b/udf/agent/py/agent.py @@ -0,0 +1,168 @@ +# Kapacitor UDF Agent implementation in Python +# +# Requires protobuf v3 +# pip install protobuf==3.0.0b2 + +import sys +import udf_pb2 +from threading import Lock, Thread +from Queue import Queue +import io + +import logging +logger = logging.getLogger() + + +# The Agent calls the appropriate methods on the Handler as requests are read off STDIN. +# +# Throwing an exception will cause the Agent to stop and an ErrorResponse to be sent. +# Some *Response objects (like SnapshotResponse) allow for returning their own error within the object itself. +# These types of errors will not stop the Agent and Kapacitor will deal with them appropriately. +# +# The Handler is called from a single thread, meaning methods will not be called concurrently. +# +# To write Points/Batches back to the Agent/Kapacitor use the Agent.write_response method, which is thread safe. +class Handler(object): + def info(): + pass + def init(init_req): + pass + def snapshot(): + pass + def restore(restore_req): + pass + def begin_batch(): + pass + def point(): + pass + def end_batch(end_req): + pass + + +# Python implementation of a Kapacitor UDF agent. +# This agent is responsible for reading and writing +# messages over STDIN and STDOUT. +# +# The Agent requires a Handler object in order to fulfill requests. +class Agent(object): + def __init__(self, _in=sys.stdin, out=sys.stdout,handler=None): + self._in = _in + self._out = out + self._thread = None + self.handler = handler + self._write_lock = Lock() + + # Start the agent. + # This method returns immediately + def start(self): + self._thread = Thread(target=self._read_loop) + self._thread.start() + + # Wait for the Agent to terminate. + # The Agent will terminate if STDIN is closed or an error occurs + def wait(self): + self._thread.join() + + # Write a response to STDOUT. + # This method is thread safe. + def write_response(self, response, flush=False): + if response is None: + raise Exception("cannot write None response") + + # Serialize message + self._write_lock.acquire() + try: + data = response.SerializeToString() + # Write message len + encodeUvarint(sys.stdout, len(data)) + # Write message + sys.stdout.write(data) + if flush: + sys.stdout.flush() + finally: + self._write_lock.release() + + # Read requests off stdin + def _read_loop(self): + request = udf_pb2.Request() + while True: + msg = 'unknown' + try: + size = decodeUvarint32(sys.stdin) + data = sys.stdin.read(size) + + request.ParseFromString(data) + + # use parsed message + msg = request.WhichOneof("message") + if msg == "info": + response = self.handler.info() + self.write_response(response, flush=True) + elif msg == "init": + response = self.handler.init(request.init) + self.write_response(response, flush=True) + elif msg == "keepalive": + response = udf_pb2.Response() + response.keepalive.time = request.keepalive.time + self.write_response(response, flush=True) + elif msg == "snapshot": + response = self.handler.snapshot() + self.write_response(response, flush=True) + elif msg == "restore": + response = self.handler.restore(request.restore) + self.write_response(response, flush=True) + elif msg == "begin": + self.handler.begin_batch() + elif msg == "point": + self.handler.point(request.point) + elif msg == "end": + self.handler.end_batch(request.end) + else: + logger.error("received unhandled request %s", msg) + except EOF: + break + except Exception as e: + error = "error processing request of type %s: %s" % (msg, e) + logger.error(error) + response = udf_pb2.Response() + response.error.error = error + self.write_response(response) + +# Indicates the end of a file/stream has been reached. +class EOF(Exception): + pass + +# Varint encode decode values +mask32uint = (1 << 32) - 1 +byteSize = 8 +shiftSize = byteSize - 1 +varintMoreMask = 2**shiftSize +varintMask = varintMoreMask - 1 + + +# Encode an unsigned varint +def encodeUvarint(writer, value): + bits = value & varintMask + value >>= shiftSize + while value: + writer.write(chr(varintMoreMask|bits)) + bits = value & varintMask + value >>= shiftSize + return writer.write(chr(bits)) + +# Decode an unsigned varint, max of 32 bits +def decodeUvarint32(reader): + result = 0 + shift = 0 + while True: + byte = reader.read(1) + if len(byte) == 0: + raise EOF + b = ord(byte) + result |= ((b & varintMask) << shift) + if not (b & varintMoreMask): + result &= mask32uint + return result + shift += shiftSize + if shift >= 32: + raise Exception("too many bytes when decoding varint, larger than 32bit uint") diff --git a/udf/agent/py/udf_pb2.py b/udf/agent/py/udf_pb2.py new file mode 100644 index 000000000..73bc66872 --- /dev/null +++ b/udf/agent/py/udf_pb2.py @@ -0,0 +1,1470 @@ +# Generated by the protocol buffer compiler. DO NOT EDIT! +# source: udf.proto + +import sys +_b=sys.version_info[0]<3 and (lambda x:x) or (lambda x:x.encode('latin1')) +from google.protobuf.internal import enum_type_wrapper +from google.protobuf import descriptor as _descriptor +from google.protobuf import message as _message +from google.protobuf import reflection as _reflection +from google.protobuf import symbol_database as _symbol_database +from google.protobuf import descriptor_pb2 +# @@protoc_insertion_point(imports) + +_sym_db = _symbol_database.Default() + + + + +DESCRIPTOR = _descriptor.FileDescriptor( + name='udf.proto', + package='udf', + syntax='proto3', + serialized_pb=_b('\n\tudf.proto\x12\x03udf\"\r\n\x0bInfoRequest\"\xbf\x01\n\x0cInfoResponse\x12\x1c\n\x05wants\x18\x01 \x01(\x0e\x32\r.udf.EdgeType\x12\x1f\n\x08provides\x18\x02 \x01(\x0e\x32\r.udf.EdgeType\x12/\n\x07options\x18\x03 \x03(\x0b\x32\x1e.udf.InfoResponse.OptionsEntry\x1a?\n\x0cOptionsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\x1e\n\x05value\x18\x02 \x01(\x0b\x32\x0f.udf.OptionInfo:\x02\x38\x01\"0\n\nOptionInfo\x12\"\n\nvalueTypes\x18\x01 \x03(\x0e\x32\x0e.udf.ValueType\"+\n\x0bInitRequest\x12\x1c\n\x07options\x18\x01 \x03(\x0b\x32\x0b.udf.Option\"8\n\x06Option\x12\x0c\n\x04name\x18\x01 \x01(\t\x12 \n\x06values\x18\x02 \x03(\x0b\x32\x10.udf.OptionValue\"\xa4\x01\n\x0bOptionValue\x12\x1c\n\x04type\x18\x01 \x01(\x0e\x32\x0e.udf.ValueType\x12\x13\n\tboolValue\x18\x02 \x01(\x08H\x00\x12\x12\n\x08intValue\x18\x03 \x01(\x03H\x00\x12\x15\n\x0b\x64oubleValue\x18\x04 \x01(\x01H\x00\x12\x15\n\x0bstringValue\x18\x05 \x01(\tH\x00\x12\x17\n\rdurationValue\x18\x06 \x01(\x03H\x00\x42\x07\n\x05value\".\n\x0cInitResponse\x12\x0f\n\x07success\x18\x01 \x01(\x08\x12\r\n\x05\x65rror\x18\x02 \x01(\t\"\x11\n\x0fSnapshotRequest\"$\n\x10SnapshotResponse\x12\x10\n\x08snapshot\x18\x01 \x01(\x0c\"\"\n\x0eRestoreRequest\x12\x10\n\x08snapshot\x18\x01 \x01(\x0c\"1\n\x0fRestoreResponse\x12\x0f\n\x07success\x18\x01 \x01(\x08\x12\r\n\x05\x65rror\x18\x02 \x01(\t\" \n\x10KeepaliveRequest\x12\x0c\n\x04time\x18\x01 \x01(\x03\"!\n\x11KeepaliveResponse\x12\x0c\n\x04time\x18\x01 \x01(\x03\"\x1e\n\rErrorResponse\x12\r\n\x05\x65rror\x18\x01 \x01(\t\"\x0c\n\nBeginBatch\"\xf4\x03\n\x05Point\x12\x0c\n\x04time\x18\x01 \x01(\x03\x12\x0c\n\x04name\x18\x02 \x01(\t\x12\x10\n\x08\x64\x61tabase\x18\x03 \x01(\t\x12\x17\n\x0fretentionPolicy\x18\x04 \x01(\t\x12\r\n\x05group\x18\x05 \x01(\t\x12\x12\n\ndimensions\x18\x06 \x03(\t\x12\"\n\x04tags\x18\x07 \x03(\x0b\x32\x14.udf.Point.TagsEntry\x12\x32\n\x0c\x66ieldsDouble\x18\x08 \x03(\x0b\x32\x1c.udf.Point.FieldsDoubleEntry\x12,\n\tfieldsInt\x18\t \x03(\x0b\x32\x19.udf.Point.FieldsIntEntry\x12\x32\n\x0c\x66ieldsString\x18\n \x03(\x0b\x32\x1c.udf.Point.FieldsStringEntry\x1a+\n\tTagsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\x1a\x33\n\x11\x46ieldsDoubleEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\x01:\x02\x38\x01\x1a\x30\n\x0e\x46ieldsIntEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\x03:\x02\x38\x01\x1a\x33\n\x11\x46ieldsStringEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\"\x89\x01\n\x08\x45ndBatch\x12\x0c\n\x04Name\x18\x01 \x01(\t\x12\r\n\x05Group\x18\x02 \x01(\t\x12\x0c\n\x04TMax\x18\x03 \x01(\x03\x12%\n\x04Tags\x18\x04 \x03(\x0b\x32\x17.udf.EndBatch.TagsEntry\x1a+\n\tTagsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\"\xb3\x02\n\x07Request\x12 \n\x04info\x18\x01 \x01(\x0b\x32\x10.udf.InfoRequestH\x00\x12 \n\x04init\x18\x02 \x01(\x0b\x32\x10.udf.InitRequestH\x00\x12*\n\tkeepalive\x18\x03 \x01(\x0b\x32\x15.udf.KeepaliveRequestH\x00\x12(\n\x08snapshot\x18\x04 \x01(\x0b\x32\x14.udf.SnapshotRequestH\x00\x12&\n\x07restore\x18\x05 \x01(\x0b\x32\x13.udf.RestoreRequestH\x00\x12 \n\x05\x62\x65gin\x18\x10 \x01(\x0b\x32\x0f.udf.BeginBatchH\x00\x12\x1b\n\x05point\x18\x11 \x01(\x0b\x32\n.udf.PointH\x00\x12\x1c\n\x03\x65nd\x18\x12 \x01(\x0b\x32\r.udf.EndBatchH\x00\x42\t\n\x07message\"\xde\x02\n\x08Response\x12!\n\x04info\x18\x01 \x01(\x0b\x32\x11.udf.InfoResponseH\x00\x12!\n\x04init\x18\x02 \x01(\x0b\x32\x11.udf.InitResponseH\x00\x12+\n\tkeepalive\x18\x03 \x01(\x0b\x32\x16.udf.KeepaliveResponseH\x00\x12)\n\x08snapshot\x18\x04 \x01(\x0b\x32\x15.udf.SnapshotResponseH\x00\x12\'\n\x07restore\x18\x05 \x01(\x0b\x32\x14.udf.RestoreResponseH\x00\x12#\n\x05\x65rror\x18\x06 \x01(\x0b\x32\x12.udf.ErrorResponseH\x00\x12 \n\x05\x62\x65gin\x18\x10 \x01(\x0b\x32\x0f.udf.BeginBatchH\x00\x12\x1b\n\x05point\x18\x11 \x01(\x0b\x32\n.udf.PointH\x00\x12\x1c\n\x03\x65nd\x18\x12 \x01(\x0b\x32\r.udf.EndBatchH\x00\x42\t\n\x07message*!\n\x08\x45\x64geType\x12\n\n\x06STREAM\x10\x00\x12\t\n\x05\x42\x41TCH\x10\x01*D\n\tValueType\x12\x08\n\x04\x42OOL\x10\x00\x12\x07\n\x03INT\x10\x01\x12\n\n\x06\x44OUBLE\x10\x02\x12\n\n\x06STRING\x10\x03\x12\x0c\n\x08\x44URATION\x10\x04\x62\x06proto3') +) +_sym_db.RegisterFileDescriptor(DESCRIPTOR) + +_EDGETYPE = _descriptor.EnumDescriptor( + name='EdgeType', + full_name='udf.EdgeType', + filename=None, + file=DESCRIPTOR, + values=[ + _descriptor.EnumValueDescriptor( + name='STREAM', index=0, number=0, + options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='BATCH', index=1, number=1, + options=None, + type=None), + ], + containing_type=None, + options=None, + serialized_start=2160, + serialized_end=2193, +) +_sym_db.RegisterEnumDescriptor(_EDGETYPE) + +EdgeType = enum_type_wrapper.EnumTypeWrapper(_EDGETYPE) +_VALUETYPE = _descriptor.EnumDescriptor( + name='ValueType', + full_name='udf.ValueType', + filename=None, + file=DESCRIPTOR, + values=[ + _descriptor.EnumValueDescriptor( + name='BOOL', index=0, number=0, + options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='INT', index=1, number=1, + options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='DOUBLE', index=2, number=2, + options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='STRING', index=3, number=3, + options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='DURATION', index=4, number=4, + options=None, + type=None), + ], + containing_type=None, + options=None, + serialized_start=2195, + serialized_end=2263, +) +_sym_db.RegisterEnumDescriptor(_VALUETYPE) + +ValueType = enum_type_wrapper.EnumTypeWrapper(_VALUETYPE) +STREAM = 0 +BATCH = 1 +BOOL = 0 +INT = 1 +DOUBLE = 2 +STRING = 3 +DURATION = 4 + + + +_INFOREQUEST = _descriptor.Descriptor( + name='InfoRequest', + full_name='udf.InfoRequest', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=18, + serialized_end=31, +) + + +_INFORESPONSE_OPTIONSENTRY = _descriptor.Descriptor( + name='OptionsEntry', + full_name='udf.InfoResponse.OptionsEntry', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='key', full_name='udf.InfoResponse.OptionsEntry.key', index=0, + number=1, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='value', full_name='udf.InfoResponse.OptionsEntry.value', index=1, + number=2, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + options=_descriptor._ParseOptions(descriptor_pb2.MessageOptions(), _b('8\001')), + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=162, + serialized_end=225, +) + +_INFORESPONSE = _descriptor.Descriptor( + name='InfoResponse', + full_name='udf.InfoResponse', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='wants', full_name='udf.InfoResponse.wants', index=0, + number=1, type=14, cpp_type=8, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='provides', full_name='udf.InfoResponse.provides', index=1, + number=2, type=14, cpp_type=8, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='options', full_name='udf.InfoResponse.options', index=2, + number=3, type=11, cpp_type=10, label=3, + has_default_value=False, default_value=[], + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + ], + extensions=[ + ], + nested_types=[_INFORESPONSE_OPTIONSENTRY, ], + enum_types=[ + ], + options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=34, + serialized_end=225, +) + + +_OPTIONINFO = _descriptor.Descriptor( + name='OptionInfo', + full_name='udf.OptionInfo', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='valueTypes', full_name='udf.OptionInfo.valueTypes', index=0, + number=1, type=14, cpp_type=8, label=3, + has_default_value=False, default_value=[], + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=227, + serialized_end=275, +) + + +_INITREQUEST = _descriptor.Descriptor( + name='InitRequest', + full_name='udf.InitRequest', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='options', full_name='udf.InitRequest.options', index=0, + number=1, type=11, cpp_type=10, label=3, + has_default_value=False, default_value=[], + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=277, + serialized_end=320, +) + + +_OPTION = _descriptor.Descriptor( + name='Option', + full_name='udf.Option', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='name', full_name='udf.Option.name', index=0, + number=1, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='values', full_name='udf.Option.values', index=1, + number=2, type=11, cpp_type=10, label=3, + has_default_value=False, default_value=[], + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=322, + serialized_end=378, +) + + +_OPTIONVALUE = _descriptor.Descriptor( + name='OptionValue', + full_name='udf.OptionValue', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='type', full_name='udf.OptionValue.type', index=0, + number=1, type=14, cpp_type=8, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='boolValue', full_name='udf.OptionValue.boolValue', index=1, + number=2, type=8, cpp_type=7, label=1, + has_default_value=False, default_value=False, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='intValue', full_name='udf.OptionValue.intValue', index=2, + number=3, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='doubleValue', full_name='udf.OptionValue.doubleValue', index=3, + number=4, type=1, cpp_type=5, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='stringValue', full_name='udf.OptionValue.stringValue', index=4, + number=5, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='durationValue', full_name='udf.OptionValue.durationValue', index=5, + number=6, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + _descriptor.OneofDescriptor( + name='value', full_name='udf.OptionValue.value', + index=0, containing_type=None, fields=[]), + ], + serialized_start=381, + serialized_end=545, +) + + +_INITRESPONSE = _descriptor.Descriptor( + name='InitResponse', + full_name='udf.InitResponse', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='success', full_name='udf.InitResponse.success', index=0, + number=1, type=8, cpp_type=7, label=1, + has_default_value=False, default_value=False, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='error', full_name='udf.InitResponse.error', index=1, + number=2, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=547, + serialized_end=593, +) + + +_SNAPSHOTREQUEST = _descriptor.Descriptor( + name='SnapshotRequest', + full_name='udf.SnapshotRequest', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=595, + serialized_end=612, +) + + +_SNAPSHOTRESPONSE = _descriptor.Descriptor( + name='SnapshotResponse', + full_name='udf.SnapshotResponse', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='snapshot', full_name='udf.SnapshotResponse.snapshot', index=0, + number=1, type=12, cpp_type=9, label=1, + has_default_value=False, default_value=_b(""), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=614, + serialized_end=650, +) + + +_RESTOREREQUEST = _descriptor.Descriptor( + name='RestoreRequest', + full_name='udf.RestoreRequest', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='snapshot', full_name='udf.RestoreRequest.snapshot', index=0, + number=1, type=12, cpp_type=9, label=1, + has_default_value=False, default_value=_b(""), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=652, + serialized_end=686, +) + + +_RESTORERESPONSE = _descriptor.Descriptor( + name='RestoreResponse', + full_name='udf.RestoreResponse', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='success', full_name='udf.RestoreResponse.success', index=0, + number=1, type=8, cpp_type=7, label=1, + has_default_value=False, default_value=False, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='error', full_name='udf.RestoreResponse.error', index=1, + number=2, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=688, + serialized_end=737, +) + + +_KEEPALIVEREQUEST = _descriptor.Descriptor( + name='KeepaliveRequest', + full_name='udf.KeepaliveRequest', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='time', full_name='udf.KeepaliveRequest.time', index=0, + number=1, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=739, + serialized_end=771, +) + + +_KEEPALIVERESPONSE = _descriptor.Descriptor( + name='KeepaliveResponse', + full_name='udf.KeepaliveResponse', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='time', full_name='udf.KeepaliveResponse.time', index=0, + number=1, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=773, + serialized_end=806, +) + + +_ERRORRESPONSE = _descriptor.Descriptor( + name='ErrorResponse', + full_name='udf.ErrorResponse', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='error', full_name='udf.ErrorResponse.error', index=0, + number=1, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=808, + serialized_end=838, +) + + +_BEGINBATCH = _descriptor.Descriptor( + name='BeginBatch', + full_name='udf.BeginBatch', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=840, + serialized_end=852, +) + + +_POINT_TAGSENTRY = _descriptor.Descriptor( + name='TagsEntry', + full_name='udf.Point.TagsEntry', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='key', full_name='udf.Point.TagsEntry.key', index=0, + number=1, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='value', full_name='udf.Point.TagsEntry.value', index=1, + number=2, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + options=_descriptor._ParseOptions(descriptor_pb2.MessageOptions(), _b('8\001')), + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=1156, + serialized_end=1199, +) + +_POINT_FIELDSDOUBLEENTRY = _descriptor.Descriptor( + name='FieldsDoubleEntry', + full_name='udf.Point.FieldsDoubleEntry', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='key', full_name='udf.Point.FieldsDoubleEntry.key', index=0, + number=1, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='value', full_name='udf.Point.FieldsDoubleEntry.value', index=1, + number=2, type=1, cpp_type=5, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + options=_descriptor._ParseOptions(descriptor_pb2.MessageOptions(), _b('8\001')), + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=1201, + serialized_end=1252, +) + +_POINT_FIELDSINTENTRY = _descriptor.Descriptor( + name='FieldsIntEntry', + full_name='udf.Point.FieldsIntEntry', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='key', full_name='udf.Point.FieldsIntEntry.key', index=0, + number=1, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='value', full_name='udf.Point.FieldsIntEntry.value', index=1, + number=2, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + options=_descriptor._ParseOptions(descriptor_pb2.MessageOptions(), _b('8\001')), + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=1254, + serialized_end=1302, +) + +_POINT_FIELDSSTRINGENTRY = _descriptor.Descriptor( + name='FieldsStringEntry', + full_name='udf.Point.FieldsStringEntry', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='key', full_name='udf.Point.FieldsStringEntry.key', index=0, + number=1, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='value', full_name='udf.Point.FieldsStringEntry.value', index=1, + number=2, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + options=_descriptor._ParseOptions(descriptor_pb2.MessageOptions(), _b('8\001')), + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=1304, + serialized_end=1355, +) + +_POINT = _descriptor.Descriptor( + name='Point', + full_name='udf.Point', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='time', full_name='udf.Point.time', index=0, + number=1, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='name', full_name='udf.Point.name', index=1, + number=2, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='database', full_name='udf.Point.database', index=2, + number=3, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='retentionPolicy', full_name='udf.Point.retentionPolicy', index=3, + number=4, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='group', full_name='udf.Point.group', index=4, + number=5, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='dimensions', full_name='udf.Point.dimensions', index=5, + number=6, type=9, cpp_type=9, label=3, + has_default_value=False, default_value=[], + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='tags', full_name='udf.Point.tags', index=6, + number=7, type=11, cpp_type=10, label=3, + has_default_value=False, default_value=[], + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='fieldsDouble', full_name='udf.Point.fieldsDouble', index=7, + number=8, type=11, cpp_type=10, label=3, + has_default_value=False, default_value=[], + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='fieldsInt', full_name='udf.Point.fieldsInt', index=8, + number=9, type=11, cpp_type=10, label=3, + has_default_value=False, default_value=[], + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='fieldsString', full_name='udf.Point.fieldsString', index=9, + number=10, type=11, cpp_type=10, label=3, + has_default_value=False, default_value=[], + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + ], + extensions=[ + ], + nested_types=[_POINT_TAGSENTRY, _POINT_FIELDSDOUBLEENTRY, _POINT_FIELDSINTENTRY, _POINT_FIELDSSTRINGENTRY, ], + enum_types=[ + ], + options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=855, + serialized_end=1355, +) + + +_ENDBATCH_TAGSENTRY = _descriptor.Descriptor( + name='TagsEntry', + full_name='udf.EndBatch.TagsEntry', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='key', full_name='udf.EndBatch.TagsEntry.key', index=0, + number=1, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='value', full_name='udf.EndBatch.TagsEntry.value', index=1, + number=2, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + options=_descriptor._ParseOptions(descriptor_pb2.MessageOptions(), _b('8\001')), + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=1156, + serialized_end=1199, +) + +_ENDBATCH = _descriptor.Descriptor( + name='EndBatch', + full_name='udf.EndBatch', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='Name', full_name='udf.EndBatch.Name', index=0, + number=1, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='Group', full_name='udf.EndBatch.Group', index=1, + number=2, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='TMax', full_name='udf.EndBatch.TMax', index=2, + number=3, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='Tags', full_name='udf.EndBatch.Tags', index=3, + number=4, type=11, cpp_type=10, label=3, + has_default_value=False, default_value=[], + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + ], + extensions=[ + ], + nested_types=[_ENDBATCH_TAGSENTRY, ], + enum_types=[ + ], + options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=1358, + serialized_end=1495, +) + + +_REQUEST = _descriptor.Descriptor( + name='Request', + full_name='udf.Request', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='info', full_name='udf.Request.info', index=0, + number=1, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='init', full_name='udf.Request.init', index=1, + number=2, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='keepalive', full_name='udf.Request.keepalive', index=2, + number=3, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='snapshot', full_name='udf.Request.snapshot', index=3, + number=4, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='restore', full_name='udf.Request.restore', index=4, + number=5, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='begin', full_name='udf.Request.begin', index=5, + number=16, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='point', full_name='udf.Request.point', index=6, + number=17, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='end', full_name='udf.Request.end', index=7, + number=18, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + _descriptor.OneofDescriptor( + name='message', full_name='udf.Request.message', + index=0, containing_type=None, fields=[]), + ], + serialized_start=1498, + serialized_end=1805, +) + + +_RESPONSE = _descriptor.Descriptor( + name='Response', + full_name='udf.Response', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='info', full_name='udf.Response.info', index=0, + number=1, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='init', full_name='udf.Response.init', index=1, + number=2, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='keepalive', full_name='udf.Response.keepalive', index=2, + number=3, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='snapshot', full_name='udf.Response.snapshot', index=3, + number=4, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='restore', full_name='udf.Response.restore', index=4, + number=5, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='error', full_name='udf.Response.error', index=5, + number=6, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='begin', full_name='udf.Response.begin', index=6, + number=16, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='point', full_name='udf.Response.point', index=7, + number=17, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + _descriptor.FieldDescriptor( + name='end', full_name='udf.Response.end', index=8, + number=18, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + options=None, + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + _descriptor.OneofDescriptor( + name='message', full_name='udf.Response.message', + index=0, containing_type=None, fields=[]), + ], + serialized_start=1808, + serialized_end=2158, +) + +_INFORESPONSE_OPTIONSENTRY.fields_by_name['value'].message_type = _OPTIONINFO +_INFORESPONSE_OPTIONSENTRY.containing_type = _INFORESPONSE +_INFORESPONSE.fields_by_name['wants'].enum_type = _EDGETYPE +_INFORESPONSE.fields_by_name['provides'].enum_type = _EDGETYPE +_INFORESPONSE.fields_by_name['options'].message_type = _INFORESPONSE_OPTIONSENTRY +_OPTIONINFO.fields_by_name['valueTypes'].enum_type = _VALUETYPE +_INITREQUEST.fields_by_name['options'].message_type = _OPTION +_OPTION.fields_by_name['values'].message_type = _OPTIONVALUE +_OPTIONVALUE.fields_by_name['type'].enum_type = _VALUETYPE +_OPTIONVALUE.oneofs_by_name['value'].fields.append( + _OPTIONVALUE.fields_by_name['boolValue']) +_OPTIONVALUE.fields_by_name['boolValue'].containing_oneof = _OPTIONVALUE.oneofs_by_name['value'] +_OPTIONVALUE.oneofs_by_name['value'].fields.append( + _OPTIONVALUE.fields_by_name['intValue']) +_OPTIONVALUE.fields_by_name['intValue'].containing_oneof = _OPTIONVALUE.oneofs_by_name['value'] +_OPTIONVALUE.oneofs_by_name['value'].fields.append( + _OPTIONVALUE.fields_by_name['doubleValue']) +_OPTIONVALUE.fields_by_name['doubleValue'].containing_oneof = _OPTIONVALUE.oneofs_by_name['value'] +_OPTIONVALUE.oneofs_by_name['value'].fields.append( + _OPTIONVALUE.fields_by_name['stringValue']) +_OPTIONVALUE.fields_by_name['stringValue'].containing_oneof = _OPTIONVALUE.oneofs_by_name['value'] +_OPTIONVALUE.oneofs_by_name['value'].fields.append( + _OPTIONVALUE.fields_by_name['durationValue']) +_OPTIONVALUE.fields_by_name['durationValue'].containing_oneof = _OPTIONVALUE.oneofs_by_name['value'] +_POINT_TAGSENTRY.containing_type = _POINT +_POINT_FIELDSDOUBLEENTRY.containing_type = _POINT +_POINT_FIELDSINTENTRY.containing_type = _POINT +_POINT_FIELDSSTRINGENTRY.containing_type = _POINT +_POINT.fields_by_name['tags'].message_type = _POINT_TAGSENTRY +_POINT.fields_by_name['fieldsDouble'].message_type = _POINT_FIELDSDOUBLEENTRY +_POINT.fields_by_name['fieldsInt'].message_type = _POINT_FIELDSINTENTRY +_POINT.fields_by_name['fieldsString'].message_type = _POINT_FIELDSSTRINGENTRY +_ENDBATCH_TAGSENTRY.containing_type = _ENDBATCH +_ENDBATCH.fields_by_name['Tags'].message_type = _ENDBATCH_TAGSENTRY +_REQUEST.fields_by_name['info'].message_type = _INFOREQUEST +_REQUEST.fields_by_name['init'].message_type = _INITREQUEST +_REQUEST.fields_by_name['keepalive'].message_type = _KEEPALIVEREQUEST +_REQUEST.fields_by_name['snapshot'].message_type = _SNAPSHOTREQUEST +_REQUEST.fields_by_name['restore'].message_type = _RESTOREREQUEST +_REQUEST.fields_by_name['begin'].message_type = _BEGINBATCH +_REQUEST.fields_by_name['point'].message_type = _POINT +_REQUEST.fields_by_name['end'].message_type = _ENDBATCH +_REQUEST.oneofs_by_name['message'].fields.append( + _REQUEST.fields_by_name['info']) +_REQUEST.fields_by_name['info'].containing_oneof = _REQUEST.oneofs_by_name['message'] +_REQUEST.oneofs_by_name['message'].fields.append( + _REQUEST.fields_by_name['init']) +_REQUEST.fields_by_name['init'].containing_oneof = _REQUEST.oneofs_by_name['message'] +_REQUEST.oneofs_by_name['message'].fields.append( + _REQUEST.fields_by_name['keepalive']) +_REQUEST.fields_by_name['keepalive'].containing_oneof = _REQUEST.oneofs_by_name['message'] +_REQUEST.oneofs_by_name['message'].fields.append( + _REQUEST.fields_by_name['snapshot']) +_REQUEST.fields_by_name['snapshot'].containing_oneof = _REQUEST.oneofs_by_name['message'] +_REQUEST.oneofs_by_name['message'].fields.append( + _REQUEST.fields_by_name['restore']) +_REQUEST.fields_by_name['restore'].containing_oneof = _REQUEST.oneofs_by_name['message'] +_REQUEST.oneofs_by_name['message'].fields.append( + _REQUEST.fields_by_name['begin']) +_REQUEST.fields_by_name['begin'].containing_oneof = _REQUEST.oneofs_by_name['message'] +_REQUEST.oneofs_by_name['message'].fields.append( + _REQUEST.fields_by_name['point']) +_REQUEST.fields_by_name['point'].containing_oneof = _REQUEST.oneofs_by_name['message'] +_REQUEST.oneofs_by_name['message'].fields.append( + _REQUEST.fields_by_name['end']) +_REQUEST.fields_by_name['end'].containing_oneof = _REQUEST.oneofs_by_name['message'] +_RESPONSE.fields_by_name['info'].message_type = _INFORESPONSE +_RESPONSE.fields_by_name['init'].message_type = _INITRESPONSE +_RESPONSE.fields_by_name['keepalive'].message_type = _KEEPALIVERESPONSE +_RESPONSE.fields_by_name['snapshot'].message_type = _SNAPSHOTRESPONSE +_RESPONSE.fields_by_name['restore'].message_type = _RESTORERESPONSE +_RESPONSE.fields_by_name['error'].message_type = _ERRORRESPONSE +_RESPONSE.fields_by_name['begin'].message_type = _BEGINBATCH +_RESPONSE.fields_by_name['point'].message_type = _POINT +_RESPONSE.fields_by_name['end'].message_type = _ENDBATCH +_RESPONSE.oneofs_by_name['message'].fields.append( + _RESPONSE.fields_by_name['info']) +_RESPONSE.fields_by_name['info'].containing_oneof = _RESPONSE.oneofs_by_name['message'] +_RESPONSE.oneofs_by_name['message'].fields.append( + _RESPONSE.fields_by_name['init']) +_RESPONSE.fields_by_name['init'].containing_oneof = _RESPONSE.oneofs_by_name['message'] +_RESPONSE.oneofs_by_name['message'].fields.append( + _RESPONSE.fields_by_name['keepalive']) +_RESPONSE.fields_by_name['keepalive'].containing_oneof = _RESPONSE.oneofs_by_name['message'] +_RESPONSE.oneofs_by_name['message'].fields.append( + _RESPONSE.fields_by_name['snapshot']) +_RESPONSE.fields_by_name['snapshot'].containing_oneof = _RESPONSE.oneofs_by_name['message'] +_RESPONSE.oneofs_by_name['message'].fields.append( + _RESPONSE.fields_by_name['restore']) +_RESPONSE.fields_by_name['restore'].containing_oneof = _RESPONSE.oneofs_by_name['message'] +_RESPONSE.oneofs_by_name['message'].fields.append( + _RESPONSE.fields_by_name['error']) +_RESPONSE.fields_by_name['error'].containing_oneof = _RESPONSE.oneofs_by_name['message'] +_RESPONSE.oneofs_by_name['message'].fields.append( + _RESPONSE.fields_by_name['begin']) +_RESPONSE.fields_by_name['begin'].containing_oneof = _RESPONSE.oneofs_by_name['message'] +_RESPONSE.oneofs_by_name['message'].fields.append( + _RESPONSE.fields_by_name['point']) +_RESPONSE.fields_by_name['point'].containing_oneof = _RESPONSE.oneofs_by_name['message'] +_RESPONSE.oneofs_by_name['message'].fields.append( + _RESPONSE.fields_by_name['end']) +_RESPONSE.fields_by_name['end'].containing_oneof = _RESPONSE.oneofs_by_name['message'] +DESCRIPTOR.message_types_by_name['InfoRequest'] = _INFOREQUEST +DESCRIPTOR.message_types_by_name['InfoResponse'] = _INFORESPONSE +DESCRIPTOR.message_types_by_name['OptionInfo'] = _OPTIONINFO +DESCRIPTOR.message_types_by_name['InitRequest'] = _INITREQUEST +DESCRIPTOR.message_types_by_name['Option'] = _OPTION +DESCRIPTOR.message_types_by_name['OptionValue'] = _OPTIONVALUE +DESCRIPTOR.message_types_by_name['InitResponse'] = _INITRESPONSE +DESCRIPTOR.message_types_by_name['SnapshotRequest'] = _SNAPSHOTREQUEST +DESCRIPTOR.message_types_by_name['SnapshotResponse'] = _SNAPSHOTRESPONSE +DESCRIPTOR.message_types_by_name['RestoreRequest'] = _RESTOREREQUEST +DESCRIPTOR.message_types_by_name['RestoreResponse'] = _RESTORERESPONSE +DESCRIPTOR.message_types_by_name['KeepaliveRequest'] = _KEEPALIVEREQUEST +DESCRIPTOR.message_types_by_name['KeepaliveResponse'] = _KEEPALIVERESPONSE +DESCRIPTOR.message_types_by_name['ErrorResponse'] = _ERRORRESPONSE +DESCRIPTOR.message_types_by_name['BeginBatch'] = _BEGINBATCH +DESCRIPTOR.message_types_by_name['Point'] = _POINT +DESCRIPTOR.message_types_by_name['EndBatch'] = _ENDBATCH +DESCRIPTOR.message_types_by_name['Request'] = _REQUEST +DESCRIPTOR.message_types_by_name['Response'] = _RESPONSE +DESCRIPTOR.enum_types_by_name['EdgeType'] = _EDGETYPE +DESCRIPTOR.enum_types_by_name['ValueType'] = _VALUETYPE + +InfoRequest = _reflection.GeneratedProtocolMessageType('InfoRequest', (_message.Message,), dict( + DESCRIPTOR = _INFOREQUEST, + __module__ = 'udf_pb2' + # @@protoc_insertion_point(class_scope:udf.InfoRequest) + )) +_sym_db.RegisterMessage(InfoRequest) + +InfoResponse = _reflection.GeneratedProtocolMessageType('InfoResponse', (_message.Message,), dict( + + OptionsEntry = _reflection.GeneratedProtocolMessageType('OptionsEntry', (_message.Message,), dict( + DESCRIPTOR = _INFORESPONSE_OPTIONSENTRY, + __module__ = 'udf_pb2' + # @@protoc_insertion_point(class_scope:udf.InfoResponse.OptionsEntry) + )) + , + DESCRIPTOR = _INFORESPONSE, + __module__ = 'udf_pb2' + # @@protoc_insertion_point(class_scope:udf.InfoResponse) + )) +_sym_db.RegisterMessage(InfoResponse) +_sym_db.RegisterMessage(InfoResponse.OptionsEntry) + +OptionInfo = _reflection.GeneratedProtocolMessageType('OptionInfo', (_message.Message,), dict( + DESCRIPTOR = _OPTIONINFO, + __module__ = 'udf_pb2' + # @@protoc_insertion_point(class_scope:udf.OptionInfo) + )) +_sym_db.RegisterMessage(OptionInfo) + +InitRequest = _reflection.GeneratedProtocolMessageType('InitRequest', (_message.Message,), dict( + DESCRIPTOR = _INITREQUEST, + __module__ = 'udf_pb2' + # @@protoc_insertion_point(class_scope:udf.InitRequest) + )) +_sym_db.RegisterMessage(InitRequest) + +Option = _reflection.GeneratedProtocolMessageType('Option', (_message.Message,), dict( + DESCRIPTOR = _OPTION, + __module__ = 'udf_pb2' + # @@protoc_insertion_point(class_scope:udf.Option) + )) +_sym_db.RegisterMessage(Option) + +OptionValue = _reflection.GeneratedProtocolMessageType('OptionValue', (_message.Message,), dict( + DESCRIPTOR = _OPTIONVALUE, + __module__ = 'udf_pb2' + # @@protoc_insertion_point(class_scope:udf.OptionValue) + )) +_sym_db.RegisterMessage(OptionValue) + +InitResponse = _reflection.GeneratedProtocolMessageType('InitResponse', (_message.Message,), dict( + DESCRIPTOR = _INITRESPONSE, + __module__ = 'udf_pb2' + # @@protoc_insertion_point(class_scope:udf.InitResponse) + )) +_sym_db.RegisterMessage(InitResponse) + +SnapshotRequest = _reflection.GeneratedProtocolMessageType('SnapshotRequest', (_message.Message,), dict( + DESCRIPTOR = _SNAPSHOTREQUEST, + __module__ = 'udf_pb2' + # @@protoc_insertion_point(class_scope:udf.SnapshotRequest) + )) +_sym_db.RegisterMessage(SnapshotRequest) + +SnapshotResponse = _reflection.GeneratedProtocolMessageType('SnapshotResponse', (_message.Message,), dict( + DESCRIPTOR = _SNAPSHOTRESPONSE, + __module__ = 'udf_pb2' + # @@protoc_insertion_point(class_scope:udf.SnapshotResponse) + )) +_sym_db.RegisterMessage(SnapshotResponse) + +RestoreRequest = _reflection.GeneratedProtocolMessageType('RestoreRequest', (_message.Message,), dict( + DESCRIPTOR = _RESTOREREQUEST, + __module__ = 'udf_pb2' + # @@protoc_insertion_point(class_scope:udf.RestoreRequest) + )) +_sym_db.RegisterMessage(RestoreRequest) + +RestoreResponse = _reflection.GeneratedProtocolMessageType('RestoreResponse', (_message.Message,), dict( + DESCRIPTOR = _RESTORERESPONSE, + __module__ = 'udf_pb2' + # @@protoc_insertion_point(class_scope:udf.RestoreResponse) + )) +_sym_db.RegisterMessage(RestoreResponse) + +KeepaliveRequest = _reflection.GeneratedProtocolMessageType('KeepaliveRequest', (_message.Message,), dict( + DESCRIPTOR = _KEEPALIVEREQUEST, + __module__ = 'udf_pb2' + # @@protoc_insertion_point(class_scope:udf.KeepaliveRequest) + )) +_sym_db.RegisterMessage(KeepaliveRequest) + +KeepaliveResponse = _reflection.GeneratedProtocolMessageType('KeepaliveResponse', (_message.Message,), dict( + DESCRIPTOR = _KEEPALIVERESPONSE, + __module__ = 'udf_pb2' + # @@protoc_insertion_point(class_scope:udf.KeepaliveResponse) + )) +_sym_db.RegisterMessage(KeepaliveResponse) + +ErrorResponse = _reflection.GeneratedProtocolMessageType('ErrorResponse', (_message.Message,), dict( + DESCRIPTOR = _ERRORRESPONSE, + __module__ = 'udf_pb2' + # @@protoc_insertion_point(class_scope:udf.ErrorResponse) + )) +_sym_db.RegisterMessage(ErrorResponse) + +BeginBatch = _reflection.GeneratedProtocolMessageType('BeginBatch', (_message.Message,), dict( + DESCRIPTOR = _BEGINBATCH, + __module__ = 'udf_pb2' + # @@protoc_insertion_point(class_scope:udf.BeginBatch) + )) +_sym_db.RegisterMessage(BeginBatch) + +Point = _reflection.GeneratedProtocolMessageType('Point', (_message.Message,), dict( + + TagsEntry = _reflection.GeneratedProtocolMessageType('TagsEntry', (_message.Message,), dict( + DESCRIPTOR = _POINT_TAGSENTRY, + __module__ = 'udf_pb2' + # @@protoc_insertion_point(class_scope:udf.Point.TagsEntry) + )) + , + + FieldsDoubleEntry = _reflection.GeneratedProtocolMessageType('FieldsDoubleEntry', (_message.Message,), dict( + DESCRIPTOR = _POINT_FIELDSDOUBLEENTRY, + __module__ = 'udf_pb2' + # @@protoc_insertion_point(class_scope:udf.Point.FieldsDoubleEntry) + )) + , + + FieldsIntEntry = _reflection.GeneratedProtocolMessageType('FieldsIntEntry', (_message.Message,), dict( + DESCRIPTOR = _POINT_FIELDSINTENTRY, + __module__ = 'udf_pb2' + # @@protoc_insertion_point(class_scope:udf.Point.FieldsIntEntry) + )) + , + + FieldsStringEntry = _reflection.GeneratedProtocolMessageType('FieldsStringEntry', (_message.Message,), dict( + DESCRIPTOR = _POINT_FIELDSSTRINGENTRY, + __module__ = 'udf_pb2' + # @@protoc_insertion_point(class_scope:udf.Point.FieldsStringEntry) + )) + , + DESCRIPTOR = _POINT, + __module__ = 'udf_pb2' + # @@protoc_insertion_point(class_scope:udf.Point) + )) +_sym_db.RegisterMessage(Point) +_sym_db.RegisterMessage(Point.TagsEntry) +_sym_db.RegisterMessage(Point.FieldsDoubleEntry) +_sym_db.RegisterMessage(Point.FieldsIntEntry) +_sym_db.RegisterMessage(Point.FieldsStringEntry) + +EndBatch = _reflection.GeneratedProtocolMessageType('EndBatch', (_message.Message,), dict( + + TagsEntry = _reflection.GeneratedProtocolMessageType('TagsEntry', (_message.Message,), dict( + DESCRIPTOR = _ENDBATCH_TAGSENTRY, + __module__ = 'udf_pb2' + # @@protoc_insertion_point(class_scope:udf.EndBatch.TagsEntry) + )) + , + DESCRIPTOR = _ENDBATCH, + __module__ = 'udf_pb2' + # @@protoc_insertion_point(class_scope:udf.EndBatch) + )) +_sym_db.RegisterMessage(EndBatch) +_sym_db.RegisterMessage(EndBatch.TagsEntry) + +Request = _reflection.GeneratedProtocolMessageType('Request', (_message.Message,), dict( + DESCRIPTOR = _REQUEST, + __module__ = 'udf_pb2' + # @@protoc_insertion_point(class_scope:udf.Request) + )) +_sym_db.RegisterMessage(Request) + +Response = _reflection.GeneratedProtocolMessageType('Response', (_message.Message,), dict( + DESCRIPTOR = _RESPONSE, + __module__ = 'udf_pb2' + # @@protoc_insertion_point(class_scope:udf.Response) + )) +_sym_db.RegisterMessage(Response) + + +_INFORESPONSE_OPTIONSENTRY.has_options = True +_INFORESPONSE_OPTIONSENTRY._options = _descriptor._ParseOptions(descriptor_pb2.MessageOptions(), _b('8\001')) +_POINT_TAGSENTRY.has_options = True +_POINT_TAGSENTRY._options = _descriptor._ParseOptions(descriptor_pb2.MessageOptions(), _b('8\001')) +_POINT_FIELDSDOUBLEENTRY.has_options = True +_POINT_FIELDSDOUBLEENTRY._options = _descriptor._ParseOptions(descriptor_pb2.MessageOptions(), _b('8\001')) +_POINT_FIELDSINTENTRY.has_options = True +_POINT_FIELDSINTENTRY._options = _descriptor._ParseOptions(descriptor_pb2.MessageOptions(), _b('8\001')) +_POINT_FIELDSSTRINGENTRY.has_options = True +_POINT_FIELDSSTRINGENTRY._options = _descriptor._ParseOptions(descriptor_pb2.MessageOptions(), _b('8\001')) +_ENDBATCH_TAGSENTRY.has_options = True +_ENDBATCH_TAGSENTRY._options = _descriptor._ParseOptions(descriptor_pb2.MessageOptions(), _b('8\001')) +# @@protoc_insertion_point(module_scope) diff --git a/udf/io.go b/udf/io.go new file mode 100644 index 000000000..cca491f17 --- /dev/null +++ b/udf/io.go @@ -0,0 +1,67 @@ +package udf + +import ( + "encoding/binary" + "fmt" + "io" + + "github.com/gogo/protobuf/proto" +) + +//go:generate protoc --gogo_out=./ --python_out=./agent/py/ udf.proto +type ByteReadReader interface { + io.Reader + io.ByteReader +} + +// Write the message to the io.Writer with a varint size header. +func WriteMessage(msg proto.Message, w io.Writer) error { + // marshal message + data, err := proto.Marshal(msg) + if err != nil { + return err + } + varint := make([]byte, binary.MaxVarintLen32) + n := binary.PutUvarint(varint, uint64(len(data))) + + _, err = w.Write(varint[:n]) + if err != nil { + return err + } + _, err = w.Write(data) + if err != nil { + return err + } + return nil +} + +// Read a message from io.ByteReader by first reading a varint size, +// and then reading and decoding the message object. +// If buf is not big enough a new buffer will be allocated to replace buf. +func ReadMessage(buf *[]byte, r ByteReadReader, msg proto.Message) error { + size, err := binary.ReadUvarint(r) + if err != nil { + return err + } + if cap(*buf) < int(size) { + *buf = make([]byte, size) + } + b := (*buf)[:size] + read := uint64(0) + + for read != size { + n, err := r.Read(b[read:]) + if err == io.EOF { + return fmt.Errorf("unexpected EOF, expected %d more bytes", size) + } + if err != nil { + return err + } + read += uint64(n) + } + err = proto.Unmarshal(b, msg) + if err != nil { + return err + } + return nil +} diff --git a/udf/io_test.go b/udf/io_test.go new file mode 100644 index 000000000..8fb9424bf --- /dev/null +++ b/udf/io_test.go @@ -0,0 +1,69 @@ +package udf_test + +import ( + "bytes" + "reflect" + "testing" + + "github.com/influxdata/kapacitor/udf" +) + +func TestMessage_ReadWrite(t *testing.T) { + req := &udf.Request{} + req.Message = &udf.Request_Keepalive{ + Keepalive: &udf.KeepaliveRequest{ + Time: 42, + }, + } + + var buf bytes.Buffer + + err := udf.WriteMessage(req, &buf) + if err != nil { + t.Fatal(err) + } + + nreq := &udf.Request{} + var b []byte + err = udf.ReadMessage(&b, &buf, nreq) + if err != nil { + t.Fatal(err) + } + + if !reflect.DeepEqual(req, nreq) { + t.Errorf("unexpected request: \ngot %v\nexp %v", nreq, req) + } +} + +func TestMessage_ReadWriteMultiple(t *testing.T) { + req := &udf.Request{} + req.Message = &udf.Request_Keepalive{ + Keepalive: &udf.KeepaliveRequest{ + Time: 42, + }, + } + + var buf bytes.Buffer + + var count int = 1e4 + for i := 0; i < count; i++ { + err := udf.WriteMessage(req, &buf) + if err != nil { + t.Fatal(err) + } + } + + nreq := &udf.Request{} + var b []byte + + for i := 0; i < count; i++ { + err := udf.ReadMessage(&b, &buf, nreq) + if err != nil { + t.Fatal(err) + } + + if !reflect.DeepEqual(req, nreq) { + t.Fatalf("unexpected request: i:%d \ngot %v\nexp %v", i, nreq, req) + } + } +} diff --git a/udf/udf.pb.go b/udf/udf.pb.go new file mode 100644 index 000000000..7afddd0d4 --- /dev/null +++ b/udf/udf.pb.go @@ -0,0 +1,1039 @@ +// Code generated by protoc-gen-gogo. +// source: udf.proto +// DO NOT EDIT! + +/* +Package udf is a generated protocol buffer package. + +It is generated from these files: + udf.proto + +It has these top-level messages: + InfoRequest + InfoResponse + OptionInfo + InitRequest + Option + OptionValue + InitResponse + SnapshotRequest + SnapshotResponse + RestoreRequest + RestoreResponse + KeepaliveRequest + KeepaliveResponse + ErrorResponse + BeginBatch + Point + EndBatch + Request + Response +*/ +package udf + +import proto "github.com/gogo/protobuf/proto" +import fmt "fmt" +import math "math" + +// Reference imports to suppress errors if they are not otherwise used. +var _ = proto.Marshal +var _ = fmt.Errorf +var _ = math.Inf + +type EdgeType int32 + +const ( + EdgeType_STREAM EdgeType = 0 + EdgeType_BATCH EdgeType = 1 +) + +var EdgeType_name = map[int32]string{ + 0: "STREAM", + 1: "BATCH", +} +var EdgeType_value = map[string]int32{ + "STREAM": 0, + "BATCH": 1, +} + +func (x EdgeType) String() string { + return proto.EnumName(EdgeType_name, int32(x)) +} + +type ValueType int32 + +const ( + ValueType_BOOL ValueType = 0 + ValueType_INT ValueType = 1 + ValueType_DOUBLE ValueType = 2 + ValueType_STRING ValueType = 3 + ValueType_DURATION ValueType = 4 +) + +var ValueType_name = map[int32]string{ + 0: "BOOL", + 1: "INT", + 2: "DOUBLE", + 3: "STRING", + 4: "DURATION", +} +var ValueType_value = map[string]int32{ + "BOOL": 0, + "INT": 1, + "DOUBLE": 2, + "STRING": 3, + "DURATION": 4, +} + +func (x ValueType) String() string { + return proto.EnumName(ValueType_name, int32(x)) +} + +// Request that the process return information about available Options. +type InfoRequest struct { +} + +func (m *InfoRequest) Reset() { *m = InfoRequest{} } +func (m *InfoRequest) String() string { return proto.CompactTextString(m) } +func (*InfoRequest) ProtoMessage() {} + +type InfoResponse struct { + Wants EdgeType `protobuf:"varint,1,opt,name=wants,proto3,enum=udf.EdgeType" json:"wants,omitempty"` + Provides EdgeType `protobuf:"varint,2,opt,name=provides,proto3,enum=udf.EdgeType" json:"provides,omitempty"` + Options map[string]*OptionInfo `protobuf:"bytes,3,rep,name=options" json:"options,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value"` +} + +func (m *InfoResponse) Reset() { *m = InfoResponse{} } +func (m *InfoResponse) String() string { return proto.CompactTextString(m) } +func (*InfoResponse) ProtoMessage() {} + +func (m *InfoResponse) GetOptions() map[string]*OptionInfo { + if m != nil { + return m.Options + } + return nil +} + +type OptionInfo struct { + ValueTypes []ValueType `protobuf:"varint,1,rep,name=valueTypes,enum=udf.ValueType" json:"valueTypes,omitempty"` +} + +func (m *OptionInfo) Reset() { *m = OptionInfo{} } +func (m *OptionInfo) String() string { return proto.CompactTextString(m) } +func (*OptionInfo) ProtoMessage() {} + +// Request that the process initialize itself with the provided options. +type InitRequest struct { + Options []*Option `protobuf:"bytes,1,rep,name=options" json:"options,omitempty"` +} + +func (m *InitRequest) Reset() { *m = InitRequest{} } +func (m *InitRequest) String() string { return proto.CompactTextString(m) } +func (*InitRequest) ProtoMessage() {} + +func (m *InitRequest) GetOptions() []*Option { + if m != nil { + return m.Options + } + return nil +} + +type Option struct { + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + Values []*OptionValue `protobuf:"bytes,2,rep,name=values" json:"values,omitempty"` +} + +func (m *Option) Reset() { *m = Option{} } +func (m *Option) String() string { return proto.CompactTextString(m) } +func (*Option) ProtoMessage() {} + +func (m *Option) GetValues() []*OptionValue { + if m != nil { + return m.Values + } + return nil +} + +type OptionValue struct { + Type ValueType `protobuf:"varint,1,opt,name=type,proto3,enum=udf.ValueType" json:"type,omitempty"` + // Types that are valid to be assigned to Value: + // *OptionValue_BoolValue + // *OptionValue_IntValue + // *OptionValue_DoubleValue + // *OptionValue_StringValue + // *OptionValue_DurationValue + Value isOptionValue_Value `protobuf_oneof:"value"` +} + +func (m *OptionValue) Reset() { *m = OptionValue{} } +func (m *OptionValue) String() string { return proto.CompactTextString(m) } +func (*OptionValue) ProtoMessage() {} + +type isOptionValue_Value interface { + isOptionValue_Value() +} + +type OptionValue_BoolValue struct { + BoolValue bool `protobuf:"varint,2,opt,name=boolValue,proto3,oneof"` +} +type OptionValue_IntValue struct { + IntValue int64 `protobuf:"varint,3,opt,name=intValue,proto3,oneof"` +} +type OptionValue_DoubleValue struct { + DoubleValue float64 `protobuf:"fixed64,4,opt,name=doubleValue,proto3,oneof"` +} +type OptionValue_StringValue struct { + StringValue string `protobuf:"bytes,5,opt,name=stringValue,proto3,oneof"` +} +type OptionValue_DurationValue struct { + DurationValue int64 `protobuf:"varint,6,opt,name=durationValue,proto3,oneof"` +} + +func (*OptionValue_BoolValue) isOptionValue_Value() {} +func (*OptionValue_IntValue) isOptionValue_Value() {} +func (*OptionValue_DoubleValue) isOptionValue_Value() {} +func (*OptionValue_StringValue) isOptionValue_Value() {} +func (*OptionValue_DurationValue) isOptionValue_Value() {} + +func (m *OptionValue) GetValue() isOptionValue_Value { + if m != nil { + return m.Value + } + return nil +} + +func (m *OptionValue) GetBoolValue() bool { + if x, ok := m.GetValue().(*OptionValue_BoolValue); ok { + return x.BoolValue + } + return false +} + +func (m *OptionValue) GetIntValue() int64 { + if x, ok := m.GetValue().(*OptionValue_IntValue); ok { + return x.IntValue + } + return 0 +} + +func (m *OptionValue) GetDoubleValue() float64 { + if x, ok := m.GetValue().(*OptionValue_DoubleValue); ok { + return x.DoubleValue + } + return 0 +} + +func (m *OptionValue) GetStringValue() string { + if x, ok := m.GetValue().(*OptionValue_StringValue); ok { + return x.StringValue + } + return "" +} + +func (m *OptionValue) GetDurationValue() int64 { + if x, ok := m.GetValue().(*OptionValue_DurationValue); ok { + return x.DurationValue + } + return 0 +} + +// XXX_OneofFuncs is for the internal use of the proto package. +func (*OptionValue) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) error, func(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error), []interface{}) { + return _OptionValue_OneofMarshaler, _OptionValue_OneofUnmarshaler, []interface{}{ + (*OptionValue_BoolValue)(nil), + (*OptionValue_IntValue)(nil), + (*OptionValue_DoubleValue)(nil), + (*OptionValue_StringValue)(nil), + (*OptionValue_DurationValue)(nil), + } +} + +func _OptionValue_OneofMarshaler(msg proto.Message, b *proto.Buffer) error { + m := msg.(*OptionValue) + // value + switch x := m.Value.(type) { + case *OptionValue_BoolValue: + t := uint64(0) + if x.BoolValue { + t = 1 + } + _ = b.EncodeVarint(2<<3 | proto.WireVarint) + _ = b.EncodeVarint(t) + case *OptionValue_IntValue: + _ = b.EncodeVarint(3<<3 | proto.WireVarint) + _ = b.EncodeVarint(uint64(x.IntValue)) + case *OptionValue_DoubleValue: + _ = b.EncodeVarint(4<<3 | proto.WireFixed64) + _ = b.EncodeFixed64(math.Float64bits(x.DoubleValue)) + case *OptionValue_StringValue: + _ = b.EncodeVarint(5<<3 | proto.WireBytes) + _ = b.EncodeStringBytes(x.StringValue) + case *OptionValue_DurationValue: + _ = b.EncodeVarint(6<<3 | proto.WireVarint) + _ = b.EncodeVarint(uint64(x.DurationValue)) + case nil: + default: + return fmt.Errorf("OptionValue.Value has unexpected type %T", x) + } + return nil +} + +func _OptionValue_OneofUnmarshaler(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error) { + m := msg.(*OptionValue) + switch tag { + case 2: // value.boolValue + if wire != proto.WireVarint { + return true, proto.ErrInternalBadWireType + } + x, err := b.DecodeVarint() + m.Value = &OptionValue_BoolValue{x != 0} + return true, err + case 3: // value.intValue + if wire != proto.WireVarint { + return true, proto.ErrInternalBadWireType + } + x, err := b.DecodeVarint() + m.Value = &OptionValue_IntValue{int64(x)} + return true, err + case 4: // value.doubleValue + if wire != proto.WireFixed64 { + return true, proto.ErrInternalBadWireType + } + x, err := b.DecodeFixed64() + m.Value = &OptionValue_DoubleValue{math.Float64frombits(x)} + return true, err + case 5: // value.stringValue + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + x, err := b.DecodeStringBytes() + m.Value = &OptionValue_StringValue{x} + return true, err + case 6: // value.durationValue + if wire != proto.WireVarint { + return true, proto.ErrInternalBadWireType + } + x, err := b.DecodeVarint() + m.Value = &OptionValue_DurationValue{int64(x)} + return true, err + default: + return false, nil + } +} + +// Respond to Kapacitor whether initialization was successful. +type InitResponse struct { + Success bool `protobuf:"varint,1,opt,name=success,proto3" json:"success,omitempty"` + Error string `protobuf:"bytes,2,opt,name=error,proto3" json:"error,omitempty"` +} + +func (m *InitResponse) Reset() { *m = InitResponse{} } +func (m *InitResponse) String() string { return proto.CompactTextString(m) } +func (*InitResponse) ProtoMessage() {} + +// Request that the process provide a snapshot of its state. +type SnapshotRequest struct { +} + +func (m *SnapshotRequest) Reset() { *m = SnapshotRequest{} } +func (m *SnapshotRequest) String() string { return proto.CompactTextString(m) } +func (*SnapshotRequest) ProtoMessage() {} + +// Respond to Kapacitor with a serialized snapshot of the running state. +type SnapshotResponse struct { + Snapshot []byte `protobuf:"bytes,1,opt,name=snapshot,proto3" json:"snapshot,omitempty"` +} + +func (m *SnapshotResponse) Reset() { *m = SnapshotResponse{} } +func (m *SnapshotResponse) String() string { return proto.CompactTextString(m) } +func (*SnapshotResponse) ProtoMessage() {} + +// Request that the process restore its state from a snapshot. +type RestoreRequest struct { + Snapshot []byte `protobuf:"bytes,1,opt,name=snapshot,proto3" json:"snapshot,omitempty"` +} + +func (m *RestoreRequest) Reset() { *m = RestoreRequest{} } +func (m *RestoreRequest) String() string { return proto.CompactTextString(m) } +func (*RestoreRequest) ProtoMessage() {} + +// Respond with success or failure to a RestoreRequest +type RestoreResponse struct { + Success bool `protobuf:"varint,1,opt,name=success,proto3" json:"success,omitempty"` + Error string `protobuf:"bytes,2,opt,name=error,proto3" json:"error,omitempty"` +} + +func (m *RestoreResponse) Reset() { *m = RestoreResponse{} } +func (m *RestoreResponse) String() string { return proto.CompactTextString(m) } +func (*RestoreResponse) ProtoMessage() {} + +// Request that the process respond with a Keepalive to verify it is responding. +type KeepaliveRequest struct { + // The number of nanoseconds since the epoch. + // Used only for debugging keepalive requests. + Time int64 `protobuf:"varint,1,opt,name=time,proto3" json:"time,omitempty"` +} + +func (m *KeepaliveRequest) Reset() { *m = KeepaliveRequest{} } +func (m *KeepaliveRequest) String() string { return proto.CompactTextString(m) } +func (*KeepaliveRequest) ProtoMessage() {} + +// Respond to KeepaliveRequest +type KeepaliveResponse struct { + // The number of nanoseconds since the epoch. + // Used only for debugging keepalive requests. + Time int64 `protobuf:"varint,1,opt,name=time,proto3" json:"time,omitempty"` +} + +func (m *KeepaliveResponse) Reset() { *m = KeepaliveResponse{} } +func (m *KeepaliveResponse) String() string { return proto.CompactTextString(m) } +func (*KeepaliveResponse) ProtoMessage() {} + +// Sent from the process to Kapacitor indicating an error has occured. +// If an ErrorResponse is received, Kapacitor will terminate the process. +type ErrorResponse struct { + Error string `protobuf:"bytes,1,opt,name=error,proto3" json:"error,omitempty"` +} + +func (m *ErrorResponse) Reset() { *m = ErrorResponse{} } +func (m *ErrorResponse) String() string { return proto.CompactTextString(m) } +func (*ErrorResponse) ProtoMessage() {} + +// Indicates the beginning of a batch. +// All subsequent points should be considered +// part of the batch until EndBatch arrives. +type BeginBatch struct { +} + +func (m *BeginBatch) Reset() { *m = BeginBatch{} } +func (m *BeginBatch) String() string { return proto.CompactTextString(m) } +func (*BeginBatch) ProtoMessage() {} + +// Message containing information about a single data point. +// Can be sent on it's own or bookended by BeginBatch and EndBatch messages. +type Point struct { + Time int64 `protobuf:"varint,1,opt,name=time,proto3" json:"time,omitempty"` + Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` + Database string `protobuf:"bytes,3,opt,name=database,proto3" json:"database,omitempty"` + RetentionPolicy string `protobuf:"bytes,4,opt,name=retentionPolicy,proto3" json:"retentionPolicy,omitempty"` + Group string `protobuf:"bytes,5,opt,name=group,proto3" json:"group,omitempty"` + Dimensions []string `protobuf:"bytes,6,rep,name=dimensions" json:"dimensions,omitempty"` + Tags map[string]string `protobuf:"bytes,7,rep,name=tags" json:"tags,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + FieldsDouble map[string]float64 `protobuf:"bytes,8,rep,name=fieldsDouble" json:"fieldsDouble,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"fixed64,2,opt,name=value,proto3"` + FieldsInt map[string]int64 `protobuf:"bytes,9,rep,name=fieldsInt" json:"fieldsInt,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` + FieldsString map[string]string `protobuf:"bytes,10,rep,name=fieldsString" json:"fieldsString,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` +} + +func (m *Point) Reset() { *m = Point{} } +func (m *Point) String() string { return proto.CompactTextString(m) } +func (*Point) ProtoMessage() {} + +func (m *Point) GetTags() map[string]string { + if m != nil { + return m.Tags + } + return nil +} + +func (m *Point) GetFieldsDouble() map[string]float64 { + if m != nil { + return m.FieldsDouble + } + return nil +} + +func (m *Point) GetFieldsInt() map[string]int64 { + if m != nil { + return m.FieldsInt + } + return nil +} + +func (m *Point) GetFieldsString() map[string]string { + if m != nil { + return m.FieldsString + } + return nil +} + +// Indicates the end of a batch and contains +// all meta data associated with the batch. +type EndBatch struct { + Name string `protobuf:"bytes,1,opt,name=Name,proto3" json:"Name,omitempty"` + Group string `protobuf:"bytes,2,opt,name=Group,proto3" json:"Group,omitempty"` + TMax int64 `protobuf:"varint,3,opt,name=TMax,proto3" json:"TMax,omitempty"` + Tags map[string]string `protobuf:"bytes,4,rep,name=Tags" json:"Tags,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` +} + +func (m *EndBatch) Reset() { *m = EndBatch{} } +func (m *EndBatch) String() string { return proto.CompactTextString(m) } +func (*EndBatch) ProtoMessage() {} + +func (m *EndBatch) GetTags() map[string]string { + if m != nil { + return m.Tags + } + return nil +} + +// Request message wrapper -- sent from Kapacitor to process +type Request struct { + // Types that are valid to be assigned to Message: + // *Request_Info + // *Request_Init + // *Request_Keepalive + // *Request_Snapshot + // *Request_Restore + // *Request_Begin + // *Request_Point + // *Request_End + Message isRequest_Message `protobuf_oneof:"message"` +} + +func (m *Request) Reset() { *m = Request{} } +func (m *Request) String() string { return proto.CompactTextString(m) } +func (*Request) ProtoMessage() {} + +type isRequest_Message interface { + isRequest_Message() +} + +type Request_Info struct { + Info *InfoRequest `protobuf:"bytes,1,opt,name=info,oneof"` +} +type Request_Init struct { + Init *InitRequest `protobuf:"bytes,2,opt,name=init,oneof"` +} +type Request_Keepalive struct { + Keepalive *KeepaliveRequest `protobuf:"bytes,3,opt,name=keepalive,oneof"` +} +type Request_Snapshot struct { + Snapshot *SnapshotRequest `protobuf:"bytes,4,opt,name=snapshot,oneof"` +} +type Request_Restore struct { + Restore *RestoreRequest `protobuf:"bytes,5,opt,name=restore,oneof"` +} +type Request_Begin struct { + Begin *BeginBatch `protobuf:"bytes,16,opt,name=begin,oneof"` +} +type Request_Point struct { + Point *Point `protobuf:"bytes,17,opt,name=point,oneof"` +} +type Request_End struct { + End *EndBatch `protobuf:"bytes,18,opt,name=end,oneof"` +} + +func (*Request_Info) isRequest_Message() {} +func (*Request_Init) isRequest_Message() {} +func (*Request_Keepalive) isRequest_Message() {} +func (*Request_Snapshot) isRequest_Message() {} +func (*Request_Restore) isRequest_Message() {} +func (*Request_Begin) isRequest_Message() {} +func (*Request_Point) isRequest_Message() {} +func (*Request_End) isRequest_Message() {} + +func (m *Request) GetMessage() isRequest_Message { + if m != nil { + return m.Message + } + return nil +} + +func (m *Request) GetInfo() *InfoRequest { + if x, ok := m.GetMessage().(*Request_Info); ok { + return x.Info + } + return nil +} + +func (m *Request) GetInit() *InitRequest { + if x, ok := m.GetMessage().(*Request_Init); ok { + return x.Init + } + return nil +} + +func (m *Request) GetKeepalive() *KeepaliveRequest { + if x, ok := m.GetMessage().(*Request_Keepalive); ok { + return x.Keepalive + } + return nil +} + +func (m *Request) GetSnapshot() *SnapshotRequest { + if x, ok := m.GetMessage().(*Request_Snapshot); ok { + return x.Snapshot + } + return nil +} + +func (m *Request) GetRestore() *RestoreRequest { + if x, ok := m.GetMessage().(*Request_Restore); ok { + return x.Restore + } + return nil +} + +func (m *Request) GetBegin() *BeginBatch { + if x, ok := m.GetMessage().(*Request_Begin); ok { + return x.Begin + } + return nil +} + +func (m *Request) GetPoint() *Point { + if x, ok := m.GetMessage().(*Request_Point); ok { + return x.Point + } + return nil +} + +func (m *Request) GetEnd() *EndBatch { + if x, ok := m.GetMessage().(*Request_End); ok { + return x.End + } + return nil +} + +// XXX_OneofFuncs is for the internal use of the proto package. +func (*Request) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) error, func(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error), []interface{}) { + return _Request_OneofMarshaler, _Request_OneofUnmarshaler, []interface{}{ + (*Request_Info)(nil), + (*Request_Init)(nil), + (*Request_Keepalive)(nil), + (*Request_Snapshot)(nil), + (*Request_Restore)(nil), + (*Request_Begin)(nil), + (*Request_Point)(nil), + (*Request_End)(nil), + } +} + +func _Request_OneofMarshaler(msg proto.Message, b *proto.Buffer) error { + m := msg.(*Request) + // message + switch x := m.Message.(type) { + case *Request_Info: + _ = b.EncodeVarint(1<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.Info); err != nil { + return err + } + case *Request_Init: + _ = b.EncodeVarint(2<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.Init); err != nil { + return err + } + case *Request_Keepalive: + _ = b.EncodeVarint(3<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.Keepalive); err != nil { + return err + } + case *Request_Snapshot: + _ = b.EncodeVarint(4<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.Snapshot); err != nil { + return err + } + case *Request_Restore: + _ = b.EncodeVarint(5<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.Restore); err != nil { + return err + } + case *Request_Begin: + _ = b.EncodeVarint(16<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.Begin); err != nil { + return err + } + case *Request_Point: + _ = b.EncodeVarint(17<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.Point); err != nil { + return err + } + case *Request_End: + _ = b.EncodeVarint(18<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.End); err != nil { + return err + } + case nil: + default: + return fmt.Errorf("Request.Message has unexpected type %T", x) + } + return nil +} + +func _Request_OneofUnmarshaler(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error) { + m := msg.(*Request) + switch tag { + case 1: // message.info + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(InfoRequest) + err := b.DecodeMessage(msg) + m.Message = &Request_Info{msg} + return true, err + case 2: // message.init + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(InitRequest) + err := b.DecodeMessage(msg) + m.Message = &Request_Init{msg} + return true, err + case 3: // message.keepalive + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(KeepaliveRequest) + err := b.DecodeMessage(msg) + m.Message = &Request_Keepalive{msg} + return true, err + case 4: // message.snapshot + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(SnapshotRequest) + err := b.DecodeMessage(msg) + m.Message = &Request_Snapshot{msg} + return true, err + case 5: // message.restore + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(RestoreRequest) + err := b.DecodeMessage(msg) + m.Message = &Request_Restore{msg} + return true, err + case 16: // message.begin + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(BeginBatch) + err := b.DecodeMessage(msg) + m.Message = &Request_Begin{msg} + return true, err + case 17: // message.point + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(Point) + err := b.DecodeMessage(msg) + m.Message = &Request_Point{msg} + return true, err + case 18: // message.end + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(EndBatch) + err := b.DecodeMessage(msg) + m.Message = &Request_End{msg} + return true, err + default: + return false, nil + } +} + +// Response message wrapper -- sent from process to Kapacitor +type Response struct { + // Types that are valid to be assigned to Message: + // *Response_Info + // *Response_Init + // *Response_Keepalive + // *Response_Snapshot + // *Response_Restore + // *Response_Error + // *Response_Begin + // *Response_Point + // *Response_End + Message isResponse_Message `protobuf_oneof:"message"` +} + +func (m *Response) Reset() { *m = Response{} } +func (m *Response) String() string { return proto.CompactTextString(m) } +func (*Response) ProtoMessage() {} + +type isResponse_Message interface { + isResponse_Message() +} + +type Response_Info struct { + Info *InfoResponse `protobuf:"bytes,1,opt,name=info,oneof"` +} +type Response_Init struct { + Init *InitResponse `protobuf:"bytes,2,opt,name=init,oneof"` +} +type Response_Keepalive struct { + Keepalive *KeepaliveResponse `protobuf:"bytes,3,opt,name=keepalive,oneof"` +} +type Response_Snapshot struct { + Snapshot *SnapshotResponse `protobuf:"bytes,4,opt,name=snapshot,oneof"` +} +type Response_Restore struct { + Restore *RestoreResponse `protobuf:"bytes,5,opt,name=restore,oneof"` +} +type Response_Error struct { + Error *ErrorResponse `protobuf:"bytes,6,opt,name=error,oneof"` +} +type Response_Begin struct { + Begin *BeginBatch `protobuf:"bytes,16,opt,name=begin,oneof"` +} +type Response_Point struct { + Point *Point `protobuf:"bytes,17,opt,name=point,oneof"` +} +type Response_End struct { + End *EndBatch `protobuf:"bytes,18,opt,name=end,oneof"` +} + +func (*Response_Info) isResponse_Message() {} +func (*Response_Init) isResponse_Message() {} +func (*Response_Keepalive) isResponse_Message() {} +func (*Response_Snapshot) isResponse_Message() {} +func (*Response_Restore) isResponse_Message() {} +func (*Response_Error) isResponse_Message() {} +func (*Response_Begin) isResponse_Message() {} +func (*Response_Point) isResponse_Message() {} +func (*Response_End) isResponse_Message() {} + +func (m *Response) GetMessage() isResponse_Message { + if m != nil { + return m.Message + } + return nil +} + +func (m *Response) GetInfo() *InfoResponse { + if x, ok := m.GetMessage().(*Response_Info); ok { + return x.Info + } + return nil +} + +func (m *Response) GetInit() *InitResponse { + if x, ok := m.GetMessage().(*Response_Init); ok { + return x.Init + } + return nil +} + +func (m *Response) GetKeepalive() *KeepaliveResponse { + if x, ok := m.GetMessage().(*Response_Keepalive); ok { + return x.Keepalive + } + return nil +} + +func (m *Response) GetSnapshot() *SnapshotResponse { + if x, ok := m.GetMessage().(*Response_Snapshot); ok { + return x.Snapshot + } + return nil +} + +func (m *Response) GetRestore() *RestoreResponse { + if x, ok := m.GetMessage().(*Response_Restore); ok { + return x.Restore + } + return nil +} + +func (m *Response) GetError() *ErrorResponse { + if x, ok := m.GetMessage().(*Response_Error); ok { + return x.Error + } + return nil +} + +func (m *Response) GetBegin() *BeginBatch { + if x, ok := m.GetMessage().(*Response_Begin); ok { + return x.Begin + } + return nil +} + +func (m *Response) GetPoint() *Point { + if x, ok := m.GetMessage().(*Response_Point); ok { + return x.Point + } + return nil +} + +func (m *Response) GetEnd() *EndBatch { + if x, ok := m.GetMessage().(*Response_End); ok { + return x.End + } + return nil +} + +// XXX_OneofFuncs is for the internal use of the proto package. +func (*Response) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) error, func(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error), []interface{}) { + return _Response_OneofMarshaler, _Response_OneofUnmarshaler, []interface{}{ + (*Response_Info)(nil), + (*Response_Init)(nil), + (*Response_Keepalive)(nil), + (*Response_Snapshot)(nil), + (*Response_Restore)(nil), + (*Response_Error)(nil), + (*Response_Begin)(nil), + (*Response_Point)(nil), + (*Response_End)(nil), + } +} + +func _Response_OneofMarshaler(msg proto.Message, b *proto.Buffer) error { + m := msg.(*Response) + // message + switch x := m.Message.(type) { + case *Response_Info: + _ = b.EncodeVarint(1<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.Info); err != nil { + return err + } + case *Response_Init: + _ = b.EncodeVarint(2<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.Init); err != nil { + return err + } + case *Response_Keepalive: + _ = b.EncodeVarint(3<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.Keepalive); err != nil { + return err + } + case *Response_Snapshot: + _ = b.EncodeVarint(4<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.Snapshot); err != nil { + return err + } + case *Response_Restore: + _ = b.EncodeVarint(5<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.Restore); err != nil { + return err + } + case *Response_Error: + _ = b.EncodeVarint(6<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.Error); err != nil { + return err + } + case *Response_Begin: + _ = b.EncodeVarint(16<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.Begin); err != nil { + return err + } + case *Response_Point: + _ = b.EncodeVarint(17<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.Point); err != nil { + return err + } + case *Response_End: + _ = b.EncodeVarint(18<<3 | proto.WireBytes) + if err := b.EncodeMessage(x.End); err != nil { + return err + } + case nil: + default: + return fmt.Errorf("Response.Message has unexpected type %T", x) + } + return nil +} + +func _Response_OneofUnmarshaler(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error) { + m := msg.(*Response) + switch tag { + case 1: // message.info + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(InfoResponse) + err := b.DecodeMessage(msg) + m.Message = &Response_Info{msg} + return true, err + case 2: // message.init + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(InitResponse) + err := b.DecodeMessage(msg) + m.Message = &Response_Init{msg} + return true, err + case 3: // message.keepalive + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(KeepaliveResponse) + err := b.DecodeMessage(msg) + m.Message = &Response_Keepalive{msg} + return true, err + case 4: // message.snapshot + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(SnapshotResponse) + err := b.DecodeMessage(msg) + m.Message = &Response_Snapshot{msg} + return true, err + case 5: // message.restore + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(RestoreResponse) + err := b.DecodeMessage(msg) + m.Message = &Response_Restore{msg} + return true, err + case 6: // message.error + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(ErrorResponse) + err := b.DecodeMessage(msg) + m.Message = &Response_Error{msg} + return true, err + case 16: // message.begin + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(BeginBatch) + err := b.DecodeMessage(msg) + m.Message = &Response_Begin{msg} + return true, err + case 17: // message.point + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(Point) + err := b.DecodeMessage(msg) + m.Message = &Response_Point{msg} + return true, err + case 18: // message.end + if wire != proto.WireBytes { + return true, proto.ErrInternalBadWireType + } + msg := new(EndBatch) + err := b.DecodeMessage(msg) + m.Message = &Response_End{msg} + return true, err + default: + return false, nil + } +} + +func init() { + proto.RegisterType((*InfoRequest)(nil), "udf.InfoRequest") + proto.RegisterType((*InfoResponse)(nil), "udf.InfoResponse") + proto.RegisterType((*OptionInfo)(nil), "udf.OptionInfo") + proto.RegisterType((*InitRequest)(nil), "udf.InitRequest") + proto.RegisterType((*Option)(nil), "udf.Option") + proto.RegisterType((*OptionValue)(nil), "udf.OptionValue") + proto.RegisterType((*InitResponse)(nil), "udf.InitResponse") + proto.RegisterType((*SnapshotRequest)(nil), "udf.SnapshotRequest") + proto.RegisterType((*SnapshotResponse)(nil), "udf.SnapshotResponse") + proto.RegisterType((*RestoreRequest)(nil), "udf.RestoreRequest") + proto.RegisterType((*RestoreResponse)(nil), "udf.RestoreResponse") + proto.RegisterType((*KeepaliveRequest)(nil), "udf.KeepaliveRequest") + proto.RegisterType((*KeepaliveResponse)(nil), "udf.KeepaliveResponse") + proto.RegisterType((*ErrorResponse)(nil), "udf.ErrorResponse") + proto.RegisterType((*BeginBatch)(nil), "udf.BeginBatch") + proto.RegisterType((*Point)(nil), "udf.Point") + proto.RegisterType((*EndBatch)(nil), "udf.EndBatch") + proto.RegisterType((*Request)(nil), "udf.Request") + proto.RegisterType((*Response)(nil), "udf.Response") + proto.RegisterEnum("udf.EdgeType", EdgeType_name, EdgeType_value) + proto.RegisterEnum("udf.ValueType", ValueType_name, ValueType_value) +} diff --git a/udf/udf.proto b/udf/udf.proto new file mode 100644 index 000000000..dda43607e --- /dev/null +++ b/udf/udf.proto @@ -0,0 +1,217 @@ +syntax = "proto3"; + +package udf; + +//------------------------------------------------------ +// RPC Messages for Kapacitor to communicate with +// a child process for data processing. +// +// The Kapacitor processes will send Request messages +// over STDIN to the child process. +// The child process can respond over STDOUT. +// +// Messages are streamed by writing a varint header +// that contains the length of the following message. +// +// To decode the stream read a varint, then read +// the determined size and decode as a protobuf message. +// There is not footer so the next varint if any begins +// right after the previous message. +// +//------------------------------------------------------ +// Management messages +// +// *Request messages are sent to the process from Kapacitor. +// *Response messages are sent to Kapacitor from the process. +// +// While there is an obvious request/response structure for communicating, +// there is a loose coupling between request and response. +// Meaning that ordering or synchronizing STDIN and STDOUT in anyway +// is not necessary. +// For example if Kapacitor requests a snapshot and the +// process is in the middle of writing a previous response or +// data points those can continue. Eventually Kapacitor will receive +// the snapshot response and act accordingly. +// +// A KeepaliveRequest/KeepaliveResponse system is used to ensure that +// the process is responsive. Every time that a KeepaliveRequest is sent +// a KeepaliveResponse must be returned within a timeout. +// If the timeout is reached than the process is considered dead and will be terminated/restarted. +// +// It is recommend to disable buffering on the STDIN and STDOUT sockets. +// Some languages like python will automatically buffer the sockets. +// To disable this behavior use the -u flag on the python interpreter. + +// Request that the process return information about available Options. +message InfoRequest { +} + +enum EdgeType { + STREAM = 0; + BATCH = 1; +} + +message InfoResponse { + EdgeType wants = 1; + EdgeType provides = 2; + map options = 3; +} + +enum ValueType { + BOOL = 0; + INT = 1; + DOUBLE = 2; + STRING = 3; + DURATION = 4; +} + +message OptionInfo { + repeated ValueType valueTypes = 1; +} + +// Request that the process initialize itself with the provided options. +message InitRequest { + repeated Option options = 1; +} + +message Option { + string name = 1; + repeated OptionValue values = 2; +} + +message OptionValue { + ValueType type = 1; + oneof value { + bool boolValue = 2; + int64 intValue = 3; + double doubleValue = 4; + string stringValue = 5; + int64 durationValue = 6; + } +} + +// Respond to Kapacitor whether initialization was successful. +message InitResponse { + bool success = 1; + string error = 2; +} + +// Request that the process provide a snapshot of its state. +message SnapshotRequest { +} + +// Respond to Kapacitor with a serialized snapshot of the running state. +message SnapshotResponse { + bytes snapshot = 1; +} + +// Request that the process restore its state from a snapshot. +message RestoreRequest { + bytes snapshot = 1; +} + +// Respond with success or failure to a RestoreRequest +message RestoreResponse { + bool success = 1; + string error = 2; +} + + +// Request that the process respond with a Keepalive to verify it is responding. +message KeepaliveRequest { + // The number of nanoseconds since the epoch. + // Used only for debugging keepalive requests. + int64 time = 1; +} + +// Respond to KeepaliveRequest +message KeepaliveResponse { + // The number of nanoseconds since the epoch. + // Used only for debugging keepalive requests. + int64 time = 1; +} + +// Sent from the process to Kapacitor indicating an error has occured. +// If an ErrorResponse is received, Kapacitor will terminate the process. +message ErrorResponse { + string error = 1; +} + +//------------------------------------------------------ +// Data flow messages +// +// Sent and received by both the process and Kapacitor + + +// Indicates the beginning of a batch. +// All subsequent points should be considered +// part of the batch until EndBatch arrives. +message BeginBatch { +} + +// Message containing information about a single data point. +// Can be sent on it's own or bookended by BeginBatch and EndBatch messages. +message Point { + int64 time = 1; + string name = 2; + string database = 3; + string retentionPolicy = 4; + string group = 5; + repeated string dimensions = 6; + map tags = 7; + map fieldsDouble = 8; + map fieldsInt = 9; + map fieldsString = 10; +} + +// Indicates the end of a batch and contains +// all meta data associated with the batch. +message EndBatch { + string Name = 1; + string Group = 2; + int64 TMax = 3; + map Tags = 4; +} + +//----------------------------------------------------------- +// Wrapper messages +// +// All messages sent over STDIN will be Request messages. +// All messages sent over STDOUT must be Response messages. + + +// Request message wrapper -- sent from Kapacitor to process +message Request { + oneof message { + // Management requests + InfoRequest info = 1; + InitRequest init = 2; + KeepaliveRequest keepalive = 3; + SnapshotRequest snapshot = 4; + RestoreRequest restore = 5; + + // Data flow responses + BeginBatch begin = 16; + Point point = 17; + EndBatch end = 18; + } +} + +// Response message wrapper -- sent from process to Kapacitor +message Response { + oneof message { + // Management responses + InfoResponse info = 1; + InitResponse init = 2; + KeepaliveResponse keepalive = 3; + SnapshotResponse snapshot = 4; + RestoreResponse restore = 5; + ErrorResponse error = 6; + + // Data flow responses + BeginBatch begin = 16; + Point point = 17; + EndBatch end = 18; + } +} + diff --git a/udf_process.go b/udf_process.go new file mode 100644 index 000000000..7104ac31d --- /dev/null +++ b/udf_process.go @@ -0,0 +1,771 @@ +package kapacitor + +import ( + "bufio" + "errors" + "fmt" + "io" + "log" + "sync" + "time" + + "github.com/influxdata/kapacitor/command" + "github.com/influxdata/kapacitor/models" + "github.com/influxdata/kapacitor/pipeline" + "github.com/influxdata/kapacitor/udf" +) + +var ErrUDFProcessStopped = errors.New("process already stopped") + +type byteReadCloser struct { + *bufio.Reader + io.Closer +} + +// Wraps an external process and sends and receives data +// over STDIN and STDOUT. Lines received over STDERR are logged +// via normal Kapacitor logging. +// +// Once a UDFProcess is created and started the owner can send points or batches +// to the subprocess by writing them to the PointIn or BatchIn channels respectively, +// and according to the type of process created. +// +// The UDFProcess may be Aborted at anytime for various reasons. It is the owner's responsibility +// via the abortCallback to stop writing to the *In channels since no more selects on the channels +// will be performed. +// +// Calling Stop on the process should only be done once the owner has stopped writing to the *In channel, +// at which point the remaining data will be processed and the subprocess will be allowed to exit cleanly. +// +// Callling Info returns information about available options the process has. +// +// Calling Init is required to process data. +// The behavior is undefined if you send points/batches to the process without calling Init. +type UDFProcess struct { + + // If the processes is Aborted (via Keepalive timeout, etc.) + // then no more data will be read off the *In channels. + // + // Optional callback if the process aborts. + // It is the owners response + abortCallback func() + abortOnce sync.Once + + pointIn chan models.Point + PointIn chan<- models.Point + batchIn chan models.Batch + BatchIn chan<- models.Batch + + pointOut chan models.Point + PointOut <-chan models.Point + batchOut chan models.Batch + BatchOut <-chan models.Batch + + stopped bool + stopping chan struct{} + aborted bool + aborting chan struct{} + // The first error that occurred or nil + err error + errMu sync.Mutex + + requests chan *udf.Request + requestsGroup sync.WaitGroup + + keepalive chan int64 + keepaliveTimeout time.Duration + + commander command.Commander + cmd command.Command + stdin io.WriteCloser + stdout byteReadCloser + stderr io.ReadCloser + + // Group for waiting on read/write goroutines + pipeWaitGroup sync.WaitGroup + // Group for waiting on the process itself + processWaitGroup sync.WaitGroup + + mu sync.Mutex + logger *log.Logger + + responseBuf []byte + response *udf.Response + + requestResponse sync.Mutex + infoResponse chan *udf.Response + initResponse chan *udf.Response + snapshotResponse chan *udf.Response + restoreResponse chan *udf.Response + + batch *models.Batch +} + +func NewUDFProcess( + commander command.Commander, + l *log.Logger, + timeout time.Duration, + abortCallback func(), +) *UDFProcess { + p := &UDFProcess{ + commander: commander, + logger: l, + requests: make(chan *udf.Request), + keepalive: make(chan int64, 1), + keepaliveTimeout: timeout, + abortCallback: abortCallback, + response: &udf.Response{}, + } + + p.pointIn = make(chan models.Point) + p.PointIn = p.pointIn + p.batchIn = make(chan models.Batch) + p.BatchIn = p.batchIn + + p.pointOut = make(chan models.Point) + p.PointOut = p.pointOut + p.batchOut = make(chan models.Batch) + p.BatchOut = p.batchOut + return p +} + +func (p *UDFProcess) setError(err error) { + p.errMu.Lock() + defer p.errMu.Unlock() + if p.err == nil { + p.err = err + } +} + +// Start the UDFProcess +func (p *UDFProcess) Start() error { + p.mu.Lock() + defer p.mu.Unlock() + p.stopped = false + p.stopping = make(chan struct{}) + p.aborted = false + p.aborting = make(chan struct{}) + + cmd := p.commander.NewCommand() + stdin, err := cmd.StdinPipe() + if err != nil { + return err + } + p.stdin = stdin + + stdout, err := cmd.StdoutPipe() + if err != nil { + return err + } + brc := byteReadCloser{ + bufio.NewReader(stdout), + stdout, + } + p.stdout = brc + + stderr, err := cmd.StderrPipe() + if err != nil { + return err + } + p.stderr = stderr + + err = cmd.Start() + if err != nil { + return err + } + p.cmd = cmd + + p.pipeWaitGroup.Add(1) + go func() { + err := p.writeData() + if err != nil { + p.setError(err) + defer p.abort() + } + p.pipeWaitGroup.Done() + }() + p.pipeWaitGroup.Add(1) + go func() { + err := p.readData() + if err != nil { + p.setError(err) + defer p.abort() + } + p.pipeWaitGroup.Done() + }() + + // Wait for process to terminate + p.processWaitGroup.Add(1) + go func() { + // First wait for the pipe read writes to finish + p.pipeWaitGroup.Wait() + err := cmd.Wait() + if err != nil { + err = fmt.Errorf("process exited unexpectedly: %v", err) + p.setError(err) + defer p.abort() + } + p.processWaitGroup.Done() + }() + + p.requestsGroup.Add(2) + go p.runKeepalive() + go p.watchKeepalive() + + p.pipeWaitGroup.Add(1) + go p.logStdErr() + + return nil +} + +// Abort the process. +// Data in-flight will not be processed. +func (p *UDFProcess) Abort(err error) { + p.setError(err) + p.abort() +} + +func (p *UDFProcess) abort() { + p.mu.Lock() + defer p.mu.Unlock() + if p.aborted { + return + } + p.aborted = true + close(p.aborting) + if p.abortCallback != nil { + p.abortOnce.Do(p.abortCallback) + } + p.stop() +} + +// Send the kill signal to the process +// it hasn't shutdown cleanly +func (p *UDFProcess) kill() { + p.cmd.Kill() +} + +// Stop the UDFProcess cleanly. +// +// Calling Stop should only be done once the owner has stopped writing to the *In channel, +// at which point the remaining data will be processed and the subprocess will be allowed to exit cleanly. +func (p *UDFProcess) Stop() error { + p.mu.Lock() + defer p.mu.Unlock() + return p.stop() +} + +// internal stop function you must acquire the lock before calling +func (p *UDFProcess) stop() error { + + if p.stopped { + return p.err + } + p.stopped = true + + close(p.stopping) + + p.requestsGroup.Wait() + + close(p.requests) + + close(p.pointIn) + close(p.batchIn) + + p.pipeWaitGroup.Wait() + p.processWaitGroup.Wait() + + // Return the error that occurred first + p.errMu.Lock() + defer p.errMu.Unlock() + return p.err +} + +type UDFProcessInfo struct { + Commander command.Commander + Timeout time.Duration + Wants pipeline.EdgeType + Provides pipeline.EdgeType + Options map[string]*udf.OptionInfo +} + +// Get information about the process, available options etc. +// Info need not be called every time a process is started. +func (p *UDFProcess) Info() (UDFProcessInfo, error) { + info := UDFProcessInfo{} + req := &udf.Request{Message: &udf.Request_Info{ + Info: &udf.InfoRequest{}, + }} + + resp, err := p.doRequestResponse(req, &p.infoResponse) + if err != nil { + return info, err + } + ri := resp.Message.(*udf.Response_Info).Info + info.Commander = p.commander + info.Timeout = p.keepaliveTimeout + info.Options = ri.Options + + switch ri.Wants { + case udf.EdgeType_STREAM: + info.Wants = pipeline.StreamEdge + case udf.EdgeType_BATCH: + info.Wants = pipeline.BatchEdge + } + switch ri.Provides { + case udf.EdgeType_STREAM: + info.Provides = pipeline.StreamEdge + case udf.EdgeType_BATCH: + info.Provides = pipeline.BatchEdge + } + return info, nil + +} + +// Initialize the process with a set of Options. +// Calling Init is required even if you do not have any specific Options, just pass nil +func (p *UDFProcess) Init(options []*udf.Option) error { + req := &udf.Request{Message: &udf.Request_Init{ + Init: &udf.InitRequest{ + Options: options, + }, + }} + resp, err := p.doRequestResponse(req, &p.initResponse) + if err != nil { + return err + } + + init := resp.Message.(*udf.Response_Init).Init + if !init.Success { + return fmt.Errorf("failed to initialize processes %s", init.Error) + } + return nil +} + +// Request a snapshot from the process. +func (p *UDFProcess) Snapshot() ([]byte, error) { + req := &udf.Request{Message: &udf.Request_Snapshot{ + Snapshot: &udf.SnapshotRequest{}, + }} + resp, err := p.doRequestResponse(req, &p.snapshotResponse) + if err != nil { + return nil, err + } + snapshot := resp.Message.(*udf.Response_Snapshot).Snapshot.Snapshot + return snapshot, nil +} + +// Request to restore a snapshot. +func (p *UDFProcess) Restore(snapshot []byte) error { + + req := &udf.Request{Message: &udf.Request_Restore{ + Restore: &udf.RestoreRequest{snapshot}, + }} + resp, err := p.doRequestResponse(req, &p.restoreResponse) + if err != nil { + return err + } + + restore := resp.Message.(*udf.Response_Restore).Restore + if !restore.Success { + return fmt.Errorf("error restoring snapshot: %s", restore.Error) + } + return nil +} + +// Request a snapshot from the process. +func (p *UDFProcess) doRequestResponse(req *udf.Request, respC *chan *udf.Response) (*udf.Response, error) { + err := func() error { + p.mu.Lock() + defer p.mu.Unlock() + if p.stopped { + if p.err != nil { + return p.err + } + return ErrUDFProcessStopped + } + p.requestsGroup.Add(1) + + p.requestResponse.Lock() + defer p.requestResponse.Unlock() + *respC = make(chan *udf.Response, 1) + return nil + }() + if err != nil { + return nil, err + } + defer p.requestsGroup.Done() + defer func() { + p.requestResponse.Lock() + defer p.requestResponse.Unlock() + *respC = nil + }() + + select { + case <-p.aborting: + return nil, p.err + case p.requests <- req: + } + + select { + case <-p.aborting: + return nil, p.err + case res := <-(*respC): + return res, nil + } +} + +func (p *UDFProcess) doResponse(response *udf.Response, respC *chan *udf.Response) { + p.requestResponse.Lock() + if *respC != nil { + p.requestResponse.Unlock() + *respC <- response + } else { + p.requestResponse.Unlock() + p.logger.Printf("E! received %T without requesting it", response.Message) + } +} + +// send KeepaliveRequest on the specified interval +func (p *UDFProcess) runKeepalive() { + defer p.requestsGroup.Done() + if p.keepaliveTimeout <= 0 { + return + } + ticker := time.NewTicker(p.keepaliveTimeout / 2) + defer ticker.Stop() + for { + select { + case <-ticker.C: + req := &udf.Request{Message: &udf.Request_Keepalive{ + Keepalive: &udf.KeepaliveRequest{ + Time: time.Now().UnixNano(), + }, + }} + select { + case p.requests <- req: + case <-p.aborting: + } + case <-p.stopping: + return + } + } +} + +// Abort the process if a keepalive timeout is reached. +func (p *UDFProcess) watchKeepalive() { + // Defer functions are called LIFO. + // We need to call p.abort after p.requestsGroup.Done so we just set a flag. + var err error + defer func() { + if err != nil { + p.setError(err) + go func() { + timeout := p.keepaliveTimeout * 2 + if timeout <= 0 { + timeout = time.Second + } + time.Sleep(timeout) + p.mu.Lock() + defer p.mu.Unlock() + if !p.stopped { + p.logger.Println("E! process not responding! killing") + p.kill() + } + }() + p.abort() + } + }() + defer p.requestsGroup.Done() + // If timeout is <= 0 then we don't ever timeout from keepalive, + // but we need to receive from p.keepalive or handleResponse will block. + // So we set a long timeout and then ignore it if its reached. + timeout := p.keepaliveTimeout + if timeout <= 0 { + timeout = time.Hour + } + last := time.Now().UnixNano() + for { + select { + case last = <-p.keepalive: + case <-time.After(timeout): + // Ignore invalid timeout + if p.keepaliveTimeout <= 0 { + break + } + err = fmt.Errorf("keepalive timedout, last keepalive received was: %s", time.Unix(0, last)) + p.logger.Println("E!", err) + return + case <-p.stopping: + return + } + } +} + +// Write Requests to the STDIN of the process. +func (p *UDFProcess) writeData() error { + defer p.stdin.Close() + for { + select { + case pt, ok := <-p.pointIn: + if ok { + err := p.writePoint(pt) + if err != nil { + return err + } + } else { + p.pointIn = nil + } + case bt, ok := <-p.batchIn: + if ok { + err := p.writeBatch(bt) + if err != nil { + return err + } + } else { + p.batchIn = nil + } + case req, ok := <-p.requests: + if ok { + err := p.writeRequest(req) + if err != nil { + return err + } + } else { + p.requests = nil + } + case <-p.aborting: + return p.err + } + if p.pointIn == nil && p.batchIn == nil && p.requests == nil { + break + } + } + return nil +} + +func (p *UDFProcess) writePoint(pt models.Point) error { + strs, floats, ints := p.fieldsToTypedMaps(pt.Fields) + udfPoint := &udf.Point{ + Time: pt.Time.UnixNano(), + Name: pt.Name, + Database: pt.Database, + RetentionPolicy: pt.RetentionPolicy, + Group: string(pt.Group), + Dimensions: pt.Dimensions, + Tags: pt.Tags, + FieldsDouble: floats, + FieldsInt: ints, + FieldsString: strs, + } + req := &udf.Request{ + Message: &udf.Request_Point{udfPoint}, + } + return p.writeRequest(req) +} + +func (p *UDFProcess) fieldsToTypedMaps(fields models.Fields) ( + strs map[string]string, + floats map[string]float64, + ints map[string]int64, +) { + for k, v := range fields { + switch value := v.(type) { + case string: + if strs == nil { + strs = make(map[string]string) + } + strs[k] = value + case float64: + if floats == nil { + floats = make(map[string]float64) + } + floats[k] = value + case int64: + if ints == nil { + ints = make(map[string]int64) + } + ints[k] = value + default: + panic("unsupported field value type") + } + } + return +} + +func (p *UDFProcess) typeMapsToFields( + strs map[string]string, + floats map[string]float64, + ints map[string]int64, +) models.Fields { + fields := make(models.Fields) + for k, v := range strs { + fields[k] = v + } + for k, v := range ints { + fields[k] = v + } + for k, v := range floats { + fields[k] = v + } + return fields +} + +func (p *UDFProcess) writeBatch(b models.Batch) error { + req := &udf.Request{ + Message: &udf.Request_Begin{&udf.BeginBatch{}}, + } + err := p.writeRequest(req) + if err != nil { + return err + } + rp := &udf.Request_Point{} + req.Message = rp + for _, pt := range b.Points { + strs, floats, ints := p.fieldsToTypedMaps(pt.Fields) + udfPoint := &udf.Point{ + Time: pt.Time.UnixNano(), + Group: string(b.Group), + Tags: pt.Tags, + FieldsDouble: floats, + FieldsInt: ints, + FieldsString: strs, + } + rp.Point = udfPoint + err := p.writeRequest(req) + if err != nil { + return err + } + } + + req.Message = &udf.Request_End{ + &udf.EndBatch{ + Name: b.Name, + }, + } + return p.writeRequest(req) +} + +func (p *UDFProcess) writeRequest(req *udf.Request) error { + err := udf.WriteMessage(req, p.stdin) + if err != nil { + err = fmt.Errorf("write error: %s", err) + } + return err +} + +// Read Responses from STDOUT of the process. +func (p *UDFProcess) readData() error { + defer func() { + close(p.pointOut) + close(p.batchOut) + }() + for { + response, err := p.readResponse() + if err == io.EOF { + return nil + } + if err != nil { + err = fmt.Errorf("read error: %s", err) + return err + } + err = p.handleResponse(response) + if err != nil { + return err + } + } +} + +func (p *UDFProcess) readResponse() (*udf.Response, error) { + err := udf.ReadMessage(&p.responseBuf, p.stdout, p.response) + if err != nil { + return nil, err + } + return p.response, nil +} + +func (p *UDFProcess) handleResponse(response *udf.Response) error { + // Always reset the keepalive timer since we received a response + select { + case p.keepalive <- time.Now().UnixNano(): + case <-p.stopping: + // No one is watching the keepalive anymore so we don't need to feed it, + // but we still want to handle the response + case <-p.aborting: + return p.err + } + // handle response + switch msg := response.Message.(type) { + case *udf.Response_Keepalive: + // Noop we already reset the keepalive timer + case *udf.Response_Info: + p.doResponse(response, &p.infoResponse) + case *udf.Response_Init: + p.doResponse(response, &p.initResponse) + case *udf.Response_Snapshot: + p.doResponse(response, &p.snapshotResponse) + case *udf.Response_Restore: + p.doResponse(response, &p.restoreResponse) + case *udf.Response_Error: + p.logger.Println("E!", msg.Error.Error) + return errors.New(msg.Error.Error) + case *udf.Response_Begin: + p.batch = &models.Batch{} + case *udf.Response_Point: + if p.batch != nil { + pt := models.BatchPoint{ + Time: time.Unix(0, msg.Point.Time).UTC(), + Tags: msg.Point.Tags, + Fields: p.typeMapsToFields( + msg.Point.FieldsString, + msg.Point.FieldsDouble, + msg.Point.FieldsInt, + ), + } + p.batch.Points = append(p.batch.Points, pt) + } else { + pt := models.Point{ + Time: time.Unix(0, msg.Point.Time).UTC(), + Name: msg.Point.Name, + Database: msg.Point.Database, + RetentionPolicy: msg.Point.RetentionPolicy, + Group: models.GroupID(msg.Point.Group), + Dimensions: msg.Point.Dimensions, + Tags: msg.Point.Tags, + Fields: p.typeMapsToFields( + msg.Point.FieldsString, + msg.Point.FieldsDouble, + msg.Point.FieldsInt, + ), + } + select { + case p.pointOut <- pt: + case <-p.aborting: + return p.err + } + } + case *udf.Response_End: + p.batch.Name = msg.End.Name + p.batch.TMax = time.Unix(0, msg.End.TMax) + p.batch.Group = models.GroupID(msg.End.Group) + p.batch.Tags = msg.End.Tags + select { + case p.batchOut <- *p.batch: + case <-p.aborting: + return p.err + } + p.batch = nil + default: + panic(fmt.Sprintf("unexpected response message %T", msg)) + } + return nil +} + +// Replay any lines from STDERR of the process to the Kapacitor log. +func (p *UDFProcess) logStdErr() { + defer p.pipeWaitGroup.Done() + scanner := bufio.NewScanner(p.stderr) + for scanner.Scan() { + p.logger.Println("I!P", scanner.Text()) + } +} diff --git a/udf_process_test.go b/udf_process_test.go new file mode 100644 index 000000000..406d5b387 --- /dev/null +++ b/udf_process_test.go @@ -0,0 +1,337 @@ +package kapacitor_test + +import ( + "errors" + "log" + "os" + "reflect" + "testing" + "time" + + "github.com/influxdata/kapacitor" + cmd_test "github.com/influxdata/kapacitor/command/test" + "github.com/influxdata/kapacitor/pipeline" + "github.com/influxdata/kapacitor/udf" +) + +func TestProcess_StartStop(t *testing.T) { + cmd := cmd_test.NewCommandHelper() + l := log.New(os.Stderr, "[TestProcess_StartStop] ", log.LstdFlags) + p := kapacitor.NewUDFProcess(cmd, l, 0, nil) + + p.Start() + + close(cmd.Responses) + p.Stop() + // read all requests and wait till the chan is closed + for range cmd.Requests { + } + if err := <-cmd.ErrC; err != nil { + t.Error(err) + } +} + +func TestProcess_StartInitStop(t *testing.T) { + cmd := cmd_test.NewCommandHelper() + l := log.New(os.Stderr, "[TestProcess_StartStop] ", log.LstdFlags) + p := kapacitor.NewUDFProcess(cmd, l, 0, nil) + go func() { + req := <-cmd.Requests + _, ok := req.Message.(*udf.Request_Init) + if !ok { + t.Errorf("expected init message got %T", req.Message) + } + res := &udf.Response{ + Message: &udf.Response_Init{ + Init: &udf.InitResponse{ + Success: true, + }, + }, + } + cmd.Responses <- res + close(cmd.Responses) + }() + + p.Start() + err := p.Init(nil) + if err != nil { + t.Fatal(err) + } + + p.Stop() + // read all requests and wait till the chan is closed + for range cmd.Requests { + } + if err := <-cmd.ErrC; err != nil { + t.Error(err) + } +} + +func TestProcess_StartInitAbort(t *testing.T) { + cmd := cmd_test.NewCommandHelper() + l := log.New(os.Stderr, "[TestProcess_StartInfoAbort] ", log.LstdFlags) + p := kapacitor.NewUDFProcess(cmd, l, 0, nil) + p.Start() + expErr := errors.New("explicit abort") + go func() { + req := <-cmd.Requests + _, ok := req.Message.(*udf.Request_Init) + if !ok { + t.Error("expected init message") + } + p.Abort(expErr) + close(cmd.Responses) + }() + err := p.Init(nil) + if err != expErr { + t.Fatal("expected explicit abort error") + } +} + +func TestProcess_StartInfoStop(t *testing.T) { + cmd := cmd_test.NewCommandHelper() + l := log.New(os.Stderr, "[TestProcess_StartInfoStop] ", log.LstdFlags) + p := kapacitor.NewUDFProcess(cmd, l, 0, nil) + go func() { + req := <-cmd.Requests + _, ok := req.Message.(*udf.Request_Info) + if !ok { + t.Errorf("expected info message got %T", req.Message) + } + res := &udf.Response{ + Message: &udf.Response_Info{ + Info: &udf.InfoResponse{ + Wants: udf.EdgeType_STREAM, + Provides: udf.EdgeType_BATCH, + }, + }, + } + cmd.Responses <- res + close(cmd.Responses) + }() + p.Start() + info, err := p.Info() + if err != nil { + t.Fatal(err) + } + if exp, got := pipeline.StreamEdge, info.Wants; got != exp { + t.Errorf("unexpected info.Wants got %v exp %v", got, exp) + } + if exp, got := pipeline.BatchEdge, info.Provides; got != exp { + t.Errorf("unexpected info.Provides got %v exp %v", got, exp) + } + + p.Stop() + // read all requests and wait till the chan is closed + for range cmd.Requests { + } + if err := <-cmd.ErrC; err != nil { + t.Error(err) + } +} + +func TestProcess_StartInfoAbort(t *testing.T) { + cmd := cmd_test.NewCommandHelper() + l := log.New(os.Stderr, "[TestProcess_StartInfoAbort] ", log.LstdFlags) + p := kapacitor.NewUDFProcess(cmd, l, 0, nil) + p.Start() + expErr := errors.New("explicit abort") + go func() { + req := <-cmd.Requests + _, ok := req.Message.(*udf.Request_Info) + if !ok { + t.Error("expected info message") + } + p.Abort(expErr) + close(cmd.Responses) + }() + _, err := p.Info() + if err != expErr { + t.Fatal("expected ErrUDFProcessAborted") + } +} + +func TestProcess_Keepalive(t *testing.T) { + cmd := cmd_test.NewCommandHelper() + l := log.New(os.Stderr, "[TestProcess_Keepalive] ", log.LstdFlags) + p := kapacitor.NewUDFProcess(cmd, l, time.Millisecond*10, nil) + p.Start() + p.Init(nil) + req := <-cmd.Requests + _, ok := req.Message.(*udf.Request_Init) + if !ok { + t.Error("expected init message") + } + select { + case req = <-cmd.Requests: + case <-time.After(time.Second): + t.Fatal("expected keepalive message") + } + if req == nil { + t.Fatal("expected keepalive message got nil") + } + _, ok = req.Message.(*udf.Request_Keepalive) + if !ok { + t.Errorf("expected keepalive message got %T", req.Message) + } + + close(cmd.Responses) + p.Stop() + // read all requests and wait till the chan is closed + for range cmd.Requests { + } + if err := <-cmd.ErrC; err != nil { + t.Error(err) + } +} + +func TestProcess_MissedKeepalive(t *testing.T) { + abortCalled := make(chan struct{}) + aborted := func() { + close(abortCalled) + } + + cmd := cmd_test.NewCommandHelper() + l := log.New(os.Stderr, "[TestProcess_MissedKeepalive] ", log.LstdFlags) + p := kapacitor.NewUDFProcess(cmd, l, 2, aborted) + p.Start() + + // Since the keepalive is missed, the process should abort on its own. + for range cmd.Requests { + } + + select { + case <-abortCalled: + case <-time.After(time.Second * 20): + t.Error("expected abort callback to be called") + } + + close(cmd.Responses) + if err := <-cmd.ErrC; err != nil { + t.Error(err) + } +} + +func TestProcess_MissedKeepaliveInit(t *testing.T) { + abortCalled := make(chan struct{}) + aborted := func() { + close(abortCalled) + } + + cmd := cmd_test.NewCommandHelper() + l := log.New(os.Stderr, "[TestProcess_MissedKeepalive] ", log.LstdFlags) + p := kapacitor.NewUDFProcess(cmd, l, 2, aborted) + p.Start() + p.Init(nil) + + // Since the keepalive is missed, the process should abort on its own. + for range cmd.Requests { + } + + select { + case <-abortCalled: + case <-time.After(time.Second * 20): + t.Error("expected abort callback to be called") + } + close(cmd.Responses) + if err := <-cmd.ErrC; err != nil { + t.Error(err) + } +} + +func TestProcess_MissedKeepaliveInfo(t *testing.T) { + abortCalled := make(chan struct{}) + aborted := func() { + close(abortCalled) + } + + cmd := cmd_test.NewCommandHelper() + l := log.New(os.Stderr, "[TestProcess_MissedKeepalive] ", log.LstdFlags) + p := kapacitor.NewUDFProcess(cmd, l, 2, aborted) + p.Start() + p.Info() + + // Since the keepalive is missed, the process should abort on its own. + for range cmd.Requests { + } + + select { + case <-abortCalled: + case <-time.After(time.Second * 20): + t.Error("expected abort callback to be called") + } + close(cmd.Responses) + if err := <-cmd.ErrC; err != nil { + t.Error(err) + } +} + +func TestProcess_SnapshotRestore(t *testing.T) { + cmd := cmd_test.NewCommandHelper() + l := log.New(os.Stderr, "[TestProcess_SnapshotRestore] ", log.LstdFlags) + p := kapacitor.NewUDFProcess(cmd, l, 0, nil) + go func() { + // Init + req := <-cmd.Requests + _, ok := req.Message.(*udf.Request_Init) + if !ok { + t.Error("expected init message") + } + cmd.Responses <- &udf.Response{ + Message: &udf.Response_Init{ + Init: &udf.InitResponse{Success: true}, + }, + } + // Snapshot + req = <-cmd.Requests + if req == nil { + t.Fatal("expected snapshot message got nil") + } + _, ok = req.Message.(*udf.Request_Snapshot) + if !ok { + t.Errorf("expected snapshot message got %T", req.Message) + } + data := []byte{42} + cmd.Responses <- &udf.Response{ + Message: &udf.Response_Snapshot{ + Snapshot: &udf.SnapshotResponse{data}, + }, + } + // Restore + req = <-cmd.Requests + if req == nil { + t.Fatal("expected restore message got nil") + } + restore, ok := req.Message.(*udf.Request_Restore) + if !ok { + t.Errorf("expected restore message got %T", req.Message) + } + if !reflect.DeepEqual(data, restore.Restore.Snapshot) { + t.Errorf("unexpected restore snapshot got %v exp %v", restore.Restore.Snapshot, data) + } + cmd.Responses <- &udf.Response{ + Message: &udf.Response_Restore{ + Restore: &udf.RestoreResponse{Success: true}, + }, + } + close(cmd.Responses) + }() + p.Start() + p.Init(nil) + snapshot, err := p.Snapshot() + if err != nil { + t.Fatal(err) + } + err = p.Restore(snapshot) + if err != nil { + t.Fatal(err) + } + + p.Stop() + // read all requests and wait till the chan is closed + for range cmd.Requests { + } + if err := <-cmd.ErrC; err != nil { + t.Error(err) + } +} diff --git a/union.go b/union.go index 55ee16061..a8836f7d2 100644 --- a/union.go +++ b/union.go @@ -1,6 +1,8 @@ package kapacitor import ( + "log" + "github.com/influxdata/kapacitor/pipeline" ) @@ -11,16 +13,16 @@ type UnionNode struct { // Create a new UnionNode which combines all parent data streams into a single stream. // No transformation of any kind is performed. -func newUnionNode(et *ExecutingTask, n *pipeline.UnionNode) (*UnionNode, error) { +func newUnionNode(et *ExecutingTask, n *pipeline.UnionNode, l *log.Logger) (*UnionNode, error) { un := &UnionNode{ u: n, - node: node{Node: n, et: et}, + node: node{Node: n, et: et, logger: l}, } un.node.runF = un.runUnion return un, nil } -func (u *UnionNode) runUnion() error { +func (u *UnionNode) runUnion([]byte) error { rename := u.u.Rename if rename == "" { //the calling node is always the last node diff --git a/where.go b/where.go index 5bb06e9d4..e65c43fab 100644 --- a/where.go +++ b/where.go @@ -2,6 +2,7 @@ package kapacitor import ( "errors" + "log" "github.com/influxdata/kapacitor/models" "github.com/influxdata/kapacitor/pipeline" @@ -16,9 +17,9 @@ type WhereNode struct { } // Create a new WhereNode which filters down the batch or stream by a condition -func newWhereNode(et *ExecutingTask, n *pipeline.WhereNode) (wn *WhereNode, err error) { +func newWhereNode(et *ExecutingTask, n *pipeline.WhereNode, l *log.Logger) (wn *WhereNode, err error) { wn = &WhereNode{ - node: node{Node: n, et: et}, + node: node{Node: n, et: et, logger: l}, w: n, expressions: make(map[models.GroupID]*tick.StatefulExpr), } @@ -29,7 +30,7 @@ func newWhereNode(et *ExecutingTask, n *pipeline.WhereNode) (wn *WhereNode, err return } -func (w *WhereNode) runWhere() error { +func (w *WhereNode) runWhere(snapshot []byte) error { switch w.Wants() { case pipeline.StreamEdge: for p, ok := w.ins[0].NextPoint(); ok; p, ok = w.ins[0].NextPoint() { diff --git a/window.go b/window.go index df467e564..ac2542b50 100644 --- a/window.go +++ b/window.go @@ -16,16 +16,16 @@ type WindowNode struct { } // Create a new WindowNode, which windows data for a period of time and emits the window. -func newWindowNode(et *ExecutingTask, n *pipeline.WindowNode) (*WindowNode, error) { +func newWindowNode(et *ExecutingTask, n *pipeline.WindowNode, l *log.Logger) (*WindowNode, error) { wn := &WindowNode{ w: n, - node: node{Node: n, et: et}, + node: node{Node: n, et: et, logger: l}, } wn.node.runF = wn.runWindow return wn, nil } -func (w *WindowNode) runWindow() error { +func (w *WindowNode) runWindow([]byte) error { windows := make(map[models.GroupID]*window) // Loops through points windowing by group for p, ok := w.ins[0].NextPoint(); ok; p, ok = w.ins[0].NextPoint() {