Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

processor: add WaitForReadyContext method #393

Merged
Merged
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
28 changes: 24 additions & 4 deletions processor.go
Original file line number Diff line number Diff line change
Expand Up @@ -734,22 +734,38 @@ func (g *Processor) Cleanup(session sarama.ConsumerGroupSession) error {
return err
}

// WaitForReady waits until the processor is ready to consume messages (or is actually consuming messages)
// 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())
}

// WaitForReadyContext is context aware option of WaitForReady.
// It either waits until the processor is ready or until context is canceled.
// If the return was caused by context it will return context reported error.
func (g *Processor) WaitForReadyContext(ctx context.Context) error {
return g.waitForReady(ctx)
}

func (g *Processor) waitForReady(ctx context.Context) error {
// wait for the processor to be started (or stopped)
select {
case <-g.state.WaitForStateMin(ProcStateStarting):
case <-g.done:
return
return nil
case <-ctx.Done():
return ctx.Err()
}
// wait that the processor is actually running
select {
case <-g.state.WaitForState(ProcStateRunning):
case <-g.done:
case <-ctx.Done():
return ctx.Err()
}

// wait for all partitionprocessors to be running
// wait for all PartitionProcessors to be running

// copy them first with the mutex so we don't run into a deadlock

Expand All @@ -764,8 +780,12 @@ func (g *Processor) WaitForReady() {
select {
case <-part.state.WaitForState(PPStateRunning):
case <-g.done:
case <-ctx.Done():
return ctx.Err()
}
}

return nil
}

// ConsumeClaim must start a consumer loop of ConsumerGroupClaim's Messages().
Expand Down Expand Up @@ -1015,7 +1035,7 @@ func ensureCopartitioned(tm TopicManager, topics []string) (int, error) {
// Note that the offset to be committed must be the offset that the consumer expects to consume next, not the offset of the message.
// See documentation at https://kafka.apache.org/25/javadoc/org/apache/kafka/clients/consumer/KafkaConsumer.html which says:
//
// Note: The committed offset should always be the offset of the next message that your application will read. Thus, when calling commitSync(offsets) you should add one to the offset of the last message processed.
// Note: The committed offset should always be the offset of the next message that your application will read. Thus, when calling commitSync(offsets) you should add one to the offset of the last message processed.
//
// This has the same semantics as sarama's implementation of session.MarkMessage (which calls MarkOffset with offset+1)
func createMessageCommitter(session sarama.ConsumerGroupSession) commitCallback {
Expand Down