Scanteianu commented on code in PR #12753:
URL: https://github.com/apache/kafka/pull/12753#discussion_r1009197631


##########
core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala:
##########
@@ -694,6 +694,76 @@ class PlaintextConsumerTest extends BaseConsumerTest {
     assertEquals(outOfRangePos.toLong, outOfRangePartitions.get(tp))
   }
 
+  @Test
+  def testFetchInvalidOffsetResetConfigEarliest(): Unit = {
+    this.consumerConfig.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, 
"earliest")
+    val consumer = createConsumer(configOverrides = this.consumerConfig)
+    val totalRecords = 10L
+
+    val producer = createProducer()
+    val startingTimestamp = 0
+    sendRecords(producer, totalRecords.toInt, tp, startingTimestamp = 
startingTimestamp)
+    consumer.assign(List(tp).asJava)
+    consumeAndVerifyRecords(consumer = consumer, numRecords = 
totalRecords.toInt, startingOffset =0)
+    // seek to out of range position
+    val outOfRangePos = totalRecords + 1
+    consumer.seek(tp, outOfRangePos)
+    // assert that poll resets to the beginning position
+    consumeAndVerifyRecords(consumer = consumer, numRecords = 1, 
startingOffset = 0)
+  }
+
+  @Test
+  def testFetchInvalidOffsetResetConfigLatest(): Unit = {
+
+    this.consumerConfig.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, 
"latest")
+    val consumer = createConsumer(configOverrides = this.consumerConfig)
+    val totalRecords = 10L
+
+    val producer = createProducer()
+    val startingTimestamp = 0
+    sendRecords(producer, totalRecords.toInt, tp, startingTimestamp = 
startingTimestamp)
+    consumer.assign(List(tp).asJava)
+    consumer.seek(tp,0)
+    consumeAndVerifyRecords(consumer = consumer, numRecords = 
totalRecords.toInt, startingOffset = 0)
+    // seek to out of range position
+    val outOfRangePos = totalRecords + 7 //arbitrary, much higher offset
+    consumer.seek(tp, outOfRangePos)
+    // assert that poll resets to the ending position
+    assertTrue(consumer.poll(Duration.ofMillis(50)).isEmpty)
+    sendRecords(producer, totalRecords.toInt, tp, startingTimestamp = 
totalRecords)
+    // ensure that new records start with the offset that was passed to seek()
+    val nextRecord = consumer.poll(Duration.ofMillis(50)).iterator().next()
+    assertEquals(outOfRangePos,nextRecord.offset())
+
+  }
+
+  @Test
+  def testFetchInvalidOffsetResetConfigLatestPastEnd(): Unit = {
+    this.consumerConfig.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, 
"latest")
+    val consumer = createConsumer(configOverrides = this.consumerConfig)
+    val totalRecords = 10L
+
+    val producer = createProducer()
+    val startingTimestamp = 0
+    sendRecords(producer, totalRecords.toInt, tp, startingTimestamp = 
startingTimestamp)
+    consumer.assign(List(tp).asJava)
+    consumer.seek(tp, 0)
+    consumeAndVerifyRecords(consumer = consumer, numRecords = 
totalRecords.toInt, startingOffset = 0)
+    // seek to out of range position
+    val outOfRangePos = totalRecords + 17 //arbitrary, much higher offset
+    consumer.seek(tp, outOfRangePos)
+    // assert that poll resets to the ending position
+    assertTrue(consumer.poll(Duration.ofMillis(50)).isEmpty)

Review Comment:
   shouldn't this reset the offset to 10, so the next seek should return offset 
10?



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

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

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

Reply via email to