mirror of
https://github.com/fnproject/fn.git
synced 2022-10-28 21:29:17 +03:00
fn: lb and pure-runner with non-blocking agent (#989)
* fn: lb and pure-runner with non-blocking agent *) Removed pure-runner capacity tracking code. This did not play well with internal agent resource tracker. *) In LB and runner gRPC comm, removed ACK. Now, upon TryCall, pure-runner quickly proceeds to call Submit. This is good since at this stage pure-runner already has all relevant data to initiate the call. *) Unless pure-runner emits a NACK, LB immediately streams http body to runners. *) For retriable requests added a CachedReader for http.Request Body. *) Idempotenty/retry is similar to previous code. After initial success in Engament, after attempting a TryCall, unless we receive NACK, we cannot retry that call. *) ch and naive places now wraps each TryExec with a cancellable context to clean up gRPC contexts quicker. * fn: err for simpler one-time read GetBody approach This allows for a more flexible approach since we let users to define GetBody() to allow repetitive http body read. In default LB case, LB executes a one-time io.ReadAll and sets of GetBody, which is detected by RunnerCall.RequestBody(). * fn: additional check for non-nil req.body * fn: attempt to override IO errors with ctx for TryExec * fn: system-tests log dest * fn: LB: EOF send handling * fn: logging for partial IO * fn: use buffer pool for IO storage in lb agent * fn: pure runner should use chunks for data msgs * fn: required config validations and pass APIErrors * fn: additional tests and gRPC proto simplification *) remove ACK/NACK messages as Finish message type works OK for this purpose. *) return resp in api tests for check for status code *) empty body json test in api tests for lb & pure-runner * fn: buffer adjustments *) setRequestBody result handling correction *) switch to bytes.Reader for read-only safety *) io.EOF can be returned for non-nil Body in request. * fn: clarify detection of 503 / Server Too Busy
This commit is contained in:
@@ -5,6 +5,8 @@ import (
|
||||
"encoding/json"
|
||||
"errors"
|
||||
"io"
|
||||
"strconv"
|
||||
"strings"
|
||||
"time"
|
||||
|
||||
"google.golang.org/grpc"
|
||||
@@ -12,13 +14,20 @@ import (
|
||||
|
||||
pb "github.com/fnproject/fn/api/agent/grpc"
|
||||
"github.com/fnproject/fn/api/common"
|
||||
"github.com/fnproject/fn/api/models"
|
||||
pool "github.com/fnproject/fn/api/runnerpool"
|
||||
"github.com/fnproject/fn/grpcutil"
|
||||
"github.com/sirupsen/logrus"
|
||||
)
|
||||
|
||||
var (
|
||||
ErrorRunnerClosed = errors.New("Runner is closed")
|
||||
ErrorRunnerClosed = errors.New("Runner is closed")
|
||||
ErrorPureRunnerNoEOF = errors.New("Purerunner missing EOF response")
|
||||
)
|
||||
|
||||
const (
|
||||
// max buffer size for grpc data messages, 10K
|
||||
MaxDataChunk = 10 * 1024
|
||||
)
|
||||
|
||||
type gRPCRunner struct {
|
||||
@@ -76,10 +85,15 @@ func (r *gRPCRunner) Address() string {
|
||||
return r.address
|
||||
}
|
||||
|
||||
func isRetriable(err error) bool {
|
||||
return models.GetAPIErrorCode(err) == models.GetAPIErrorCode(models.ErrCallTimeoutServerBusy)
|
||||
}
|
||||
|
||||
func (r *gRPCRunner) TryExec(ctx context.Context, call pool.RunnerCall) (bool, error) {
|
||||
logrus.WithField("runner_addr", r.address).Debug("Attempting to place call")
|
||||
if !r.shutWg.AddSession(1) {
|
||||
return true, ErrorRunnerClosed
|
||||
// try another runner if this one is closed.
|
||||
return false, ErrorRunnerClosed
|
||||
}
|
||||
defer r.shutWg.DoneSession()
|
||||
|
||||
@@ -90,6 +104,7 @@ func (r *gRPCRunner) TryExec(ctx context.Context, call pool.RunnerCall) (bool, e
|
||||
// If we can't encode the model, no runner will ever be able to run this. Give up.
|
||||
return true, err
|
||||
}
|
||||
|
||||
runnerConnection, err := r.client.Engage(ctx)
|
||||
if err != nil {
|
||||
logrus.WithError(err).Error("Unable to create client to runner node")
|
||||
@@ -97,113 +112,171 @@ func (r *gRPCRunner) TryExec(ctx context.Context, call pool.RunnerCall) (bool, e
|
||||
return false, err
|
||||
}
|
||||
|
||||
// After this point, we assume "COMMITTED" unless pure runner
|
||||
// send explicit NACK
|
||||
err = runnerConnection.Send(&pb.ClientMsg{Body: &pb.ClientMsg_Try{Try: &pb.TryCall{ModelsCallJson: string(modelJSON)}}})
|
||||
if err != nil {
|
||||
logrus.WithError(err).Error("Failed to send message to runner node")
|
||||
return false, err
|
||||
}
|
||||
msg, err := runnerConnection.Recv()
|
||||
if err != nil {
|
||||
logrus.WithError(err).Error("Failed to receive first message from runner node")
|
||||
return false, err
|
||||
return true, err
|
||||
}
|
||||
|
||||
switch body := msg.Body.(type) {
|
||||
case *pb.RunnerMsg_Acknowledged:
|
||||
if !body.Acknowledged.Committed {
|
||||
logrus.Debugf("Runner didn't commit invocation request: %v", body.Acknowledged.Details)
|
||||
return false, nil
|
||||
// Try the next runner
|
||||
}
|
||||
logrus.Debug("Runner committed invocation request, sending data frames")
|
||||
done := make(chan error)
|
||||
go receiveFromRunner(runnerConnection, call, done)
|
||||
sendToRunner(call, runnerConnection)
|
||||
return true, <-done
|
||||
recvDone := make(chan error, 1)
|
||||
|
||||
default:
|
||||
logrus.Errorf("Unhandled message type received from runner: %v\n", msg)
|
||||
return true, nil
|
||||
go receiveFromRunner(runnerConnection, call, recvDone)
|
||||
go sendToRunner(runnerConnection, call)
|
||||
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
logrus.Infof("Engagement Context ended ctxErr=%v", ctx.Err())
|
||||
return true, ctx.Err()
|
||||
case recvErr := <-recvDone:
|
||||
return !isRetriable(recvErr), recvErr
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
func sendToRunner(call pool.RunnerCall, protocolClient pb.RunnerProtocol_EngageClient) error {
|
||||
func sendToRunner(protocolClient pb.RunnerProtocol_EngageClient, call pool.RunnerCall) {
|
||||
bodyReader := call.RequestBody()
|
||||
writeBufferSize := 10 * 1024 // 10KB
|
||||
writeBuffer := make([]byte, writeBufferSize)
|
||||
for {
|
||||
n, err := bodyReader.Read(writeBuffer)
|
||||
logrus.Debugf("Wrote %v bytes to the runner", n)
|
||||
writeBuffer := make([]byte, MaxDataChunk)
|
||||
|
||||
if err == io.EOF {
|
||||
err = protocolClient.Send(&pb.ClientMsg{
|
||||
Body: &pb.ClientMsg_Data{
|
||||
Data: &pb.DataFrame{
|
||||
Data: writeBuffer,
|
||||
Eof: true,
|
||||
},
|
||||
},
|
||||
})
|
||||
if err != nil {
|
||||
logrus.WithError(err).Error("Failed to send data frame with EOF to runner")
|
||||
}
|
||||
break
|
||||
// IMPORTANT: IO Read below can fail in multiple go-routine cases (in retry
|
||||
// case especially if receiveFromRunner go-routine receives a NACK while sendToRunner is
|
||||
// already blocked on a read) or in the case of reading the http body multiple times (retries.)
|
||||
// Normally http.Request.Body can be read once. However runner_client users should implement/add
|
||||
// http.Request.GetBody() function and cache the body content in the request.
|
||||
// See lb_agent setRequestGetBody() which handles this. With GetBody installed,
|
||||
// the 'Read' below is an actually non-blocking operation since GetBody() should hand out
|
||||
// a new instance of io.ReadCloser() that allows repetitive reads on the http body.
|
||||
for {
|
||||
// WARNING: blocking read.
|
||||
n, err := bodyReader.Read(writeBuffer)
|
||||
if err != nil && err != io.EOF {
|
||||
logrus.WithError(err).Error("Failed to receive data from http client body")
|
||||
}
|
||||
err = protocolClient.Send(&pb.ClientMsg{
|
||||
|
||||
// any IO error or n == 0 is an EOF for pure-runner
|
||||
isEOF := err != nil || n == 0
|
||||
data := writeBuffer[:n]
|
||||
|
||||
logrus.Debugf("Sending %d bytes of data isEOF=%v to runner", n, isEOF)
|
||||
sendErr := protocolClient.Send(&pb.ClientMsg{
|
||||
Body: &pb.ClientMsg_Data{
|
||||
Data: &pb.DataFrame{
|
||||
Data: writeBuffer,
|
||||
Eof: false,
|
||||
Data: data,
|
||||
Eof: isEOF,
|
||||
},
|
||||
},
|
||||
})
|
||||
if err != nil {
|
||||
logrus.WithError(err).Error("Failed to send data frame")
|
||||
return err
|
||||
if sendErr != nil {
|
||||
logrus.WithError(sendErr).Errorf("Failed to send data frame size=%d isEOF=%v", n, isEOF)
|
||||
return
|
||||
}
|
||||
if isEOF {
|
||||
return
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func parseError(details string) error {
|
||||
tokens := strings.SplitN(details, ":", 2)
|
||||
if len(tokens) != 2 || tokens[0] == "" || tokens[1] == "" {
|
||||
return errors.New(details)
|
||||
}
|
||||
code, err := strconv.ParseInt(tokens[0], 10, 64)
|
||||
if err != nil {
|
||||
return errors.New(details)
|
||||
}
|
||||
if code != 0 {
|
||||
return models.NewAPIError(int(code), errors.New(tokens[1]))
|
||||
}
|
||||
return errors.New(tokens[1])
|
||||
}
|
||||
|
||||
func tryQueueError(err error, done chan error) {
|
||||
select {
|
||||
case done <- err:
|
||||
default:
|
||||
}
|
||||
}
|
||||
|
||||
func receiveFromRunner(protocolClient pb.RunnerProtocol_EngageClient, c pool.RunnerCall, done chan error) {
|
||||
w := c.ResponseWriter()
|
||||
defer close(done)
|
||||
|
||||
isPartialWrite := false
|
||||
|
||||
DataLoop:
|
||||
for {
|
||||
msg, err := protocolClient.Recv()
|
||||
if err != nil {
|
||||
logrus.WithError(err).Error("Failed to receive message from runner")
|
||||
done <- err
|
||||
logrus.WithError(err).Info("Receive error from runner")
|
||||
tryQueueError(err, done)
|
||||
return
|
||||
}
|
||||
|
||||
switch body := msg.Body.(type) {
|
||||
|
||||
// Process HTTP header/status message. This may not arrive depending on
|
||||
// pure runners behavior. (Eg. timeout & no IO received from function)
|
||||
case *pb.RunnerMsg_ResultStart:
|
||||
switch meta := body.ResultStart.Meta.(type) {
|
||||
case *pb.CallResultStart_Http:
|
||||
logrus.Debugf("Received meta http result from runner Status=%v", meta.Http.StatusCode)
|
||||
for _, header := range meta.Http.Headers {
|
||||
w.Header().Set(header.Key, header.Value)
|
||||
}
|
||||
if meta.Http.StatusCode > 0 {
|
||||
w.WriteHeader(int(meta.Http.StatusCode))
|
||||
}
|
||||
default:
|
||||
logrus.Errorf("Unhandled meta type in start message: %v", meta)
|
||||
}
|
||||
|
||||
// May arrive if function has output. We ignore EOF.
|
||||
case *pb.RunnerMsg_Data:
|
||||
w.Write(body.Data.Data)
|
||||
logrus.Debugf("Received data from runner len=%d isEOF=%v", len(body.Data.Data), body.Data.Eof)
|
||||
if !isPartialWrite {
|
||||
// WARNING: blocking write
|
||||
n, err := w.Write(body.Data.Data)
|
||||
if n != len(body.Data.Data) {
|
||||
isPartialWrite = true
|
||||
logrus.WithError(err).Infof("Failed to write full response (%d of %d) to client", n, len(body.Data.Data))
|
||||
if err == nil {
|
||||
err = io.ErrShortWrite
|
||||
}
|
||||
tryQueueError(err, done)
|
||||
}
|
||||
}
|
||||
|
||||
// Finish messages required for finish/finalize the processing.
|
||||
case *pb.RunnerMsg_Finished:
|
||||
if body.Finished.Success {
|
||||
logrus.Infof("Call finished successfully: %v", body.Finished.Details)
|
||||
} else {
|
||||
logrus.Infof("Call finished unsuccessfully: %v", body.Finished.Details)
|
||||
logrus.Infof("Call finished Success=%v %v", body.Finished.Success, body.Finished.Details)
|
||||
if !body.Finished.Success {
|
||||
err := parseError(body.Finished.GetDetails())
|
||||
tryQueueError(err, done)
|
||||
}
|
||||
// There should be an EOF following the last packet
|
||||
if _, err := protocolClient.Recv(); err != io.EOF {
|
||||
logrus.WithError(err).Error("Did not receive expected EOF from runner stream")
|
||||
done <- err
|
||||
}
|
||||
close(done)
|
||||
return
|
||||
break DataLoop
|
||||
|
||||
default:
|
||||
logrus.Errorf("Unhandled message type from runner: %v", body)
|
||||
logrus.Error("Ignoring unknown message type %T from runner, possible client/server mismatch", body)
|
||||
}
|
||||
}
|
||||
|
||||
// There should be an EOF following the last packet
|
||||
for {
|
||||
msg, err := protocolClient.Recv()
|
||||
if err == io.EOF {
|
||||
break
|
||||
}
|
||||
if err != nil {
|
||||
logrus.WithError(err).Infof("Call Waiting EOF received error")
|
||||
tryQueueError(err, done)
|
||||
break
|
||||
}
|
||||
|
||||
switch body := msg.Body.(type) {
|
||||
default:
|
||||
logrus.Infof("Call Waiting EOF ignoring message %T", body)
|
||||
}
|
||||
tryQueueError(ErrorPureRunnerNoEOF, done)
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user