Uploaded image for project: 'Kafka'
  1. Kafka
  2. KAFKA-16777

New consumer should throw NoOffsetForPartitionException on continuous poll zero if no reset strategy

    XMLWordPrintableJSON

Details

    Description

      If the consumer does not define an offset reset strategy, a call to poll should fail with NoOffsetForPartitionException. That works as expected on the new consumer when polling with a timeout > 0 (existing integration test here), but fails when polling continuously with ZERO timeout.

      This can be easily reproduced with a new integration test like this (passes for the legacy consumer but fails for the new consumer). We should add it as part of the fix, for better coverage:

        @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
        @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
        def testNoOffsetForPartitionExceptionOnPollZero(quorum: String, groupProtocol: String): Unit = {
          this.consumerConfig.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "none")
          val consumer = createConsumer(configOverrides = this.consumerConfig)
      
          consumer.assign(List(tp).asJava)
      
          // continuous poll should eventually fail because there is no offset reset strategy set (fail only when resetting positions after coordinator is known)
          TestUtils.tryUntilNoAssertionError() {
            assertThrows(classOf[NoOffsetForPartitionException], () => consumer.poll(Duration.ZERO))
          }
        }
      

      Also this is covered in the unit test KafkaConsumerTest.testMissingOffsetNoResetPolicy, that is currently enabled only for the LegacyConsumer. After fixing this issue we should be able to enable it for the new consumer too.

      The issue seems to be around calling poll with ZERO timeout, that even when called continuously, the consumer is not able to initWithCommittedOffsetsIfNeeded, so the updateFetchPositions never makes it to resetInitializingPositions, where the exception is thrown.

       

      There is the related issue https://issues.apache.org/jira/browse/KAFKA-16637, but filing this one to provide more context and point out the test failures and suggested new tests,. All fail even with the current patch in KAFKA-16637 so needs investigation. 

      Attachments

        Issue Links

          Activity

            People

              kirktrue Kirk True
              lianetm Lianet Magrans
              Bruno Cadonna Bruno Cadonna
              Votes:
              0 Vote for this issue
              Watchers:
              3 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: