Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14026#discussion_r69828515
  
    --- Diff: 
external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/ConsumerStrategy.scala
 ---
    @@ -79,8 +81,71 @@ private case class Subscribe[K, V](
       def onStart(currentOffsets: ju.Map[TopicPartition, jl.Long]): 
Consumer[K, V] = {
         val consumer = new KafkaConsumer[K, V](kafkaParams)
         consumer.subscribe(topics)
    -    if (currentOffsets.isEmpty) {
    -      offsets.asScala.foreach { case (topicPartition, offset) =>
    +    val toSeek = if (currentOffsets.isEmpty) {
    +      offsets
    +    } else {
    +      currentOffsets
    +    }
    +    if (!toSeek.isEmpty) {
    +      // work around KAFKA-3370 when reset is none
    +      val aor = kafkaParams.get(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG)
    +      val shouldSuppress = aor != null && 
aor.asInstanceOf[String].toUpperCase == "NONE"
    +      try {
    +        consumer.poll(0)
    +      } catch {
    +        case x: NoOffsetForPartitionException if shouldSuppress =>
    +          // silence exception
    +      }
    +      toSeek.asScala.foreach { case (topicPartition, offset) =>
    +          consumer.seek(topicPartition, offset)
    +      }
    +    }
    +
    +    consumer
    +  }
    +}
    +
    +/**
    + * Subscribe to all topics matching specified pattern to get dynamically 
assigned partitions.
    + * The pattern matching will be done periodically against topics existing 
at the time of check.
    + * @param pattern pattern to subscribe to
    + * @param kafkaParams Kafka
    + * <a href="http://kafka.apache.org/documentation.html#newconsumerconfigs";>
    + * configuration parameters</a> to be used on driver. The same params will 
be used on executors,
    + * with minor automatic modifications applied.
    + *  Requires "bootstrap.servers" to be set
    + * with Kafka broker(s) specified in host1:port1,host2:port2 form.
    + * @param offsets: offsets to begin at on initial startup.  If no offset 
is given for a
    + * TopicPartition, the committed offset (if applicable) or kafka param
    + * auto.offset.reset will be used.
    + */
    +private case class SubscribePattern[K, V](
    +    pattern: ju.regex.Pattern,
    +    kafkaParams: ju.Map[String, Object],
    +    offsets: ju.Map[TopicPartition, jl.Long]
    +  ) extends ConsumerStrategy[K, V] {
    +
    +  def executorKafkaParams: ju.Map[String, Object] = kafkaParams
    +
    +  def onStart(currentOffsets: ju.Map[TopicPartition, jl.Long]): 
Consumer[K, V] = {
    +    val consumer = new KafkaConsumer[K, V](kafkaParams)
    +    consumer.subscribe(pattern, new NoOpConsumerRebalanceListener())
    +    val toSeek = if (currentOffsets.isEmpty) {
    +      offsets
    +    } else {
    +      currentOffsets
    +    }
    +    if (!toSeek.isEmpty) {
    +      // work around KAFKA-3370 when reset is none
    +      val aor = kafkaParams.get(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG)
    +      val shouldSuppress = aor != null && 
aor.asInstanceOf[String].toUpperCase == "NONE"
    +      try {
    +        consumer.poll(0)
    +      } catch {
    +        case x: NoOffsetForPartitionException if shouldSuppress =>
    --- End diff --
    
    same here. log as warn.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org

Reply via email to