Skip to content

Commit

Permalink
Added context wrapper option for processors (#420)
Browse files Browse the repository at this point in the history
* Added context wrapper option for processors
* remove explicit poll-timeout
frairon authored Apr 4, 2023

Verified

This commit was signed with the committer’s verified signature. The key has expired.
lilnasy Arsh
1 parent d17c843 commit 61c9842
Showing 15 changed files with 166 additions and 61 deletions.
16 changes: 16 additions & 0 deletions examples/docker-compose.yml
Original file line number Diff line number Diff line change
@@ -10,10 +10,18 @@ services:
ZOO_MY_ID: 1
ZOO_PORT: 2181
ZOO_SERVERS: server.1=zoo1:2888:3888
ulimits:
nofile:
soft: 65536
hard: 65536
kafka1:
image: confluentinc/cp-kafka:5.4.0
hostname: kafka1
container_name: kafka1
ulimits:
nofile:
soft: 65536
hard: 65536
ports:
- "9092:9092"
environment:
@@ -33,6 +41,10 @@ services:
image: confluentinc/cp-kafka:5.4.0
hostname: kafka2
container_name: kafka2
ulimits:
nofile:
soft: 65536
hard: 65536
ports:
- "9093:9093"
environment:
@@ -52,6 +64,10 @@ services:
image: confluentinc/cp-kafka:5.4.0
hostname: kafka3
container_name: kafka3
ulimits:
nofile:
soft: 65536
hard: 65536
ports:
- "9094:9094"
environment:
3 changes: 2 additions & 1 deletion graph.go
Original file line number Diff line number Diff line change
@@ -168,7 +168,8 @@ func (gg *GroupGraph) joint(topic string) bool {
// DefineGroup creates a group graph with a given group name and a list of
// edges.
func DefineGroup(group Group, edges ...Edge) *GroupGraph {
gg := GroupGraph{group: string(group),
gg := GroupGraph{
group: string(group),
codecs: make(map[string]Codec),
callbacks: make(map[string]ProcessCallback),
joinCheck: make(map[string]bool),
70 changes: 70 additions & 0 deletions integrationtest/processor_test.go
Original file line number Diff line number Diff line change
@@ -223,6 +223,76 @@ func TestProcessorVisit(t *testing.T) {
require.NoError(t, err)
}

type (
gokaCtx = goka.Context
wrapper struct {
gokaCtx
value int64
}
)

func (w *wrapper) SetValue(value interface{}, options ...goka.ContextOption) {
val := value.(int64)
w.value = val
w.gokaCtx.SetValue(val + 1)
}

func TestProcessorContextWrapper(t *testing.T) {
gkt := tester.New(t)

// holds the last wrapper
var w *wrapper

// create a new processor, registering the tester
proc, _ := goka.NewProcessor([]string{}, goka.DefineGroup("proc",
goka.Input("input", new(codec.Int64), func(ctx goka.Context, msg interface{}) {
ctx.SetValue(msg)
}),
goka.Visitor("visit", func(ctx goka.Context, msg interface{}) {
ctx.SetValue(msg.(int64))
}),
goka.Persist(new(codec.Int64)),
),
goka.WithTester(gkt),
goka.WithContextWrapper(func(ctx goka.Context) goka.Context {
w = &wrapper{
gokaCtx: ctx,
}
return w
}),
)

ctx, cancel := context.WithCancel(context.Background())
done := make(chan struct{})

// start it
go func() {
defer close(done)
err := proc.Run(ctx)
if err != nil {
t.Errorf("error running processor: %v", err)
}
}()

// send a message
gkt.Consume("input", "key", int64(23))

// both wrapper value and real value are set
require.EqualValues(t, 23, w.value)
require.EqualValues(t, 24, gkt.TableValue("proc-table", "key"))

// also the visitor should wrap the context
err := proc.VisitAll(ctx, "visit", int64(815))
require.NoError(t, err)

// both values are set again
require.EqualValues(t, 815, w.value)
require.EqualValues(t, 816, gkt.TableValue("proc-table", "key"))

cancel()
<-done
}

/*
import (
"context"
12 changes: 12 additions & 0 deletions options.go
Original file line number Diff line number Diff line change
@@ -129,6 +129,7 @@ type poptions struct {

updateCallback UpdateCallback
rebalanceCallback RebalanceCallback
contextWrapper ContextWrapper
partitionChannelSize int
hasher func() hash.Hash32
nilHandling NilHandling
@@ -147,6 +148,15 @@ type poptions struct {
}
}

// WithContextWrapper allows to intercept the context passed to each callback invocation.
// The wrapper function will be called concurrently across all partitions the returned context
// must not be shared.
func WithContextWrapper(wrapper ContextWrapper) ProcessorOption {
return func(o *poptions, gg *GroupGraph) {
o.contextWrapper = wrapper
}
}

// WithUpdateCallback defines the callback called upon recovering a message
// from the log.
func WithUpdateCallback(cb UpdateCallback) ProcessorOption {
@@ -329,6 +339,8 @@ func (opt *poptions) applyOptions(gg *GroupGraph, opts ...ProcessorOption) error
opt.log = defaultLogger
opt.hasher = DefaultHasher()
opt.backoffResetTime = defaultBackoffResetTime
// default context wrapper returns the original context
opt.contextWrapper = func(ctx Context) Context { return ctx }

for _, o := range opts {
o(opt, gg)
11 changes: 6 additions & 5 deletions partition_processor.go
Original file line number Diff line number Diff line change
@@ -47,6 +47,8 @@ type visit struct {

type commitCallback func(msg *message, meta string)

type ContextWrapper func(ctx Context) Context

// PartitionProcessor handles message processing of one partition by serializing
// messages from different input topics.
// It also handles joined tables as well as lookup views (managed by `Processor`).
@@ -595,7 +597,6 @@ func (pp *PartitionProcessor) processVisit(ctx context.Context, wg *sync.WaitGro
emitterDefaultHeaders: pp.opts.producerDefaultHeaders,
table: pp.table,
}

// start context and call the ProcessorCallback cb
msgContext.start()

@@ -610,8 +611,8 @@ func (pp *PartitionProcessor) processVisit(ctx context.Context, wg *sync.WaitGro
}
}()

// now call cb
cb(msgContext, v.meta)
// now call cb, wrap the context
cb(pp.opts.contextWrapper(msgContext), v.meta)
msgContext.finish(nil)
return
}
@@ -673,7 +674,7 @@ func (pp *PartitionProcessor) processMessage(ctx context.Context, wg *sync.WaitG
msgContext.start()

// now call cb
cb(msgContext, m)
cb(pp.opts.contextWrapper(msgContext), m)
msgContext.finish(nil)
return nil
}
@@ -706,7 +707,7 @@ func (pp *PartitionProcessor) VisitValues(ctx context.Context, name string, meta
}

defer it.Release()

stopping, doneWaitingForStop := pp.stopping()
defer doneWaitingForStop()

5 changes: 3 additions & 2 deletions processor.go
Original file line number Diff line number Diff line change
@@ -40,6 +40,7 @@ type Processor struct {
log logger
brokers []string

// hook used to be notified whenever the processor has rebalanced to a new assignment
rebalanceCallback RebalanceCallback

// rwmutex protecting read/write of partitions and lookuptables.
@@ -737,8 +738,8 @@ func (g *Processor) Cleanup(session sarama.ConsumerGroupSession) error {
// WaitForReady waits until the processor is ready to consume messages
// (or is actually consuming messages)
// i.e., it is done catching up all partition tables, joins and lookup tables
func (g *Processor) WaitForReady() {
g.waitForReady(context.Background())
func (g *Processor) WaitForReady() error {
return g.waitForReady(context.Background())
}

// WaitForReadyContext is context aware option of WaitForReady.
12 changes: 4 additions & 8 deletions systemtest/commit_test.go
Original file line number Diff line number Diff line change
@@ -12,10 +12,6 @@ import (
"github.com/stretchr/testify/require"
)

const (
pollWaitSecs = 15.0
)

// TestAutoCommit tests/demonstrates the behavior of disabling the auto-commit functionality.
// The autocommiter sends the offsets of the marked messages to the broker regularily. If the processor shuts down
// (or the group rebalances), the offsets are sent one last time, so just turning it of is not enough.
@@ -78,7 +74,7 @@ func TestAutoCommit(t *testing.T) {

// run the first processor
_, cancel, done := runProc(createProc())
pollTimed(t, "all-received1", pollWaitSecs, func() bool {
pollTimed(t, "all-received1", func() bool {
return len(offsets) == 10 && offsets[0] == 0
})

@@ -96,7 +92,7 @@ func TestAutoCommit(t *testing.T) {

// --> we'll receive all messages again
// --> i.e., no offsets were committed
pollTimed(t, "all-received2", pollWaitSecs, func() bool {
pollTimed(t, "all-received2", func() bool {
return len(offsets) == 10 && offsets[0] == 0
})

@@ -153,7 +149,7 @@ func TestUnmarkedMessages(t *testing.T) {

// run the first processor
runProc(createProc())
pollTimed(t, "all-received1", pollWaitSecs, func() bool {
pollTimed(t, "all-received1", func() bool {
return len(values) == 2 && values[0] == 1
})

@@ -162,7 +158,7 @@ func TestUnmarkedMessages(t *testing.T) {

// restart -> we'll only receive the second message
runProc(createProc())
pollTimed(t, "all-received2", pollWaitSecs, func() bool {
pollTimed(t, "all-received2", func() bool {
return len(values) == 1 && values[0] == 2
})
}
2 changes: 1 addition & 1 deletion systemtest/emitter_disconnect_test.go
Original file line number Diff line number Diff line change
@@ -78,7 +78,7 @@ func TestEmitter_KafkaDisconnect(t *testing.T) {
}
}()

pollTimed(t, "emitter emitted something successfully", 10, func() bool {
pollTimed(t, "emitter emitted something successfully", func() bool {
return atomic.LoadInt64(&success) > 0
})

4 changes: 2 additions & 2 deletions systemtest/multitopic_test.go
Original file line number Diff line number Diff line change
@@ -85,7 +85,7 @@ func TestMultiTopics(t *testing.T) {
})

log.Printf("waiting for processor/view to be running")
pollTimed(t, "proc and view are recovered", 10.0, proc.Recovered, view.Recovered)
pollTimed(t, "proc and view are recovered", proc.Recovered, view.Recovered)
log.Printf("...done")

var sum int64
@@ -110,7 +110,7 @@ func TestMultiTopics(t *testing.T) {
}

// poll the view and the processor until we're sure that we have
pollTimed(t, "all messages have been transferred", 10.0,
pollTimed(t, "all messages have been transferred",
func() bool {
value, err := view.Get("key")
require.NoError(t, err)
2 changes: 1 addition & 1 deletion systemtest/proc_disconnect_test.go
Original file line number Diff line number Diff line change
@@ -80,7 +80,7 @@ func TestProcessorShutdown_KafkaDisconnect(t *testing.T) {
errg.Go(func() error {
return proc.Run(ctx)
})
pollTimed(t, "proc running", 10, proc.Recovered, func() bool {
pollTimed(t, "proc running", proc.Recovered, func() bool {
if val, _ := proc.Get("key-15"); val != nil && val.(int64) > 0 {
return true
}
Loading

0 comments on commit 61c9842

Please sign in to comment.