MINOR: Improve exception messages in FileChannelRecordBatch#6068
Conversation
Improve message in `FileChannelRecordBatch.loadBatchWithSize`. The initial description was removed as part of 81f0c1e, reuse the description that was present before. Replace `channel` by `file` in KafkaException potentially thrown when loading/writing FileChannelRecordBatch. This makes exception messages more readable (channel only show an object hashcode, file shows the path of the file being read).
|
Failing tests pass locally |
hachikuji
left a comment
There was a problem hiding this comment.
Thanks for the patch. Left one suggestion for consideration.
| public abstract static class FileChannelRecordBatch extends AbstractRecordBatch { | ||
| protected final long offset; | ||
| protected final byte magic; | ||
| protected final File file; |
There was a problem hiding this comment.
If we need both the File and FileChannel, I wonder if we may as well pass a reference to FileRecords. We can get to the channel using FileRecords.channel() and we can rely on FileRecords.toString() for a nicer representation in logs and exception messages.
There was a problem hiding this comment.
Good point 🙂
I published an update to address this. I had to add implementations for FileRecords.{equals, hashCode} (they're just leveraging the internal channel, to mimic the previous behaviour). Let me know if you see any issue with this.
Also implement equals/hashCode for FileRecords to make FileLogInputStream.equals more readable. It should mimic the same behaviour as previously (equality of channel means equality of FileRecords).
|
The other failing test is also passing locally |
| } | ||
|
|
||
| @Override | ||
| public boolean equals(Object o) { |
There was a problem hiding this comment.
I wonder if we should include some additional fields. It would be a little odd if slices of different ranges are considered equal. Perhaps we can include start, end and isSlice? Admittedly, this would make less sense from the perspective of FileChannelRecordBatch if it relies on the implementation from FileRecords. Alternatively, perhaps we could have FileChannelRecordBatch use fileRecords.channel() in its equals and hashCode?
Sigh, if only FileChannel gave us a more useful toString 😞
There was a problem hiding this comment.
I had a go with the second choice, it does not look super good, but it should get the job done 😄
Alternately, FileChannelRecordBatch could expose a protected FileChannel channel() method.
The equals bit would look something like:
return ...
...
(channel() == null ? that.channel() == null : channel().equals(that.channel());No strong opinion between these options
hachikuji
left a comment
There was a problem hiding this comment.
LGTM. Thanks for the patch!
Replace `channel` by `fileRecords` in potentially thrown KafkaException descriptions when loading/writing `FileChannelRecordBatch`. This makes exception messages more readable (channel only shows an object hashcode, fileRecords shows the path of the file being read and start/end positions in the file). Reviewers: Jason Gustafson <jason@confluent.io>
Replace `channel` by `fileRecords` in potentially thrown KafkaException descriptions when loading/writing `FileChannelRecordBatch`. This makes exception messages more readable (channel only shows an object hashcode, fileRecords shows the path of the file being read and start/end positions in the file). Reviewers: Jason Gustafson <jason@confluent.io>
) Replace `channel` by `fileRecords` in potentially thrown KafkaException descriptions when loading/writing `FileChannelRecordBatch`. This makes exception messages more readable (channel only shows an object hashcode, fileRecords shows the path of the file being read and start/end positions in the file). Reviewers: Jason Gustafson <jason@confluent.io>
Improve exception message in
FileChannelRecordBatch.loadBatchWithSize.The original description was removed as part of 81f0c1e, reuse the
description that was present before.
Replace
channelbyfileRecordsin potentially thrown KafkaExceptiondescriptions when loading/writing
FileChannelRecordBatch. This makes exceptionmessages more readable (channel only shows an object hashcode, fileRecords shows
the path of the file being read and start/end positions in the file).
Context
If loading a Kafka segment throws an
IOException(fromFileChannel.readin
Utils.readFully), the exception is wrapped into aKafkaException.If this happens when loading/recovering a segment at start time, Kafka will shutdown
and the current exception message does not help to pinpoint the origin of the issue.
Example stacktrace: (from a broker running 1.1.0)
After applying the proposed changes:
Committer Checklist (excluded from commit message)