You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2021/04/07 04:39:28 UTC

[GitHub] [kafka] chia7712 commented on a change in pull request #10470: KAFKA-12612: Remove checksum from ConsumerRecord and RecordMetadata for 3.0

chia7712 commented on a change in pull request #10470:
URL: https://github.com/apache/kafka/pull/10470#discussion_r608332079



##########
File path: clients/src/main/java/org/apache/kafka/clients/producer/RecordMetadata.java
##########
@@ -42,20 +41,26 @@
     private final int serializedValueSize;
     private final TopicPartition topicPartition;
 
-    private volatile Long checksum;
-
     public RecordMetadata(TopicPartition topicPartition, long baseOffset, long relativeOffset, long timestamp,
-                          Long checksum, int serializedKeySize, int serializedValueSize) {
+                          int serializedKeySize, int serializedValueSize) {
         // ignore the relativeOffset if the base offset is -1,
         // since this indicates the offset is unknown
         this.offset = baseOffset == -1 ? baseOffset : baseOffset + relativeOffset;
         this.timestamp = timestamp;
-        this.checksum = checksum;
         this.serializedKeySize = serializedKeySize;
         this.serializedValueSize = serializedValueSize;
         this.topicPartition = topicPartition;
     }
 
+    /**
+     * @deprecated use constructor without `checksum` parameter.
+     */
+    @Deprecated
+    public RecordMetadata(TopicPartition topicPartition, long baseOffset, long relativeOffset, long timestamp,

Review comment:
       Not sure whether we have to offer this constructor. We remove the `checksum` from all constructors of `ConsumerRecord` even though we don't add deprecation annotation to  constructors before.
   
   

##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
##########
@@ -1385,7 +1385,7 @@ private void handleOffsetOutOfRange(FetchPosition fetchPosition, TopicPartition
             byte[] valueByteArray = valueBytes == null ? null : Utils.toArray(valueBytes);
             V value = valueBytes == null ? null : this.valueDeserializer.deserialize(partition.topic(), headers, valueByteArray);
             return new ConsumerRecord<>(partition.topic(), partition.partition(), offset,
-                                        timestamp, timestampType, record.checksumOrNull(),

Review comment:
       Could we remove `checksumOrNull` from `Record`? That method works for `AbstractLegacyRecordBatch` only.

##########
File path: clients/src/test/java/org/apache/kafka/clients/producer/internals/ProducerBatchTest.java
##########
@@ -65,7 +64,6 @@ public void testChecksumNullForMagicV2() {
         ProducerBatch batch = new ProducerBatch(new TopicPartition("topic", 1), memoryRecordsBuilder, now);
         FutureRecordMetadata future = batch.tryAppend(now, null, new byte[10], Record.EMPTY_HEADERS, null, now);
         assertNotNull(future);
-        assertNull(future.checksumOrNull());

Review comment:
       this test case (`testChecksumNullForMagicV2`) is meaningless after we remove  `assertNull(future.checksumOrNull())`. It seems to me it is ok to remove whole test case.

##########
File path: clients/src/main/java/org/apache/kafka/clients/producer/RecordMetadata.java
##########
@@ -89,25 +94,6 @@ public long timestamp() {
         return this.timestamp;
     }
 
-    /**
-     * The checksum (CRC32) of the record.
-     *
-     * @deprecated As of Kafka 0.11.0. Because of the potential for message format conversion on the broker, the
-     *             computed checksum may not match what was stored on the broker, or what will be returned to the consumer.
-     *             It is therefore unsafe to depend on this checksum for end-to-end delivery guarantees. Additionally,
-     *             message format v2 does not include a record-level checksum (for performance, the record checksum
-     *             was replaced with a batch checksum). To maintain compatibility, a partial checksum computed from
-     *             the record timestamp, serialized key size, and serialized value size is returned instead, but
-     *             this should not be depended on for end-to-end reliability.
-     */
-    @Deprecated
-    public long checksum() {
-        if (checksum == null)
-            // The checksum is null only for message format v2 and above, which do not have a record-level checksum.
-            this.checksum = DefaultRecord.computePartialChecksum(timestamp, serializedKeySize, serializedValueSize);

Review comment:
       Could you remove `computePartialChecksum`? It is unused now.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org