Skip to content

Commit

Permalink
Reader Next returns on closed consumer
Browse files Browse the repository at this point in the history
Signed-off-by: Gaylor Bosson <gb@taurusgroup.ch>
  • Loading branch information
Gilthoniel committed May 15, 2024
1 parent 8e90873 commit bd5e740
Show file tree
Hide file tree
Showing 2 changed files with 26 additions and 0 deletions.
2 changes: 2 additions & 0 deletions pulsar/reader_impl.go
Original file line number Diff line number Diff line change
Expand Up @@ -171,6 +171,8 @@ func (r *reader) Next(ctx context.Context) (Message, error) {
return nil, err
}
return cm.Message, nil
case <-r.c.closeCh:
return nil, newError(ConsumerClosed, "consumer closed")
case <-ctx.Done():
return nil, ctx.Err()
}
Expand Down
24 changes: 24 additions & 0 deletions pulsar/reader_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1035,3 +1035,27 @@ func TestReaderHasNextRetryFailed(t *testing.T) {
}

}

func TestReaderNextReturnsOnClosedConsumer(t *testing.T) {
client, err := NewClient(ClientOptions{
URL: serviceURL,
OperationTimeout: 2 * time.Second,
})
assert.NoError(t, err)
topic := newTopicName()
reader, err := client.CreateReader(ReaderOptions{
Topic: topic,
StartMessageID: EarliestMessageID(),
})
assert.Nil(t, err)

reader.Close()

ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second)
defer cancel()

var e *Error
_, err = reader.Next(ctx)
assert.ErrorAs(t, err, &e)
assert.Equal(t, ConsumerClosed, e.Result())
}

0 comments on commit bd5e740

Please sign in to comment.