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

a lot of unexpected EOF error when reading from partition #1286

Open
acynothia opened this issue Apr 30, 2024 · 2 comments · May be fixed by #1287
Open

a lot of unexpected EOF error when reading from partition #1286

acynothia opened this issue Apr 30, 2024 · 2 comments · May be fixed by #1287
Labels

Comments

@acynothia
Copy link

acynothia commented Apr 30, 2024

Describe the bug

A clear and concise description of what the bug is.

a lot of unexpected EOF error when reading from partition cause here.

kafka-go/reader.go

Lines 1421 to 1433 in 2af3101

default:
var kafkaError Error
if errors.As(err, &kafkaError) {
r.sendError(ctx, err)
} else {
r.withErrorLogger(func(log Logger) {
log.Printf("the kafka reader got an unknown error reading partition %d of %s at offset %d: %s", r.partition, r.topic, toHumanOffset(offset), err)
})
r.stats.errors.observe(1)
conn.Close()
break readLoop
}
}

I was try to log the detail found that when highWaterMark + 1 == offset and remain is 0. newMessageSetReader will return io.EOF(because remain is 0, read head first offset with io.EOF error)

kafka-go/conn.go

Lines 859 to 866 in 2af3101

var msgs *messageSetReader
if err == nil {
if highWaterMark == offset {
msgs = &messageSetReader{empty: true}
} else {
msgs, err = newMessageSetReader(&c.rbuf, remain)
}
}

It's cause a lot of error log and conn.Close a lot of recreate connection and TIME_WAIT.

Kafka Version

  • What version(s) of Kafka are you testing against?
    v2.2.0
  • What version of kafka-go are you using?
    v0.4.47

To Reproduce

Resources to reproduce the behavior:

---
# docker-compose.yaml
#
# Adding a docker-compose file will help the maintainers setup the environment
# to reproduce the issue.
#
# If one the docker-compose files available in the repository may be used,
# mentioning it is also a useful alternative.
...
package main

import (
    "github.com/segmentio/kafka-go"
)

func main() {
    // Adding a fully reproducible example will help maintainers provide
    // assistance to debug the issues.
    ...
}

Expected Behavior

A clear and concise description of what you expected to happen.

Don't return error when highWaterMark + 1 == offset and remain is 0.

Shall we just add remain == 0 condition ?

if highWaterMark == offset {

  	if highWaterMark == offset || remain == 0 { 

IMO, when remain is 0 return empty is reasonable.

Observed Behavior

A clear and concise description of the behavior you observed.

Often times, pasting the logging output from a kafka.Reader or kafka.Writer will
provide useful details to help maintainers investigate the issue and provide a
fix. If possible, providing stack traces or CPU/memory profiles may also contain
valuable information to understand the conditions that triggered the issue.

Additional Context

Add any other context about the problem here.

@acynothia acynothia added the bug label Apr 30, 2024
acynothia added a commit to acynothia/kafka-go that referenced this issue May 2, 2024
@taguhoiya
Copy link

taguhoiya commented May 16, 2024

Hi! i'm also using segmentio/kafka-go and now I've been also facing this issue now as well.
I can't figure out the cause here...

can you help us out?
@achille-roussel @rhansen2

@acynothia
Copy link
Author

I have already product with the PR#1287 over 2 weeks and consumes over 100 billion messges. It's seems working well.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
Projects
None yet
Development

Successfully merging a pull request may close this issue.

2 participants