Skip to content

Commit

Permalink
Allow request and init hooks to update the context.
Browse files Browse the repository at this point in the history
It's useful for a request hook to enrich the context with information,
so it can pass values to downstream processing. The hook should return the
originally provided context if no modifications are made. Likewise,
init hooks can establish the base context used by the harness.

Fix the aggregated context to actually be returned.

Added context propagation to init.

Added tests for both modes.
  • Loading branch information
wcn3 committed Apr 24, 2018
1 parent e981b43 commit adb78f6
Show file tree
Hide file tree
Showing 6 changed files with 95 additions and 24 deletions.
8 changes: 4 additions & 4 deletions sdks/go/pkg/beam/core/runtime/harness/harness.go
Original file line number Diff line number Diff line change
Expand Up @@ -105,11 +105,11 @@ func Main(ctx context.Context, loggingEndpoint, controlEndpoint string) error {

// Launch a goroutine to handle the control message.
// TODO(wcn): implement a rate limiter for 'heavy' messages?
fn := func() {
fn := func(ctx context.Context, req *fnpb.InstructionRequest) {
log.Debugf(ctx, "RECV: %v", proto.MarshalTextString(req))
recordInstructionRequest(req)

hooks.RunRequestHooks(ctx, req)
ctx = hooks.RunRequestHooks(ctx, req)
resp := ctrl.handleInstruction(ctx, req)

hooks.RunResponseHooks(ctx, req, resp)
Expand All @@ -123,9 +123,9 @@ func Main(ctx context.Context, loggingEndpoint, controlEndpoint string) error {
if req.GetProcessBundle() != nil {
// Only process bundles in a goroutine. We at least need to process instructions for
// each plan serially. Perhaps just invoke plan.Execute async?
go fn()
go fn(ctx, req)
} else {
fn()
fn(ctx, req)
}
}
}
Expand Down
4 changes: 2 additions & 2 deletions sdks/go/pkg/beam/core/runtime/harness/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -203,12 +203,12 @@ var captureHookRegistry = make(map[string]CaptureHookFactory)
func init() {
hf := func(opts []string) hooks.Hook {
return hooks.Hook{
Init: func(_ context.Context) error {
Init: func(ctx context.Context) (context.Context, error) {
if len(opts) > 0 {
name, opts := hooks.Decode(opts[0])
capture = captureHookRegistry[name](opts)
}
return nil
return ctx, nil
},
}
}
Expand Down
23 changes: 14 additions & 9 deletions sdks/go/pkg/beam/core/util/hooks/hooks.go
Original file line number Diff line number Diff line change
Expand Up @@ -59,7 +59,7 @@ type Hook struct {

// InitHook is a hook that is called when the harness
// initializes.
type InitHook func(context.Context) error
type InitHook func(context.Context) (context.Context, error)

// HookFactory is a function that produces a Hook from the supplied arguments.
type HookFactory func([]string) Hook
Expand All @@ -71,34 +71,39 @@ func RegisterHook(name string, h HookFactory) {
}

// RunInitHooks runs the init hooks.
func RunInitHooks(ctx context.Context) error {
func RunInitHooks(ctx context.Context) (context.Context, error) {
// If an init hook fails to complete, the invariants of the
// system are compromised and we can't run a workflow.
// The hooks can run in any order. They should not be
// interdependent or interfere with each other.
for _, h := range activeHooks {
if h.Init != nil {
if err := h.Init(ctx); err != nil {
return err
var err error
if ctx, err = h.Init(ctx); err != nil {
return ctx, err
}
}
}
return nil
return ctx, nil
}

// RequestHook is called when handling a FnAPI instruction.
type RequestHook func(context.Context, *fnpb.InstructionRequest) error
// RequestHook is called when handling a FnAPI instruction. It can return an updated
// context to pass additional information to downstream callers, or return the
// original context provided.
type RequestHook func(context.Context, *fnpb.InstructionRequest) (context.Context, error)

// RunRequestHooks runs the hooks that handle a FnAPI request.
func RunRequestHooks(ctx context.Context, req *fnpb.InstructionRequest) {
func RunRequestHooks(ctx context.Context, req *fnpb.InstructionRequest) context.Context {
// The request hooks should not modify the request.
for n, h := range activeHooks {
if h.Req != nil {
if err := h.Req(ctx, req); err != nil {
var err error
if ctx, err = h.Req(ctx, req); err != nil {
log.Infof(ctx, "request hook %s failed: %v", n, err)
}
}
}
return ctx
}

// ResponseHook is called when sending a FnAPI instruction response.
Expand Down
65 changes: 65 additions & 0 deletions sdks/go/pkg/beam/core/util/hooks/hooks_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,65 @@
// Licensed to the Apache Software Foundation (ASF) under one or more
// contributor license agreements. See the NOTICE file distributed with
// this work for additional information regarding copyright ownership.
// The ASF licenses this file to You under the Apache License, Version 2.0
// (the "License"); you may not use this file except in compliance with
// the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

package hooks

import (
"context"
"fmt"
"testing"

fnpb "github.com/apache/beam/sdks/go/pkg/beam/model/fnexecution_v1"
)

type contextKey string

func initializeHooks() {
activeHooks["test"] = Hook{
Init: func(ctx context.Context) (context.Context, error) {
return context.WithValue(ctx, contextKey("init_key"), "value"), nil
},
Req: func(ctx context.Context, req *fnpb.InstructionRequest) (context.Context, error) {
return context.WithValue(ctx, contextKey("req_key"), "value"), nil
},
}
}

func TestInitContextPropagation(t *testing.T) {
initializeHooks()
ctx := context.Background()
var err error

expected := `context.Background.WithValue("init_key", "value")`
ctx, err = RunInitHooks(ctx)
if err != nil {
t.Errorf("got %v error, wanted no error", err)
}
actual := ctx.(fmt.Stringer).String()
if actual != expected {
t.Errorf("Got %s, wanted %s", actual, expected)
}
}

func TestRequestContextPropagation(t *testing.T) {
initializeHooks()
ctx := context.Background()

expected := `context.Background.WithValue("req_key", "value")`
ctx = RunRequestHooks(ctx, nil)
actual := ctx.(fmt.Stringer).String()
if actual != expected {
t.Errorf("Got %s, wanted %s", actual, expected)
}
}
7 changes: 4 additions & 3 deletions sdks/go/pkg/beam/util/grpcx/hook.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@ type Hook struct {
// TODO(wcn): expose other hooks here.
}

// HookFactory is a function that creates hooks from supplied arguments.
type HookFactory func([]string) Hook

var hookRegistry = make(map[string]HookFactory)
Expand All @@ -49,17 +50,17 @@ func RegisterHook(name string, c HookFactory) {

hf := func(opts []string) hooks.Hook {
return hooks.Hook{
Init: func(_ context.Context) error {
Init: func(ctx context.Context) (context.Context, error) {
if len(opts) == 0 {
return nil
return ctx, nil
}

name, opts := hooks.Decode(opts[0])
grpcHook := hookRegistry[name](opts)
if grpcHook.Dialer != nil {
Dial = grpcHook.Dialer
}
return nil
return ctx, nil
},
}
}
Expand Down
12 changes: 6 additions & 6 deletions sdks/go/pkg/beam/x/hooks/perf/perf.go
Original file line number Diff line number Diff line change
Expand Up @@ -49,12 +49,12 @@ func init() {
enabled := len(enabledProfCaptureHooks) > 0
var cpuProfBuf bytes.Buffer
return hooks.Hook{
Req: func(_ context.Context, _ *fnpb.InstructionRequest) error {
Req: func(ctx context.Context, _ *fnpb.InstructionRequest) (context.Context, error) {
if !enabled {
return nil
return ctx, nil
}
cpuProfBuf.Reset()
return pprof.StartCPUProfile(&cpuProfBuf)
return ctx, pprof.StartCPUProfile(&cpuProfBuf)
},
Resp: func(ctx context.Context, req *fnpb.InstructionRequest, _ *fnpb.InstructionResponse) error {
if !enabled {
Expand All @@ -78,12 +78,12 @@ func init() {
enabledTraceCaptureHooks = opts
enabled := len(enabledTraceCaptureHooks) > 0
return hooks.Hook{
Req: func(_ context.Context, _ *fnpb.InstructionRequest) error {
Req: func(ctx context.Context, _ *fnpb.InstructionRequest) (context.Context, error) {
if !enabled {
return nil
return ctx, nil
}
traceProfBuf.Reset()
return trace.Start(&traceProfBuf)
return ctx, trace.Start(&traceProfBuf)
},
Resp: func(ctx context.Context, req *fnpb.InstructionRequest, _ *fnpb.InstructionResponse) error {
if !enabled {
Expand Down

0 comments on commit adb78f6

Please sign in to comment.