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/03/18 01:07:06 UTC

[GitHub] [kafka] hachikuji commented on a change in pull request #10276: KAFKA-12253: Add tests that cover all of the cases for ReplicatedLog's validateOffsetAndEpoch

hachikuji commented on a change in pull request #10276:
URL: https://github.com/apache/kafka/pull/10276#discussion_r596481018



##########
File path: raft/src/main/java/org/apache/kafka/raft/ValidOffsetAndEpoch.java
##########
@@ -16,40 +16,56 @@
  */
 package org.apache.kafka.raft;
 
+import java.util.Objects;
+
 public final class ValidOffsetAndEpoch {
-    final private Type type;
+    final private Kind kind;

Review comment:
       It's always been a little odd to me that `ValidOffsetAndEpoch` had different types (or kinds), including "VALID." If this were scala, we would probably use case classes to represent this. The nice thing about that is that we can use different field names for each case instead of having to call it `offsetAndEpoch` in all cases. Even though we don't have anything as convenient in Java, I do wonder if it might be clearer to approximate that pattern with an interface and separate implementations for each case. For example:
   
   ```java
   interface ValidationResult
   
   class OutOfRange implements ValidationResult {
     OffsetAndEpoch latestSnapshotId;
   }
   
   class DivergingEpoch implements ValidationResult {
     OffsetAndEpoch divergingEpoch;
   }
   
   class ValidOffsetAndEpoch implements ValidationResult {
   }
   ```
   
   The benefit is that we make the expectation about the `OffsetAndEpoch` in the result explicit. The downside is that we have to use `instanceof` checks for each case and we don't get any guarantees from the type system that we have handled them all. Kind of painful either way.
   

##########
File path: raft/src/main/java/org/apache/kafka/raft/ValidOffsetAndEpoch.java
##########
@@ -16,40 +16,56 @@
  */
 package org.apache.kafka.raft;
 
+import java.util.Objects;
+
 public final class ValidOffsetAndEpoch {
-    final private Type type;
+    final private Kind kind;
     final private OffsetAndEpoch offsetAndEpoch;
 
-    ValidOffsetAndEpoch(Type type, OffsetAndEpoch offsetAndEpoch) {
-        this.type = type;
+    ValidOffsetAndEpoch(Kind kind, OffsetAndEpoch offsetAndEpoch) {
+        this.kind = kind;
         this.offsetAndEpoch = offsetAndEpoch;
     }
 
-    public Type type() {
-        return type;
+    public Kind kind() {
+        return kind;
     }
 
     public OffsetAndEpoch offsetAndEpoch() {
         return offsetAndEpoch;
     }
 
-    public static enum Type {
+    public static enum Kind {

Review comment:
       nit: `static` is redundant

##########
File path: core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala
##########
@@ -485,6 +485,170 @@ final class KafkaMetadataLogTest {
     batchBuilder.build()
   }
 
+  @Test
+  def testValidateEpochGreaterThanLastKnownEpoch(): Unit = {
+    val log = buildMetadataLog(tempDir, mockTime)
+
+    val numberOfRecords = 1
+    val epoch = 1
+
+    append(log, numberOfRecords, epoch)
+
+    val resultOffsetAndEpoch = log.validateOffsetAndEpoch(numberOfRecords, epoch + 1)
+    assertEquals(ValidOffsetAndEpoch.Kind.DIVERGING, resultOffsetAndEpoch.kind)
+    assertEquals(new OffsetAndEpoch(log.endOffset.offset, epoch), resultOffsetAndEpoch.offsetAndEpoch())
+  }
+
+  @Test
+  def testValidateEpochLessThanOldestSnapshotEpoch(): Unit = {
+    val log = buildMetadataLog(tempDir, mockTime)
+
+    val numberOfRecords = 10
+    val epoch = 1
+
+    append(log, numberOfRecords, epoch)
+    log.updateHighWatermark(new LogOffsetMetadata(numberOfRecords))
+
+    val snapshotId = new OffsetAndEpoch(numberOfRecords, epoch)
+    TestUtils.resource(log.createSnapshot(snapshotId)) { snapshot =>
+      snapshot.freeze()
+    }
+    assertTrue(log.deleteBeforeSnapshot(snapshotId))
+
+    val resultOffsetAndEpoch = log.validateOffsetAndEpoch(numberOfRecords, epoch - 1)
+    assertEquals(ValidOffsetAndEpoch.Kind.SNAPSHOT, resultOffsetAndEpoch.kind)
+    assertEquals(new OffsetAndEpoch(numberOfRecords, epoch), resultOffsetAndEpoch.offsetAndEpoch())

Review comment:
       nit: use `snapshotId`? There are a few similar cases below and in `MockLogTest`.




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