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

Do a final commit on end consumer group generation for immediate commits #715

Merged
merged 2 commits into from
Jan 21, 2022
Merged
Show file tree
Hide file tree
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
27 changes: 22 additions & 5 deletions reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -212,6 +212,25 @@ func (r *Reader) commitLoopImmediate(ctx context.Context, gen *Generation) {
for {
select {
case <-ctx.Done():
// drain the commit channel and prepare a single, final commit.
// the commit will combine any outstanding requests and the result
// will be sent back to all the callers of CommitMessages so that
// they can return.
var errchs []chan<- error
for hasCommits := true; hasCommits; {
select {
case req := <-r.commits:
offsets.merge(req.commits)
errchs = append(errchs, req.errch)
default:
hasCommits = false
}
}
err := r.commitOffsetsWithRetry(gen, offsets, defaultCommitRetries)
for _, errch := range errchs {
// NOTE : this will be a buffered channel and will not block.
errch <- err
}
return

case req := <-r.commits:
Expand Down Expand Up @@ -509,7 +528,6 @@ type ReaderConfig struct {

// Validate method validates ReaderConfig properties.
func (config *ReaderConfig) Validate() error {

if len(config.Brokers) == 0 {
return errors.New("cannot create a new kafka reader with an empty list of broker addresses")
}
Expand Down Expand Up @@ -854,7 +872,7 @@ func (r *Reader) CommitMessages(ctx context.Context, msgs ...Message) error {
}

var errch <-chan error
var creq = commitRequest{
creq := commitRequest{
commits: makeCommits(msgs...),
}

Expand Down Expand Up @@ -1323,7 +1341,6 @@ func (r *reader) run(ctx context.Context, offset int64) {

case OffsetOutOfRange:
first, last, err := r.readOffsets(conn)

if err != nil {
r.withErrorLogger(func(log Logger) {
log.Printf("the kafka reader got an error while attempting to determine whether it was reading before the first offset or after the last offset of partition %d of %s: %s", r.partition, r.topic, err)
Expand Down Expand Up @@ -1383,7 +1400,7 @@ func (r *reader) run(ctx context.Context, offset int64) {

func (r *reader) initialize(ctx context.Context, offset int64) (conn *Conn, start int64, err error) {
for i := 0; i != len(r.brokers) && conn == nil; i++ {
var broker = r.brokers[i]
broker := r.brokers[i]
var first, last int64

t0 := time.Now()
Expand Down Expand Up @@ -1532,7 +1549,7 @@ func (r *reader) withErrorLogger(do func(Logger)) {
// extractTopics returns the unique list of topics represented by the set of
// provided members
func extractTopics(members []GroupMember) []string {
var visited = map[string]struct{}{}
visited := map[string]struct{}{}
var topics []string

for _, member := range members {
Expand Down
50 changes: 48 additions & 2 deletions reader_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -436,7 +436,7 @@ func makeTestSequence(n int) []Message {
}

func prepareReader(t *testing.T, ctx context.Context, r *Reader, msgs ...Message) {
var config = r.Config()
config := r.Config()
var conn *Conn
var err error

Expand Down Expand Up @@ -637,7 +637,6 @@ func TestReaderPartitionWhenConsumerGroupsEnabled(t *testing.T) {
if !invoke() {
t.Fatalf("expected panic; but NewReader worked?!")
}

}

func TestExtractTopics(t *testing.T) {
Expand Down Expand Up @@ -1208,6 +1207,53 @@ func TestValidateReader(t *testing.T) {
}
}

func TestCommitLoopImmediateFlushOnGenerationEnd(t *testing.T) {
t.Parallel()
var committedOffset int64
var commitCount int
gen := &Generation{
conn: mockCoordinator{
offsetCommitFunc: func(r offsetCommitRequestV2) (offsetCommitResponseV2, error) {
commitCount++
committedOffset = r.Topics[0].Partitions[0].Offset
return offsetCommitResponseV2{}, nil
},
},
done: make(chan struct{}),
log: func(func(Logger)) {},
logError: func(func(Logger)) {},
}

// initialize commits so that the commitLoopImmediate select statement blocks
r := &Reader{stctx: context.Background(), commits: make(chan commitRequest, 100)}

for i := 0; i < 100; i++ {
cr := commitRequest{
commits: []commit{{
topic: "topic",
partition: 0,
offset: int64(i) + 1,
}},
errch: make(chan<- error, 1),
}
r.commits <- cr
}

gen.Start(func(ctx context.Context) {
r.commitLoopImmediate(ctx, gen)
})

gen.close()

if committedOffset != 100 {
t.Fatalf("expected commited offset to be 100 but got %d", committedOffset)
}

if commitCount >= 100 {
t.Fatalf("expected a single final commit on generation end got %d", commitCount)
}
}

func TestCommitOffsetsWithRetry(t *testing.T) {
offsets := offsetStash{"topic": {0: 0}}

Expand Down