functions: hot containers (#332)

* functions: modify datastore to accomodate hot containers support

* functions: protocol between functions and hot containers

* functions: add hot containers clockwork

* fn: add hot containers support
This commit is contained in:
C Cirello
2016-11-28 18:45:35 +01:00
committed by Pedro Nasser
parent d0429c3dfd
commit ac0044f7d9
31 changed files with 809 additions and 170 deletions

View File

@@ -342,6 +342,8 @@ func (ds *BoltDatastore) UpdateRoute(ctx context.Context, newroute *models.Route
if route.Type != "" {
route.Type = newroute.Type
}
route.Format = newroute.Format
route.MaxConcurrency = newroute.MaxConcurrency
if newroute.Headers != nil {
if route.Headers == nil {
route.Headers = map[string][]string{}

View File

@@ -18,8 +18,11 @@ const routesTableCreate = `
CREATE TABLE IF NOT EXISTS routes (
app_name character varying(256) NOT NULL,
path text NOT NULL,
image character varying(256) NOT NULL,
image character varying(256) NOT NULL,
format character varying(16) NOT NULL,
maxc integer NOT NULL,
memory integer NOT NULL,
type character varying(16) NOT NULL,
headers text NOT NULL,
config text NOT NULL,
PRIMARY KEY (app_name, path)
@@ -35,7 +38,7 @@ const extrasTableCreate = `CREATE TABLE IF NOT EXISTS extras (
value character varying(256) NOT NULL
);`
const routeSelector = `SELECT app_name, path, image, memory, headers, config FROM routes`
const routeSelector = `SELECT app_name, path, image, format, maxc, memory, type, headers, config FROM routes`
type rowScanner interface {
Scan(dest ...interface{}) error
@@ -260,15 +263,21 @@ func (ds *PostgresDatastore) InsertRoute(ctx context.Context, route *models.Rout
app_name,
path,
image,
format,
maxc,
memory,
type,
headers,
config
)
VALUES ($1, $2, $3, $4, $5, $6);`,
VALUES ($1, $2, $3, $4, $5, $6, $7, $8, $9);`,
route.AppName,
route.Path,
route.Image,
route.Format,
route.MaxConcurrency,
route.Memory,
route.Type,
string(hbyte),
string(cbyte),
)
@@ -301,14 +310,20 @@ func (ds *PostgresDatastore) UpdateRoute(ctx context.Context, route *models.Rout
res, err := ds.db.Exec(`
UPDATE routes SET
image = $3,
memory = $4,
headers = $5,
config = $6
format = $4,
memory = $5,
maxc = $6,
type = $7,
headers = $8,
config = $9
WHERE app_name = $1 AND path = $2;`,
route.AppName,
route.Path,
route.Image,
route.Format,
route.Memory,
route.MaxConcurrency,
route.Type,
string(hbyte),
string(cbyte),
)
@@ -367,7 +382,10 @@ func scanRoute(scanner rowScanner, route *models.Route) error {
&route.AppName,
&route.Path,
&route.Image,
&route.Format,
&route.Memory,
&route.MaxConcurrency,
&route.Type,
&headerStr,
&configStr,
)

View File

@@ -57,6 +57,8 @@ func TestPostgres(t *testing.T) {
AppName: testApp.Name,
Path: "/test",
Image: "iron/hello",
Type: "sync",
Format: "http",
}
// Testing insert app

View File

@@ -25,19 +25,22 @@ var (
type Routes []*Route
type Route struct {
AppName string `json:"app_name,omitempty"`
Path string `json:"path,omitempty"`
Image string `json:"image,omitempty"`
Memory uint64 `json:"memory,omitempty"`
Headers http.Header `json:"headers,omitempty"`
Type string `json:"type,omitempty"`
Config `json:"config"`
AppName string `json:"app_name,omitempty"`
Path string `json:"path,omitempty"`
Image string `json:"image,omitempty"`
Memory uint64 `json:"memory,omitempty"`
Headers http.Header `json:"headers,omitempty"`
Type string `json:"type,omitempty"`
Format string `json:"format,omitempty"`
MaxConcurrency int `json:"max_concurrency,omitempty"`
Config `json:"config"`
}
var (
ErrRoutesValidationFoundDynamicURL = errors.New("Dynamic URL is not allowed")
ErrRoutesValidationInvalidPath = errors.New("Invalid Path format")
ErrRoutesValidationInvalidType = errors.New("Invalid route Type")
ErrRoutesValidationInvalidFormat = errors.New("Invalid route Format")
ErrRoutesValidationMissingAppName = errors.New("Missing route AppName")
ErrRoutesValidationMissingImage = errors.New("Missing route Image")
ErrRoutesValidationMissingName = errors.New("Missing route Name")
@@ -82,6 +85,14 @@ func (r *Route) Validate() error {
res = append(res, ErrRoutesValidationInvalidType)
}
if r.Format != FormatDefault && r.Format != FormatHTTP {
res = append(res, ErrRoutesValidationInvalidFormat)
}
if r.MaxConcurrency == 0 && r.Format == FormatHTTP {
r.MaxConcurrency = 1
}
if len(res) > 0 {
return apiErrors.CompositeValidationError(res...)
}

View File

@@ -21,6 +21,13 @@ const (
TypeAsync = "async"
)
const (
// FormatDefault ...
FormatDefault = ""
// FormatHTTP ...
FormatHTTP = "http"
)
/*Task task
swagger:model Task

View File

@@ -14,6 +14,7 @@ import (
"github.com/Sirupsen/logrus"
"github.com/iron-io/functions/api/models"
"github.com/iron-io/functions/api/runner/task"
"github.com/iron-io/runner/common"
)
@@ -40,18 +41,18 @@ func getTask(ctx context.Context, url string) (*models.Task, error) {
return &task, nil
}
func getCfg(task *models.Task) *Config {
// TODO: should limit the size of this, error if gets too big. akin to: https://golang.org/pkg/io/#LimitReader
if task.Timeout == nil {
func getCfg(t *models.Task) *task.Config {
if t.Timeout == nil {
timeout := int32(30)
task.Timeout = &timeout
t.Timeout = &timeout
}
cfg := &Config{
Image: *task.Image,
Timeout: time.Duration(*task.Timeout) * time.Second,
ID: task.ID,
AppName: task.AppName,
Env: task.EnvVars,
cfg := &task.Config{
Image: *t.Image,
Timeout: time.Duration(*t.Timeout) * time.Second,
ID: t.ID,
AppName: t.AppName,
Env: t.EnvVars,
}
return cfg
}
@@ -82,14 +83,14 @@ func deleteTask(url string, task *models.Task) error {
}
// RunAsyncRunner pulls tasks off a queue and processes them
func RunAsyncRunner(ctx context.Context, tasksrv string, tasks chan TaskRequest, rnr *Runner) {
func RunAsyncRunner(ctx context.Context, tasksrv string, tasks chan task.Request, rnr *Runner) {
u := tasksrvURL(tasksrv)
startAsyncRunners(ctx, u, tasks, rnr)
<-ctx.Done()
}
func startAsyncRunners(ctx context.Context, url string, tasks chan TaskRequest, rnr *Runner) {
func startAsyncRunners(ctx context.Context, url string, tasks chan task.Request, rnr *Runner) {
var wg sync.WaitGroup
ctx, log := common.LoggerWithFields(ctx, logrus.Fields{"runner": "async"})
for {

View File

@@ -17,6 +17,7 @@ import (
"github.com/gin-gonic/gin"
"github.com/iron-io/functions/api/models"
"github.com/iron-io/functions/api/mqs"
"github.com/iron-io/functions/api/runner/task"
)
func setLogBuffer() *bytes.Buffer {
@@ -202,13 +203,13 @@ func TestAsyncRunnersGracefulShutdown(t *testing.T) {
ts := getTestServer([]*models.Task{&mockTask})
defer ts.Close()
tasks := make(chan TaskRequest)
tasks := make(chan task.Request)
ctx, cancel := context.WithTimeout(context.Background(), 2*time.Second)
defer cancel()
defer close(tasks)
go func() {
for t := range tasks {
t.Response <- TaskResponse{
t.Response <- task.Response{
Result: nil,
Err: nil,
}

View File

@@ -0,0 +1,19 @@
package protocol
import (
"context"
"github.com/iron-io/functions/api/runner/task"
)
// DefaultProtocol is the protocol used by cold-containers
type DefaultProtocol struct {
}
func (p *DefaultProtocol) IsStreamable() bool {
return false
}
func (p *DefaultProtocol) Dispatch(ctx context.Context, t task.Request) error {
return nil
}

View File

@@ -0,0 +1,52 @@
package protocol
import (
"context"
"errors"
"io"
"github.com/iron-io/functions/api/models"
"github.com/iron-io/functions/api/runner/task"
)
var errInvalidProtocol = errors.New("Invalid Protocol")
// ContainerIO defines the interface used to talk to a hot container.
// Internally, a protocol must know when to alternate between stdin and stdout.
// It returns any protocol error, if present.
type ContainerIO interface {
IsStreamable() bool
Dispatch(ctx context.Context, t task.Request) error
}
// Protocol defines all protocols that operates a ContainerIO.
type Protocol string
// Hot container protocols
const (
Default Protocol = models.FormatDefault
HTTP Protocol = models.FormatHTTP
)
// New creates a valid protocol handler from a I/O pipe representing containers
// stdin/stdout.
func New(p Protocol, in io.Writer, out io.Reader) (ContainerIO, error) {
switch p {
case HTTP:
return &HTTPProtocol{in, out}, nil
case Default:
return &DefaultProtocol{}, nil
default:
return nil, errInvalidProtocol
}
}
// IsStreamable says whether the given protocol can be used for streaming into
// hot containers.
func IsStreamable(p string) (bool, error) {
proto, err := New(Protocol(p), nil, nil)
if err != nil {
return false, err
}
return proto.IsStreamable(), nil
}

View File

@@ -0,0 +1,73 @@
package protocol
import (
"bufio"
"bytes"
"context"
"fmt"
"io"
"net/http"
"net/http/httputil"
"time"
"github.com/iron-io/functions/api/models"
"github.com/iron-io/functions/api/runner/task"
)
// HTTPProtocol converts stdin/stdout streams into HTTP/1.1 compliant
// communication. It relies on Content-Length to know when to stop reading from
// containers stdout. It also mandates valid HTTP headers back and forth, thus
// returning errors in case of parsing problems.
type HTTPProtocol struct {
in io.Writer
out io.Reader
}
func (p *HTTPProtocol) IsStreamable() bool {
return true
}
func (p *HTTPProtocol) Dispatch(ctx context.Context, t task.Request) error {
var retErr error
done := make(chan struct{})
go func() {
var body bytes.Buffer
io.Copy(&body, t.Config.Stdin)
req, err := http.NewRequest("GET", "/", &body)
if err != nil {
retErr = err
return
}
for k, v := range t.Config.Env {
req.Header.Set(k, v)
}
req.Header.Set("Content-Length", fmt.Sprint(body.Len()))
req.Header.Set("Task-ID", t.Config.ID)
raw, err := httputil.DumpRequest(req, true)
if err != nil {
retErr = err
return
}
p.in.Write(raw)
res, err := http.ReadResponse(bufio.NewReader(p.out), req)
if err != nil {
retErr = err
return
}
io.Copy(t.Config.Stdout, res.Body)
done <- struct{}{}
}()
timeout := time.After(t.Config.Timeout)
select {
case <-ctx.Done():
return ctx.Err()
case <-timeout:
return models.ErrRunnerTimeout
case <-done:
return retErr
}
}

View File

@@ -5,7 +5,6 @@ import (
"context"
"errors"
"fmt"
"io"
"io/ioutil"
"os"
"runtime"
@@ -15,6 +14,7 @@ import (
"time"
"github.com/Sirupsen/logrus"
"github.com/iron-io/functions/api/runner/task"
"github.com/iron-io/runner/common"
"github.com/iron-io/runner/drivers"
driverscommon "github.com/iron-io/runner/drivers"
@@ -22,19 +22,6 @@ import (
"github.com/iron-io/runner/drivers/mock"
)
type Config struct {
ID string
Image string
Timeout time.Duration
AppName string
Path string
Memory uint64
Env map[string]string
Stdin io.Reader
Stdout io.Writer
Stderr io.Writer
}
type Runner struct {
driver drivers.Driver
taskQueue chan *containerTask
@@ -155,7 +142,7 @@ func (r *Runner) checkMemAndUse(req uint64) bool {
return true
}
func (r *Runner) Run(ctx context.Context, cfg *Config) (drivers.RunResult, error) {
func (r *Runner) Run(ctx context.Context, cfg *task.Config) (drivers.RunResult, error) {
var err error
if cfg.Memory == 0 {
@@ -224,7 +211,7 @@ func (r *Runner) Run(ctx context.Context, cfg *Config) (drivers.RunResult, error
return result, nil
}
func (r Runner) EnsureImageExists(ctx context.Context, cfg *Config) error {
func (r Runner) EnsureImageExists(ctx context.Context, cfg *task.Config) error {
ctask := &containerTask{
cfg: cfg,
}

View File

@@ -9,6 +9,7 @@ import (
"time"
"github.com/iron-io/functions/api/models"
"github.com/iron-io/functions/api/runner/task"
)
func TestRunnerHello(t *testing.T) {
@@ -31,7 +32,7 @@ func TestRunnerHello(t *testing.T) {
{&models.Route{Image: "iron/hello"}, `{"name": "test"}`, "success", "Hello test!", ""},
} {
var stdout, stderr bytes.Buffer
cfg := &Config{
cfg := &task.Config{
ID: fmt.Sprintf("hello-%d-%d", i, time.Now().Unix()),
Image: test.route.Image,
Timeout: 5 * time.Second,
@@ -84,7 +85,7 @@ func TestRunnerError(t *testing.T) {
{&models.Route{Image: "iron/error"}, `{"name": "test"}`, "error", "", "RuntimeError"},
} {
var stdout, stderr bytes.Buffer
cfg := &Config{
cfg := &task.Config{
ID: fmt.Sprintf("err-%d-%d", i, time.Now().Unix()),
Image: test.route.Image,
Timeout: 5 * time.Second,

View File

@@ -6,12 +6,13 @@ import (
"time"
"github.com/fsouza/go-dockerclient"
"github.com/iron-io/functions/api/runner/task"
"github.com/iron-io/runner/drivers"
)
type containerTask struct {
ctx context.Context
cfg *Config
cfg *task.Config
canRun chan bool
}

40
api/runner/task/task.go Normal file
View File

@@ -0,0 +1,40 @@
package task
import (
"context"
"io"
"time"
"github.com/iron-io/runner/drivers"
)
type Config struct {
ID string
Path string
Image string
Timeout time.Duration
AppName string
Memory uint64
Env map[string]string
Format string
MaxConcurrency int
Stdin io.Reader
Stdout io.Writer
Stderr io.Writer
}
// Request stores the task to be executed by the common concurrency stream,
// whatever type the ask actually is, either sync or async. It holds in itself
// the channel to return its response to its caller.
type Request struct {
Ctx context.Context
Config *Config
Response chan Response
}
// Response holds the response metainformation of a Request
type Response struct {
Result drivers.RunResult
Err error
}

View File

@@ -1,52 +1,367 @@
package runner
import (
"bufio"
"context"
"fmt"
"io"
"sync"
"time"
"github.com/Sirupsen/logrus"
"github.com/iron-io/functions/api/runner/protocol"
"github.com/iron-io/functions/api/runner/task"
"github.com/iron-io/runner/drivers"
)
type TaskRequest struct {
Ctx context.Context
Config *Config
Response chan TaskResponse
}
// Hot containers - theory of operation
//
// A function is converted into a hot container if its `Format` is either
// a streamable format/protocol. At the very first task request a hot
// container shall be started and run it. Each hot container has an internal
// clock that actually halts the container if it goes idle long enough. In the
// absence of workload, it just stops the whole clockwork.
//
// Internally, the hot container uses a modified Config whose Stdin and Stdout
// are bound to an internal pipe. This internal pipe is fed with incoming tasks
// Stdin and feeds incoming tasks with Stdout.
//
// Each execution is the alternation of feeding hot containers stdin with tasks
// stdin, and reading the answer back from containers stdout. For all `Format`s
// we send embedded into the message metadata to help the container to know when
// to stop reading from its stdin and Functions expect the container to do the
// same. Refer to api/runner/protocol.go for details of these communications.
//
// Hot Containers implementation relies in two moving parts (drawn below):
// htcntrmgr and htcntr. Refer to their respective comments for
// details.
// │
// Incoming
// Task
// │
// ▼
// ┌───────────────┐
// │ Task Request │
// │ Main Loop │
// └───────────────┘
// │
// ┌──────▼────────┐
// ┌┴──────────────┐│
// │ Per Function ││ non-streamable f()
// ┌───────│ Container │├──────┐───────────────┐
// │ │ Manager ├┘ │ │
// │ └───────────────┘ │ │
// │ │ │ │
// ▼ ▼ ▼ ▼
// ┌───────────┐ ┌───────────┐ ┌───────────┐ ┌───────────┐
// │ Hot │ │ Hot │ │ Hot │ │ Cold │
// │ Container │ │ Container │ │ Container │ │ Container │
// └───────────┘ └───────────┘ └───────────┘ └───────────┘
// Timeout
// Terminate
// (internal clock)
type TaskResponse struct {
Result drivers.RunResult
Err error
}
const (
// Terminate hot container after this timeout
htcntrScaleDownTimeout = 30 * time.Second
)
// StartWorkers handle incoming tasks and spawns self-regulating container
// workers.
func StartWorkers(ctx context.Context, rnr *Runner, tasks <-chan TaskRequest) {
var wg sync.WaitGroup
for {
select {
case <-ctx.Done():
wg.Wait()
return
case task := <-tasks:
wg.Add(1)
go func(task TaskRequest) {
defer wg.Done()
result, err := rnr.Run(task.Ctx, task.Config)
select {
case task.Response <- TaskResponse{result, err}:
close(task.Response)
default:
}
}(task)
}
}
}
func RunTask(tasks chan TaskRequest, ctx context.Context, cfg *Config) (drivers.RunResult, error) {
tresp := make(chan TaskResponse)
treq := TaskRequest{Ctx: ctx, Config: cfg, Response: tresp}
// RunTask helps sending a task.Request into the common concurrency stream.
// Refer to StartWorkers() to understand what this is about.
func RunTask(tasks chan task.Request, ctx context.Context, cfg *task.Config) (drivers.RunResult, error) {
tresp := make(chan task.Response)
treq := task.Request{Ctx: ctx, Config: cfg, Response: tresp}
tasks <- treq
resp := <-treq.Response
return resp.Result, resp.Err
}
// StartWorkers operates the common concurrency stream, ie, it will process all
// IronFunctions tasks, either sync or async. In the process, it also dispatches
// the workload to either regular or hot containers.
func StartWorkers(ctx context.Context, rnr *Runner, tasks <-chan task.Request) {
var wg sync.WaitGroup
defer wg.Wait()
var hcmgr htcntrmgr
for {
select {
case <-ctx.Done():
return
case task := <-tasks:
p := hcmgr.getPipe(ctx, rnr, task.Config)
if p == nil {
wg.Add(1)
go runTaskReq(rnr, &wg, task)
continue
}
select {
case <-ctx.Done():
return
case p <- task:
}
}
}
}
// htcntrmgr is the intermediate between the common concurrency stream and
// hot containers. All hot containers share a single task.Request stream per
// function (chn), but each function may have more than one hot container (hc).
type htcntrmgr struct {
chn map[string]chan task.Request
hc map[string]*htcntrsvr
}
func (h *htcntrmgr) getPipe(ctx context.Context, rnr *Runner, cfg *task.Config) chan task.Request {
isStream, err := protocol.IsStreamable(cfg.Format)
if err != nil {
logrus.WithError(err).Info("could not detect container IO protocol")
return nil
} else if !isStream {
return nil
}
if h.chn == nil {
h.chn = make(map[string]chan task.Request)
h.hc = make(map[string]*htcntrsvr)
}
// TODO(ccirello): re-implement this without memory allocation (fmt.Sprint)
fn := fmt.Sprint(cfg.AppName, ",", cfg.Path, cfg.Image, cfg.Timeout, cfg.Memory, cfg.Format, cfg.MaxConcurrency)
tasks, ok := h.chn[fn]
if !ok {
h.chn[fn] = make(chan task.Request)
tasks = h.chn[fn]
svr := newhtcntrsvr(ctx, cfg, rnr, tasks)
if err := svr.launch(ctx); err != nil {
logrus.WithError(err).Error("cannot start hot container supervisor")
return nil
}
h.hc[fn] = svr
}
return tasks
}
// htcntrsvr is part of htcntrmgr, abstracted apart for simplicity, its only
// purpose is to test for hot containers saturation and try starting as many as
// needed. In case of absence of workload, it will stop trying to start new hot
// containers.
type htcntrsvr struct {
cfg *task.Config
rnr *Runner
tasksin <-chan task.Request
tasksout chan task.Request
maxc chan struct{}
}
func newhtcntrsvr(ctx context.Context, cfg *task.Config, rnr *Runner, tasks <-chan task.Request) *htcntrsvr {
svr := &htcntrsvr{
cfg: cfg,
rnr: rnr,
tasksin: tasks,
tasksout: make(chan task.Request, 1),
maxc: make(chan struct{}, cfg.MaxConcurrency),
}
// This pipe will take all incoming tasks and just forward them to the
// started hot containers. The catch here is that it feeds a buffered
// channel from an unbuffered one. And this buffered channel is
// then used to determine the presence of running hot containers.
// If no hot container is available, tasksout will fill up to its
// capacity and pipe() will start them.
go svr.pipe(ctx)
return svr
}
func (svr *htcntrsvr) pipe(ctx context.Context) {
for {
select {
case t := <-svr.tasksin:
svr.tasksout <- t
if len(svr.tasksout) > 0 {
if err := svr.launch(ctx); err != nil {
logrus.WithError(err).Error("cannot start more hot containers")
}
}
case <-ctx.Done():
return
}
}
}
func (svr *htcntrsvr) launch(ctx context.Context) error {
select {
case svr.maxc <- struct{}{}:
hc, err := newhtcntr(
svr.cfg,
protocol.Protocol(svr.cfg.Format),
svr.tasksout,
svr.rnr,
)
if err != nil {
return err
}
go func() {
hc.serve(ctx)
<-svr.maxc
}()
default:
}
return nil
}
// htcntr actually interfaces an incoming task from the common concurrency
// stream into a long lived container. If idle long enough, it will stop. It
// uses route configuration to determine which protocol to use.
type htcntr struct {
cfg *task.Config
proto protocol.ContainerIO
tasks <-chan task.Request
// Side of the pipe that takes information from outer world
// and injects into the container.
in io.Writer
out io.Reader
// Receiving side of the container.
containerIn io.Reader
containerOut io.Writer
rnr *Runner
}
func newhtcntr(cfg *task.Config, proto protocol.Protocol, tasks <-chan task.Request, rnr *Runner) (*htcntr, error) {
stdinr, stdinw := io.Pipe()
stdoutr, stdoutw := io.Pipe()
p, err := protocol.New(proto, stdinw, stdoutr)
if err != nil {
return nil, err
}
hc := &htcntr{
cfg: cfg,
proto: p,
tasks: tasks,
in: stdinw,
out: stdoutr,
containerIn: stdinr,
containerOut: stdoutw,
rnr: rnr,
}
return hc, nil
}
func (hc *htcntr) serve(ctx context.Context) {
lctx, cancel := context.WithCancel(ctx)
var wg sync.WaitGroup
wg.Add(1)
go func() {
defer wg.Done()
for {
inactivity := time.After(htcntrScaleDownTimeout)
select {
case <-lctx.Done():
return
case <-inactivity:
cancel()
case t := <-hc.tasks:
if err := hc.proto.Dispatch(lctx, t); err != nil {
logrus.WithField("ctx", lctx).Info("task failed")
t.Response <- task.Response{
&runResult{StatusValue: "error", error: err},
err,
}
continue
}
t.Response <- task.Response{
&runResult{StatusValue: "success"},
nil,
}
}
}
}()
cfg := *hc.cfg
cfg.Timeout = 0 // add a timeout to simulate ab.end. failure.
cfg.Stdin = hc.containerIn
cfg.Stdout = hc.containerOut
// Why can we not attach stderr to the task like we do for stdin and
// stdout?
//
// Stdin/Stdout are completely known to the scope of the task. You must
// have a task stdin to feed containers stdin, and also the other way
// around when reading from stdout. So both are directly related to the
// life cycle of the request.
//
// Stderr, on the other hand, can be written by anything any time:
// failure between requests, failures inside requests and messages send
// right after stdout has been finished being transmitted. Thus, with
// hot containers, there is not a 1:1 relation between stderr and tasks.
//
// Still, we do pass - at protocol level - a Task-ID header, from which
// the application running inside the hot container can use to identify
// its own stderr output.
errr, errw := io.Pipe()
cfg.Stderr = errw
wg.Add(1)
go func() {
defer wg.Done()
scanner := bufio.NewScanner(errr)
for scanner.Scan() {
logrus.WithFields(logrus.Fields{
"app": cfg.AppName,
"route": cfg.Path,
"image": cfg.Image,
"memory": cfg.Memory,
"format": cfg.Format,
"max_concurrency": cfg.MaxConcurrency,
}).Info(scanner.Text())
}
}()
result, err := hc.rnr.Run(lctx, &cfg)
if err != nil {
logrus.WithError(err).Error("hot container failure detected")
}
cancel()
errw.Close()
wg.Wait()
logrus.WithField("result", result).Info("hot container terminated")
}
func runTaskReq(rnr *Runner, wg *sync.WaitGroup, t task.Request) {
defer wg.Done()
result, err := rnr.Run(t.Ctx, t.Config)
select {
case t.Response <- task.Response{result, err}:
close(t.Response)
default:
}
}
type runResult struct {
error
StatusValue string
}
func (r *runResult) Error() string {
if r.error == nil {
return ""
}
return r.error.Error()
}
func (r *runResult) Status() string { return r.StatusValue }

View File

@@ -12,7 +12,7 @@ import (
"github.com/iron-io/functions/api/datastore"
"github.com/iron-io/functions/api/models"
"github.com/iron-io/functions/api/mqs"
"github.com/iron-io/functions/api/runner"
"github.com/iron-io/functions/api/runner/task"
)
func setLogBuffer() *bytes.Buffer {
@@ -25,8 +25,8 @@ func setLogBuffer() *bytes.Buffer {
return &buf
}
func mockTasksConduit() chan runner.TaskRequest {
tasks := make(chan runner.TaskRequest)
func mockTasksConduit() chan task.Request {
tasks := make(chan task.Request)
go func() {
for range tasks {
}

View File

@@ -6,7 +6,7 @@ import (
"github.com/gin-gonic/gin"
"github.com/iron-io/functions/api/models"
"github.com/iron-io/functions/api/runner"
"github.com/iron-io/functions/api/runner/task"
"github.com/iron-io/runner/common"
)
@@ -41,7 +41,7 @@ func (s *Server) handleRouteCreate(c *gin.Context) {
c.JSON(http.StatusBadRequest, simpleError(models.ErrRoutesValidationMissingImage))
return
}
err = Api.Runner.EnsureImageExists(ctx, &runner.Config{
err = Api.Runner.EnsureImageExists(ctx, &task.Config{
Image: wroute.Route.Image,
})
if err != nil {

View File

@@ -7,7 +7,7 @@ import (
"github.com/gin-gonic/gin"
"github.com/iron-io/functions/api/models"
"github.com/iron-io/functions/api/runner"
"github.com/iron-io/functions/api/runner/task"
"github.com/iron-io/runner/common"
)
@@ -34,7 +34,7 @@ func handleRouteUpdate(c *gin.Context) {
wroute.Route.Path = path.Clean(c.Param("route"))
if wroute.Route.Image != "" {
err = Api.Runner.EnsureImageExists(ctx, &runner.Config{
err = Api.Runner.EnsureImageExists(ctx, &task.Config{
Image: wroute.Route.Image,
})
if err != nil {

View File

@@ -14,6 +14,7 @@ import (
"github.com/gin-gonic/gin"
"github.com/iron-io/functions/api/models"
"github.com/iron-io/functions/api/runner"
"github.com/iron-io/functions/api/runner/task"
"github.com/iron-io/runner/common"
uuid "github.com/satori/go.uuid"
)
@@ -167,15 +168,18 @@ func (s *Server) serve(ctx context.Context, c *gin.Context, appName string, foun
envVars[ToEnvName("HEADER", header)] = strings.Join(value, " ")
}
cfg := &runner.Config{
Image: found.Image,
Timeout: 30 * time.Second,
ID: reqID,
AppName: appName,
Stdout: &stdout,
Env: envVars,
Memory: found.Memory,
Stdin: payload,
cfg := &task.Config{
AppName: appName,
Path: found.Path,
Env: envVars,
Format: found.Format,
ID: reqID,
Image: found.Image,
MaxConcurrency: found.MaxConcurrency,
Memory: found.Memory,
Stdin: payload,
Stdout: &stdout,
Timeout: 30 * time.Second,
}
switch found.Type {

View File

@@ -13,10 +13,11 @@ import (
"github.com/iron-io/functions/api/models"
"github.com/iron-io/functions/api/mqs"
"github.com/iron-io/functions/api/runner"
"github.com/iron-io/functions/api/runner/task"
"github.com/iron-io/runner/common"
)
func testRouterAsync(ds models.Datastore, mq models.MessageQueue, rnr *runner.Runner, tasks chan runner.TaskRequest, enqueue models.Enqueue) *gin.Engine {
func testRouterAsync(ds models.Datastore, mq models.MessageQueue, rnr *runner.Runner, tasks chan task.Request, enqueue models.Enqueue) *gin.Engine {
ctx := context.Background()
s := New(ctx, ds, mq, rnr, tasks, enqueue)
r := s.Router

View File

@@ -11,6 +11,7 @@ import (
"github.com/iron-io/functions/api/models"
"github.com/iron-io/functions/api/mqs"
"github.com/iron-io/functions/api/runner"
"github.com/iron-io/functions/api/runner/task"
)
func testRunner(t *testing.T) *runner.Runner {
@@ -106,7 +107,7 @@ func TestRouteRunnerPost(t *testing.T) {
func TestRouteRunnerExecution(t *testing.T) {
buf := setLogBuffer()
tasks := make(chan runner.TaskRequest)
tasks := make(chan task.Request)
ctx, cancel := context.WithCancel(context.Background())
defer cancel()

View File

@@ -15,6 +15,7 @@ import (
"github.com/iron-io/functions/api/ifaces"
"github.com/iron-io/functions/api/models"
"github.com/iron-io/functions/api/runner"
"github.com/iron-io/functions/api/runner/task"
"github.com/iron-io/functions/api/server/internal/routecache"
"github.com/iron-io/runner/common"
)
@@ -31,7 +32,7 @@ type Server struct {
SpecialHandlers []ifaces.SpecialHandler
Enqueue models.Enqueue
tasks chan runner.TaskRequest
tasks chan task.Request
mu sync.Mutex // protects hotroutes
hotroutes map[string]*routecache.Cache
@@ -40,7 +41,7 @@ type Server struct {
Datastore models.Datastore
}
func New(ctx context.Context, ds models.Datastore, mq models.MessageQueue, r *runner.Runner, tasks chan runner.TaskRequest, enqueue models.Enqueue) *Server {
func New(ctx context.Context, ds models.Datastore, mq models.MessageQueue, r *runner.Runner, tasks chan task.Request, enqueue models.Enqueue) *Server {
Api = &Server{
Runner: r,
Router: gin.New(),

View File

@@ -16,12 +16,13 @@ import (
"github.com/iron-io/functions/api/models"
"github.com/iron-io/functions/api/mqs"
"github.com/iron-io/functions/api/runner"
"github.com/iron-io/functions/api/runner/task"
"github.com/iron-io/runner/common"
)
var tmpBolt = "/tmp/func_test_bolt.db"
func testRouter(ds models.Datastore, mq models.MessageQueue, rnr *runner.Runner, tasks chan runner.TaskRequest) *gin.Engine {
func testRouter(ds models.Datastore, mq models.MessageQueue, rnr *runner.Runner, tasks chan task.Request) *gin.Engine {
ctx := context.Background()
s := New(ctx, ds, mq, rnr, tasks, DefaultEnqueue)
r := s.Router
@@ -90,7 +91,7 @@ func TestFullStack(t *testing.T) {
ds, closeBolt := prepareBolt(t)
defer closeBolt()
tasks := make(chan runner.TaskRequest)
tasks := make(chan task.Request)
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
go runner.StartWorkers(ctx, testRunner(t), tasks)

View File

@@ -2,31 +2,25 @@
This document will describe the details of how a function works, inputs/outputs, etc.
## Input
## Formats
### STDIN and Environment Variables
While wanting to keep things simple, flexible and expandable, we decided to go back to the basics, using Unix input and output. Standard in is easy to use in any
language and doesn't require anything extra. It also allows streaming input so we can do things like keeping a container running some time and stream
requests into the container.
While wanting to keep things simple, flexible and expandable, we decided to go back to the basics, using Unix input and output. Standard in is easy to use in any language and doesn't require anything extra. It also allows streaming input so we can do things like keeping a container running some time and stream requests into the container.
Configuration values, environment information and other things will be passed in through environment variables.
### Input Formats
The goals of the input format are the following:
* Very easy to use and parse
* Streamable for increasing performance (more than one call per container execution)
* Ability to build higher level abstractions on top (ie: Lambda syntax compatible)
The format is still up for discussion and in order to move forward and remain flexible, it's likely we will just allow different input formats and the
function creator can decide what they want, on a per function basis. Default being the simplest format to use.
The format is still up for discussion and in order to move forward and remain flexible, it's likely we will just allow different input formats and the function creator can decide what they want, on a per function basis. Default being the simplest format to use.
#### Default Input Format
#### Default I/O Format
The default input format is simply the request body itself plus some environment variables. For instance, if someone were to post a JSON body, the unmodified body would
be sent in via STDIN.
The default I/O format is simply the request body itself plus some environment variables. For instance, if someone were to post a JSON body, the unmodified body would be sent in via STDIN. The result comes via STDOUT. When task is done, pipes are closed and the container running the function is terminated.
Pros:
@@ -36,23 +30,31 @@ Cons:
* Not streamable
#### HTTP/1 Input Format (Not implemented)
#### HTTP I/O Format
`--input-format http`
`--format http`
HTTP format could be a good option as it is in very common use obviously, most languages have some semi-easy way to parse it, and it's streamable. The basic format
is:
HTTP format could be a good option as it is in very common use obviously, most languages have some semi-easy way to parse it, and it's streamable. The response will look like a HTTP response. The communication is still done via stdin/stdout, but these pipes are never closed unless the container is explicitly terminated. The basic format is:
Request:
```
REQUEST LINE
HEADER
BLANK LINE
BODY
GET / HTTP/1.1
Content-Length: 5
world
```
Response:
```
HTTP/1.1 200 OK
Content-Length: 11
hello world
```
The header keys and values would be populated with information about the function call such as the request URL and query parameters.
Body length is determined by the [Content-Length](https://tools.ietf.org/html/rfc7230#section-3.3.3) header, which is mandatory.
`Content-Length` is determined by the [Content-Length](https://tools.ietf.org/html/rfc7230#section-3.3.3) header, which is mandatory both for input and output. It is used by IronFunctions to know when stop writing to STDIN and reading from STDOUT.
Pros:
@@ -64,9 +66,9 @@ Cons:
* Requires a parsing library or fair amount of code to parse headers properly
* Double parsing - headers + body (if body is to be parsed, such as json)
#### JSON/HTTP Input Format (Not implemented)
#### JSON I/O Format (not implemented)
`--input-format json-http`
`--format json`
The idea here is to keep the HTTP base structure, but make it a bit easier to parse by making the `request line` and `headers` a JSON struct.
Eg:
@@ -91,17 +93,6 @@ Cons:
* New, unknown format
## Output
### STDOUT
For synchronous: True to form, whatever is written to standard out is returned as the response. If you want to return some JSON output, just write it directly to STDOUT.
TODO: How to change response headers? Perhaps a similar style as input? Headers, then body. Default headers can be defined on the route and overridden on output.
For asynchronous: STDOUT will be written to /dev/null until [further notice](https://github.com/iron-io/functions/issues/173). We do not want to write this
to the logs now, then change it later, otherwise people will start to depend on it.
### STDERR
Standard error is reserved for logging, like it was meant to be. Anything you output to STDERR will show up in the logs. And if you use a log

View File

@@ -6,7 +6,7 @@ swagger: '2.0'
info:
title: IronFunctions
description: The open source serverless platform.
version: "0.1.17"
version: "0.1.18"
# the domain of the service
host: "127.0.0.1:8080"
# array of all schemes that your API supports
@@ -338,6 +338,16 @@ definitions:
- sync
- async
description: Route type
format:
enum:
- default
- http
- json
description: Payload format sent into function.
max_concurrency:
type: integer
format: int32
description: Maximum number of hot containers concurrency
config:
type: object
description: Route configuration - overrides application configuration

View File

@@ -0,0 +1,45 @@
package main
import (
"bufio"
"bytes"
"fmt"
"io/ioutil"
"net/http"
"os"
"strconv"
)
func main() {
for {
res := http.Response{
Proto: "HTTP/1.1",
ProtoMajor: 1,
ProtoMinor: 1,
StatusCode: 200,
Status: "OK",
}
r := bufio.NewReader(os.Stdin)
req, err := http.ReadRequest(r)
var buf bytes.Buffer
if err != nil {
res.StatusCode = 500
res.Status = http.StatusText(res.StatusCode)
fmt.Fprintln(&buf, err)
} else {
l, _ := strconv.Atoi(req.Header.Get("Content-Length"))
p := make([]byte, l)
r.Read(p)
fmt.Fprintf(&buf, "Hello %s\n", p)
for k, vs := range req.Header {
fmt.Fprintf(&buf, "ENV: %s %#v\n", k, vs)
}
}
res.Body = ioutil.NopCloser(&buf)
res.ContentLength = int64(buf.Len())
res.Write(os.Stdout)
}
}

View File

@@ -0,0 +1,9 @@
{"route":{
"app_name": "myapp",
"path": "/hot",
"image": "USERNAME/hchttp",
"memory": 64,
"type": "sync",
"config": null,
"format": "http"
}}

View File

@@ -23,16 +23,18 @@ var (
)
type funcfile struct {
App *string `yaml:"app,omitempty",json:"app,omitempty"`
Name string `yaml:"name,omitempty",json:"name,omitempty"`
Version string `yaml:"version,omitempty",json:"version,omitempty"`
Runtime *string `yaml:"runtime,omitempty",json:"runtime,omitempty"`
Entrypoint *string `yaml:"entrypoint,omitempty",json:"entrypoint,omitempty"`
Route *string `yaml:"route,omitempty",json:"route,omitempty"`
Type *string `yaml:"type,omitempty",json:"type,omitempty"`
Memory *int64 `yaml:"memory,omitempty",json:"memory,omitempty"`
Config map[string]string `yaml:"config,omitempty",json:"config,omitempty"`
Build []string `yaml:"build,omitempty",json:"build,omitempty"`
App *string `yaml:"app,omitempty",json:"app,omitempty"`
Name string `yaml:"name,omitempty",json:"name,omitempty"`
Version string `yaml:"version,omitempty",json:"version,omitempty"`
Runtime *string `yaml:"runtime,omitempty",json:"runtime,omitempty"`
Entrypoint *string `yaml:"entrypoint,omitempty",json:"entrypoint,omitempty"`
Route *string `yaml:"route,omitempty",json:"route,omitempty"`
Type *string `yaml:"type,omitempty",json:"type,omitempty"`
Memory *int64 `yaml:"memory,omitempty",json:"memory,omitempty"`
Format *string `yaml:"format,omitempty",json:"format,omitempty"`
MaxConcurrency *int `yaml:"int,omitempty",json:"int,omitempty"`
Config map[string]string `yaml:"config,omitempty",json:"config,omitempty"`
Build []string `yaml:"build,omitempty",json:"build,omitempty"`
}
func (ff *funcfile) FullName() string {

View File

@@ -40,10 +40,12 @@ func init() {
}
type initFnCmd struct {
name string
force bool
runtime string
entrypoint string
name string
force bool
runtime string
entrypoint string
format string
maxConcurrency int
}
func initFn() cli.Command {
@@ -71,6 +73,18 @@ func initFn() cli.Command {
Usage: "entrypoint is the command to run to start this function - equivalent to Dockerfile ENTRYPOINT.",
Destination: &a.entrypoint,
},
cli.StringFlag{
Name: "format",
Usage: "hot container IO format - json or http",
Destination: &a.format,
Value: "",
},
cli.IntFlag{
Name: "max-concurrency",
Usage: "maximum concurrency for hot container",
Destination: &a.maxConcurrency,
Value: 1,
},
},
}
}
@@ -92,10 +106,12 @@ func (a *initFnCmd) init(c *cli.Context) error {
}
ff := &funcfile{
Name: a.name,
Runtime: &a.runtime,
Version: initialVersion,
Entrypoint: &a.entrypoint,
Name: a.name,
Runtime: &a.runtime,
Version: initialVersion,
Entrypoint: &a.entrypoint,
Format: &a.format,
MaxConcurrency: &a.maxConcurrency,
}
if err := encodeFuncfileYAML("func.yaml", ff); err != nil {

View File

@@ -64,6 +64,16 @@ func routes() cli.Command {
Name: "config,c",
Usage: "route configuration",
},
cli.StringFlag{
Name: "format,f",
Usage: "hot container IO format - json or http",
Value: "",
},
cli.IntFlag{
Name: "max-concurrency,m",
Usage: "maximum concurrency for hot container",
Value: 1,
},
},
},
{
@@ -229,6 +239,8 @@ func (a *routesCmd) create(c *cli.Context) error {
appName := c.Args().Get(0)
route := c.Args().Get(1)
image := c.Args().Get(2)
var format string
var maxC int
if image == "" {
ff, err := findFuncfile()
if err != nil {
@@ -239,16 +251,31 @@ func (a *routesCmd) create(c *cli.Context) error {
}
}
image = ff.FullName()
if ff.Format != nil {
format = *ff.Format
}
if ff.MaxConcurrency != nil {
maxC = *ff.MaxConcurrency
}
}
if f := c.String("format"); f != "" {
format = f
}
if m := c.Int("max-concurrency"); m > 0 {
maxC = m
}
body := functions.RouteWrapper{
Route: functions.Route{
AppName: appName,
Path: route,
Image: image,
Memory: c.Int64("memory"),
Type_: c.String("type"),
Config: extractEnvConfig(c.StringSlice("config")),
AppName: appName,
Path: route,
Image: image,
Memory: c.Int64("memory"),
Type_: c.String("type"),
Config: extractEnvConfig(c.StringSlice("config")),
Format: format,
MaxConcurrency: int32(maxC),
},
}

View File

@@ -13,6 +13,7 @@ import (
"github.com/iron-io/functions/api/datastore"
"github.com/iron-io/functions/api/mqs"
"github.com/iron-io/functions/api/runner"
"github.com/iron-io/functions/api/runner/task"
"github.com/iron-io/functions/api/server"
"github.com/spf13/viper"
)
@@ -84,7 +85,7 @@ func main() {
},
}
tasks := make(chan runner.TaskRequest)
tasks := make(chan task.Request)
svr.AddFunc(func(ctx context.Context) {
runner.StartWorkers(ctx, rnr, tasks)