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

    https://github.com/apache/spark/pull/16686#discussion_r99332003
  
    --- Diff: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaTopicPartitionOffsetReader.scala
 ---
    @@ -0,0 +1,300 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.kafka010
    +
    +import java.{util => ju}
    +import java.util.concurrent.{Executors, ThreadFactory}
    +
    +import scala.collection.JavaConverters._
    +import scala.concurrent.{ExecutionContext, Future}
    +import scala.concurrent.duration.Duration
    +import scala.util.control.NonFatal
    +
    +import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig, 
KafkaConsumer}
    +import org.apache.kafka.common.TopicPartition
    +
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.sql.types._
    +import org.apache.spark.util.{ThreadUtils, UninterruptibleThread}
    +
    +/**
    + * This class uses Kafka's own [[KafkaConsumer]] API to read data offsets 
from Kafka.
    + * The [[ConsumerStrategy]] class defines which Kafka topics and 
partitions should be read
    + * by this source. These strategies directly correspond to the different 
consumption options
    + * in. This class is designed to return a configured [[KafkaConsumer]] 
that is used by the
    + * [[KafkaSource]] to query for the offsets. See the docs on
    + * [[org.apache.spark.sql.kafka010.ConsumerStrategy]]
    + * for more details.
    + *
    + * Note: This class is not ThreadSafe
    + */
    +private[kafka010] class KafkaTopicPartitionOffsetReader(
    +    consumerStrategy: ConsumerStrategy,
    +    driverKafkaParams: ju.Map[String, Object],
    +    readerOptions: Map[String, String],
    +    driverGroupIdPrefix: String) extends Logging {
    +  /**
    +   * Used to ensure execute fetch operations execute in an 
UninterruptibleThread
    +   */
    +  val kafkaReaderThread = Executors.newSingleThreadExecutor(new 
ThreadFactory {
    +    override def newThread(r: Runnable): Thread = {
    +      val t = new UninterruptibleThread("Kafka Offset Reader") {
    +        override def run(): Unit = {
    +          r.run()
    +        }
    +      }
    +      t.setDaemon(true)
    +      t
    +    }
    +  })
    +  val execContext = ExecutionContext.fromExecutorService(kafkaReaderThread)
    +
    +  /**
    +   * A KafkaConsumer used in the driver to query the latest Kafka offsets. 
This only queries the
    +   * offsets and never commits them.
    +   */
    +  protected var consumer = createConsumer()
    +
    +  private val maxOffsetFetchAttempts =
    +    readerOptions.getOrElse("fetchOffset.numRetries", "3").toInt
    +
    +  private val offsetFetchAttemptIntervalMs =
    +    readerOptions.getOrElse("fetchOffset.retryIntervalMs", "1000").toLong
    +
    +  private var groupId: String = null
    +
    +  private var nextId = 0
    +
    +  private def nextGroupId(): String = {
    +    groupId = driverGroupIdPrefix + "-" + nextId
    +    nextId += 1
    +    groupId
    +  }
    +
    +  override def toString(): String = consumerStrategy.toString
    +
    +  /**
    +   * Closes the connection to Kafka, and cleans up state.
    +   */
    +  def close(): Unit = {
    +    consumer.close()
    +    kafkaReaderThread.shutdownNow()
    +  }
    +
    +  /**
    +   * @return The Set of TopicPartitions for a given topic
    +   */
    +  def fetchTopicPartitions(): Set[TopicPartition] = runUninterruptibly {
    +    assert(Thread.currentThread().isInstanceOf[UninterruptibleThread])
    +    // Poll to get the latest assigned partitions
    +    consumer.poll(0)
    +    val partitions = consumer.assignment()
    +    consumer.pause(partitions)
    +    partitions.asScala.toSet
    +  }
    +
    +  /**
    +   * Resolves the specific offsets based on Kafka seek positions.
    +   * This method resolves offset value -1 to the latest and -2 to the
    +   * earliest Kafka seek position.
    +   */
    +  def fetchSpecificOffsets(
    +      partitionOffsets: Map[TopicPartition, Long]): Map[TopicPartition, 
Long] =
    +    runUninterruptibly {
    +      withRetriesWithoutInterrupt {
    +        // Poll to get the latest assigned partitions
    +        consumer.poll(0)
    +        val partitions = consumer.assignment()
    +        consumer.pause(partitions)
    +        assert(partitions.asScala == partitionOffsets.keySet,
    +          "If startingOffsets contains specific offsets, you must specify 
all TopicPartitions.\n" +
    +            "Use -1 for latest, -2 for earliest, if you don't care.\n" +
    +            s"Specified: ${partitionOffsets.keySet} Assigned: 
${partitions.asScala}")
    +        logDebug(s"Partitions assigned to consumer: $partitions. Seeking 
to $partitionOffsets")
    +
    +        partitionOffsets.foreach {
    +          case (tp, KafkaOffsets.LATEST) => 
consumer.seekToEnd(ju.Arrays.asList(tp))
    +          case (tp, KafkaOffsets.EARLIEST) => 
consumer.seekToBeginning(ju.Arrays.asList(tp))
    +          case (tp, off) => consumer.seek(tp, off)
    +        }
    +        partitionOffsets.map {
    +          case (tp, _) => tp -> consumer.position(tp)
    +        }
    +      }
    +    }
    +
    +  /**
    +   * Fetch the earliest offsets of partitions.
    --- End diff --
    
    can you specify which partitions? maybe "offsets of all partitions to be 
consumed according the consumer strategy"
    
    same for docs of other methods that do not take a specific list of 
partitions.


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