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

Fix: Closing kafka Writer during WriteMessages causes a potential hang #1307 #1312

Open
wants to merge 1 commit into
base: main
Choose a base branch
from
Open
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
13 changes: 10 additions & 3 deletions writer.go
Original file line number Diff line number Diff line change
Expand Up @@ -663,7 +663,10 @@ func (w *Writer) WriteMessages(ctx context.Context, msgs ...Message) error {
assignments[key] = append(assignments[key], int32(i))
}

batches := w.batchMessages(msgs, assignments)
batches, err := w.batchMessages(msgs, assignments)
if err != nil {
return err
}
if w.Async {
return nil
}
Expand Down Expand Up @@ -695,7 +698,7 @@ func (w *Writer) WriteMessages(ctx context.Context, msgs ...Message) error {
return werr
}

func (w *Writer) batchMessages(messages []Message, assignments map[topicPartition][]int32) map[*writeBatch][]int32 {
func (w *Writer) batchMessages(messages []Message, assignments map[topicPartition][]int32) (map[*writeBatch][]int32, error) {
var batches map[*writeBatch][]int32
if !w.Async {
batches = make(map[*writeBatch][]int32, len(assignments))
Expand All @@ -704,6 +707,10 @@ func (w *Writer) batchMessages(messages []Message, assignments map[topicPartitio
w.mutex.Lock()
defer w.mutex.Unlock()

if w.closed {
return nil, io.ErrClosedPipe
}

if w.writers == nil {
w.writers = map[topicPartition]*partitionWriter{}
}
Expand All @@ -721,7 +728,7 @@ func (w *Writer) batchMessages(messages []Message, assignments map[topicPartitio
}
}

return batches
return batches, nil
}

func (w *Writer) produce(key topicPartition, batch *writeBatch) (*ProduceResponse, error) {
Expand Down
44 changes: 44 additions & 0 deletions writer_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -191,6 +191,10 @@ func TestWriter(t *testing.T) {
scenario: "test write message with writer data",
function: testWriteMessageWithWriterData,
},
{
scenario: "test no new partition writers after close",
function: testWriterNoNewPartitionWritersAfterClose,
},
}

for _, test := range tests {
Expand Down Expand Up @@ -1030,6 +1034,46 @@ func testWriterOverrideConfigStats(t *testing.T) {
}
}

func testWriterNoNewPartitionWritersAfterClose(t *testing.T) {
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
defer cancel()
topic1 := makeTopic()
createTopic(t, topic1, 1)
defer deleteTopic(t, topic1)

w := newTestWriter(WriterConfig{
Topic: topic1,
})
defer w.Close() // try and close anyway after test finished

// using balancer to close writer right between first mutex is released and second mutex is taken to make map of partition writers
w.Balancer = mockBalancerFunc(func(m Message, i ...int) int {
go w.Close() // close is blocking so run in goroutine
for { // wait until writer is marked as closed
w.mutex.Lock()
if w.closed {
w.mutex.Unlock()
break
}
w.mutex.Unlock()
}
return 0
})

msg := Message{Value: []byte("Hello World")} // no topic

if err := w.WriteMessages(ctx, msg); !errors.Is(err, io.ErrClosedPipe) {
t.Errorf("expected error: %v got: %v", io.ErrClosedPipe, err)
return
}
}

type mockBalancerFunc func(msg Message, partitions ...int) (partition int)

func (b mockBalancerFunc) Balance(msg Message, partitions ...int) int {
return b(msg, partitions...)
}

type staticBalancer struct {
partition int
}
Expand Down