Skip to content

KAFKA-14104: Add CRC validation when iterating over Metadata Log Records#12457

Merged
hachikuji merged 4 commits intoapache:trunkfrom
niket-goel:kafka-14104-crc-checks
Aug 8, 2022
Merged

KAFKA-14104: Add CRC validation when iterating over Metadata Log Records#12457
hachikuji merged 4 commits intoapache:trunkfrom
niket-goel:kafka-14104-crc-checks

Conversation

@niket-goel
Copy link
Copy Markdown
Contributor

@niket-goel niket-goel commented Jul 28, 2022

This commit adds a check to ensure the RecordBatch CRC is valid when
iterating over a Batch of Records using the RecordsIterator. The
RecordsIterator is used by both Snapshot reads and Log Records reads in
Kraft. The check can be turned off by a class parameter and is on by default.

Copy link
Copy Markdown
Member

@jsancio jsancio left a comment

Choose a reason for hiding this comment

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

RecordsIterator is used to read cluster metadata log and snapshot.

Comment thread core/src/main/scala/kafka/log/UnifiedLog.scala Outdated
Comment thread raft/src/main/java/org/apache/kafka/snapshot/RecordsSnapshotReader.java Outdated
This commit adds a check to ensure the RecordBatch CRC is valid when
iterating over a Batch of Records using the RecordsIterator. The
RecordsIterator is used by both Snapshot reads and Log Records reads in
Kraft. The check can be turned off by a class parameter and is on by default.
@niket-goel niket-goel force-pushed the kafka-14104-crc-checks branch from 8386642 to 178db2d Compare August 3, 2022 00:41
@niket-goel niket-goel closed this Aug 3, 2022
@niket-goel niket-goel reopened this Aug 3, 2022
@niket-goel niket-goel changed the title WIP -- added some CRC checking KAFKA-14104: Add CRC validation when iterating over Metadata Log Records Aug 3, 2022
@niket-goel niket-goel marked this pull request as ready for review August 3, 2022 00:45
@niket-goel
Copy link
Copy Markdown
Contributor Author

niket-goel commented Aug 3, 2022

Failing Tests:

Build / JDK 17 and Scala 2.13 / testLargeAssignmentAndGroupWithNonEqualSubscription() – org.apache.kafka.clients.consumer.CooperativeStickyAssignorTest1m 11s
Build / JDK 17 and Scala 2.13 / testLargeAssignmentAndGroupWithNonEqualSubscription() – org.apache.kafka.clients.consumer.StickyAssignorTest1m 13s
Build / JDK 11 and Scala 2.13 / testLargeAssignmentAndGroupWithNonEqualSubscription() – org.apache.kafka.clients.consumer.CooperativeStickyAssignorTest1m 29s
Build / JDK 11 and Scala 2.13 / testReturnRecordsDuringRebalance() – org.apache.kafka.clients.consumer.KafkaConsumerTest<1s
Build / JDK 11 and Scala 2.13 / testLargeAssignmentAndGroupWithNonEqualSubscription() – org.apache.kafka.clients.consumer.StickyAssignorTest1m 21s

The same tests all pass locally, and do not seem related to this change.

Copy link
Copy Markdown
Member

@jsancio jsancio 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 changes @niket-goel

Comment thread raft/src/main/java/org/apache/kafka/raft/internals/RecordsIterator.java Outdated
Comment thread raft/src/main/java/org/apache/kafka/raft/internals/RecordsIterator.java Outdated
Comment thread raft/src/test/java/org/apache/kafka/raft/internals/RecordsIteratorTest.java Outdated
Comment thread raft/src/test/java/org/apache/kafka/raft/internals/RecordsIteratorTest.java Outdated
Comment thread raft/src/test/java/org/apache/kafka/raft/internals/RecordsIteratorTest.java Outdated
Copy link
Copy Markdown
Member

@jsancio jsancio left a comment

Choose a reason for hiding this comment

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

Should be good to merge after this comment.

Comment thread raft/src/main/java/org/apache/kafka/raft/internals/RecordsIterator.java Outdated
Copy link
Copy Markdown
Member

@jsancio jsancio left a comment

Choose a reason for hiding this comment

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

LGTM

private Batch<T> readBatch(DefaultRecordBatch batch) {
if (doCrcValidation) {
// Perform a CRC validity check on this batch
batch.ensureValid();
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

This throws an exception that doesn't include a stacktrace. This is a rare event so we want to pay the cost of generating a stacktrace as it will help us debug the issue.

@jsancio jsancio added core Kafka Broker 3.3 labels Aug 5, 2022
@hachikuji hachikuji merged commit 48caba9 into apache:trunk Aug 8, 2022
hachikuji pushed a commit that referenced this pull request Aug 8, 2022
…rds (#12457)

This commit adds a check to ensure the RecordBatch CRC is valid when
iterating over a Batch of Records using the RecordsIterator. The
RecordsIterator is used by both Snapshot reads and Log Records reads in
Kraft. The check can be turned off by a class parameter and is on by default.

Reviewers: José Armando García Sancio <jsancio@users.noreply.github.com>
ijuma added a commit to confluentinc/kafka that referenced this pull request Aug 10, 2022
…(10 August 2022)

Trivial conflict in gradle/dependencies.gradle due to the newer Netty
version in confluentinc/kafka.

* apache-github/trunk:
MINOR: Upgrade gradle to 7.5.1 and bump other build/test dependencies
(apache#12495)
KAFKA-14140: Ensure an offline or in-controlled-shutdown replica is
not eligible to join ISR in ZK mode (apache#12487)
  KAFKA-14114: Add Metadata Error Related Metrics
MINOR: BrokerMetadataSnapshotter must avoid exceeding batch size
(apache#12486)
  MINOR: Upgrade mockito test dependencies (apache#12460)
KAFKA-14144:; Compare AlterPartition LeaderAndIsr before fencing
partition epoch (apache#12489)
KAFKA-14134: Replace EasyMock with Mockito for WorkerConnectorTest
(apache#12472)
  MINOR: Update scala version in bin scripts to 2.13.8 (apache#12477)
KAFKA-14104; Add CRC validation when iterating over Metadata Log
Records (apache#12457)
  MINOR: add :server-common test dependency to :storage (apache#12488)
  KAFKA-14107: Upgrade Jetty version for CVE fixes (apache#12440)
  KAFKA-14124: improve quorum controller fault handling (apache#12447)
ijuma added a commit to franz1981/kafka that referenced this pull request Aug 12, 2022
* apache-github/trunk: (447 commits)
  KAFKA-13959: Controller should unfence Broker with busy metadata log (apache#12274)
  KAFKA-10199: Expose read only task from state updater (apache#12497)
  KAFKA-14154; Return NOT_CONTROLLER from AlterPartition if leader is ahead of controller (apache#12506)
  KAFKA-13986; Brokers should include node.id in fetches to metadata quorum (apache#12498)
  KAFKA-14163; Retry compilation after zinc compile cache error (apache#12507)
  Remove duplicate common.message.* from clients:test jar file (apache#12407)
  KAFKA-13060: Replace EasyMock and PowerMock with Mockito in WorkerGroupMemberTest.java (apache#12484)
  Fix the rate window size calculation for edge cases (apache#12184)
  MINOR: Upgrade gradle to 7.5.1 and bump other build/test dependencies (apache#12495)
  KAFKA-14140: Ensure an offline or in-controlled-shutdown replica is not eligible to join ISR in ZK mode (apache#12487)
  KAFKA-14114: Add Metadata Error Related Metrics
  MINOR: BrokerMetadataSnapshotter must avoid exceeding batch size (apache#12486)
  MINOR: Upgrade mockito test dependencies (apache#12460)
  KAFKA-14144:; Compare AlterPartition LeaderAndIsr before fencing partition epoch (apache#12489)
  KAFKA-14134: Replace EasyMock with Mockito for WorkerConnectorTest (apache#12472)
  MINOR: Update scala version in bin scripts to 2.13.8 (apache#12477)
  KAFKA-14104; Add CRC validation when iterating over Metadata Log Records (apache#12457)
  MINOR: add :server-common test dependency to :storage (apache#12488)
  KAFKA-14107: Upgrade Jetty version for CVE fixes (apache#12440)
  KAFKA-14124: improve quorum controller fault handling (apache#12447)
  ...
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

core Kafka Broker

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants