KAFKA-9206: throw a KafkaException when encountering CORRUPT_MESSAGE#8111
KAFKA-9206: throw a KafkaException when encountering CORRUPT_MESSAGE#8111hachikuji merged 7 commits intoapache:trunkfrom
Conversation
|
Note: I found |
There was a problem hiding this comment.
Why is this needed? Supposedly the slf4jlog4j dependency brings this too?
There was a problem hiding this comment.
I guess you're doing it because we refer to log4j at compile time now?
There was a problem hiding this comment.
yes, it's an import for LogCaptureAppender
There was a problem hiding this comment.
I think the main question here is whether we want to raise this error to the user. If we happen to see message corruption in the fetched data, we raise that to the user, so should this case be different?
There was a problem hiding this comment.
Ack. As discussed, will (1) change this to correspond to the current behavior for corruption in fetched data, i.e. throwing a KafkaException that wraps around a CorruptRecordException, and (2) remove the log message expectation.
There was a problem hiding this comment.
Correction: just a KafkaException (the wrapping around I referred to was the text of the CorruptRecordException, not the exception itself)
|
Updated, please take a look. |
soondenana
left a comment
There was a problem hiding this comment.
i.e. throwing a KafkaException that wraps around a CorruptRecordException
Are you planing to have a CorruptRecordException exception too?
| // Trigger the exception. | ||
| assertThrows(KafkaException.class, () -> { | ||
| Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> partitionRecords = | ||
| fetchedRecords(); |
There was a problem hiding this comment.
nit: just calling fetchedRecords() should work here, or is compiler not liking it and forcing the return value assignment?
There was a problem hiding this comment.
Agree, made the change to discard the return value.
No, clarified with a comment above too, I initially thought we were wrapping the exception in the "data corruption" case, but we're just wrapping its text. |
| } else if (error == Errors.UNKNOWN_SERVER_ERROR) { | ||
| log.warn("Unknown error fetching data for topic-partition {}", tp); | ||
| } else if (error == Errors.CORRUPT_MESSAGE) { | ||
| throw new KafkaException("Encountered corrupt message when fetching offset " |
There was a problem hiding this comment.
While we're at it, could we add the fetch offset to the IllegalStateException message below and the UNKNOWN_SERVER_ERROR log message above?
|
ok to test |
|
Test failure seems unrelated to this change: (will rerun tests once) |
|
retest this please |
|
The test failure is known flaky. I have a fix in #8101. |
| log.debug("Received unknown leader epoch error in fetch for partition {}", tp); | ||
| } else if (error == Errors.UNKNOWN_SERVER_ERROR) { | ||
| log.warn("Unknown error fetching data for topic-partition {}", tp); | ||
| log.warn("Unknown error fetching data while fetching offset " |
There was a problem hiding this comment.
nit: for log messages, can we use the {} placeholders?
|
retest this please |
|
ok to test |
|
Couple of tests that failed in this run but not the previous run, likely flaky:
|
hachikuji
left a comment
There was a problem hiding this comment.
LGTM. Thanks for the PR!
|
@agam One other small thing before we merge. Can you add CORRUPT_MESSAGE to the list of error codes in the javadoc for |
|
retest this please |
|
ok to test |
|
retest this please |
|
Two test failures:
|
* apache-github/trunk: (23 commits) KAFKA-9530; Fix flaky test `testDescribeGroupWithShortInitializationTimeout` (apache#8154) HOTFIX: fix NPE in Kafka Streams IQ (apache#8158) MINOR: set scala version automatically based on gradle.properties KAFKA-9577; SaslClientAuthenticator incorrectly negotiates SASL_HANDSHAKE version (apache#8142) KAFKA-9441: Add internal TransactionManager (apache#8105) MINOR: Document endpoints for connector topic tracking (KIP-558) MINOR: Standby task commit needed when offsets updated (apache#8146) KAFKA-9206; Throw KafkaException on CORRUPT_MESSAGE error in Fetch response (apache#8111) MINOR: Remove unwanted regexReplace on tests/kafkatest/__init__.py KAFKA-9586: Fix errored json filename in ops documentation KAFKA-9575: Mention ZooKeeper 3.5.7 upgrade KAFKA-9481: Graceful handling TaskMigrated and TaskCorrupted (apache#8058) HOTFIX: don't try to remove uninitialized changelogs from assignment & don't prematurely mark task closed (apache#8140) MINOR: Fix javadoc at org.apache.kafka.clients.producer.KafkaProducer.InterceptorCallback#onCompletion (apache#7337) MINOR: Improve EOS example exception handling (apache#8052) MINOR: Fix a number of warnings in clients test (apache#8073) MINOR: Update shell scripts to support z/OS system (apache#7913) MINOR: Wording fix in Streams DSL docs (apache#5692) MINOR: Add missing @test annotation to MetadataTest#testMetadataMerge (apache#8141) KAFKA-9533: ValueTransform forwards `null` values (apache#8108) ...
…etrics-common * confluent/master: (76 commits) KAFKA-9530; Fix flaky test `testDescribeGroupWithShortInitializationTimeout` (apache#8154) HOTFIX: fix NPE in Kafka Streams IQ (apache#8158) MINOR: set scala version automatically based on gradle.properties KAFKA-9577; SaslClientAuthenticator incorrectly negotiates SASL_HANDSHAKE version (apache#8142) KAFKA-9441: Add internal TransactionManager (apache#8105) MINOR: Document endpoints for connector topic tracking (KIP-558) MINOR: Standby task commit needed when offsets updated (apache#8146) Changes to migrate to Artifactory (#263) KAFKA-9206; Throw KafkaException on CORRUPT_MESSAGE error in Fetch response (apache#8111) MINOR: Remove unwanted regexReplace on tests/kafkatest/__init__.py KAFKA-9586: Fix errored json filename in ops documentation KAFKA-9575: Mention ZooKeeper 3.5.7 upgrade KAFKA-9481: Graceful handling TaskMigrated and TaskCorrupted (apache#8058) HOTFIX: don't try to remove uninitialized changelogs from assignment & don't prematurely mark task closed (apache#8140) MINOR: Fix javadoc at org.apache.kafka.clients.producer.KafkaProducer.InterceptorCallback#onCompletion (apache#7337) MINOR: Improve EOS example exception handling (apache#8052) MINOR: Fix a number of warnings in clients test (apache#8073) MINOR: Update shell scripts to support z/OS system (apache#7913) MINOR: Wording fix in Streams DSL docs (apache#5692) MINOR: Add missing @test annotation to MetadataTest#testMetadataMerge (apache#8141) ...
If the completed fetch has an error code signifying a corrupt message, throw a
KafkaExceptionthat notes the offset and the topic-partition.Also added a test that triggers the warning and verifies it is thrown.
Jira: https://issues.apache.org/jira/browse/KAFKA-9206
Committer Checklist (excluded from commit message)