Skip to content

MINOR: Improve exception messages in FileChannelRecordBatch#6068

Merged
hachikuji merged 5 commits intoapache:trunkfrom
flavray:improve-file-log-input-stream-description
Dec 28, 2018
Merged

MINOR: Improve exception messages in FileChannelRecordBatch#6068
hachikuji merged 5 commits intoapache:trunkfrom
flavray:improve-file-log-input-stream-description

Conversation

@flavray
Copy link
Copy Markdown
Contributor

@flavray flavray commented Dec 26, 2018

Improve exception message in FileChannelRecordBatch.loadBatchWithSize.
The original description was removed as part of 81f0c1e, reuse the
description that was present before.

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).

Context

If loading a Kafka segment throws an IOException (from FileChannel.read
in Utils.readFully), the exception is wrapped into a KafkaException.
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)

[2018-12-24 23:17:44,727] ERROR {main} There was an error in one of the threads during logs loading: org.apache.kafka.common.KafkaException: java.io.IOException: Input/output error (kafka.log.LogManager)
[2018-12-24 23:17:44,732] ERROR {main} [KafkaServer id=172329974] Fatal error during KafkaServer startup. Prepare to shutdown (kafka.server.KafkaServer)
org.apache.kafka.common.KafkaException: java.io.IOException: Input/output error
        at org.apache.kafka.common.record.FileLogInputStream$FileChannelRecordBatch.loadBatchWithSize(FileLogInputStream.java:214)
        ...
        at java.lang.Thread.run(Thread.java:745)
Caused by: java.io.IOException: Input/output error
        at sun.nio.ch.FileDispatcherImpl.pread0(Native Method)
        at sun.nio.ch.FileDispatcherImpl.pread(FileDispatcherImpl.java:52)
        at sun.nio.ch.IOUtil.readIntoNativeBuffer(IOUtil.java:220)
        at sun.nio.ch.IOUtil.read(IOUtil.java:197)
        at sun.nio.ch.FileChannelImpl.readInternal(FileChannelImpl.java:741)
        at sun.nio.ch.FileChannelImpl.read(FileChannelImpl.java:727)
        at org.apache.kafka.common.utils.Utils.readFully(Utils.java:831)
        at org.apache.kafka.common.utils.Utils.readFullyOrFail(Utils.java:804)
        at org.apache.kafka.common.record.FileLogInputStream$FileChannelRecordBatch.loadBatchWithSize(FileLogInputStream.java:210)
        ... 28 more

After applying the proposed changes:

org.apache.kafka.common.KafkaException: Failed to load record batch at position 12868 from FileRecords(file= /usr/local/var/lib/kafka-logs/__consumer_offsets-9/00000000000000000000.log, start=0, end=2147483647)
	at org.apache.kafka.common.record.FileLogInputStream$FileChannelRecordBatch.loadBatchWithSize(FileLogInputStream.java:218)
	...
	at java.lang.Thread.run(Thread.java:748)
Caused by: java.io.IOException: Input/output error
        at sun.nio.ch.FileDispatcherImpl.pread0(Native Method)
        at sun.nio.ch.FileDispatcherImpl.pread(FileDispatcherImpl.java:52)
        at sun.nio.ch.IOUtil.readIntoNativeBuffer(IOUtil.java:220)
        at sun.nio.ch.IOUtil.read(IOUtil.java:197)
        at sun.nio.ch.FileChannelImpl.readInternal(FileChannelImpl.java:741)
        at sun.nio.ch.FileChannelImpl.read(FileChannelImpl.java:727)
        at org.apache.kafka.common.utils.Utils.readFully(Utils.java:831)
        at org.apache.kafka.common.utils.Utils.readFullyOrFail(Utils.java:804)
        at org.apache.kafka.common.record.FileLogInputStream$FileChannelRecordBatch.loadBatchWithSize(FileLogInputStream.java:213)

Committer Checklist (excluded from commit message)

  • Verify design and implementation
  • Verify test coverage and CI build status
  • Verify documentation (including upgrade notes)

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).
@flavray
Copy link
Copy Markdown
Contributor Author

flavray commented Dec 27, 2018

Failing tests pass locally

>> ./gradlew connect:runtime:test --tests org.apache.kafka.connect.runtime.rest.RestServerTest.testCORSEnabled

> Configure project :
Building project 'core' with Scala version 2.12.7
Building project 'streams-scala' with Scala version 2.12.7

> Task :connect:runtime:test

org.apache.kafka.connect.runtime.rest.RestServerTest > testCORSEnabled PASSED

Deprecated Gradle features were used in this build, making it incompatible with Gradle 6.0.
Use '--warning-mode all' to show the individual deprecation warnings.
See https://docs.gradle.org/5.0/userguide/command_line_interface.html#sec:command_line_warnings

BUILD SUCCESSFUL in 37s
27 actionable tasks: 2 executed, 25 up-to-date
>> ./gradlew clients:test --tests org.apache.kafka.clients.admin.KafkaAdminClientTest.testCreateTopicsRetryBackoff

> Configure project :
Building project 'core' with Scala version 2.12.7
Building project 'streams-scala' with Scala version 2.12.7

> Task :clients:test

org.apache.kafka.clients.admin.KafkaAdminClientTest > testCreateTopicsRetryBackoff PASSED

Deprecated Gradle features were used in this build, making it incompatible with Gradle 6.0.
Use '--warning-mode all' to show the individual deprecation warnings.
See https://docs.gradle.org/5.0/userguide/command_line_interface.html#sec:command_line_warnings

BUILD SUCCESSFUL in 7s
8 actionable tasks: 1 executed, 7 up-to-date

Copy link
Copy Markdown
Contributor

@hachikuji hachikuji left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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;
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

Flavien Raynaud added 2 commits December 28, 2018 13:19
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).
@flavray
Copy link
Copy Markdown
Contributor Author

flavray commented Dec 28, 2018

The other failing test is also passing locally

>> ./gradlew clients:test --tests org.apache.kafka.clients.admin.KafkaAdminClientTest.testUnreachableBootstrapServer

> Configure project :
Building project 'core' with Scala version 2.12.7
Building project 'streams-scala' with Scala version 2.12.7

> Task :clients:test

org.apache.kafka.clients.admin.KafkaAdminClientTest > testUnreachableBootstrapServer PASSED

Deprecated Gradle features were used in this build, making it incompatible with Gradle 6.0.
Use '--warning-mode all' to show the individual deprecation warnings.
See https://docs.gradle.org/5.0/userguide/command_line_interface.html#sec:command_line_warnings

BUILD SUCCESSFUL in 7s
8 actionable tasks: 1 executed, 7 up-to-date

}

@Override
public boolean equals(Object o) {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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 😞

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

Copy link
Copy Markdown
Contributor

@hachikuji hachikuji left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM. Thanks for the patch!

@hachikuji hachikuji merged commit 9295444 into apache:trunk Dec 28, 2018
hachikuji pushed a commit that referenced this pull request Dec 28, 2018
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>
hachikuji pushed a commit that referenced this pull request Dec 28, 2018
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>
pengxiaolong pushed a commit to pengxiaolong/kafka that referenced this pull request Jun 14, 2019
)

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>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

2 participants