You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2020/09/11 13:48:41 UTC

[GitHub] [spark] gaborgsomogyi opened a new pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of poll(long) API

gaborgsomogyi opened a new pull request #29729:
URL: https://github.com/apache/spark/pull/29729


   ### What changes were proposed in this pull request?
   Deprecated `KafkaConsumer.poll(long)` API calls may cause infinite wait in the driver. In this PR I've changed `KafkaConsumer` calls to `AdminClient` calls. Please see the following [doc](https://docs.google.com/document/d/1gAh0pKgZUgyqO2Re3sAy-fdYpe_SxpJ6DkeXE8R1P7E/edit?usp=sharing) for further details.
   
   The PR contains the following changes:
   * Replaced `KafkaConsumer` calls to `AdminClient` calls
   * GroupId prefix feature removed from driver (`AdminClient` doesn't need any GroupId)
   * GroupId override feature removed from driver (`AdminClient` doesn't need any GroupId)
   * Additional unit tests
   * Code comment changes
   * Minor bugfixes here and there
   * Removed Kafka auto topic creation feature (please see doc for rationale). In short, it's super hidden, not sure anybody ever used in production + error prone.
   
   ### Why are the changes needed?
   Driver may hang forever.
   
   ### Does this PR introduce _any_ user-facing change?
   No.
   
   ### How was this patch tested?
   Existing + additional unit tests.
   Cluster test with simple Kafka topic to another topic query.
   


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



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


[GitHub] [spark] gaborgsomogyi commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-698211449






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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-701984943






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



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


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #29729:
URL: https://github.com/apache/spark/pull/29729#discussion_r487682836



##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -33,10 +36,10 @@ import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap
 import org.apache.spark.util.{UninterruptibleThread, UninterruptibleThreadRunner}
 
 /**
- * This class uses Kafka's own [[KafkaConsumer]] API to read data offsets from Kafka.
+ * This class uses Kafka's its own [[Admin]] API to read data offsets from Kafka.

Review comment:
       Fixed.




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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-691856280


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/128622/
   Test FAILed.


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



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


[GitHub] [spark] SparkQA commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-703610985


   **[Test build #129413 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129413/testReport)** for PR 29729 at commit [`34333b9`](https://github.com/apache/spark/commit/34333b915981e115999a9c0649c6fab607aa4e29).


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-731996154


   Merged build finished. Test FAILed.


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



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


[GitHub] [spark] SparkQA commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-691923042


   **[Test build #128634 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128634/testReport)** for PR 29729 at commit [`030d3e7`](https://github.com/apache/spark/commit/030d3e7ad8a6ac849d4c18ae6a71ca2ca960f8d5).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


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



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


[GitHub] [spark] Tagar commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
Tagar commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-697969651


   Just wondering if this authorization any different in Confluent Kafka (vs OSS Kafka)? 
   I was trying to follow this 
   https://docs.confluent.io/current/kafka/authorization.html 
   ``` 
   **Group**
   Groups in the brokers. 
   All protocol calls that work with groups, such as joining a group, 
   must have corresponding privileges with the group in the subject. 
   
   Group (**group.id**) can mean Consumer Group, Stream Group (application.id), 
   Connect Worker Group, or any other group that uses the 
   Consumer Group protocol, like Schema Registry cluster.
   ```
   
   


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



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


[GitHub] [spark] SparkQA commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-703628914


   **[Test build #129413 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129413/testReport)** for PR 29729 at commit [`34333b9`](https://github.com/apache/spark/commit/34333b915981e115999a9c0649c6fab607aa4e29).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-691877478






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



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


[GitHub] [spark] viirya commented on a change in pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #29729:
URL: https://github.com/apache/spark/pull/29729#discussion_r499167165



##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/ConsumerStrategy.scala
##########
@@ -36,31 +37,40 @@ import org.apache.spark.kafka010.KafkaConfigUpdater
  * All three strategies have overloaded constructors that allow you to specify
  * the starting offset for a particular partition.
  */
-private[kafka010] sealed trait ConsumerStrategy {
-  /** Create a [[KafkaConsumer]] and subscribe to topics according to a desired strategy */
-  def createConsumer(kafkaParams: ju.Map[String, Object]): Consumer[Array[Byte], Array[Byte]]
-
-  /**
-   * Updates the parameters with security if needed.
-   * Added a function to hide internals and reduce code duplications because all strategy uses it.
-   */
-  protected def setAuthenticationConfigIfNeeded(kafkaParams: ju.Map[String, Object]) =
-    KafkaConfigUpdater("source", kafkaParams.asScala.toMap)
+private[kafka010] sealed trait ConsumerStrategy extends Logging {
+  /** Creates an [[org.apache.kafka.clients.admin.AdminClient]] */
+  def createAdmin(kafkaParams: ju.Map[String, Object]): Admin = {
+    val updatedKafkaParams = KafkaConfigUpdater("source", kafkaParams.asScala.toMap)
       .setAuthenticationConfigIfNeeded()
       .build()
+    logDebug(s"Admin params: ${KafkaRedactionUtil.redactParams(updatedKafkaParams.asScala.toSeq)}")
+    Admin.create(updatedKafkaParams)
+  }
+
+  /** Returns the assigned or subscribed [[TopicPartition]] */
+  def assignedTopicPartitions(admin: Admin): Set[TopicPartition]
+
+  protected def retrieveAllPartitions(admin: Admin, topics: Set[String]): Set[TopicPartition] = {
+    admin.describeTopics(topics.asJava).all().get().asScala.filterNot(_._2.isInternal).flatMap {
+      case (topic, topicDescription) =>
+        topicDescription.partitions().asScala.map { topicPartitionInfo =>
+          val partition = topicPartitionInfo.partition()
+          logDebug(s"Partition added: $topic:$partition")

Review comment:
       Will this message mislead to users? For example, `AssignStrategy` is only interesting to specified partitions among all partitions here.




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



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


[GitHub] [spark] viirya commented on a change in pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #29729:
URL: https://github.com/apache/spark/pull/29729#discussion_r499873147



##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -46,39 +49,40 @@ import org.apache.spark.util.{UninterruptibleThread, UninterruptibleThreadRunner
 private[kafka010] class KafkaOffsetReader(
     consumerStrategy: ConsumerStrategy,
     val driverKafkaParams: ju.Map[String, Object],
-    readerOptions: CaseInsensitiveMap[String],
-    driverGroupIdPrefix: String) extends Logging {
+    readerOptions: CaseInsensitiveMap[String]) extends Logging {
 
   /**
-   * [[UninterruptibleThreadRunner]] ensures that all [[KafkaConsumer]] communication called in an
+   * [[UninterruptibleThreadRunner]] ensures that all Kafka communication called in an
    * [[UninterruptibleThread]]. In the case of streaming queries, we are already running in an
    * [[UninterruptibleThread]], however for batch mode this is not the case.
    */
   val uninterruptibleThreadRunner = new UninterruptibleThreadRunner("Kafka Offset Reader")
 
-  /**
-   * Place [[groupId]] and [[nextId]] here so that they are initialized before any consumer is
-   * created -- see SPARK-19564.
-   */
-  private var groupId: String = null
-  private var nextId = 0
-
-  /**
-   * A KafkaConsumer used in the driver to query the latest Kafka offsets. This only queries the
-   * offsets and never commits them.
-   */

Review comment:
       E.g., An `AdminClient` used in the driver to query the latest Kafka offsets. This only queries the offsets because AdminClient has no functionality to commit offsets?




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



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


[GitHub] [spark] SparkQA commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-691110067


   **[Test build #128570 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128570/testReport)** for PR 29729 at commit [`6236737`](https://github.com/apache/spark/commit/62367376f7d79390ed26bda233323171b2177373).


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



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


[GitHub] [spark] SparkQA commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-701993096


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/33919/
   


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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-691122214






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



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


[GitHub] [spark] HeartSaVioR commented on a change in pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #29729:
URL: https://github.com/apache/spark/pull/29729#discussion_r489217921



##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -105,34 +109,16 @@ private[kafka010] class KafkaOffsetReader(
     minPartitions.map(_ > numTopicPartitions).getOrElse(false)
   }
 
-  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 = {
-    if (_consumer != null) uninterruptibleThreadRunner.runUninterruptibly { stopConsumer() }
+    if (_admin != null) uninterruptibleThreadRunner.runUninterruptibly { stopAdmin() }
     uninterruptibleThreadRunner.shutdown()
   }
 
-  /**
-   * @return The Set of TopicPartitions for a given topic
-   */
-  def fetchTopicPartitions(): Set[TopicPartition] = uninterruptibleThreadRunner.runUninterruptibly {
-    assert(Thread.currentThread().isInstanceOf[UninterruptibleThread])

Review comment:
       Having a separate PR is OK for me - that would avoid unnecessary hold on this PR.




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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-703693488






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



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


[GitHub] [spark] xuanyuanking commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
xuanyuanking commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-716444949


   Thanks @gaborgsomogyi for the solid work! Leave some investigations on my side:
   
   - For `group.id` based authorization: The exception, `org.apache.kafka.common.errors.AuthorizationException`, used for groupId authorization in KafkaConsumer is no longer used in the AdminClient API. The only one use case in AdminClient is for the topic based authorization in `createPartitions`.
   - For backward compatibility: The comment in Admin(https://github.com/apache/kafka/blob/2.6/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java#L49-L51) is consistent with the test result by Gabor in https://github.com/apache/spark/pull/29729#issuecomment-712798812
   
   Some small suggestions:
   
   - Should we add some guides of the alternative method for the original usage of group.id based authorization? Maybe link the Kafka doc is enough.
   - IMO, It would be great to link all the investigations/tests in the PR description.
   


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



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


[GitHub] [spark] SparkQA commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-702003442


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/33919/
   


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



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


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #29729:
URL: https://github.com/apache/spark/pull/29729#discussion_r488466661



##########
File path: external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
##########
@@ -689,57 +691,6 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase {
     )
   }
 
-  test("allow group.id prefix") {

Review comment:
       OK, resolving this comment.




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



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


[GitHub] [spark] gaborgsomogyi commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of poll(long) API

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-691138952


   cc @HeartSaVioR @zsxwing 


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



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


[GitHub] [spark] SparkQA commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-691892312


   **[Test build #128630 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128630/testReport)** for PR 29729 at commit [`3f82562`](https://github.com/apache/spark/commit/3f825622fff9b251f32c51fdaa1a279ea718b36b).
    * This patch **fails to build**.
    * This patch merges cleanly.
    * This patch adds no public classes.


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-735817105






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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-701380676






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



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


[GitHub] [spark] SparkQA commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-704149885


   **[Test build #129440 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129440/testReport)** for PR 29729 at commit [`d0e4c63`](https://github.com/apache/spark/commit/d0e4c63018beb4e0d36bfea1753c3e5d8f13659e).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


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



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


[GitHub] [spark] SparkQA commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-733670191


   **[Test build #131774 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131774/testReport)** for PR 29729 at commit [`f429129`](https://github.com/apache/spark/commit/f42912919432e91e372af4e478bb968995003868).


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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-701361361






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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-703669704






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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-691859185






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



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


[GitHub] [spark] HeartSaVioR commented on a change in pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #29729:
URL: https://github.com/apache/spark/pull/29729#discussion_r489215803



##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -105,34 +109,16 @@ private[kafka010] class KafkaOffsetReader(
     minPartitions.map(_ > numTopicPartitions).getOrElse(false)
   }
 
-  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 = {
-    if (_consumer != null) uninterruptibleThreadRunner.runUninterruptibly { stopConsumer() }
+    if (_admin != null) uninterruptibleThreadRunner.runUninterruptibly { stopAdmin() }
     uninterruptibleThreadRunner.shutdown()
   }
 
-  /**
-   * @return The Set of TopicPartitions for a given topic
-   */
-  def fetchTopicPartitions(): Set[TopicPartition] = uninterruptibleThreadRunner.runUninterruptibly {
-    assert(Thread.currentThread().isInstanceOf[UninterruptibleThread])

Review comment:
       Sorry I meant Kafka related, driver side only. (In executors we still leverage KafkaConsumer so not 100% sure we can get rid of it. Though still better to get rid if possible.)




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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-691859197


   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/33247/
   Test PASSed.


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



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


[GitHub] [spark] gaborgsomogyi commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-703649627


   retest this please


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



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


[GitHub] [spark] SparkQA commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-701969080


   **[Test build #129304 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129304/testReport)** for PR 29729 at commit [`bc6fbdc`](https://github.com/apache/spark/commit/bc6fbdc8ad08ed1f4e67eef2607b61cbb46c9313).


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



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


[GitHub] [spark] SparkQA commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-701361190


   **[Test build #129271 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129271/testReport)** for PR 29729 at commit [`40031cc`](https://github.com/apache/spark/commit/40031ccaae05fcffc9206729ad742a249f370edf).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-701470089






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



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


[GitHub] [spark] gaborgsomogyi commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-702491670


   @zsxwing you understand it right, actually it removes permissions needed.


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



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


[GitHub] [spark] gaborgsomogyi commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-735818247


   I've just updated the description to reflect the actual stand of the PR.


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-692105566






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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-692571650






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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-732049402






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



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


[GitHub] [spark] gaborgsomogyi commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-698439672


   @Tagar yeah, authorization w/ vanilla and confluent Kafka still works and a good alternative is the topic based restriction since `AdminClient` is not using `group.id`. I've tested this and it works like charm.


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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-691923296






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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-691892342


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/128630/
   Test FAILed.


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



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


[GitHub] [spark] gaborgsomogyi commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-701470701


   retest this please


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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-732028920






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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-731996167


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/131540/
   Test FAILed.


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-702003476






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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-735826161






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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-691877478






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



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


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of poll(long) API

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #29729:
URL: https://github.com/apache/spark/pull/29729#discussion_r487063789



##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -105,34 +102,16 @@ private[kafka010] class KafkaOffsetReader(
     minPartitions.map(_ > numTopicPartitions).getOrElse(false)
   }
 
-  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 = {
-    if (_consumer != null) uninterruptibleThreadRunner.runUninterruptibly { stopConsumer() }
+    if (_admin != null) uninterruptibleThreadRunner.runUninterruptibly { stopAdmin() }

Review comment:
       I've left `UninterruptibleThreadRunner` for now but I think it can be removed since there is no evidence that `AdminClient` suffers from the same issue.

##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -213,64 +180,70 @@ private[kafka010] class KafkaOffsetReader(
       assert(partitions.asScala == partitionTimestamps.keySet,
         "If starting/endingOffsetsByTimestamp contains specific offsets, you must specify all " +
           s"topics. Specified: ${partitionTimestamps.keySet} Assigned: ${partitions.asScala}")
-      logDebug(s"Partitions assigned to consumer: $partitions. Seeking to $partitionTimestamps")
+      logDebug(s"Assigned partitions: $partitions. Seeking to $partitionTimestamps")
     }
 
     val fnRetrievePartitionOffsets: ju.Set[TopicPartition] => Map[TopicPartition, Long] = { _ => {
-        val converted = partitionTimestamps.map { case (tp, timestamp) =>
-          tp -> java.lang.Long.valueOf(timestamp)
+        val listOffsetsParams = partitionTimestamps.map { p =>
+          p._1 -> OffsetSpec.forTimestamp(p._2)
         }.asJava
+        admin.listOffsets(listOffsetsParams, listOffsetsOptions()).all().get().asScala.map {
+          case (tp, offsetSpec) =>
+            if (failsOnNoMatchingOffset) {
+              assert(offsetSpec.offset() != OffsetFetchResponse.INVALID_OFFSET, "No offset " +
+                s"matched from request of topic-partition $tp and timestamp " +
+                s"${partitionTimestamps(tp)}.")
+            }
 
-        val offsetForTime: ju.Map[TopicPartition, OffsetAndTimestamp] =
-          consumer.offsetsForTimes(converted)
-
-        offsetForTime.asScala.map { case (tp, offsetAndTimestamp) =>
-          if (failsOnNoMatchingOffset) {
-            assert(offsetAndTimestamp != null, "No offset matched from request of " +
-              s"topic-partition $tp and timestamp ${partitionTimestamps(tp)}.")
-          }
-
-          if (offsetAndTimestamp == null) {
-            tp -> KafkaOffsetRangeLimit.LATEST
-          } else {
-            tp -> offsetAndTimestamp.offset()
-          }
+            if (offsetSpec == null) {

Review comment:
       Different API, different result. `offsetSpec` can be null.

##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala
##########
@@ -96,8 +96,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
     val kafkaOffsetReader = new KafkaOffsetReader(
       strategy(caseInsensitiveParameters),
       kafkaParamsForDriver(specifiedKafkaParams),
-      caseInsensitiveParameters,
-      driverGroupIdPrefix = s"$uniqueGroupId-driver")

Review comment:
       GroupId removal in this file since `AdminClient` doesn't require it.

##########
File path: external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
##########
@@ -689,57 +691,6 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase {
     )
   }
 
-  test("allow group.id prefix") {

Review comment:
       No need the tests because driver is not user `KafkaConsumer` => no GroupId needed.

##########
File path: external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala
##########
@@ -465,41 +463,6 @@ abstract class KafkaRelationSuiteBase extends QueryTest with SharedSparkSession
     testBadOptions("subscribePattern" -> "")("pattern to subscribe is empty")
   }
 
-  test("allow group.id prefix") {

Review comment:
       Same here.

##########
File path: external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
##########
@@ -608,7 +608,9 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase {
     // in executors.
     val query = kafka.map(kv => kv._2.toInt).writeStream.foreach(new ForeachWriter[Int] {
       override def open(partitionId: Long, version: Long): Boolean = {
+        // Re-create topic since Kafka auto topic creation is not supported by Spark
         KafkaSourceSuite.globalTestUtils.deleteTopic(topic)
+        KafkaSourceSuite.globalTestUtils.createTopic(topic)

Review comment:
       This needed because we don't use `KafkaConsumer` and `AdminClient` doesn't provide auto topic creation. Please see doc for the rationale.

##########
File path: external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
##########
@@ -689,57 +691,6 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase {
     )
   }
 
-  test("allow group.id prefix") {

Review comment:
       No need the tests because driver is not using `KafkaConsumer` => no GroupId needed.

##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -105,34 +102,16 @@ private[kafka010] class KafkaOffsetReader(
     minPartitions.map(_ > numTopicPartitions).getOrElse(false)
   }
 
-  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 = {
-    if (_consumer != null) uninterruptibleThreadRunner.runUninterruptibly { stopConsumer() }
+    if (_admin != null) uninterruptibleThreadRunner.runUninterruptibly { stopAdmin() }

Review comment:
       I've left `UninterruptibleThreadRunner` for now but I think it can be removed since there is no evidence that `AdminClient` suffers from the same issue.

##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -213,64 +180,70 @@ private[kafka010] class KafkaOffsetReader(
       assert(partitions.asScala == partitionTimestamps.keySet,
         "If starting/endingOffsetsByTimestamp contains specific offsets, you must specify all " +
           s"topics. Specified: ${partitionTimestamps.keySet} Assigned: ${partitions.asScala}")
-      logDebug(s"Partitions assigned to consumer: $partitions. Seeking to $partitionTimestamps")
+      logDebug(s"Assigned partitions: $partitions. Seeking to $partitionTimestamps")
     }
 
     val fnRetrievePartitionOffsets: ju.Set[TopicPartition] => Map[TopicPartition, Long] = { _ => {
-        val converted = partitionTimestamps.map { case (tp, timestamp) =>
-          tp -> java.lang.Long.valueOf(timestamp)
+        val listOffsetsParams = partitionTimestamps.map { p =>
+          p._1 -> OffsetSpec.forTimestamp(p._2)
         }.asJava
+        admin.listOffsets(listOffsetsParams, listOffsetsOptions()).all().get().asScala.map {
+          case (tp, offsetSpec) =>
+            if (failsOnNoMatchingOffset) {
+              assert(offsetSpec.offset() != OffsetFetchResponse.INVALID_OFFSET, "No offset " +
+                s"matched from request of topic-partition $tp and timestamp " +
+                s"${partitionTimestamps(tp)}.")
+            }
 
-        val offsetForTime: ju.Map[TopicPartition, OffsetAndTimestamp] =
-          consumer.offsetsForTimes(converted)
-
-        offsetForTime.asScala.map { case (tp, offsetAndTimestamp) =>
-          if (failsOnNoMatchingOffset) {
-            assert(offsetAndTimestamp != null, "No offset matched from request of " +
-              s"topic-partition $tp and timestamp ${partitionTimestamps(tp)}.")
-          }
-
-          if (offsetAndTimestamp == null) {
-            tp -> KafkaOffsetRangeLimit.LATEST
-          } else {
-            tp -> offsetAndTimestamp.offset()
-          }
+            if (offsetSpec == null) {

Review comment:
       Different API, different result. `offsetSpec` can be null.

##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala
##########
@@ -96,8 +96,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
     val kafkaOffsetReader = new KafkaOffsetReader(
       strategy(caseInsensitiveParameters),
       kafkaParamsForDriver(specifiedKafkaParams),
-      caseInsensitiveParameters,
-      driverGroupIdPrefix = s"$uniqueGroupId-driver")

Review comment:
       GroupId removal in this file since `AdminClient` doesn't require it.

##########
File path: external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
##########
@@ -689,57 +691,6 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase {
     )
   }
 
-  test("allow group.id prefix") {

Review comment:
       No need the tests because driver is not user `KafkaConsumer` => no GroupId needed.

##########
File path: external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala
##########
@@ -465,41 +463,6 @@ abstract class KafkaRelationSuiteBase extends QueryTest with SharedSparkSession
     testBadOptions("subscribePattern" -> "")("pattern to subscribe is empty")
   }
 
-  test("allow group.id prefix") {

Review comment:
       Same here.

##########
File path: external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
##########
@@ -608,7 +608,9 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase {
     // in executors.
     val query = kafka.map(kv => kv._2.toInt).writeStream.foreach(new ForeachWriter[Int] {
       override def open(partitionId: Long, version: Long): Boolean = {
+        // Re-create topic since Kafka auto topic creation is not supported by Spark
         KafkaSourceSuite.globalTestUtils.deleteTopic(topic)
+        KafkaSourceSuite.globalTestUtils.createTopic(topic)

Review comment:
       This needed because we don't use `KafkaConsumer` and `AdminClient` doesn't provide auto topic creation. Please see doc for the rationale.

##########
File path: external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
##########
@@ -689,57 +691,6 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase {
     )
   }
 
-  test("allow group.id prefix") {

Review comment:
       No need the tests because driver is not using `KafkaConsumer` => no GroupId needed.

##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -105,34 +102,16 @@ private[kafka010] class KafkaOffsetReader(
     minPartitions.map(_ > numTopicPartitions).getOrElse(false)
   }
 
-  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 = {
-    if (_consumer != null) uninterruptibleThreadRunner.runUninterruptibly { stopConsumer() }
+    if (_admin != null) uninterruptibleThreadRunner.runUninterruptibly { stopAdmin() }

Review comment:
       I've left `UninterruptibleThreadRunner` for now but I think it can be removed since there is no evidence that `AdminClient` suffers from the same issue.

##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -213,64 +180,70 @@ private[kafka010] class KafkaOffsetReader(
       assert(partitions.asScala == partitionTimestamps.keySet,
         "If starting/endingOffsetsByTimestamp contains specific offsets, you must specify all " +
           s"topics. Specified: ${partitionTimestamps.keySet} Assigned: ${partitions.asScala}")
-      logDebug(s"Partitions assigned to consumer: $partitions. Seeking to $partitionTimestamps")
+      logDebug(s"Assigned partitions: $partitions. Seeking to $partitionTimestamps")
     }
 
     val fnRetrievePartitionOffsets: ju.Set[TopicPartition] => Map[TopicPartition, Long] = { _ => {
-        val converted = partitionTimestamps.map { case (tp, timestamp) =>
-          tp -> java.lang.Long.valueOf(timestamp)
+        val listOffsetsParams = partitionTimestamps.map { p =>
+          p._1 -> OffsetSpec.forTimestamp(p._2)
         }.asJava
+        admin.listOffsets(listOffsetsParams, listOffsetsOptions()).all().get().asScala.map {
+          case (tp, offsetSpec) =>
+            if (failsOnNoMatchingOffset) {
+              assert(offsetSpec.offset() != OffsetFetchResponse.INVALID_OFFSET, "No offset " +
+                s"matched from request of topic-partition $tp and timestamp " +
+                s"${partitionTimestamps(tp)}.")
+            }
 
-        val offsetForTime: ju.Map[TopicPartition, OffsetAndTimestamp] =
-          consumer.offsetsForTimes(converted)
-
-        offsetForTime.asScala.map { case (tp, offsetAndTimestamp) =>
-          if (failsOnNoMatchingOffset) {
-            assert(offsetAndTimestamp != null, "No offset matched from request of " +
-              s"topic-partition $tp and timestamp ${partitionTimestamps(tp)}.")
-          }
-
-          if (offsetAndTimestamp == null) {
-            tp -> KafkaOffsetRangeLimit.LATEST
-          } else {
-            tp -> offsetAndTimestamp.offset()
-          }
+            if (offsetSpec == null) {

Review comment:
       Different API, different result. `offsetSpec` can be null.

##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala
##########
@@ -96,8 +96,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
     val kafkaOffsetReader = new KafkaOffsetReader(
       strategy(caseInsensitiveParameters),
       kafkaParamsForDriver(specifiedKafkaParams),
-      caseInsensitiveParameters,
-      driverGroupIdPrefix = s"$uniqueGroupId-driver")

Review comment:
       GroupId removal in this file since `AdminClient` doesn't require it.

##########
File path: external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
##########
@@ -689,57 +691,6 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase {
     )
   }
 
-  test("allow group.id prefix") {

Review comment:
       No need the tests because driver is not user `KafkaConsumer` => no GroupId needed.

##########
File path: external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala
##########
@@ -465,41 +463,6 @@ abstract class KafkaRelationSuiteBase extends QueryTest with SharedSparkSession
     testBadOptions("subscribePattern" -> "")("pattern to subscribe is empty")
   }
 
-  test("allow group.id prefix") {

Review comment:
       Same here.

##########
File path: external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
##########
@@ -608,7 +608,9 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase {
     // in executors.
     val query = kafka.map(kv => kv._2.toInt).writeStream.foreach(new ForeachWriter[Int] {
       override def open(partitionId: Long, version: Long): Boolean = {
+        // Re-create topic since Kafka auto topic creation is not supported by Spark
         KafkaSourceSuite.globalTestUtils.deleteTopic(topic)
+        KafkaSourceSuite.globalTestUtils.createTopic(topic)

Review comment:
       This needed because we don't use `KafkaConsumer` and `AdminClient` doesn't provide auto topic creation. Please see doc for the rationale.

##########
File path: external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
##########
@@ -689,57 +691,6 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase {
     )
   }
 
-  test("allow group.id prefix") {

Review comment:
       No need the tests because driver is not using `KafkaConsumer` => no GroupId needed.




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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-702003476






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



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


[GitHub] [spark] HeartSaVioR edited a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
HeartSaVioR edited a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-735677230


   @zsxwing @viirya @xuanyuanking 
   Appreciate the next round of review. Thanks. As this was proposed in a couple of months ago and we all agree about the direction, I'll make sure this is included in 3.1.0. 
   (The goal is to merge this in this week, before the branch cut. That said, I'll probably merge without waiting days once my comments are addressed and there's no major concern from others in between.)


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-701361361






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



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


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #29729:
URL: https://github.com/apache/spark/pull/29729#discussion_r489221735



##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -213,64 +187,68 @@ private[kafka010] class KafkaOffsetReader(
       assert(partitions.asScala == partitionTimestamps.keySet,
         "If starting/endingOffsetsByTimestamp contains specific offsets, you must specify all " +
           s"topics. Specified: ${partitionTimestamps.keySet} Assigned: ${partitions.asScala}")
-      logDebug(s"Partitions assigned to consumer: $partitions. Seeking to $partitionTimestamps")
+      logDebug(s"Assigned partitions: $partitions. Seeking to $partitionTimestamps")
     }
 
     val fnRetrievePartitionOffsets: ju.Set[TopicPartition] => Map[TopicPartition, Long] = { _ => {
-        val converted = partitionTimestamps.map { case (tp, timestamp) =>
-          tp -> java.lang.Long.valueOf(timestamp)
+        val listOffsetsParams = partitionTimestamps.map { case (tp, timestamp) =>
+          tp -> OffsetSpec.forTimestamp(timestamp)
         }.asJava
+        admin.listOffsets(listOffsetsParams, listOffsetsOptions).all().get().asScala.map {
+          case (tp, offsetSpec) =>
+            if (failsOnNoMatchingOffset) {
+              assert(offsetSpec.offset() != OffsetFetchResponse.INVALID_OFFSET, "No offset " +
+                s"matched from request of topic-partition $tp and timestamp " +
+                s"${partitionTimestamps(tp)}.")
+            }
 
-        val offsetForTime: ju.Map[TopicPartition, OffsetAndTimestamp] =
-          consumer.offsetsForTimes(converted)
-
-        offsetForTime.asScala.map { case (tp, offsetAndTimestamp) =>
-          if (failsOnNoMatchingOffset) {
-            assert(offsetAndTimestamp != null, "No offset matched from request of " +
-              s"topic-partition $tp and timestamp ${partitionTimestamps(tp)}.")
-          }
-
-          if (offsetAndTimestamp == null) {
-            tp -> KafkaOffsetRangeLimit.LATEST
-          } else {
-            tp -> offsetAndTimestamp.offset()
-          }
+            if (offsetSpec.offset() == OffsetFetchResponse.INVALID_OFFSET) {
+              tp -> KafkaOffsetRangeLimit.LATEST
+            } else {
+              tp -> offsetSpec.offset()
+            }
         }.toMap
       }
     }
 
-    val fnAssertFetchedOffsets: Map[TopicPartition, Long] => Unit = { _ => }
-
-    fetchSpecificOffsets0(fnAssertParametersWithPartitions, fnRetrievePartitionOffsets,
-      fnAssertFetchedOffsets)
+    fetchSpecificOffsets0(fnAssertParametersWithPartitions, fnRetrievePartitionOffsets)
   }
 
   private def fetchSpecificOffsets0(
       fnAssertParametersWithPartitions: ju.Set[TopicPartition] => Unit,
-      fnRetrievePartitionOffsets: ju.Set[TopicPartition] => Map[TopicPartition, Long],
-      fnAssertFetchedOffsets: Map[TopicPartition, Long] => Unit): KafkaSourceOffset = {
+      fnRetrievePartitionOffsets: ju.Set[TopicPartition] => Map[TopicPartition, Long]

Review comment:
       It mustn't be unless the code you've pinpointed can fire up somehow: https://github.com/apache/spark/pull/29729#discussion_r488831782




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



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


[GitHub] [spark] zsxwing commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
zsxwing commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-709391739


   Sorry for the delay. By the way, I forgot to ask one question. Does AdminClient use the same RPC requests as Kafka consumer? If not, are the requests sent by AdminClient compatible with All Kafka clusters since 0.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.

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



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


[GitHub] [spark] zsxwing commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
zsxwing commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-696862302


   @gaborgsomogyi Thanks for the explanation. So the Kafka group based authorization does nothing when fetching the data? And after this change, the users need to change their Kafka cluster security settings to allow client id authorization?


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



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


[GitHub] [spark] HeartSaVioR commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-735020039


   I see there's no major change in both `KafkaOffsetReaderConsumer` and `KafkaOffsetReaderAdmin`. For this case I'd be OK to refactor here as well, though I'd just change `KafkaOffserReader` to trait and have a companion object `KafkaOffsetReaderConsumer` which provides implementation depending on the config. We no longer need to require wrapper then.


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-736116493






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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-733798910






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



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


[GitHub] [spark] SparkQA commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-692596400


   **[Test build #128708 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128708/testReport)** for PR 29729 at commit [`54a09f6`](https://github.com/apache/spark/commit/54a09f69a9e6e9e1039f0f913e2c34accca600d8).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


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



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


[GitHub] [spark] gaborgsomogyi commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-701424151


   There is still an issue w/ Scala 2.13, fixing...
   ```
   Error: ] /home/runner/work/spark/spark/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala:306: type mismatch;
    found   : scala.collection.mutable.ArrayBuffer[(org.apache.kafka.common.TopicPartition, Long, Long)]
    required: Seq[(org.apache.kafka.common.TopicPartition, Long, Long)]
   ```
   


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



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


[GitHub] [spark] SparkQA commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-732028663


   **[Test build #131543 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131543/testReport)** for PR 29729 at commit [`1252c4f`](https://github.com/apache/spark/commit/1252c4f25f541141651a3b01d9767e6b2c367d0e).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


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



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


[GitHub] [spark] HeartSaVioR commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-713554471


   My suggestion assumes the case when AdminClient requires Kafka 1.x. If AdminClient (at least the list of methods what we call) works with Kafka 0.11 then I'm happy to support it.
   
   @zsxwing Looks like version compatibility is also addressed. What's your take on this PR as of now?


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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-732755010






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



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


[GitHub] [spark] SparkQA commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-735991709


   **[Test build #132005 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/132005/testReport)** for PR 29729 at commit [`314299f`](https://github.com/apache/spark/commit/314299f0d9cc737b1c539822b558bd887aeb8b14).


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



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


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #29729:
URL: https://github.com/apache/spark/pull/29729#discussion_r499568889



##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -46,39 +49,40 @@ import org.apache.spark.util.{UninterruptibleThread, UninterruptibleThreadRunner
 private[kafka010] class KafkaOffsetReader(
     consumerStrategy: ConsumerStrategy,
     val driverKafkaParams: ju.Map[String, Object],
-    readerOptions: CaseInsensitiveMap[String],
-    driverGroupIdPrefix: String) extends Logging {
+    readerOptions: CaseInsensitiveMap[String]) extends Logging {
 
   /**
-   * [[UninterruptibleThreadRunner]] ensures that all [[KafkaConsumer]] communication called in an
+   * [[UninterruptibleThreadRunner]] ensures that all Kafka communication called in an
    * [[UninterruptibleThread]]. In the case of streaming queries, we are already running in an
    * [[UninterruptibleThread]], however for batch mode this is not the case.
    */
   val uninterruptibleThreadRunner = new UninterruptibleThreadRunner("Kafka Offset Reader")
 
-  /**
-   * Place [[groupId]] and [[nextId]] here so that they are initialized before any consumer is
-   * created -- see SPARK-19564.
-   */
-  private var groupId: String = null
-  private var nextId = 0
-
-  /**
-   * A KafkaConsumer used in the driver to query the latest Kafka offsets. This only queries the
-   * offsets and never commits them.
-   */
-  @volatile protected var _consumer: Consumer[Array[Byte], Array[Byte]] = null
+  @volatile protected var _admin: Admin = null
 
-  protected def consumer: Consumer[Array[Byte], Array[Byte]] = synchronized {
+  protected def admin: Admin = synchronized {
     assert(Thread.currentThread().isInstanceOf[UninterruptibleThread])
-    if (_consumer == null) {
-      val newKafkaParams = new ju.HashMap[String, Object](driverKafkaParams)
-      if (driverKafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG) == null) {
-        newKafkaParams.put(ConsumerConfig.GROUP_ID_CONFIG, nextGroupId())

Review comment:
       There were another discussions touching this area but it's scattered so making a summary here. In driver the `group.id` is not used anymore but in the executors we still do. That's the reason why the configuration possibility is still there. The end user impact is that `group.id` authorization is not working anymore (as written a good alternative is topic based authorization). It's breaking.




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



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


[GitHub] [spark] viirya commented on a change in pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #29729:
URL: https://github.com/apache/spark/pull/29729#discussion_r499874995



##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -563,12 +526,6 @@ private[kafka010] class KafkaOffsetReader(
         && !Thread.currentThread().isInterrupted) {
         Thread.currentThread match {
           case ut: UninterruptibleThread =>
-            // "KafkaConsumer.poll" may hang forever if the thread is interrupted (E.g., the query
-            // is stopped)(KAFKA-1894). Hence, we just make sure we don't interrupt it.
-            //
-            // If the broker addresses are wrong, or Kafka cluster is down, "KafkaConsumer.poll" may
-            // hang forever as well. This cannot be resolved in KafkaSource until Kafka fixes the
-            // issue.

Review comment:
       Ah, I see. Good to know.




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



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


[GitHub] [spark] SparkQA removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-691110067






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



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


[GitHub] [spark] SparkQA commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-733789888


   **[Test build #131770 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131770/testReport)** for PR 29729 at commit [`cd653e6`](https://github.com/apache/spark/commit/cd653e600165866599c2c63c4a9a1f0a96aac139).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


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



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


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #29729:
URL: https://github.com/apache/spark/pull/29729#discussion_r487692224



##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -105,34 +102,16 @@ private[kafka010] class KafkaOffsetReader(
     minPartitions.map(_ > numTopicPartitions).getOrElse(false)
   }
 
-  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 = {
-    if (_consumer != null) uninterruptibleThreadRunner.runUninterruptibly { stopConsumer() }
+    if (_admin != null) uninterruptibleThreadRunner.runUninterruptibly { stopAdmin() }

Review comment:
       I've already checked this deeply w/ the Kafka guys and this statement is the result of that discussion. I agree w/ the direction but I wanted to file another PR not to mix intentions. I trust the Kafka guys but I would like to execute cluster tests where I want to simulate `AdminClient` interruptions in mass.




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



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


[GitHub] [spark] SparkQA commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-731983334


   **[Test build #131540 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131540/testReport)** for PR 29729 at commit [`1252c4f`](https://github.com/apache/spark/commit/1252c4f25f541141651a3b01d9767e6b2c367d0e).


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



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


[GitHub] [spark] SparkQA removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-691854890


   **[Test build #128622 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128622/testReport)** for PR 29729 at commit [`1608b72`](https://github.com/apache/spark/commit/1608b7240efba205004191fba5feedf75e7979df).


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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-704150056






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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-701494756






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



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


[GitHub] [spark] gaborgsomogyi edited a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi edited a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-701271287


   Here is the result of the deepdive with the Kafka guys: https://gist.github.com/gaborgsomogyi/06361fa4d96055a5963d133577aae4ab
   I'm going to write the extract into the migration guide.


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



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


[GitHub] [spark] SparkQA commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-692130237


   **[Test build #128658 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128658/testReport)** for PR 29729 at commit [`8261259`](https://github.com/apache/spark/commit/8261259103c3af8de1502b77f6580c020631187c).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-691106987






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



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


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #29729:
URL: https://github.com/apache/spark/pull/29729#discussion_r487742388



##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -213,64 +180,70 @@ private[kafka010] class KafkaOffsetReader(
       assert(partitions.asScala == partitionTimestamps.keySet,
         "If starting/endingOffsetsByTimestamp contains specific offsets, you must specify all " +
           s"topics. Specified: ${partitionTimestamps.keySet} Assigned: ${partitions.asScala}")
-      logDebug(s"Partitions assigned to consumer: $partitions. Seeking to $partitionTimestamps")
+      logDebug(s"Assigned partitions: $partitions. Seeking to $partitionTimestamps")
     }
 
     val fnRetrievePartitionOffsets: ju.Set[TopicPartition] => Map[TopicPartition, Long] = { _ => {
-        val converted = partitionTimestamps.map { case (tp, timestamp) =>
-          tp -> java.lang.Long.valueOf(timestamp)
+        val listOffsetsParams = partitionTimestamps.map { p =>
+          p._1 -> OffsetSpec.forTimestamp(p._2)
         }.asJava
+        admin.listOffsets(listOffsetsParams, listOffsetsOptions()).all().get().asScala.map {
+          case (tp, offsetSpec) =>
+            if (failsOnNoMatchingOffset) {
+              assert(offsetSpec.offset() != OffsetFetchResponse.INVALID_OFFSET, "No offset " +
+                s"matched from request of topic-partition $tp and timestamp " +
+                s"${partitionTimestamps(tp)}.")
+            }
 
-        val offsetForTime: ju.Map[TopicPartition, OffsetAndTimestamp] =
-          consumer.offsetsForTimes(converted)
-
-        offsetForTime.asScala.map { case (tp, offsetAndTimestamp) =>
-          if (failsOnNoMatchingOffset) {
-            assert(offsetAndTimestamp != null, "No offset matched from request of " +
-              s"topic-partition $tp and timestamp ${partitionTimestamps(tp)}.")
-          }
-
-          if (offsetAndTimestamp == null) {
-            tp -> KafkaOffsetRangeLimit.LATEST
-          } else {
-            tp -> offsetAndTimestamp.offset()
-          }
+            if (offsetSpec == null) {

Review comment:
       NPE is nice catch, re-evaluate this code part...




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



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


[GitHub] [spark] SparkQA removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-731983334


   **[Test build #131540 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131540/testReport)** for PR 29729 at commit [`1252c4f`](https://github.com/apache/spark/commit/1252c4f25f541141651a3b01d9767e6b2c367d0e).


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



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


[GitHub] [spark] SparkQA commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-733639162


   **[Test build #131770 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131770/testReport)** for PR 29729 at commit [`cd653e6`](https://github.com/apache/spark/commit/cd653e600165866599c2c63c4a9a1f0a96aac139).


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



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


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #29729:
URL: https://github.com/apache/spark/pull/29729#discussion_r487693574



##########
File path: external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
##########
@@ -689,57 +691,6 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase {
     )
   }
 
-  test("allow group.id prefix") {

Review comment:
       Executors still take advantage of this. How do you mean remove?




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



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


[GitHub] [spark] SparkQA commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-703693453


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34022/
   


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



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


[GitHub] [spark] SparkQA commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-703651449


   **[Test build #129415 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129415/testReport)** for PR 29729 at commit [`34333b9`](https://github.com/apache/spark/commit/34333b915981e115999a9c0649c6fab607aa4e29).


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



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


[GitHub] [spark] SparkQA commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-703649065


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34020/
   


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



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


[GitHub] [spark] gaborgsomogyi commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-726206068


   Sure, will start to implement it.


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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-691106987






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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-703649108


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/34020/
   Test FAILed.


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



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


[GitHub] [spark] viirya commented on a change in pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #29729:
URL: https://github.com/apache/spark/pull/29729#discussion_r499167330



##########
File path: docs/ss-migration-guide.md
##########
@@ -26,6 +26,19 @@ Note that this migration guide describes the items specific to Structured Stream
 Many items of SQL migration can be applied when migrating Structured Streaming to higher versions.
 Please refer [Migration Guide: SQL, Datasets and DataFrame](sql-migration-guide.html).
 
+## Upgrading from Structured Streaming 3.0 to 3.1
+
+- In Spark 3.0 and below, secure Kafka processing needed the following ACLs from driver perspective:
+  * Topic resource describe operation
+  * Topic resource read operation
+  * Group resource read operation
+
+  Since Spark 3.1, offsets are obtained with `AdminClient` instead of `KafkaConsumer` and now the following ACLs needed from driver perspective:
+  * Topic resource describe operation
+
+  Since `AdminClient` in driver is not connecting to consumer group, `group.id` based authorization will not work anymore (executors never done group based authorization).

Review comment:
       I'm not sure about this point. Can you explain more? What the impact it could be?




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



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


[GitHub] [spark] SparkQA removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-703651449


   **[Test build #129415 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129415/testReport)** for PR 29729 at commit [`34333b9`](https://github.com/apache/spark/commit/34333b915981e115999a9c0649c6fab607aa4e29).


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-691851987






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



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


[GitHub] [spark] gaborgsomogyi commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-709429484


   @zsxwing important question and hard to answer. In order to provide exhaustive result I need to sit together with the Kafka guys and go through all the used APIs. Will take some time...


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



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


[GitHub] [spark] SparkQA removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-733639162


   **[Test build #131770 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131770/testReport)** for PR 29729 at commit [`cd653e6`](https://github.com/apache/spark/commit/cd653e600165866599c2c63c4a9a1f0a96aac139).


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



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


[GitHub] [spark] SparkQA commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-701494575


   **[Test build #129280 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129280/testReport)** for PR 29729 at commit [`bc6fbdc`](https://github.com/apache/spark/commit/bc6fbdc8ad08ed1f4e67eef2607b61cbb46c9313).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


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



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


[GitHub] [spark] gaborgsomogyi commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-701967347


   retest this please


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



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


[GitHub] [spark] zsxwing commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
zsxwing commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-696230769


   Does the AdminClient API have the same permission set of the old consumer API? We saw some users would like to block unknown access, so we added the `group.id` config. Since the AdminClient API doesn't have the `group.id`, do people need to change their security policies? 


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



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


[GitHub] [spark] viirya commented on a change in pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #29729:
URL: https://github.com/apache/spark/pull/29729#discussion_r499167707



##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -46,39 +49,40 @@ import org.apache.spark.util.{UninterruptibleThread, UninterruptibleThreadRunner
 private[kafka010] class KafkaOffsetReader(
     consumerStrategy: ConsumerStrategy,
     val driverKafkaParams: ju.Map[String, Object],
-    readerOptions: CaseInsensitiveMap[String],
-    driverGroupIdPrefix: String) extends Logging {
+    readerOptions: CaseInsensitiveMap[String]) extends Logging {
 
   /**
-   * [[UninterruptibleThreadRunner]] ensures that all [[KafkaConsumer]] communication called in an
+   * [[UninterruptibleThreadRunner]] ensures that all Kafka communication called in an
    * [[UninterruptibleThread]]. In the case of streaming queries, we are already running in an
    * [[UninterruptibleThread]], however for batch mode this is not the case.
    */
   val uninterruptibleThreadRunner = new UninterruptibleThreadRunner("Kafka Offset Reader")
 
-  /**
-   * Place [[groupId]] and [[nextId]] here so that they are initialized before any consumer is
-   * created -- see SPARK-19564.
-   */
-  private var groupId: String = null
-  private var nextId = 0
-
-  /**
-   * A KafkaConsumer used in the driver to query the latest Kafka offsets. This only queries the
-   * offsets and never commits them.
-   */

Review comment:
       Can we leave a similar comment on how the admin queries offsets?




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



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


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #29729:
URL: https://github.com/apache/spark/pull/29729#discussion_r489206957



##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -182,28 +170,14 @@ private[kafka010] class KafkaOffsetReader(
         "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")
+      logDebug(s"Assigned partitions: $partitions. Seeking to $partitionOffsets")
     }
 
     val fnRetrievePartitionOffsets: ju.Set[TopicPartition] => Map[TopicPartition, Long] = { _ =>
       partitionOffsets
     }
 
-    val fnAssertFetchedOffsets: Map[TopicPartition, Long] => Unit = { fetched =>
-      partitionOffsets.foreach {
-        case (tp, off) if off != KafkaOffsetRangeLimit.LATEST &&
-          off != KafkaOffsetRangeLimit.EARLIEST =>
-          if (fetched(tp) != off) {
-            reportDataLoss(

Review comment:
       Yeah, I was not able to create a scenario where the condition fired (see the standalone app). If somebody can, feel free to add and we can re-evaluate this code part.




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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-732049410


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/36146/
   Test FAILed.


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-704166487






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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-736116493






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



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


[GitHub] [spark] gaborgsomogyi commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-712837431


   @HeartSaVioR 
   
   > Personally I wouldn't mind if it's not compatible with 0.10/0.11 as 1.0 is released in Oct 2017 (already 3 years passed), but probably someone would mind if it's not supported in 1.x.
   
   Since we know the facts (`0.11.0.0+` is supported) I would say it's a fair price to eliminate a cluster stop issue (only 1.x support would be different). Naming the connector still `0-10...` is maybe not super elegant but I think renaming it would cause quite some trouble (this was the case last time when `kafka-clients` dependency increased to 2.x or so).
   
   > Btw, while doing exhaustive test is great, we might also want to ask this to the Kafka community, as we could claim again to provide a method on metadata update changing only client side if they confirm KIP-396 cannot be the resolution of issue on replacing poll(0).
   
   We can resurrect this part but I've heard from many sources that the offset fetching must happen from `AdminClient`. This is strategic thing from their perspective (and all the rest is hack). I'm maybe pessimistic but I don't see that metadata update functionality would appear in `Consumer` any time.
   


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



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


[GitHub] [spark] SparkQA commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-701470057


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/33895/
   


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



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


[GitHub] [spark] HeartSaVioR commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-699721072


   @zsxwing @gaborgsomogyi 
   
   Let's talk about the remaining tasks to move this forward. I see the change of security requirement is probably a breaking change, but shouldn't be a blocker (as it's required to fix a real-world issue).
   
   Which would be the best way to notice the change to the end users? I'd propose to add the change on the "SS migration note", and optionally describe the security section on "SQL Kafka integration". Adding it to the release note would be also great as well.
   
   I might probably be missing something, or trying to be too exhaustive. Open to any suggestion. Thanks!


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



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


[GitHub] [spark] SparkQA commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-731996094


   **[Test build #131540 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131540/testReport)** for PR 29729 at commit [`1252c4f`](https://github.com/apache/spark/commit/1252c4f25f541141651a3b01d9767e6b2c367d0e).
    * This patch **fails due to an unknown error code, -9**.
    * This patch merges cleanly.
    * This patch adds no public classes.


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



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


[GitHub] [spark] SparkQA commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-701380650


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/33888/
   


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



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


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #29729:
URL: https://github.com/apache/spark/pull/29729#discussion_r500117329



##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -46,39 +49,40 @@ import org.apache.spark.util.{UninterruptibleThread, UninterruptibleThreadRunner
 private[kafka010] class KafkaOffsetReader(
     consumerStrategy: ConsumerStrategy,
     val driverKafkaParams: ju.Map[String, Object],
-    readerOptions: CaseInsensitiveMap[String],
-    driverGroupIdPrefix: String) extends Logging {
+    readerOptions: CaseInsensitiveMap[String]) extends Logging {
 
   /**
-   * [[UninterruptibleThreadRunner]] ensures that all [[KafkaConsumer]] communication called in an
+   * [[UninterruptibleThreadRunner]] ensures that all Kafka communication called in an
    * [[UninterruptibleThread]]. In the case of streaming queries, we are already running in an
    * [[UninterruptibleThread]], however for batch mode this is not the case.
    */
   val uninterruptibleThreadRunner = new UninterruptibleThreadRunner("Kafka Offset Reader")
 
-  /**
-   * Place [[groupId]] and [[nextId]] here so that they are initialized before any consumer is
-   * created -- see SPARK-19564.
-   */
-  private var groupId: String = null
-  private var nextId = 0
-
-  /**
-   * A KafkaConsumer used in the driver to query the latest Kafka offsets. This only queries the
-   * offsets and never commits them.
-   */

Review comment:
       OK, added the comment but mentioned `KafkaConsumer` why offset commit is important.




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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-732016650






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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-691859471






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



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


[GitHub] [spark] Tagar commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
Tagar commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-699580277


   @HeartSaVioR I concur - we have seen issues with hanging kafka consumers in Spark Streaming applications. This would be great to fix.
   


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-703669704






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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-733833288






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



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


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of poll(long) API

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #29729:
URL: https://github.com/apache/spark/pull/29729#discussion_r487067464



##########
File path: external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
##########
@@ -689,57 +691,6 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase {
     )
   }
 
-  test("allow group.id prefix") {

Review comment:
       No need the tests because driver is not using `KafkaConsumer` => no GroupId needed.




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



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


[GitHub] [spark] HeartSaVioR commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-706655853


   @zsxwing @viirya 
   Would you mind doing another round of reviews? Looks like we only concern about security aspect which is most likely about documentation issue. (As the consensus here looks like supporting this fix instead of keeping this as it is.)


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-701508266


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/33897/
   Test FAILed.


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



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


[GitHub] [spark] Tagar commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
Tagar commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-698396402


   @gaborgsomogyi that's good to know confluent will be the same with Confluent Kafka. I didn't know if this change would break some of the authorization assumptions there. That's great it doesn't. 
   
   Nice to see this improvement in Kafka connectivity. Thanks! 


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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-733798910






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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-701449029






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



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


[GitHub] [spark] HeartSaVioR commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-699721791


   @gaborgsomogyi 
   
   I'm sorry, but if you don't mind, it would be really appreciated if we can verify there's no change on security requirement to use AdminClient. We may want to check both things: 1) security requirements which will be dropped after the change (like group.id) 2) security requirements which will be added after the change


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



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


[GitHub] [spark] SparkQA commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-701496598


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/33897/
   


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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-735997779






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



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


[GitHub] [spark] gaborgsomogyi commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-694154890


   I've just filed SPARK-32910 and started to work on that as we've agreed. Intended to file a PR if this has merged.


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



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


[GitHub] [spark] xuanyuanking edited a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
xuanyuanking edited a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-722946805


   +1 for option 1.
   Personally, I think this PR is good enough and we must fix the hanging issue. For taking care of both the legacy behavior for the group.id auth and bug fix, I think to add a config, and deprecate in future version should be acceptable. It might make the code base messy now, but I also can't find another good way to move forward.


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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-703693488






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



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


[GitHub] [spark] HyukjinKwon commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-691752353


   cc @xuanyuanking too FYI


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-703649098


   Merged build finished. Test FAILed.


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



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


[GitHub] [spark] HeartSaVioR commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-723316632


   Once we pick the option 1 I'd say we should just allow the redundancy for offset reader and consumer strategy (having different classes with all the content copied), so that we can just delete the old one when we drop the legacy behavior.


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-691111838


   Merged build finished. Test FAILed.


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



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


[GitHub] [spark] SparkQA removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-732536951


   **[Test build #131605 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131605/testReport)** for PR 29729 at commit [`1252c4f`](https://github.com/apache/spark/commit/1252c4f25f541141651a3b01d9767e6b2c367d0e).


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



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


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #29729:
URL: https://github.com/apache/spark/pull/29729#discussion_r532589093



##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReaderAdmin.scala
##########
@@ -50,15 +50,16 @@ private[kafka010] class KafkaOffsetReaderAdmin(
     consumerStrategy: ConsumerStrategy,
     override val driverKafkaParams: ju.Map[String, Object],
     readerOptions: CaseInsensitiveMap[String],
-    driverGroupIdPrefix: String)
-  extends KafkaOffsetReaderBase(
-    consumerStrategy,
-    driverKafkaParams,
-    readerOptions,
-    driverGroupIdPrefix) with Logging {
+    driverGroupIdPrefix: String) extends KafkaOffsetReader with Logging {
+
+  private[kafka010] val maxOffsetFetchAttempts =
+    readerOptions.getOrElse(KafkaSourceProvider.FETCH_OFFSET_NUM_RETRY, "3").toInt
+
+  private[kafka010] val offsetFetchAttemptIntervalMs =
+    readerOptions.getOrElse(KafkaSourceProvider.FETCH_OFFSET_RETRY_INTERVAL_MS, "1000").toLong
 
   /**
-   * [[UninterruptibleThreadRunner]] ensures that all [[KafkaConsumer]] communication called in an
+   * [[UninterruptibleThreadRunner]] ensures that all [[Admin]] communication called in an

Review comment:
       This is not related but found a bug in the original commit.

##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReaderConsumer.scala
##########
@@ -126,7 +127,7 @@ private[kafka010] class KafkaOffsetReaderConsumer(
   /**
    * @return The Set of TopicPartitions for a given topic
    */
-  override def fetchTopicPartitions(): Set[TopicPartition] =
+  private def fetchTopicPartitions(): Set[TopicPartition] =

Review comment:
       This is not related but since `fetchTopicPartitions` only used in the consumer based implementation I've made this private and not added as public API in the trait.




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



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


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #29729:
URL: https://github.com/apache/spark/pull/29729#discussion_r499549277



##########
File path: docs/ss-migration-guide.md
##########
@@ -26,6 +26,19 @@ Note that this migration guide describes the items specific to Structured Stream
 Many items of SQL migration can be applied when migrating Structured Streaming to higher versions.
 Please refer [Migration Guide: SQL, Datasets and DataFrame](sql-migration-guide.html).
 
+## Upgrading from Structured Streaming 3.0 to 3.1
+
+- In Spark 3.0 and below, secure Kafka processing needed the following ACLs from driver perspective:
+  * Topic resource describe operation
+  * Topic resource read operation
+  * Group resource read operation
+
+  Since Spark 3.1, offsets are obtained with `AdminClient` instead of `KafkaConsumer` and now the following ACLs needed from driver perspective:
+  * Topic resource describe operation
+
+  Since `AdminClient` in driver is not connecting to consumer group, `group.id` based authorization will not work anymore (executors never done group based authorization).

Review comment:
       Some of the users authorizing based on `group.id` which will not work with this PR (topic based authorization is a good alternative). Nothing more, nothing less. If you can help how to make it more descriptive it's welcome.




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



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


[GitHub] [spark] SparkQA commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-691902088


   **[Test build #128634 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128634/testReport)** for PR 29729 at commit [`030d3e7`](https://github.com/apache/spark/commit/030d3e7ad8a6ac849d4c18ae6a71ca2ca960f8d5).


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



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


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #29729:
URL: https://github.com/apache/spark/pull/29729#discussion_r488469495



##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -305,11 +277,12 @@ private[kafka010] class KafkaOffsetReader(
   def fetchLatestOffsets(
       knownOffsets: Option[PartitionOffsetMap]): PartitionOffsetMap =
     partitionsAssignedToConsumer { partitions => {
-      logDebug("Seeking to the end.")
-
       if (knownOffsets.isEmpty) {
-        consumer.seekToEnd(partitions)
-        partitions.asScala.map(p => p -> consumer.position(p)).toMap
+        val listOffsetsParams = partitions.asScala.map(_ -> OffsetSpec.latest()).toMap.asJava

Review comment:
       Moved up.




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



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


[GitHub] [spark] gaborgsomogyi commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-716492068


   @xuanyuanking Thansk for your efforts making this PR better! Let me react your suggestions:
   > Should we add some guides of the alternative method for the original usage of group.id based authorization? Maybe link the Kafka doc is enough.
   
   The `group.id` based authorization uses Kafka ACLs and when one need to switch to topic based then it's the same feature but another field need to be used. Since the feature is the same I don't feel the need.
   
   > IMO, It would be great to link all the investigations/tests in the PR description.
   
   Updated the doc which is linked.
   


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



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


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #29729:
URL: https://github.com/apache/spark/pull/29729#discussion_r500102479



##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -46,39 +49,40 @@ import org.apache.spark.util.{UninterruptibleThread, UninterruptibleThreadRunner
 private[kafka010] class KafkaOffsetReader(
     consumerStrategy: ConsumerStrategy,
     val driverKafkaParams: ju.Map[String, Object],
-    readerOptions: CaseInsensitiveMap[String],
-    driverGroupIdPrefix: String) extends Logging {
+    readerOptions: CaseInsensitiveMap[String]) extends Logging {
 
   /**
-   * [[UninterruptibleThreadRunner]] ensures that all [[KafkaConsumer]] communication called in an
+   * [[UninterruptibleThreadRunner]] ensures that all Kafka communication called in an
    * [[UninterruptibleThread]]. In the case of streaming queries, we are already running in an
    * [[UninterruptibleThread]], however for batch mode this is not the case.
    */
   val uninterruptibleThreadRunner = new UninterruptibleThreadRunner("Kafka Offset Reader")
 
-  /**
-   * Place [[groupId]] and [[nextId]] here so that they are initialized before any consumer is
-   * created -- see SPARK-19564.
-   */
-  private var groupId: String = null
-  private var nextId = 0
-
-  /**
-   * A KafkaConsumer used in the driver to query the latest Kafka offsets. This only queries the
-   * offsets and never commits them.
-   */
-  @volatile protected var _consumer: Consumer[Array[Byte], Array[Byte]] = null
+  @volatile protected var _admin: Admin = null
 
-  protected def consumer: Consumer[Array[Byte], Array[Byte]] = synchronized {
+  protected def admin: Admin = synchronized {
     assert(Thread.currentThread().isInstanceOf[UninterruptibleThread])
-    if (_consumer == null) {
-      val newKafkaParams = new ju.HashMap[String, Object](driverKafkaParams)
-      if (driverKafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG) == null) {
-        newKafkaParams.put(ConsumerConfig.GROUP_ID_CONFIG, nextGroupId())

Review comment:
       > Is it easy to switch from group.id authorization to topic based authorization?
   
   It's easy, one need to use the topic based ACL on broker.
   
   > Is there possible use case that topic based authorization cannot fully replace group.id authorization?
   
   I can't really come up with such a use-case but this doesn't mean there is no such. But one thing is for sure: with the actual Kafka API after I've tried many other possibilities this one is the best (spent on this net a month or so). The price here is that the `group.id` authorization must be converted to topic based.
   




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



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


[GitHub] [spark] viirya commented on a change in pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #29729:
URL: https://github.com/apache/spark/pull/29729#discussion_r499870369



##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -46,39 +49,40 @@ import org.apache.spark.util.{UninterruptibleThread, UninterruptibleThreadRunner
 private[kafka010] class KafkaOffsetReader(
     consumerStrategy: ConsumerStrategy,
     val driverKafkaParams: ju.Map[String, Object],
-    readerOptions: CaseInsensitiveMap[String],
-    driverGroupIdPrefix: String) extends Logging {
+    readerOptions: CaseInsensitiveMap[String]) extends Logging {
 
   /**
-   * [[UninterruptibleThreadRunner]] ensures that all [[KafkaConsumer]] communication called in an
+   * [[UninterruptibleThreadRunner]] ensures that all Kafka communication called in an
    * [[UninterruptibleThread]]. In the case of streaming queries, we are already running in an
    * [[UninterruptibleThread]], however for batch mode this is not the case.
    */
   val uninterruptibleThreadRunner = new UninterruptibleThreadRunner("Kafka Offset Reader")
 
-  /**
-   * Place [[groupId]] and [[nextId]] here so that they are initialized before any consumer is
-   * created -- see SPARK-19564.
-   */
-  private var groupId: String = null
-  private var nextId = 0
-
-  /**
-   * A KafkaConsumer used in the driver to query the latest Kafka offsets. This only queries the
-   * offsets and never commits them.
-   */
-  @volatile protected var _consumer: Consumer[Array[Byte], Array[Byte]] = null
+  @volatile protected var _admin: Admin = null
 
-  protected def consumer: Consumer[Array[Byte], Array[Byte]] = synchronized {
+  protected def admin: Admin = synchronized {
     assert(Thread.currentThread().isInstanceOf[UninterruptibleThread])
-    if (_consumer == null) {
-      val newKafkaParams = new ju.HashMap[String, Object](driverKafkaParams)
-      if (driverKafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG) == null) {
-        newKafkaParams.put(ConsumerConfig.GROUP_ID_CONFIG, nextGroupId())

Review comment:
       Is it easy to switch from `group.id` authorization to topic based authorization? Is there possible use case that topic based authorization cannot fully replace `group.id` authorization?




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



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


[GitHub] [spark] SparkQA commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-691876668


   **[Test build #128628 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128628/testReport)** for PR 29729 at commit [`ce3f27c`](https://github.com/apache/spark/commit/ce3f27ccd7085a64951ba5dff3c4b55d152e4558).


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



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


[GitHub] [spark] viirya commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
viirya commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-717629594


   Because I don't handle the operation of Kafka cluster, I don't have clear idea how hard to change from group.id based authorization to topic based one from Kafka infra perspective. As I learn recently from our Kafka infra, seems we are using group.id based authorization. So I feel this is a radical change and we don't have other option instead of switching authorization method in Kafka infra. This might be a hurdle for some users to adopt new Spark version 3.1.0 in near future. cc @dbtsai @dongjoon-hyun 


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



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


[GitHub] [spark] HeartSaVioR commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-736473108


   Thanks all for great efforts of reviewing, and thanks @gaborgsomogyi for the great contribution! I merged this to master.


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



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


[GitHub] [spark] SparkQA removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-691876668


   **[Test build #128628 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128628/testReport)** for PR 29729 at commit [`ce3f27c`](https://github.com/apache/spark/commit/ce3f27ccd7085a64951ba5dff3c4b55d152e4558).


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



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


[GitHub] [spark] gaborgsomogyi commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-735947655


   retest this please


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



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


[GitHub] [spark] SparkQA commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-732030549


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/36146/
   


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-703629122






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



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


[GitHub] [spark] gaborgsomogyi commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-723435991


   I think that is the way to go but it's not so simple since strategy classes are API classes. The internal behaviour of them must be subclassed.


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



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


[GitHub] [spark] SparkQA commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-692570887


   **[Test build #128708 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128708/testReport)** for PR 29729 at commit [`54a09f6`](https://github.com/apache/spark/commit/54a09f69a9e6e9e1039f0f913e2c34accca600d8).


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-692571650






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



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


[GitHub] [spark] SparkQA removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-691858525


   **[Test build #128623 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128623/testReport)** for PR 29729 at commit [`7367274`](https://github.com/apache/spark/commit/736727499a0ea5317273b6816004ae251cedc23f).


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



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


[GitHub] [spark] SparkQA commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-691858525


   **[Test build #128623 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128623/testReport)** for PR 29729 at commit [`7367274`](https://github.com/apache/spark/commit/736727499a0ea5317273b6816004ae251cedc23f).


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



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


[GitHub] [spark] SparkQA commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-691854890


   **[Test build #128622 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128622/testReport)** for PR 29729 at commit [`1608b72`](https://github.com/apache/spark/commit/1608b7240efba205004191fba5feedf75e7979df).


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-732595837






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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-691138581






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



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


[GitHub] [spark] HeartSaVioR commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-736470062


   Github Action passed. I haven't found any new review comments except me. I'll merge this now to ensure this to be included in 3.1.0.


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



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


[GitHub] [spark] SparkQA commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-701448720


   **[Test build #129278 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129278/testReport)** for PR 29729 at commit [`bc6fbdc`](https://github.com/apache/spark/commit/bc6fbdc8ad08ed1f4e67eef2607b61cbb46c9313).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


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



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


[GitHub] [spark] HeartSaVioR commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-717622023


   @zsxwing @viirya @xuanyuanking 
   Could you please go through reviewing again? If there's no further comments in a couple of days I'll merge this in. 


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



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


[GitHub] [spark] gaborgsomogyi edited a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi edited a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-733652656


   @zsxwing @viirya @HeartSaVioR @xuanyuanking I've added the asked change and I would like to you ask to have a look please.
   
   Since it's quite a heavy change I'm listing here the main changes:
   * Introduced the asked config flag `spark.sql.streaming.kafka.useDeprecatedOffsetFetching` with default `true`
   * `KafkaOffsetReader` moved to `KafkaOffsetReaderConsumer` as-is
   * Moved the already reviewed `Admin` based offset reader to `KafkaOffsetReaderAdmin` as-is
   * Made `KafkaOffsetReader` as a wrapper to choose between the 2 implementation
   * Re-added the original implementation of `createConsumer` to `ConsumerStrategy` classes
   * Re-added group ID prefix implementation for the old offset fetching way (the tests only added back in the later commit)
   * Added `KafkaMicroBatchV1SourceWithAdminSuite`, `KafkaMicroBatchV2SourceWithAdminSuite`, `KafkaRelationSuiteWithAdminV1 ` and `KafkaRelationSuiteWithAdminV2` to test batch and streaming with the `Admin` based approach
   * Changed `KafkaOffsetReaderSuite` to test both offset fetching way
   * Adapted the `ss-migration-guide.md` to reflect the new behavior
   
   When we agree that the way is good I'm intended to change the PR description.
   


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-691902589


   Merged build finished. Test PASSed.


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



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


[GitHub] [spark] HeartSaVioR commented on a change in pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #29729:
URL: https://github.com/apache/spark/pull/29729#discussion_r487698354



##########
File path: external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
##########
@@ -689,57 +691,6 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase {
     )
   }
 
-  test("allow group.id prefix") {

Review comment:
       Ah OK I missed. We can still remove the configuration for "exact" assign of groupId (`kafka.group.id`).
   
   Btw, given the executor still leverage the config, can the removed tests be used to verify the executor side?




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



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


[GitHub] [spark] SparkQA removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-701969080


   **[Test build #129304 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129304/testReport)** for PR 29729 at commit [`bc6fbdc`](https://github.com/apache/spark/commit/bc6fbdc8ad08ed1f4e67eef2607b61cbb46c9313).


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-691881771


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/128628/
   Test FAILed.


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



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


[GitHub] [spark] SparkQA commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-701471444


   **[Test build #129280 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129280/testReport)** for PR 29729 at commit [`bc6fbdc`](https://github.com/apache/spark/commit/bc6fbdc8ad08ed1f4e67eef2607b61cbb46c9313).


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



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


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of poll(long) API

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #29729:
URL: https://github.com/apache/spark/pull/29729#discussion_r487063789



##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -105,34 +102,16 @@ private[kafka010] class KafkaOffsetReader(
     minPartitions.map(_ > numTopicPartitions).getOrElse(false)
   }
 
-  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 = {
-    if (_consumer != null) uninterruptibleThreadRunner.runUninterruptibly { stopConsumer() }
+    if (_admin != null) uninterruptibleThreadRunner.runUninterruptibly { stopAdmin() }

Review comment:
       I've left `UninterruptibleThreadRunner` for now but I think it can be removed since there is no evidence that `AdminClient` suffers from the same issue.

##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -213,64 +180,70 @@ private[kafka010] class KafkaOffsetReader(
       assert(partitions.asScala == partitionTimestamps.keySet,
         "If starting/endingOffsetsByTimestamp contains specific offsets, you must specify all " +
           s"topics. Specified: ${partitionTimestamps.keySet} Assigned: ${partitions.asScala}")
-      logDebug(s"Partitions assigned to consumer: $partitions. Seeking to $partitionTimestamps")
+      logDebug(s"Assigned partitions: $partitions. Seeking to $partitionTimestamps")
     }
 
     val fnRetrievePartitionOffsets: ju.Set[TopicPartition] => Map[TopicPartition, Long] = { _ => {
-        val converted = partitionTimestamps.map { case (tp, timestamp) =>
-          tp -> java.lang.Long.valueOf(timestamp)
+        val listOffsetsParams = partitionTimestamps.map { p =>
+          p._1 -> OffsetSpec.forTimestamp(p._2)
         }.asJava
+        admin.listOffsets(listOffsetsParams, listOffsetsOptions()).all().get().asScala.map {
+          case (tp, offsetSpec) =>
+            if (failsOnNoMatchingOffset) {
+              assert(offsetSpec.offset() != OffsetFetchResponse.INVALID_OFFSET, "No offset " +
+                s"matched from request of topic-partition $tp and timestamp " +
+                s"${partitionTimestamps(tp)}.")
+            }
 
-        val offsetForTime: ju.Map[TopicPartition, OffsetAndTimestamp] =
-          consumer.offsetsForTimes(converted)
-
-        offsetForTime.asScala.map { case (tp, offsetAndTimestamp) =>
-          if (failsOnNoMatchingOffset) {
-            assert(offsetAndTimestamp != null, "No offset matched from request of " +
-              s"topic-partition $tp and timestamp ${partitionTimestamps(tp)}.")
-          }
-
-          if (offsetAndTimestamp == null) {
-            tp -> KafkaOffsetRangeLimit.LATEST
-          } else {
-            tp -> offsetAndTimestamp.offset()
-          }
+            if (offsetSpec == null) {

Review comment:
       Different API, different result. `offsetSpec` can be null.

##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala
##########
@@ -96,8 +96,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
     val kafkaOffsetReader = new KafkaOffsetReader(
       strategy(caseInsensitiveParameters),
       kafkaParamsForDriver(specifiedKafkaParams),
-      caseInsensitiveParameters,
-      driverGroupIdPrefix = s"$uniqueGroupId-driver")

Review comment:
       GroupId removal in this file since `AdminClient` doesn't require it.

##########
File path: external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
##########
@@ -689,57 +691,6 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase {
     )
   }
 
-  test("allow group.id prefix") {

Review comment:
       No need the tests because driver is not user `KafkaConsumer` => no GroupId needed.

##########
File path: external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala
##########
@@ -465,41 +463,6 @@ abstract class KafkaRelationSuiteBase extends QueryTest with SharedSparkSession
     testBadOptions("subscribePattern" -> "")("pattern to subscribe is empty")
   }
 
-  test("allow group.id prefix") {

Review comment:
       Same here.

##########
File path: external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
##########
@@ -608,7 +608,9 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase {
     // in executors.
     val query = kafka.map(kv => kv._2.toInt).writeStream.foreach(new ForeachWriter[Int] {
       override def open(partitionId: Long, version: Long): Boolean = {
+        // Re-create topic since Kafka auto topic creation is not supported by Spark
         KafkaSourceSuite.globalTestUtils.deleteTopic(topic)
+        KafkaSourceSuite.globalTestUtils.createTopic(topic)

Review comment:
       This needed because we don't use `KafkaConsumer` and `AdminClient` doesn't provide auto topic creation. Please see doc for the rationale.

##########
File path: external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
##########
@@ -689,57 +691,6 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase {
     )
   }
 
-  test("allow group.id prefix") {

Review comment:
       No need the tests because driver is not using `KafkaConsumer` => no GroupId needed.

##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -105,34 +102,16 @@ private[kafka010] class KafkaOffsetReader(
     minPartitions.map(_ > numTopicPartitions).getOrElse(false)
   }
 
-  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 = {
-    if (_consumer != null) uninterruptibleThreadRunner.runUninterruptibly { stopConsumer() }
+    if (_admin != null) uninterruptibleThreadRunner.runUninterruptibly { stopAdmin() }

Review comment:
       I've left `UninterruptibleThreadRunner` for now but I think it can be removed since there is no evidence that `AdminClient` suffers from the same issue.

##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -213,64 +180,70 @@ private[kafka010] class KafkaOffsetReader(
       assert(partitions.asScala == partitionTimestamps.keySet,
         "If starting/endingOffsetsByTimestamp contains specific offsets, you must specify all " +
           s"topics. Specified: ${partitionTimestamps.keySet} Assigned: ${partitions.asScala}")
-      logDebug(s"Partitions assigned to consumer: $partitions. Seeking to $partitionTimestamps")
+      logDebug(s"Assigned partitions: $partitions. Seeking to $partitionTimestamps")
     }
 
     val fnRetrievePartitionOffsets: ju.Set[TopicPartition] => Map[TopicPartition, Long] = { _ => {
-        val converted = partitionTimestamps.map { case (tp, timestamp) =>
-          tp -> java.lang.Long.valueOf(timestamp)
+        val listOffsetsParams = partitionTimestamps.map { p =>
+          p._1 -> OffsetSpec.forTimestamp(p._2)
         }.asJava
+        admin.listOffsets(listOffsetsParams, listOffsetsOptions()).all().get().asScala.map {
+          case (tp, offsetSpec) =>
+            if (failsOnNoMatchingOffset) {
+              assert(offsetSpec.offset() != OffsetFetchResponse.INVALID_OFFSET, "No offset " +
+                s"matched from request of topic-partition $tp and timestamp " +
+                s"${partitionTimestamps(tp)}.")
+            }
 
-        val offsetForTime: ju.Map[TopicPartition, OffsetAndTimestamp] =
-          consumer.offsetsForTimes(converted)
-
-        offsetForTime.asScala.map { case (tp, offsetAndTimestamp) =>
-          if (failsOnNoMatchingOffset) {
-            assert(offsetAndTimestamp != null, "No offset matched from request of " +
-              s"topic-partition $tp and timestamp ${partitionTimestamps(tp)}.")
-          }
-
-          if (offsetAndTimestamp == null) {
-            tp -> KafkaOffsetRangeLimit.LATEST
-          } else {
-            tp -> offsetAndTimestamp.offset()
-          }
+            if (offsetSpec == null) {

Review comment:
       Different API, different result. `offsetSpec` can be null.

##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala
##########
@@ -96,8 +96,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
     val kafkaOffsetReader = new KafkaOffsetReader(
       strategy(caseInsensitiveParameters),
       kafkaParamsForDriver(specifiedKafkaParams),
-      caseInsensitiveParameters,
-      driverGroupIdPrefix = s"$uniqueGroupId-driver")

Review comment:
       GroupId removal in this file since `AdminClient` doesn't require it.

##########
File path: external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
##########
@@ -689,57 +691,6 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase {
     )
   }
 
-  test("allow group.id prefix") {

Review comment:
       No need the tests because driver is not user `KafkaConsumer` => no GroupId needed.

##########
File path: external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala
##########
@@ -465,41 +463,6 @@ abstract class KafkaRelationSuiteBase extends QueryTest with SharedSparkSession
     testBadOptions("subscribePattern" -> "")("pattern to subscribe is empty")
   }
 
-  test("allow group.id prefix") {

Review comment:
       Same here.

##########
File path: external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
##########
@@ -608,7 +608,9 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase {
     // in executors.
     val query = kafka.map(kv => kv._2.toInt).writeStream.foreach(new ForeachWriter[Int] {
       override def open(partitionId: Long, version: Long): Boolean = {
+        // Re-create topic since Kafka auto topic creation is not supported by Spark
         KafkaSourceSuite.globalTestUtils.deleteTopic(topic)
+        KafkaSourceSuite.globalTestUtils.createTopic(topic)

Review comment:
       This needed because we don't use `KafkaConsumer` and `AdminClient` doesn't provide auto topic creation. Please see doc for the rationale.

##########
File path: external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
##########
@@ -689,57 +691,6 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase {
     )
   }
 
-  test("allow group.id prefix") {

Review comment:
       No need the tests because driver is not using `KafkaConsumer` => no GroupId needed.

##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -105,34 +102,16 @@ private[kafka010] class KafkaOffsetReader(
     minPartitions.map(_ > numTopicPartitions).getOrElse(false)
   }
 
-  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 = {
-    if (_consumer != null) uninterruptibleThreadRunner.runUninterruptibly { stopConsumer() }
+    if (_admin != null) uninterruptibleThreadRunner.runUninterruptibly { stopAdmin() }

Review comment:
       I've left `UninterruptibleThreadRunner` for now but I think it can be removed since there is no evidence that `AdminClient` suffers from the same issue.

##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -213,64 +180,70 @@ private[kafka010] class KafkaOffsetReader(
       assert(partitions.asScala == partitionTimestamps.keySet,
         "If starting/endingOffsetsByTimestamp contains specific offsets, you must specify all " +
           s"topics. Specified: ${partitionTimestamps.keySet} Assigned: ${partitions.asScala}")
-      logDebug(s"Partitions assigned to consumer: $partitions. Seeking to $partitionTimestamps")
+      logDebug(s"Assigned partitions: $partitions. Seeking to $partitionTimestamps")
     }
 
     val fnRetrievePartitionOffsets: ju.Set[TopicPartition] => Map[TopicPartition, Long] = { _ => {
-        val converted = partitionTimestamps.map { case (tp, timestamp) =>
-          tp -> java.lang.Long.valueOf(timestamp)
+        val listOffsetsParams = partitionTimestamps.map { p =>
+          p._1 -> OffsetSpec.forTimestamp(p._2)
         }.asJava
+        admin.listOffsets(listOffsetsParams, listOffsetsOptions()).all().get().asScala.map {
+          case (tp, offsetSpec) =>
+            if (failsOnNoMatchingOffset) {
+              assert(offsetSpec.offset() != OffsetFetchResponse.INVALID_OFFSET, "No offset " +
+                s"matched from request of topic-partition $tp and timestamp " +
+                s"${partitionTimestamps(tp)}.")
+            }
 
-        val offsetForTime: ju.Map[TopicPartition, OffsetAndTimestamp] =
-          consumer.offsetsForTimes(converted)
-
-        offsetForTime.asScala.map { case (tp, offsetAndTimestamp) =>
-          if (failsOnNoMatchingOffset) {
-            assert(offsetAndTimestamp != null, "No offset matched from request of " +
-              s"topic-partition $tp and timestamp ${partitionTimestamps(tp)}.")
-          }
-
-          if (offsetAndTimestamp == null) {
-            tp -> KafkaOffsetRangeLimit.LATEST
-          } else {
-            tp -> offsetAndTimestamp.offset()
-          }
+            if (offsetSpec == null) {

Review comment:
       Different API, different result. `offsetSpec` can be null.

##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala
##########
@@ -96,8 +96,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
     val kafkaOffsetReader = new KafkaOffsetReader(
       strategy(caseInsensitiveParameters),
       kafkaParamsForDriver(specifiedKafkaParams),
-      caseInsensitiveParameters,
-      driverGroupIdPrefix = s"$uniqueGroupId-driver")

Review comment:
       GroupId removal in this file since `AdminClient` doesn't require it.

##########
File path: external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
##########
@@ -689,57 +691,6 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase {
     )
   }
 
-  test("allow group.id prefix") {

Review comment:
       No need the tests because driver is not user `KafkaConsumer` => no GroupId needed.

##########
File path: external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala
##########
@@ -465,41 +463,6 @@ abstract class KafkaRelationSuiteBase extends QueryTest with SharedSparkSession
     testBadOptions("subscribePattern" -> "")("pattern to subscribe is empty")
   }
 
-  test("allow group.id prefix") {

Review comment:
       Same here.

##########
File path: external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
##########
@@ -608,7 +608,9 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase {
     // in executors.
     val query = kafka.map(kv => kv._2.toInt).writeStream.foreach(new ForeachWriter[Int] {
       override def open(partitionId: Long, version: Long): Boolean = {
+        // Re-create topic since Kafka auto topic creation is not supported by Spark
         KafkaSourceSuite.globalTestUtils.deleteTopic(topic)
+        KafkaSourceSuite.globalTestUtils.createTopic(topic)

Review comment:
       This needed because we don't use `KafkaConsumer` and `AdminClient` doesn't provide auto topic creation. Please see doc for the rationale.

##########
File path: external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
##########
@@ -689,57 +691,6 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase {
     )
   }
 
-  test("allow group.id prefix") {

Review comment:
       No need the tests because driver is not using `KafkaConsumer` => no GroupId needed.

##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -105,34 +102,16 @@ private[kafka010] class KafkaOffsetReader(
     minPartitions.map(_ > numTopicPartitions).getOrElse(false)
   }
 
-  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 = {
-    if (_consumer != null) uninterruptibleThreadRunner.runUninterruptibly { stopConsumer() }
+    if (_admin != null) uninterruptibleThreadRunner.runUninterruptibly { stopAdmin() }

Review comment:
       I've left `UninterruptibleThreadRunner` for now but I think it can be removed since there is no evidence that `AdminClient` suffers from the same issue.

##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -213,64 +180,70 @@ private[kafka010] class KafkaOffsetReader(
       assert(partitions.asScala == partitionTimestamps.keySet,
         "If starting/endingOffsetsByTimestamp contains specific offsets, you must specify all " +
           s"topics. Specified: ${partitionTimestamps.keySet} Assigned: ${partitions.asScala}")
-      logDebug(s"Partitions assigned to consumer: $partitions. Seeking to $partitionTimestamps")
+      logDebug(s"Assigned partitions: $partitions. Seeking to $partitionTimestamps")
     }
 
     val fnRetrievePartitionOffsets: ju.Set[TopicPartition] => Map[TopicPartition, Long] = { _ => {
-        val converted = partitionTimestamps.map { case (tp, timestamp) =>
-          tp -> java.lang.Long.valueOf(timestamp)
+        val listOffsetsParams = partitionTimestamps.map { p =>
+          p._1 -> OffsetSpec.forTimestamp(p._2)
         }.asJava
+        admin.listOffsets(listOffsetsParams, listOffsetsOptions()).all().get().asScala.map {
+          case (tp, offsetSpec) =>
+            if (failsOnNoMatchingOffset) {
+              assert(offsetSpec.offset() != OffsetFetchResponse.INVALID_OFFSET, "No offset " +
+                s"matched from request of topic-partition $tp and timestamp " +
+                s"${partitionTimestamps(tp)}.")
+            }
 
-        val offsetForTime: ju.Map[TopicPartition, OffsetAndTimestamp] =
-          consumer.offsetsForTimes(converted)
-
-        offsetForTime.asScala.map { case (tp, offsetAndTimestamp) =>
-          if (failsOnNoMatchingOffset) {
-            assert(offsetAndTimestamp != null, "No offset matched from request of " +
-              s"topic-partition $tp and timestamp ${partitionTimestamps(tp)}.")
-          }
-
-          if (offsetAndTimestamp == null) {
-            tp -> KafkaOffsetRangeLimit.LATEST
-          } else {
-            tp -> offsetAndTimestamp.offset()
-          }
+            if (offsetSpec == null) {

Review comment:
       Different API, different result. `offsetSpec` can be null.

##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala
##########
@@ -96,8 +96,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
     val kafkaOffsetReader = new KafkaOffsetReader(
       strategy(caseInsensitiveParameters),
       kafkaParamsForDriver(specifiedKafkaParams),
-      caseInsensitiveParameters,
-      driverGroupIdPrefix = s"$uniqueGroupId-driver")

Review comment:
       GroupId removal in this file since `AdminClient` doesn't require it.

##########
File path: external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
##########
@@ -689,57 +691,6 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase {
     )
   }
 
-  test("allow group.id prefix") {

Review comment:
       No need the tests because driver is not user `KafkaConsumer` => no GroupId needed.

##########
File path: external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala
##########
@@ -465,41 +463,6 @@ abstract class KafkaRelationSuiteBase extends QueryTest with SharedSparkSession
     testBadOptions("subscribePattern" -> "")("pattern to subscribe is empty")
   }
 
-  test("allow group.id prefix") {

Review comment:
       Same here.

##########
File path: external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
##########
@@ -608,7 +608,9 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase {
     // in executors.
     val query = kafka.map(kv => kv._2.toInt).writeStream.foreach(new ForeachWriter[Int] {
       override def open(partitionId: Long, version: Long): Boolean = {
+        // Re-create topic since Kafka auto topic creation is not supported by Spark
         KafkaSourceSuite.globalTestUtils.deleteTopic(topic)
+        KafkaSourceSuite.globalTestUtils.createTopic(topic)

Review comment:
       This needed because we don't use `KafkaConsumer` and `AdminClient` doesn't provide auto topic creation. Please see doc for the rationale.

##########
File path: external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
##########
@@ -689,57 +691,6 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase {
     )
   }
 
-  test("allow group.id prefix") {

Review comment:
       No need the tests because driver is not using `KafkaConsumer` => no GroupId needed.

##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -105,34 +102,16 @@ private[kafka010] class KafkaOffsetReader(
     minPartitions.map(_ > numTopicPartitions).getOrElse(false)
   }
 
-  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 = {
-    if (_consumer != null) uninterruptibleThreadRunner.runUninterruptibly { stopConsumer() }
+    if (_admin != null) uninterruptibleThreadRunner.runUninterruptibly { stopAdmin() }

Review comment:
       I've left `UninterruptibleThreadRunner` for now but I think it can be removed since there is no evidence that `AdminClient` suffers from the same issue.

##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -213,64 +180,70 @@ private[kafka010] class KafkaOffsetReader(
       assert(partitions.asScala == partitionTimestamps.keySet,
         "If starting/endingOffsetsByTimestamp contains specific offsets, you must specify all " +
           s"topics. Specified: ${partitionTimestamps.keySet} Assigned: ${partitions.asScala}")
-      logDebug(s"Partitions assigned to consumer: $partitions. Seeking to $partitionTimestamps")
+      logDebug(s"Assigned partitions: $partitions. Seeking to $partitionTimestamps")
     }
 
     val fnRetrievePartitionOffsets: ju.Set[TopicPartition] => Map[TopicPartition, Long] = { _ => {
-        val converted = partitionTimestamps.map { case (tp, timestamp) =>
-          tp -> java.lang.Long.valueOf(timestamp)
+        val listOffsetsParams = partitionTimestamps.map { p =>
+          p._1 -> OffsetSpec.forTimestamp(p._2)
         }.asJava
+        admin.listOffsets(listOffsetsParams, listOffsetsOptions()).all().get().asScala.map {
+          case (tp, offsetSpec) =>
+            if (failsOnNoMatchingOffset) {
+              assert(offsetSpec.offset() != OffsetFetchResponse.INVALID_OFFSET, "No offset " +
+                s"matched from request of topic-partition $tp and timestamp " +
+                s"${partitionTimestamps(tp)}.")
+            }
 
-        val offsetForTime: ju.Map[TopicPartition, OffsetAndTimestamp] =
-          consumer.offsetsForTimes(converted)
-
-        offsetForTime.asScala.map { case (tp, offsetAndTimestamp) =>
-          if (failsOnNoMatchingOffset) {
-            assert(offsetAndTimestamp != null, "No offset matched from request of " +
-              s"topic-partition $tp and timestamp ${partitionTimestamps(tp)}.")
-          }
-
-          if (offsetAndTimestamp == null) {
-            tp -> KafkaOffsetRangeLimit.LATEST
-          } else {
-            tp -> offsetAndTimestamp.offset()
-          }
+            if (offsetSpec == null) {

Review comment:
       Different API, different result. `offsetSpec` can be null.

##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala
##########
@@ -96,8 +96,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
     val kafkaOffsetReader = new KafkaOffsetReader(
       strategy(caseInsensitiveParameters),
       kafkaParamsForDriver(specifiedKafkaParams),
-      caseInsensitiveParameters,
-      driverGroupIdPrefix = s"$uniqueGroupId-driver")

Review comment:
       GroupId removal in this file since `AdminClient` doesn't require it.

##########
File path: external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
##########
@@ -689,57 +691,6 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase {
     )
   }
 
-  test("allow group.id prefix") {

Review comment:
       No need the tests because driver is not user `KafkaConsumer` => no GroupId needed.

##########
File path: external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala
##########
@@ -465,41 +463,6 @@ abstract class KafkaRelationSuiteBase extends QueryTest with SharedSparkSession
     testBadOptions("subscribePattern" -> "")("pattern to subscribe is empty")
   }
 
-  test("allow group.id prefix") {

Review comment:
       Same here.

##########
File path: external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
##########
@@ -608,7 +608,9 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase {
     // in executors.
     val query = kafka.map(kv => kv._2.toInt).writeStream.foreach(new ForeachWriter[Int] {
       override def open(partitionId: Long, version: Long): Boolean = {
+        // Re-create topic since Kafka auto topic creation is not supported by Spark
         KafkaSourceSuite.globalTestUtils.deleteTopic(topic)
+        KafkaSourceSuite.globalTestUtils.createTopic(topic)

Review comment:
       This needed because we don't use `KafkaConsumer` and `AdminClient` doesn't provide auto topic creation. Please see doc for the rationale.

##########
File path: external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
##########
@@ -689,57 +691,6 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase {
     )
   }
 
-  test("allow group.id prefix") {

Review comment:
       No need the tests because driver is not using `KafkaConsumer` => no GroupId needed.

##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -105,34 +102,16 @@ private[kafka010] class KafkaOffsetReader(
     minPartitions.map(_ > numTopicPartitions).getOrElse(false)
   }
 
-  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 = {
-    if (_consumer != null) uninterruptibleThreadRunner.runUninterruptibly { stopConsumer() }
+    if (_admin != null) uninterruptibleThreadRunner.runUninterruptibly { stopAdmin() }

Review comment:
       I've left `UninterruptibleThreadRunner` for now but I think it can be removed since there is no evidence that `AdminClient` suffers from the same issue.

##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -213,64 +180,70 @@ private[kafka010] class KafkaOffsetReader(
       assert(partitions.asScala == partitionTimestamps.keySet,
         "If starting/endingOffsetsByTimestamp contains specific offsets, you must specify all " +
           s"topics. Specified: ${partitionTimestamps.keySet} Assigned: ${partitions.asScala}")
-      logDebug(s"Partitions assigned to consumer: $partitions. Seeking to $partitionTimestamps")
+      logDebug(s"Assigned partitions: $partitions. Seeking to $partitionTimestamps")
     }
 
     val fnRetrievePartitionOffsets: ju.Set[TopicPartition] => Map[TopicPartition, Long] = { _ => {
-        val converted = partitionTimestamps.map { case (tp, timestamp) =>
-          tp -> java.lang.Long.valueOf(timestamp)
+        val listOffsetsParams = partitionTimestamps.map { p =>
+          p._1 -> OffsetSpec.forTimestamp(p._2)
         }.asJava
+        admin.listOffsets(listOffsetsParams, listOffsetsOptions()).all().get().asScala.map {
+          case (tp, offsetSpec) =>
+            if (failsOnNoMatchingOffset) {
+              assert(offsetSpec.offset() != OffsetFetchResponse.INVALID_OFFSET, "No offset " +
+                s"matched from request of topic-partition $tp and timestamp " +
+                s"${partitionTimestamps(tp)}.")
+            }
 
-        val offsetForTime: ju.Map[TopicPartition, OffsetAndTimestamp] =
-          consumer.offsetsForTimes(converted)
-
-        offsetForTime.asScala.map { case (tp, offsetAndTimestamp) =>
-          if (failsOnNoMatchingOffset) {
-            assert(offsetAndTimestamp != null, "No offset matched from request of " +
-              s"topic-partition $tp and timestamp ${partitionTimestamps(tp)}.")
-          }
-
-          if (offsetAndTimestamp == null) {
-            tp -> KafkaOffsetRangeLimit.LATEST
-          } else {
-            tp -> offsetAndTimestamp.offset()
-          }
+            if (offsetSpec == null) {

Review comment:
       Different API, different result. `offsetSpec` can be null.

##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala
##########
@@ -96,8 +96,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister
     val kafkaOffsetReader = new KafkaOffsetReader(
       strategy(caseInsensitiveParameters),
       kafkaParamsForDriver(specifiedKafkaParams),
-      caseInsensitiveParameters,
-      driverGroupIdPrefix = s"$uniqueGroupId-driver")

Review comment:
       GroupId removal in this file since `AdminClient` doesn't require it.

##########
File path: external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
##########
@@ -689,57 +691,6 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase {
     )
   }
 
-  test("allow group.id prefix") {

Review comment:
       No need the tests because driver is not user `KafkaConsumer` => no GroupId needed.

##########
File path: external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala
##########
@@ -465,41 +463,6 @@ abstract class KafkaRelationSuiteBase extends QueryTest with SharedSparkSession
     testBadOptions("subscribePattern" -> "")("pattern to subscribe is empty")
   }
 
-  test("allow group.id prefix") {

Review comment:
       Same here.

##########
File path: external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
##########
@@ -608,7 +608,9 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase {
     // in executors.
     val query = kafka.map(kv => kv._2.toInt).writeStream.foreach(new ForeachWriter[Int] {
       override def open(partitionId: Long, version: Long): Boolean = {
+        // Re-create topic since Kafka auto topic creation is not supported by Spark
         KafkaSourceSuite.globalTestUtils.deleteTopic(topic)
+        KafkaSourceSuite.globalTestUtils.createTopic(topic)

Review comment:
       This needed because we don't use `KafkaConsumer` and `AdminClient` doesn't provide auto topic creation. Please see doc for the rationale.

##########
File path: external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
##########
@@ -689,57 +691,6 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase {
     )
   }
 
-  test("allow group.id prefix") {

Review comment:
       No need the tests because driver is not using `KafkaConsumer` => no GroupId needed.




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



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


[GitHub] [spark] SparkQA commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-703635017


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34020/
   


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



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


[GitHub] [spark] SparkQA commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-703682089


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34022/
   


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



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


[GitHub] [spark] gaborgsomogyi commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-698211449


   @Tagar not sure what you mean when you say `this`.
   ACL possibility is available in the vanilla Kafka too: https://kafka.apache.org/documentation/#security_authz
   


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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-704166487






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



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


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #29729:
URL: https://github.com/apache/spark/pull/29729#discussion_r499569640



##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -563,12 +526,6 @@ private[kafka010] class KafkaOffsetReader(
         && !Thread.currentThread().isInterrupted) {
         Thread.currentThread match {
           case ut: UninterruptibleThread =>
-            // "KafkaConsumer.poll" may hang forever if the thread is interrupted (E.g., the query
-            // is stopped)(KAFKA-1894). Hence, we just make sure we don't interrupt it.
-            //
-            // If the broker addresses are wrong, or Kafka cluster is down, "KafkaConsumer.poll" may
-            // hang forever as well. This cannot be resolved in KafkaSource until Kafka fixes the
-            // issue.

Review comment:
       There is an agreement to remove uninterruptible thread usage here in the next commit.




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



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


[GitHub] [spark] SparkQA commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-701346824


   **[Test build #129271 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129271/testReport)** for PR 29729 at commit [`40031cc`](https://github.com/apache/spark/commit/40031ccaae05fcffc9206729ad742a249f370edf).


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



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


[GitHub] [spark] gaborgsomogyi commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-696621270


   @zsxwing it's not explicitly written but the short answer is that the user needs to migrate the application.
   A little bit more detailed if one wants to authorize with `AdminClient` then `client.id` can be used as alternative (if similar solution is the expectation).
   
   @HeartSaVioR the weird behavior related `assign` is true but I would like to add a correction.
   > One can commit back offests with a consumer and such case group id is used.
   
   Of course this doesn't apply here at all since Spark is not committing any offsets. I've no idea about future but I have a strong feeling this stays like this (I mean executors won't commit back offsets).
   
   As a general thought I'm not super big fan of `group.id` or now `client.id`. It can be considered as a shared secret which is not the top solution of security(though the possibility is there). My personal opinion is to go on with ACLs.
   


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



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


[GitHub] [spark] HeartSaVioR commented on a change in pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #29729:
URL: https://github.com/apache/spark/pull/29729#discussion_r488463637



##########
File path: external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
##########
@@ -689,57 +691,6 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase {
     )
   }
 
-  test("allow group.id prefix") {

Review comment:
       Ah OK, executors also can use fixed group id. My bad. I missed.




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



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


[GitHub] [spark] HeartSaVioR commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-693198878


   Looks OK except the cleanup of unnecessary workarounds for `consumer.poll` as a new commit.
   
   cc. @zsxwing 


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



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


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #29729:
URL: https://github.com/apache/spark/pull/29729#discussion_r488467044



##########
File path: external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala
##########
@@ -465,41 +463,6 @@ abstract class KafkaRelationSuiteBase extends QueryTest with SharedSparkSession
     testBadOptions("subscribePattern" -> "")("pattern to subscribe is empty")
   }
 
-  test("allow group.id prefix") {

Review comment:
       Same reason here, resolving it.




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



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


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #29729:
URL: https://github.com/apache/spark/pull/29729#discussion_r488470443



##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -213,64 +180,70 @@ private[kafka010] class KafkaOffsetReader(
       assert(partitions.asScala == partitionTimestamps.keySet,
         "If starting/endingOffsetsByTimestamp contains specific offsets, you must specify all " +
           s"topics. Specified: ${partitionTimestamps.keySet} Assigned: ${partitions.asScala}")
-      logDebug(s"Partitions assigned to consumer: $partitions. Seeking to $partitionTimestamps")
+      logDebug(s"Assigned partitions: $partitions. Seeking to $partitionTimestamps")
     }
 
     val fnRetrievePartitionOffsets: ju.Set[TopicPartition] => Map[TopicPartition, Long] = { _ => {
-        val converted = partitionTimestamps.map { case (tp, timestamp) =>
-          tp -> java.lang.Long.valueOf(timestamp)
+        val listOffsetsParams = partitionTimestamps.map { p =>
+          p._1 -> OffsetSpec.forTimestamp(p._2)
         }.asJava
+        admin.listOffsets(listOffsetsParams, listOffsetsOptions()).all().get().asScala.map {
+          case (tp, offsetSpec) =>
+            if (failsOnNoMatchingOffset) {
+              assert(offsetSpec.offset() != OffsetFetchResponse.INVALID_OFFSET, "No offset " +
+                s"matched from request of topic-partition $tp and timestamp " +
+                s"${partitionTimestamps(tp)}.")
+            }
 
-        val offsetForTime: ju.Map[TopicPartition, OffsetAndTimestamp] =
-          consumer.offsetsForTimes(converted)
-
-        offsetForTime.asScala.map { case (tp, offsetAndTimestamp) =>
-          if (failsOnNoMatchingOffset) {
-            assert(offsetAndTimestamp != null, "No offset matched from request of " +
-              s"topic-partition $tp and timestamp ${partitionTimestamps(tp)}.")
-          }
-
-          if (offsetAndTimestamp == null) {
-            tp -> KafkaOffsetRangeLimit.LATEST
-          } else {
-            tp -> offsetAndTimestamp.offset()
-          }
+            if (offsetSpec == null) {

Review comment:
       Resolving this, drop some message here and we can re-open if still unclear.




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



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


[GitHub] [spark] HeartSaVioR commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-695899573


   As I commented earlier, I'll wait for a day to see there's any valid concerns, and merge this. Please comment if anyone needs more days to review through. Thanks!
   cc. @zsxwing @xuanyuanking 


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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-732595837






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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-692130438






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



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


[GitHub] [spark] SparkQA commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-704166471


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34047/
   


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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-735826161






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



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


[GitHub] [spark] HeartSaVioR commented on a change in pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #29729:
URL: https://github.com/apache/spark/pull/29729#discussion_r489177227



##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -105,34 +109,16 @@ private[kafka010] class KafkaOffsetReader(
     minPartitions.map(_ > numTopicPartitions).getOrElse(false)
   }
 
-  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 = {
-    if (_consumer != null) uninterruptibleThreadRunner.runUninterruptibly { stopConsumer() }
+    if (_admin != null) uninterruptibleThreadRunner.runUninterruptibly { stopAdmin() }
     uninterruptibleThreadRunner.shutdown()
   }
 
-  /**
-   * @return The Set of TopicPartitions for a given topic
-   */
-  def fetchTopicPartitions(): Set[TopicPartition] = uninterruptibleThreadRunner.runUninterruptibly {
-    assert(Thread.currentThread().isInstanceOf[UninterruptibleThread])

Review comment:
       I'm OK to keep the check, but in the following commit I expect we get rid of using UninterruptibleThread altogether.
   
   https://github.com/apache/spark/pull/29729/files#r488436868




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



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


[GitHub] [spark] SparkQA commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-735786583


   **[Test build #131990 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131990/testReport)** for PR 29729 at commit [`314299f`](https://github.com/apache/spark/commit/314299f0d9cc737b1c539822b558bd887aeb8b14).


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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-691859185


   Merged build finished. Test PASSed.


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



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


[GitHub] [spark] SparkQA commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-691138402


   **[Test build #128572 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128572/testReport)** for PR 29729 at commit [`c17582d`](https://github.com/apache/spark/commit/c17582d2f0bded4840f045f28a994b3cc09461dd).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-732016650






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



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


[GitHub] [spark] HeartSaVioR commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-695899573






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



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


[GitHub] [spark] gaborgsomogyi commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-735355267


   @HeartSaVioR Thanks for the new round of review, it's really a huge change and appreciate your effort!
   I agree on your suggestions and will apply them starting from Monday.


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



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


[GitHub] [spark] SparkQA commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-691887604


   **[Test build #128630 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128630/testReport)** for PR 29729 at commit [`3f82562`](https://github.com/apache/spark/commit/3f825622fff9b251f32c51fdaa1a279ea718b36b).


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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-692130438






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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-735960523






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



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


[GitHub] [spark] SparkQA removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-701346824


   **[Test build #129271 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129271/testReport)** for PR 29729 at commit [`40031cc`](https://github.com/apache/spark/commit/40031ccaae05fcffc9206729ad742a249f370edf).


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-701449029






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



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


[GitHub] [spark] SparkQA removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-735991709


   **[Test build #132005 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/132005/testReport)** for PR 29729 at commit [`314299f`](https://github.com/apache/spark/commit/314299f0d9cc737b1c539822b558bd887aeb8b14).


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



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


[GitHub] [spark] gaborgsomogyi commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of poll(long) API

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-691138952






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



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


[GitHub] [spark] SparkQA removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-735786583


   **[Test build #131990 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131990/testReport)** for PR 29729 at commit [`314299f`](https://github.com/apache/spark/commit/314299f0d9cc737b1c539822b558bd887aeb8b14).


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



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


[GitHub] [spark] SparkQA commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-704133484


   **[Test build #129440 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129440/testReport)** for PR 29729 at commit [`d0e4c63`](https://github.com/apache/spark/commit/d0e4c63018beb4e0d36bfea1753c3e5d8f13659e).


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



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


[GitHub] [spark] xuanyuanking commented on a change in pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
xuanyuanking commented on a change in pull request #29729:
URL: https://github.com/apache/spark/pull/29729#discussion_r488842486



##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -213,64 +187,68 @@ private[kafka010] class KafkaOffsetReader(
       assert(partitions.asScala == partitionTimestamps.keySet,
         "If starting/endingOffsetsByTimestamp contains specific offsets, you must specify all " +
           s"topics. Specified: ${partitionTimestamps.keySet} Assigned: ${partitions.asScala}")
-      logDebug(s"Partitions assigned to consumer: $partitions. Seeking to $partitionTimestamps")
+      logDebug(s"Assigned partitions: $partitions. Seeking to $partitionTimestamps")
     }
 
     val fnRetrievePartitionOffsets: ju.Set[TopicPartition] => Map[TopicPartition, Long] = { _ => {
-        val converted = partitionTimestamps.map { case (tp, timestamp) =>
-          tp -> java.lang.Long.valueOf(timestamp)
+        val listOffsetsParams = partitionTimestamps.map { case (tp, timestamp) =>
+          tp -> OffsetSpec.forTimestamp(timestamp)
         }.asJava
+        admin.listOffsets(listOffsetsParams, listOffsetsOptions).all().get().asScala.map {
+          case (tp, offsetSpec) =>
+            if (failsOnNoMatchingOffset) {
+              assert(offsetSpec.offset() != OffsetFetchResponse.INVALID_OFFSET, "No offset " +
+                s"matched from request of topic-partition $tp and timestamp " +
+                s"${partitionTimestamps(tp)}.")
+            }
 
-        val offsetForTime: ju.Map[TopicPartition, OffsetAndTimestamp] =
-          consumer.offsetsForTimes(converted)
-
-        offsetForTime.asScala.map { case (tp, offsetAndTimestamp) =>
-          if (failsOnNoMatchingOffset) {
-            assert(offsetAndTimestamp != null, "No offset matched from request of " +
-              s"topic-partition $tp and timestamp ${partitionTimestamps(tp)}.")
-          }
-
-          if (offsetAndTimestamp == null) {
-            tp -> KafkaOffsetRangeLimit.LATEST
-          } else {
-            tp -> offsetAndTimestamp.offset()
-          }
+            if (offsetSpec.offset() == OffsetFetchResponse.INVALID_OFFSET) {
+              tp -> KafkaOffsetRangeLimit.LATEST
+            } else {
+              tp -> offsetSpec.offset()
+            }
         }.toMap
       }
     }
 
-    val fnAssertFetchedOffsets: Map[TopicPartition, Long] => Unit = { _ => }
-
-    fetchSpecificOffsets0(fnAssertParametersWithPartitions, fnRetrievePartitionOffsets,
-      fnAssertFetchedOffsets)
+    fetchSpecificOffsets0(fnAssertParametersWithPartitions, fnRetrievePartitionOffsets)
   }
 
   private def fetchSpecificOffsets0(
       fnAssertParametersWithPartitions: ju.Set[TopicPartition] => Unit,
-      fnRetrievePartitionOffsets: ju.Set[TopicPartition] => Map[TopicPartition, Long],
-      fnAssertFetchedOffsets: Map[TopicPartition, Long] => Unit): KafkaSourceOffset = {
+      fnRetrievePartitionOffsets: ju.Set[TopicPartition] => Map[TopicPartition, Long]

Review comment:
       Is it a behavior change of removing the `fnAssertFetchedOffsets`?

##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -105,34 +109,16 @@ private[kafka010] class KafkaOffsetReader(
     minPartitions.map(_ > numTopicPartitions).getOrElse(false)
   }
 
-  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 = {
-    if (_consumer != null) uninterruptibleThreadRunner.runUninterruptibly { stopConsumer() }
+    if (_admin != null) uninterruptibleThreadRunner.runUninterruptibly { stopAdmin() }
     uninterruptibleThreadRunner.shutdown()
   }
 
-  /**
-   * @return The Set of TopicPartitions for a given topic
-   */
-  def fetchTopicPartitions(): Set[TopicPartition] = uninterruptibleThreadRunner.runUninterruptibly {
-    assert(Thread.currentThread().isInstanceOf[UninterruptibleThread])

Review comment:
       Shall we keep the UninterruptibleThread checking in https://github.com/apache/spark/pull/29729/files#diff-870db3f661cb22047ad39d67c7c8b22fR139?

##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -182,28 +170,14 @@ private[kafka010] class KafkaOffsetReader(
         "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")
+      logDebug(s"Assigned partitions: $partitions. Seeking to $partitionOffsets")
     }
 
     val fnRetrievePartitionOffsets: ju.Set[TopicPartition] => Map[TopicPartition, Long] = { _ =>
       partitionOffsets
     }
 
-    val fnAssertFetchedOffsets: Map[TopicPartition, Long] => Unit = { fetched =>
-      partitionOffsets.foreach {
-        case (tp, off) if off != KafkaOffsetRangeLimit.LATEST &&
-          off != KafkaOffsetRangeLimit.EARLIEST =>
-          if (fetched(tp) != off) {
-            reportDataLoss(

Review comment:
       The `reportDataLoss` is not called in the new version of `def fetchSpecificOffsets`.




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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-691106987






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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-701508256






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



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


[GitHub] [spark] SparkQA removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-691121536


   **[Test build #128572 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128572/testReport)** for PR 29729 at commit [`c17582d`](https://github.com/apache/spark/commit/c17582d2f0bded4840f045f28a994b3cc09461dd).


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



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


[GitHub] [spark] SparkQA removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-733670191


   **[Test build #131774 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131774/testReport)** for PR 29729 at commit [`f429129`](https://github.com/apache/spark/commit/f42912919432e91e372af4e478bb968995003868).


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-701984943






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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-691856264






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



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


[GitHub] [spark] HeartSaVioR commented on a change in pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #29729:
URL: https://github.com/apache/spark/pull/29729#discussion_r487652330



##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/ConsumerStrategy.scala
##########
@@ -36,31 +37,37 @@ import org.apache.spark.kafka010.KafkaConfigUpdater
  * All three strategies have overloaded constructors that allow you to specify
  * the starting offset for a particular partition.
  */
-private[kafka010] sealed trait ConsumerStrategy {
-  /** Create a [[KafkaConsumer]] and subscribe to topics according to a desired strategy */
-  def createConsumer(kafkaParams: ju.Map[String, Object]): Consumer[Array[Byte], Array[Byte]]
-
-  /**
-   * Updates the parameters with security if needed.
-   * Added a function to hide internals and reduce code duplications because all strategy uses it.
-   */
-  protected def setAuthenticationConfigIfNeeded(kafkaParams: ju.Map[String, Object]) =
-    KafkaConfigUpdater("source", kafkaParams.asScala.toMap)
+private[kafka010] sealed trait ConsumerStrategy extends Logging {
+  /** Creates an [[org.apache.kafka.clients.admin.AdminClient]] */
+  def createAdmin(kafkaParams: ju.Map[String, Object]): Admin = {
+    val updatedKafkaParams = KafkaConfigUpdater("source", kafkaParams.asScala.toMap)
       .setAuthenticationConfigIfNeeded()
       .build()
+    logDebug(s"Admin params: ${KafkaRedactionUtil.redactParams(updatedKafkaParams.asScala.toSeq)}")
+    Admin.create(updatedKafkaParams)
+  }
+
+  /** Returns the assigned or subscribed [[TopicPartition]] */
+  def assignedTopicPartitions(admin: Admin): Set[TopicPartition]
 }

Review comment:
       Looks like we can extract the duplicated code among Strategies, which does retrieving all topic partitions via AdminClient.
   
   ```suggestion
   protected def retrieveAllPartitions(admin: Admin, topics: Set[String]): Set[TopicPartition] = {
       admin.describeTopics(topics.asJava).all().get().asScala.filterNot(_._2.isInternal).flatMap {
         case (topic, topicDescription) =>
           topicDescription.partitions().asScala.map { topicPartitionInfo =>
             val partition = topicPartitionInfo.partition()
             logDebug(s"Partition added: $topic:$partition")
             new TopicPartition(topic, partition)
           }
       }.toSet
     }
   }
   ```

##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -46,41 +49,35 @@ import org.apache.spark.util.{UninterruptibleThread, UninterruptibleThreadRunner
 private[kafka010] class KafkaOffsetReader(
     consumerStrategy: ConsumerStrategy,
     val driverKafkaParams: ju.Map[String, Object],
-    readerOptions: CaseInsensitiveMap[String],
-    driverGroupIdPrefix: String) extends Logging {
+    readerOptions: CaseInsensitiveMap[String]) extends Logging {
 
   /**
-   * [[UninterruptibleThreadRunner]] ensures that all [[KafkaConsumer]] communication called in an
+   * [[UninterruptibleThreadRunner]] ensures that all Kafka communication called in an
    * [[UninterruptibleThread]]. In the case of streaming queries, we are already running in an
    * [[UninterruptibleThread]], however for batch mode this is not the case.
    */
   val uninterruptibleThreadRunner = new UninterruptibleThreadRunner("Kafka Offset Reader")
 
-  /**
-   * Place [[groupId]] and [[nextId]] here so that they are initialized before any consumer is
-   * created -- see SPARK-19564.
-   */
-  private var groupId: String = null
-  private var nextId = 0
-
-  /**
-   * A KafkaConsumer used in the driver to query the latest Kafka offsets. This only queries the
-   * offsets and never commits them.
-   */
-  @volatile protected var _consumer: Consumer[Array[Byte], Array[Byte]] = null
+  @volatile protected var _admin: Admin = null
 
-  protected def consumer: Consumer[Array[Byte], Array[Byte]] = synchronized {
+  protected def admin: Admin = synchronized {
     assert(Thread.currentThread().isInstanceOf[UninterruptibleThread])
-    if (_consumer == null) {
-      val newKafkaParams = new ju.HashMap[String, Object](driverKafkaParams)
-      if (driverKafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG) == null) {
-        newKafkaParams.put(ConsumerConfig.GROUP_ID_CONFIG, nextGroupId())
-      }
-      _consumer = consumerStrategy.createConsumer(newKafkaParams)
+    if (_admin == null) {
+      _admin = consumerStrategy.createAdmin(driverKafkaParams)
     }
-    _consumer
+    _admin
   }
 
+  def isolationLevel(): IsolationLevel = {
+    driverKafkaParams.get(ConsumerConfig.ISOLATION_LEVEL_CONFIG) match {

Review comment:
       It's a bit confusing as null can be assigned to String, and pattern matching is a Scala one where null is not a welcoming one in Scala. How about wrapping it as Option like below:
   
   ```suggestion
       Option(driverKafkaParams.get(ConsumerConfig.ISOLATION_LEVEL_CONFIG)) match {
         case Some(s: String) => IsolationLevel.valueOf(s.toUpperCase(Locale.ROOT))
         case _ => IsolationLevel.valueOf(
           ConsumerConfig.DEFAULT_ISOLATION_LEVEL.toUpperCase(Locale.ROOT))
       }
   ```

##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -46,41 +49,35 @@ import org.apache.spark.util.{UninterruptibleThread, UninterruptibleThreadRunner
 private[kafka010] class KafkaOffsetReader(
     consumerStrategy: ConsumerStrategy,
     val driverKafkaParams: ju.Map[String, Object],
-    readerOptions: CaseInsensitiveMap[String],
-    driverGroupIdPrefix: String) extends Logging {
+    readerOptions: CaseInsensitiveMap[String]) extends Logging {
 
   /**
-   * [[UninterruptibleThreadRunner]] ensures that all [[KafkaConsumer]] communication called in an
+   * [[UninterruptibleThreadRunner]] ensures that all Kafka communication called in an
    * [[UninterruptibleThread]]. In the case of streaming queries, we are already running in an
    * [[UninterruptibleThread]], however for batch mode this is not the case.
    */
   val uninterruptibleThreadRunner = new UninterruptibleThreadRunner("Kafka Offset Reader")
 
-  /**
-   * Place [[groupId]] and [[nextId]] here so that they are initialized before any consumer is
-   * created -- see SPARK-19564.
-   */
-  private var groupId: String = null
-  private var nextId = 0
-
-  /**
-   * A KafkaConsumer used in the driver to query the latest Kafka offsets. This only queries the
-   * offsets and never commits them.
-   */
-  @volatile protected var _consumer: Consumer[Array[Byte], Array[Byte]] = null
+  @volatile protected var _admin: Admin = null
 
-  protected def consumer: Consumer[Array[Byte], Array[Byte]] = synchronized {
+  protected def admin: Admin = synchronized {
     assert(Thread.currentThread().isInstanceOf[UninterruptibleThread])
-    if (_consumer == null) {
-      val newKafkaParams = new ju.HashMap[String, Object](driverKafkaParams)
-      if (driverKafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG) == null) {
-        newKafkaParams.put(ConsumerConfig.GROUP_ID_CONFIG, nextGroupId())
-      }
-      _consumer = consumerStrategy.createConsumer(newKafkaParams)
+    if (_admin == null) {
+      _admin = consumerStrategy.createAdmin(driverKafkaParams)
     }
-    _consumer
+    _admin
   }
 
+  def isolationLevel(): IsolationLevel = {

Review comment:
       Unless we assume the driverKafkaParams can be changed, it can be just `val`, or `lazy val`.

##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -33,10 +36,10 @@ import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap
 import org.apache.spark.util.{UninterruptibleThread, UninterruptibleThreadRunner}
 
 /**
- * This class uses Kafka's own [[KafkaConsumer]] API to read data offsets from Kafka.
+ * This class uses Kafka's its own [[Admin]] API to read data offsets from Kafka.

Review comment:
       nit: Sounds like `Kafka's` and `its` are same and redundant.

##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -305,11 +277,12 @@ private[kafka010] class KafkaOffsetReader(
   def fetchLatestOffsets(

Review comment:
       Looks like all of workarounds applied here are related to the consumer (group) - it's no longer relevant given we change to use Admin. If we can agree to delete workaround for consumer (here and other places as well) we can reduce complexity heavily, but let's hear others' voices on this. 

##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -105,34 +102,16 @@ private[kafka010] class KafkaOffsetReader(
     minPartitions.map(_ > numTopicPartitions).getOrElse(false)
   }
 
-  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 = {
-    if (_consumer != null) uninterruptibleThreadRunner.runUninterruptibly { stopConsumer() }
+    if (_admin != null) uninterruptibleThreadRunner.runUninterruptibly { stopAdmin() }

Review comment:
       It would be nice if we can get rid of that, as it brings complexity - better to double-check with Kafka community.

##########
File path: external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
##########
@@ -689,57 +691,6 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase {
     )
   }
 
-  test("allow group.id prefix") {

Review comment:
       Let's remove them from the doc as well. (There should be configuration as well as explanation around group id.)

##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -213,64 +180,70 @@ private[kafka010] class KafkaOffsetReader(
       assert(partitions.asScala == partitionTimestamps.keySet,
         "If starting/endingOffsetsByTimestamp contains specific offsets, you must specify all " +
           s"topics. Specified: ${partitionTimestamps.keySet} Assigned: ${partitions.asScala}")
-      logDebug(s"Partitions assigned to consumer: $partitions. Seeking to $partitionTimestamps")
+      logDebug(s"Assigned partitions: $partitions. Seeking to $partitionTimestamps")
     }
 
     val fnRetrievePartitionOffsets: ju.Set[TopicPartition] => Map[TopicPartition, Long] = { _ => {
-        val converted = partitionTimestamps.map { case (tp, timestamp) =>
-          tp -> java.lang.Long.valueOf(timestamp)
+        val listOffsetsParams = partitionTimestamps.map { p =>

Review comment:
       I think `case (tp, timestamp)` is better in point of readability.

##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -213,64 +180,70 @@ private[kafka010] class KafkaOffsetReader(
       assert(partitions.asScala == partitionTimestamps.keySet,
         "If starting/endingOffsetsByTimestamp contains specific offsets, you must specify all " +
           s"topics. Specified: ${partitionTimestamps.keySet} Assigned: ${partitions.asScala}")
-      logDebug(s"Partitions assigned to consumer: $partitions. Seeking to $partitionTimestamps")
+      logDebug(s"Assigned partitions: $partitions. Seeking to $partitionTimestamps")
     }
 
     val fnRetrievePartitionOffsets: ju.Set[TopicPartition] => Map[TopicPartition, Long] = { _ => {
-        val converted = partitionTimestamps.map { case (tp, timestamp) =>
-          tp -> java.lang.Long.valueOf(timestamp)
+        val listOffsetsParams = partitionTimestamps.map { p =>
+          p._1 -> OffsetSpec.forTimestamp(p._2)
         }.asJava
+        admin.listOffsets(listOffsetsParams, listOffsetsOptions()).all().get().asScala.map {
+          case (tp, offsetSpec) =>
+            if (failsOnNoMatchingOffset) {
+              assert(offsetSpec.offset() != OffsetFetchResponse.INVALID_OFFSET, "No offset " +
+                s"matched from request of topic-partition $tp and timestamp " +
+                s"${partitionTimestamps(tp)}.")
+            }
 
-        val offsetForTime: ju.Map[TopicPartition, OffsetAndTimestamp] =
-          consumer.offsetsForTimes(converted)
-
-        offsetForTime.asScala.map { case (tp, offsetAndTimestamp) =>
-          if (failsOnNoMatchingOffset) {
-            assert(offsetAndTimestamp != null, "No offset matched from request of " +
-              s"topic-partition $tp and timestamp ${partitionTimestamps(tp)}.")
-          }
-
-          if (offsetAndTimestamp == null) {
-            tp -> KafkaOffsetRangeLimit.LATEST
-          } else {
-            tp -> offsetAndTimestamp.offset()
-          }
+            if (offsetSpec == null) {

Review comment:
       Could you please elaborate when? I'd like to check thoughtfully on the change.

##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -213,64 +180,70 @@ private[kafka010] class KafkaOffsetReader(
       assert(partitions.asScala == partitionTimestamps.keySet,
         "If starting/endingOffsetsByTimestamp contains specific offsets, you must specify all " +
           s"topics. Specified: ${partitionTimestamps.keySet} Assigned: ${partitions.asScala}")
-      logDebug(s"Partitions assigned to consumer: $partitions. Seeking to $partitionTimestamps")
+      logDebug(s"Assigned partitions: $partitions. Seeking to $partitionTimestamps")
     }
 
     val fnRetrievePartitionOffsets: ju.Set[TopicPartition] => Map[TopicPartition, Long] = { _ => {
-        val converted = partitionTimestamps.map { case (tp, timestamp) =>
-          tp -> java.lang.Long.valueOf(timestamp)
+        val listOffsetsParams = partitionTimestamps.map { p =>
+          p._1 -> OffsetSpec.forTimestamp(p._2)
         }.asJava
+        admin.listOffsets(listOffsetsParams, listOffsetsOptions()).all().get().asScala.map {
+          case (tp, offsetSpec) =>
+            if (failsOnNoMatchingOffset) {
+              assert(offsetSpec.offset() != OffsetFetchResponse.INVALID_OFFSET, "No offset " +
+                s"matched from request of topic-partition $tp and timestamp " +
+                s"${partitionTimestamps(tp)}.")
+            }
 
-        val offsetForTime: ju.Map[TopicPartition, OffsetAndTimestamp] =
-          consumer.offsetsForTimes(converted)
-
-        offsetForTime.asScala.map { case (tp, offsetAndTimestamp) =>
-          if (failsOnNoMatchingOffset) {
-            assert(offsetAndTimestamp != null, "No offset matched from request of " +
-              s"topic-partition $tp and timestamp ${partitionTimestamps(tp)}.")
-          }
-
-          if (offsetAndTimestamp == null) {
-            tp -> KafkaOffsetRangeLimit.LATEST
-          } else {
-            tp -> offsetAndTimestamp.offset()
-          }
+            if (offsetSpec == null) {

Review comment:
       And if `offsetSpec` can be null, `offsetSpec.offset()` in below if statement will throw NPE.

##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -447,7 +422,7 @@ private[kafka010] class KafkaOffsetReader(
     }
   }
 
-  private def getSortedExecutorList(): Array[String] = {
+  private def getSortedExecutorList: Array[String] = {

Review comment:
       This looks to be unnecessary change, though it doesn't look to have side effect so semantically OK.

##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -305,11 +277,12 @@ private[kafka010] class KafkaOffsetReader(
   def fetchLatestOffsets(
       knownOffsets: Option[PartitionOffsetMap]): PartitionOffsetMap =
     partitionsAssignedToConsumer { partitions => {
-      logDebug("Seeking to the end.")
-
       if (knownOffsets.isEmpty) {
-        consumer.seekToEnd(partitions)
-        partitions.asScala.map(p => p -> consumer.position(p)).toMap
+        val listOffsetsParams = partitions.asScala.map(_ -> OffsetSpec.latest()).toMap.asJava

Review comment:
       This can be defined earlier to deduplicate.

##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -213,64 +180,70 @@ private[kafka010] class KafkaOffsetReader(
       assert(partitions.asScala == partitionTimestamps.keySet,
         "If starting/endingOffsetsByTimestamp contains specific offsets, you must specify all " +
           s"topics. Specified: ${partitionTimestamps.keySet} Assigned: ${partitions.asScala}")
-      logDebug(s"Partitions assigned to consumer: $partitions. Seeking to $partitionTimestamps")
+      logDebug(s"Assigned partitions: $partitions. Seeking to $partitionTimestamps")
     }
 
     val fnRetrievePartitionOffsets: ju.Set[TopicPartition] => Map[TopicPartition, Long] = { _ => {
-        val converted = partitionTimestamps.map { case (tp, timestamp) =>
-          tp -> java.lang.Long.valueOf(timestamp)
+        val listOffsetsParams = partitionTimestamps.map { p =>
+          p._1 -> OffsetSpec.forTimestamp(p._2)
         }.asJava
+        admin.listOffsets(listOffsetsParams, listOffsetsOptions()).all().get().asScala.map {
+          case (tp, offsetSpec) =>
+            if (failsOnNoMatchingOffset) {
+              assert(offsetSpec.offset() != OffsetFetchResponse.INVALID_OFFSET, "No offset " +
+                s"matched from request of topic-partition $tp and timestamp " +
+                s"${partitionTimestamps(tp)}.")
+            }
 
-        val offsetForTime: ju.Map[TopicPartition, OffsetAndTimestamp] =
-          consumer.offsetsForTimes(converted)
-
-        offsetForTime.asScala.map { case (tp, offsetAndTimestamp) =>
-          if (failsOnNoMatchingOffset) {
-            assert(offsetAndTimestamp != null, "No offset matched from request of " +
-              s"topic-partition $tp and timestamp ${partitionTimestamps(tp)}.")
-          }
-
-          if (offsetAndTimestamp == null) {
-            tp -> KafkaOffsetRangeLimit.LATEST
-          } else {
-            tp -> offsetAndTimestamp.offset()
-          }
+            if (offsetSpec == null) {
+              tp -> KafkaOffsetRangeLimit.LATEST
+            } else {
+              tp -> offsetSpec.offset()
+            }
         }.toMap
       }
     }
 
-    val fnAssertFetchedOffsets: Map[TopicPartition, Long] => Unit = { _ => }
-
-    fetchSpecificOffsets0(fnAssertParametersWithPartitions, fnRetrievePartitionOffsets,
-      fnAssertFetchedOffsets)
+    fetchSpecificOffsets0(fnAssertParametersWithPartitions, fnRetrievePartitionOffsets)
   }
 
   private def fetchSpecificOffsets0(
       fnAssertParametersWithPartitions: ju.Set[TopicPartition] => Unit,
-      fnRetrievePartitionOffsets: ju.Set[TopicPartition] => Map[TopicPartition, Long],
-      fnAssertFetchedOffsets: Map[TopicPartition, Long] => Unit): KafkaSourceOffset = {
+      fnRetrievePartitionOffsets: ju.Set[TopicPartition] => Map[TopicPartition, Long]
+    ): KafkaSourceOffset = {
     val fetched = partitionsAssignedToConsumer {
       partitions => {
         fnAssertParametersWithPartitions(partitions)
 
         val partitionOffsets = fnRetrievePartitionOffsets(partitions)
 
-        partitionOffsets.foreach {
-          case (tp, KafkaOffsetRangeLimit.LATEST) =>
-            consumer.seekToEnd(ju.Arrays.asList(tp))
-          case (tp, KafkaOffsetRangeLimit.EARLIEST) =>
-            consumer.seekToBeginning(ju.Arrays.asList(tp))
-          case (tp, off) => consumer.seek(tp, off)
+        val listOffsetsParams = partitionOffsets.filter { case (_, off) =>
+          off == KafkaOffsetRangeLimit.LATEST || off == KafkaOffsetRangeLimit.EARLIEST
+        }.map { case (tp, off) =>
+          off match {
+            case KafkaOffsetRangeLimit.LATEST =>
+              tp -> OffsetSpec.latest()
+            case KafkaOffsetRangeLimit.EARLIEST =>
+              tp -> OffsetSpec.earliest()
+          }
         }
-
-        partitionOffsets.map {
-          case (tp, _) => tp -> consumer.position(tp)
+        val resolvedPartitionOffsets = admin.listOffsets(listOffsetsParams.asJava,

Review comment:
       Below line looks to be used multiple times (4 times roughly) - worth to extract.
   
   ```
   admin.listOffsets(listOffsetsParams, listOffsetsOptions).all().get().asScala
   .map(result => result._1 -> result._2.offset()).toMap
   ```

##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -46,41 +49,35 @@ import org.apache.spark.util.{UninterruptibleThread, UninterruptibleThreadRunner
 private[kafka010] class KafkaOffsetReader(
     consumerStrategy: ConsumerStrategy,
     val driverKafkaParams: ju.Map[String, Object],
-    readerOptions: CaseInsensitiveMap[String],
-    driverGroupIdPrefix: String) extends Logging {
+    readerOptions: CaseInsensitiveMap[String]) extends Logging {
 
   /**
-   * [[UninterruptibleThreadRunner]] ensures that all [[KafkaConsumer]] communication called in an
+   * [[UninterruptibleThreadRunner]] ensures that all Kafka communication called in an
    * [[UninterruptibleThread]]. In the case of streaming queries, we are already running in an
    * [[UninterruptibleThread]], however for batch mode this is not the case.
    */
   val uninterruptibleThreadRunner = new UninterruptibleThreadRunner("Kafka Offset Reader")
 
-  /**
-   * Place [[groupId]] and [[nextId]] here so that they are initialized before any consumer is
-   * created -- see SPARK-19564.
-   */
-  private var groupId: String = null
-  private var nextId = 0
-
-  /**
-   * A KafkaConsumer used in the driver to query the latest Kafka offsets. This only queries the
-   * offsets and never commits them.
-   */
-  @volatile protected var _consumer: Consumer[Array[Byte], Array[Byte]] = null
+  @volatile protected var _admin: Admin = null
 
-  protected def consumer: Consumer[Array[Byte], Array[Byte]] = synchronized {
+  protected def admin: Admin = synchronized {
     assert(Thread.currentThread().isInstanceOf[UninterruptibleThread])
-    if (_consumer == null) {
-      val newKafkaParams = new ju.HashMap[String, Object](driverKafkaParams)
-      if (driverKafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG) == null) {
-        newKafkaParams.put(ConsumerConfig.GROUP_ID_CONFIG, nextGroupId())
-      }
-      _consumer = consumerStrategy.createConsumer(newKafkaParams)
+    if (_admin == null) {
+      _admin = consumerStrategy.createAdmin(driverKafkaParams)
     }
-    _consumer
+    _admin
   }
 
+  def isolationLevel(): IsolationLevel = {
+    driverKafkaParams.get(ConsumerConfig.ISOLATION_LEVEL_CONFIG) match {
+      case s: String => IsolationLevel.valueOf(s.toUpperCase(Locale.ROOT))
+      case null => IsolationLevel.valueOf(
+        ConsumerConfig.DEFAULT_ISOLATION_LEVEL.toUpperCase(Locale.ROOT))
+    }
+  }
+
+  private def listOffsetsOptions() = new ListOffsetsOptions(isolationLevel())

Review comment:
       Same here; if the instance can be reused, `val` or `lazy val`.




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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-701470103


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/33895/
   Test FAILed.


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



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


[GitHub] [spark] SparkQA removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-691902088


   **[Test build #128634 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128634/testReport)** for PR 29729 at commit [`030d3e7`](https://github.com/apache/spark/commit/030d3e7ad8a6ac849d4c18ae6a71ca2ca960f8d5).


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



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


[GitHub] [spark] HeartSaVioR closed pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
HeartSaVioR closed pull request #29729:
URL: https://github.com/apache/spark/pull/29729


   


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-691106987






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



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


[GitHub] [spark] gaborgsomogyi commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-697198305


   @zsxwing 
   > So the Kafka group based authorization does nothing when fetching the data?
   
   In case of the current Spark code the answer is yes. It's noop from authorization perspective.
   
   > And after this change, the users need to change their Kafka cluster security settings to allow client id authorization?
   
   Yeah, the user must change the security settings but please be aware `client.id` is just a possibility and not working out of the box with vanilla Kafka. Custom authorization code needed on the broker side.
   Just a side note here. Previously I've mentioned that I vote on ACLs. This doesn't mean much since this is just a personal opinion. Which is more important I see similar high level direction in the Kafka community.
   


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



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


[GitHub] [spark] gaborgsomogyi commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-701271287


   Here is the result f the deepdive with the Kafka guys: https://gist.github.com/gaborgsomogyi/06361fa4d96055a5963d133577aae4ab
   I'm going to write the extract into the migration guide.


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



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


[GitHub] [spark] HeartSaVioR commented on a change in pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #29729:
URL: https://github.com/apache/spark/pull/29729#discussion_r487709477



##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/ConsumerStrategy.scala
##########
@@ -36,31 +37,47 @@ import org.apache.spark.kafka010.KafkaConfigUpdater
  * All three strategies have overloaded constructors that allow you to specify
  * the starting offset for a particular partition.
  */
-private[kafka010] sealed trait ConsumerStrategy {
-  /** Create a [[KafkaConsumer]] and subscribe to topics according to a desired strategy */
-  def createConsumer(kafkaParams: ju.Map[String, Object]): Consumer[Array[Byte], Array[Byte]]
-
-  /**
-   * Updates the parameters with security if needed.
-   * Added a function to hide internals and reduce code duplications because all strategy uses it.
-   */
-  protected def setAuthenticationConfigIfNeeded(kafkaParams: ju.Map[String, Object]) =
-    KafkaConfigUpdater("source", kafkaParams.asScala.toMap)
+private[kafka010] sealed trait ConsumerStrategy extends Logging {
+  /** Creates an [[org.apache.kafka.clients.admin.AdminClient]] */
+  def createAdmin(kafkaParams: ju.Map[String, Object]): Admin = {
+    val updatedKafkaParams = KafkaConfigUpdater("source", kafkaParams.asScala.toMap)
       .setAuthenticationConfigIfNeeded()
       .build()
+    logDebug(s"Admin params: ${KafkaRedactionUtil.redactParams(updatedKafkaParams.asScala.toSeq)}")
+    Admin.create(updatedKafkaParams)
+  }
+
+  /** Returns the assigned or subscribed [[TopicPartition]] */
+  def assignedTopicPartitions(admin: Admin): Set[TopicPartition]
+protected def retrieveAllPartitions(admin: Admin, topics: Set[String]): Set[TopicPartition] = {

Review comment:
       Unfortunately my suggestion breaks the indentation - please add an empty line and indent correctly. And most importantly, please deduplicate the code from all strategies using the method 




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



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


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #29729:
URL: https://github.com/apache/spark/pull/29729#discussion_r533134611



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
##########
@@ -1415,6 +1415,15 @@ object SQLConf {
       .booleanConf
       .createWithDefault(true)
 
+  val USE_DEPRECATED_KAFKA_OFFSET_FETCHING =
+    buildConf("spark.sql.streaming.kafka.useDeprecatedOffsetFetching")
+      .internal()
+      .doc("When true, the deprecated Consumer based offset fetching used which could cause " +

Review comment:
       Fixed. Updating the description because I've double checked the generated HTML files too.




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



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


[GitHub] [spark] SparkQA commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-691110067






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



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


[GitHub] [spark] SparkQA commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-691881740


   **[Test build #128628 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128628/testReport)** for PR 29729 at commit [`ce3f27c`](https://github.com/apache/spark/commit/ce3f27ccd7085a64951ba5dff3c4b55d152e4558).
    * This patch **fails to build**.
    * This patch merges cleanly.
    * This patch adds no public classes.


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



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


[GitHub] [spark] SparkQA removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-692109062


   **[Test build #128658 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128658/testReport)** for PR 29729 at commit [`8261259`](https://github.com/apache/spark/commit/8261259103c3af8de1502b77f6580c020631187c).


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



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


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #29729:
URL: https://github.com/apache/spark/pull/29729#discussion_r499562051



##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -46,39 +49,40 @@ import org.apache.spark.util.{UninterruptibleThread, UninterruptibleThreadRunner
 private[kafka010] class KafkaOffsetReader(
     consumerStrategy: ConsumerStrategy,
     val driverKafkaParams: ju.Map[String, Object],
-    readerOptions: CaseInsensitiveMap[String],
-    driverGroupIdPrefix: String) extends Logging {
+    readerOptions: CaseInsensitiveMap[String]) extends Logging {
 
   /**
-   * [[UninterruptibleThreadRunner]] ensures that all [[KafkaConsumer]] communication called in an
+   * [[UninterruptibleThreadRunner]] ensures that all Kafka communication called in an
    * [[UninterruptibleThread]]. In the case of streaming queries, we are already running in an
    * [[UninterruptibleThread]], however for batch mode this is not the case.
    */
   val uninterruptibleThreadRunner = new UninterruptibleThreadRunner("Kafka Offset Reader")
 
-  /**
-   * Place [[groupId]] and [[nextId]] here so that they are initialized before any consumer is
-   * created -- see SPARK-19564.
-   */
-  private var groupId: String = null
-  private var nextId = 0
-
-  /**
-   * A KafkaConsumer used in the driver to query the latest Kafka offsets. This only queries the
-   * offsets and never commits them.
-   */

Review comment:
       `AdminClient` has no functionality to commit offsets but `KafkaConsumer` do. This side-effect is documented here. What do you think we can write here?




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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-691892337


   Merged build finished. Test FAILed.


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



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


[GitHub] [spark] SparkQA commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-691110067






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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-692596682






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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-703649098






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



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


[GitHub] [spark] viirya commented on a change in pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #29729:
URL: https://github.com/apache/spark/pull/29729#discussion_r499874400



##########
File path: docs/ss-migration-guide.md
##########
@@ -26,6 +26,19 @@ Note that this migration guide describes the items specific to Structured Stream
 Many items of SQL migration can be applied when migrating Structured Streaming to higher versions.
 Please refer [Migration Guide: SQL, Datasets and DataFrame](sql-migration-guide.html).
 
+## Upgrading from Structured Streaming 3.0 to 3.1
+
+- In Spark 3.0 and below, secure Kafka processing needed the following ACLs from driver perspective:
+  * Topic resource describe operation
+  * Topic resource read operation
+  * Group resource read operation
+
+  Since Spark 3.1, offsets are obtained with `AdminClient` instead of `KafkaConsumer` and now the following ACLs needed from driver perspective:
+  * Topic resource describe operation
+
+  Since `AdminClient` in driver is not connecting to consumer group, `group.id` based authorization will not work anymore (executors never done group based authorization).

Review comment:
       What it means `executors never done group based authorization`? Seems on executors, we use `group.id` if it is set by users, no?




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



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


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #29729:
URL: https://github.com/apache/spark/pull/29729#discussion_r488436868



##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -105,34 +102,16 @@ private[kafka010] class KafkaOffsetReader(
     minPartitions.map(_ > numTopicPartitions).getOrElse(false)
   }
 
-  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 = {
-    if (_consumer != null) uninterruptibleThreadRunner.runUninterruptibly { stopConsumer() }
+    if (_admin != null) uninterruptibleThreadRunner.runUninterruptibly { stopAdmin() }

Review comment:
       I've created a standalone test app which simulates mass `AdminClient` usage with interruptions: https://github.com/gaborgsomogyi/kafka-admin-interruption
   
   I've put this into a loop and not seen any infinite lock so we can say this has proven Kafka the guys statement. I'm going to remove `UninterruptibleThread` at the end of this PR in a separate commit which is easily revertible (just in case if somebody objects it).




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



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


[GitHub] [spark] SparkQA removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-692570887


   **[Test build #128708 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128708/testReport)** for PR 29729 at commit [`54a09f6`](https://github.com/apache/spark/commit/54a09f69a9e6e9e1039f0f913e2c34accca600d8).


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



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


[GitHub] [spark] gaborgsomogyi commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-731955328


   I would like to see it too. Just seen the discussion related cut date which I tought will be a bit later so this will be my first task to jump on heavily.


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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-691111838






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



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


[GitHub] [spark] HeartSaVioR commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-696425757


   Kafka consumer in executors also use assign, and Kafka checks group id authorization even with assign although group.id is not needed at all.
   
   There's an interesting observation though... Please refer https://github.com/apache/spark/pull/28623#issuecomment-633257746
   
   If you do "assign", the group id authorization is "conditionally" checked according to the usage pattern, and executors don't trigger authorization even it's using "assign" and passes group id. Driver triggers authorization even for "assign". Would we consider this as Kafka's bug? If then is it something we should be aware and have workaround?
   
   Btw, if they're pretty serious about the security, I think it's already insecure if attackers can successfully create AdminClient and request informations about topics which will be done in driver. Attackers will even be able to delete something like topics. They should have secured their Kafka via user based ACL or so & allowed operation set, not via group id.


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-735960523






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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-701508256


   Merged build finished. Test FAILed.


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



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


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #29729:
URL: https://github.com/apache/spark/pull/29729#discussion_r489218487



##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -105,34 +109,16 @@ private[kafka010] class KafkaOffsetReader(
     minPartitions.map(_ > numTopicPartitions).getOrElse(false)
   }
 
-  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 = {
-    if (_consumer != null) uninterruptibleThreadRunner.runUninterruptibly { stopConsumer() }
+    if (_admin != null) uninterruptibleThreadRunner.runUninterruptibly { stopAdmin() }
     uninterruptibleThreadRunner.shutdown()
   }
 
-  /**
-   * @return The Set of TopicPartitions for a given topic
-   */
-  def fetchTopicPartitions(): Set[TopicPartition] = uninterruptibleThreadRunner.runUninterruptibly {
-    assert(Thread.currentThread().isInstanceOf[UninterruptibleThread])

Review comment:
       I agree w/ this way.




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



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


[GitHub] [spark] SparkQA removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-691110067


   **[Test build #128570 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128570/testReport)** for PR 29729 at commit [`6236737`](https://github.com/apache/spark/commit/62367376f7d79390ed26bda233323171b2177373).


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-691138581






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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-691106987






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



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


[GitHub] [spark] gaborgsomogyi commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-736300563


   Hmmmm, jenkins is not lightning fast nowadays.


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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-691902589






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



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


[GitHub] [spark] gaborgsomogyi commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-691900949


   Hmmm, not sure why but my IDE not always recompiles things, sigh.


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



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


[GitHub] [spark] SparkQA removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-701426624


   **[Test build #129278 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129278/testReport)** for PR 29729 at commit [`bc6fbdc`](https://github.com/apache/spark/commit/bc6fbdc8ad08ed1f4e67eef2607b61cbb46c9313).


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-692596682






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



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


[GitHub] [spark] gaborgsomogyi commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-712798812


   @zsxwing @HeartSaVioR 
   We've just had a deepdive to find out the exact version compatibility.
   I've created an extract related all the used API calls: https://gist.github.com/gaborgsomogyi/529523dd38e8fbbb979062c6df230686
   Additionally I've created standlone applications to show that the calls are just working fine: https://github.com/gaborgsomogyi/kafka-admin-backward-compatibility
   
   Final conclusion: `Kafka 2.6.0 AdminClient is compatible with 0.11.0.0+ brokers.`
   


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



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


[GitHub] [spark] SparkQA removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-704133484


   **[Test build #129440 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129440/testReport)** for PR 29729 at commit [`d0e4c63`](https://github.com/apache/spark/commit/d0e4c63018beb4e0d36bfea1753c3e5d8f13659e).


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-691859490


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/128623/
   Test FAILed.


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



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


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #29729:
URL: https://github.com/apache/spark/pull/29729#discussion_r500109389



##########
File path: docs/ss-migration-guide.md
##########
@@ -26,6 +26,19 @@ Note that this migration guide describes the items specific to Structured Stream
 Many items of SQL migration can be applied when migrating Structured Streaming to higher versions.
 Please refer [Migration Guide: SQL, Datasets and DataFrame](sql-migration-guide.html).
 
+## Upgrading from Structured Streaming 3.0 to 3.1
+
+- In Spark 3.0 and below, secure Kafka processing needed the following ACLs from driver perspective:
+  * Topic resource describe operation
+  * Topic resource read operation
+  * Group resource read operation
+
+  Since Spark 3.1, offsets are obtained with `AdminClient` instead of `KafkaConsumer` and now the following ACLs needed from driver perspective:
+  * Topic resource describe operation
+
+  Since `AdminClient` in driver is not connecting to consumer group, `group.id` based authorization will not work anymore (executors never done group based authorization).

Review comment:
       Many users who I've spoken with thought that driver and executors are doing group based authorization which makes sense from far distance. Both cases `KafkaConsumer` used and both cases `group.id` is set. The truth is that executors using `KafkaConsumer` in a special way:
   * `assign` strategy used only
   * Auto offset commit is turned off
   * Manual offset commit not called
   
   Such case `KafkaConsumer` is not joining to any group and not doing any group based authorization (this is true with and without this PR).
   
   Not relevant but worth to mention, providing the possibility to either prefix or override executor side `group.id` still has value from user perspective so it's not modified.
   




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



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


[GitHub] [spark] SparkQA commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-732010128


   **[Test build #131543 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131543/testReport)** for PR 29729 at commit [`1252c4f`](https://github.com/apache/spark/commit/1252c4f25f541141651a3b01d9767e6b2c367d0e).


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



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


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #29729:
URL: https://github.com/apache/spark/pull/29729#discussion_r487684341



##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -46,41 +49,35 @@ import org.apache.spark.util.{UninterruptibleThread, UninterruptibleThreadRunner
 private[kafka010] class KafkaOffsetReader(
     consumerStrategy: ConsumerStrategy,
     val driverKafkaParams: ju.Map[String, Object],
-    readerOptions: CaseInsensitiveMap[String],
-    driverGroupIdPrefix: String) extends Logging {
+    readerOptions: CaseInsensitiveMap[String]) extends Logging {
 
   /**
-   * [[UninterruptibleThreadRunner]] ensures that all [[KafkaConsumer]] communication called in an
+   * [[UninterruptibleThreadRunner]] ensures that all Kafka communication called in an
    * [[UninterruptibleThread]]. In the case of streaming queries, we are already running in an
    * [[UninterruptibleThread]], however for batch mode this is not the case.
    */
   val uninterruptibleThreadRunner = new UninterruptibleThreadRunner("Kafka Offset Reader")
 
-  /**
-   * Place [[groupId]] and [[nextId]] here so that they are initialized before any consumer is
-   * created -- see SPARK-19564.
-   */
-  private var groupId: String = null
-  private var nextId = 0
-
-  /**
-   * A KafkaConsumer used in the driver to query the latest Kafka offsets. This only queries the
-   * offsets and never commits them.
-   */
-  @volatile protected var _consumer: Consumer[Array[Byte], Array[Byte]] = null
+  @volatile protected var _admin: Admin = null
 
-  protected def consumer: Consumer[Array[Byte], Array[Byte]] = synchronized {
+  protected def admin: Admin = synchronized {
     assert(Thread.currentThread().isInstanceOf[UninterruptibleThread])
-    if (_consumer == null) {
-      val newKafkaParams = new ju.HashMap[String, Object](driverKafkaParams)
-      if (driverKafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG) == null) {
-        newKafkaParams.put(ConsumerConfig.GROUP_ID_CONFIG, nextGroupId())
-      }
-      _consumer = consumerStrategy.createConsumer(newKafkaParams)
+    if (_admin == null) {
+      _admin = consumerStrategy.createAdmin(driverKafkaParams)
     }
-    _consumer
+    _admin
   }
 
+  def isolationLevel(): IsolationLevel = {
+    driverKafkaParams.get(ConsumerConfig.ISOLATION_LEVEL_CONFIG) match {

Review comment:
       Such pattern can be found in Spark but I'm fine w/ option too so merged.




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



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


[GitHub] [spark] HeartSaVioR commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-699269552


   Worth noting that the issue is not just occurred in theory, but I've seen the case multiple times around community report, customers, etc. Probably we'd feel better to document the change on security viewpoint (release note as well?) to notice the end users, but I hope the change on security requirement doesn't block resolving "real world" issue.


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



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


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #29729:
URL: https://github.com/apache/spark/pull/29729#discussion_r487686563



##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -46,41 +49,35 @@ import org.apache.spark.util.{UninterruptibleThread, UninterruptibleThreadRunner
 private[kafka010] class KafkaOffsetReader(
     consumerStrategy: ConsumerStrategy,
     val driverKafkaParams: ju.Map[String, Object],
-    readerOptions: CaseInsensitiveMap[String],
-    driverGroupIdPrefix: String) extends Logging {
+    readerOptions: CaseInsensitiveMap[String]) extends Logging {
 
   /**
-   * [[UninterruptibleThreadRunner]] ensures that all [[KafkaConsumer]] communication called in an
+   * [[UninterruptibleThreadRunner]] ensures that all Kafka communication called in an
    * [[UninterruptibleThread]]. In the case of streaming queries, we are already running in an
    * [[UninterruptibleThread]], however for batch mode this is not the case.
    */
   val uninterruptibleThreadRunner = new UninterruptibleThreadRunner("Kafka Offset Reader")
 
-  /**
-   * Place [[groupId]] and [[nextId]] here so that they are initialized before any consumer is
-   * created -- see SPARK-19564.
-   */
-  private var groupId: String = null
-  private var nextId = 0
-
-  /**
-   * A KafkaConsumer used in the driver to query the latest Kafka offsets. This only queries the
-   * offsets and never commits them.
-   */
-  @volatile protected var _consumer: Consumer[Array[Byte], Array[Byte]] = null
+  @volatile protected var _admin: Admin = null
 
-  protected def consumer: Consumer[Array[Byte], Array[Byte]] = synchronized {
+  protected def admin: Admin = synchronized {
     assert(Thread.currentThread().isInstanceOf[UninterruptibleThread])
-    if (_consumer == null) {
-      val newKafkaParams = new ju.HashMap[String, Object](driverKafkaParams)
-      if (driverKafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG) == null) {
-        newKafkaParams.put(ConsumerConfig.GROUP_ID_CONFIG, nextGroupId())
-      }
-      _consumer = consumerStrategy.createConsumer(newKafkaParams)
+    if (_admin == null) {
+      _admin = consumerStrategy.createAdmin(driverKafkaParams)
     }
-    _consumer
+    _admin
   }
 
+  def isolationLevel(): IsolationLevel = {
+    driverKafkaParams.get(ConsumerConfig.ISOLATION_LEVEL_CONFIG) match {
+      case s: String => IsolationLevel.valueOf(s.toUpperCase(Locale.ROOT))
+      case null => IsolationLevel.valueOf(
+        ConsumerConfig.DEFAULT_ISOLATION_LEVEL.toUpperCase(Locale.ROOT))
+    }
+  }
+
+  private def listOffsetsOptions() = new ListOffsetsOptions(isolationLevel())

Review comment:
       Same fix here.




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



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


[GitHub] [spark] HeartSaVioR commented on a change in pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #29729:
URL: https://github.com/apache/spark/pull/29729#discussion_r489181522



##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -182,28 +170,14 @@ private[kafka010] class KafkaOffsetReader(
         "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")
+      logDebug(s"Assigned partitions: $partitions. Seeking to $partitionOffsets")
     }
 
     val fnRetrievePartitionOffsets: ju.Set[TopicPartition] => Map[TopicPartition, Long] = { _ =>
       partitionOffsets
     }
 
-    val fnAssertFetchedOffsets: Map[TopicPartition, Long] => Unit = { fetched =>
-      partitionOffsets.foreach {
-        case (tp, off) if off != KafkaOffsetRangeLimit.LATEST &&
-          off != KafkaOffsetRangeLimit.EARLIEST =>
-          if (fetched(tp) != off) {
-            reportDataLoss(

Review comment:
       Please go through reading the doc in the comment on JIRA issue - https://docs.google.com/document/d/1gAh0pKgZUgyqO2Re3sAy-fdYpe_SxpJ6DkeXE8R1P7E/edit?pli=1#
   
   According to his explanation, the check is never be false regardless of the offset.




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



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


[GitHub] [spark] SparkQA commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-735667576


   **[Test build #131978 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/131978/testReport)** for PR 29729 at commit [`6ed215d`](https://github.com/apache/spark/commit/6ed215da52724b88464a2c33f1e2045959a48e34).


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



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


[GitHub] [spark] SparkQA commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-732005056


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/36143/
   


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



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


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #29729:
URL: https://github.com/apache/spark/pull/29729#discussion_r489212011



##########
File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala
##########
@@ -105,34 +109,16 @@ private[kafka010] class KafkaOffsetReader(
     minPartitions.map(_ > numTopicPartitions).getOrElse(false)
   }
 
-  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 = {
-    if (_consumer != null) uninterruptibleThreadRunner.runUninterruptibly { stopConsumer() }
+    if (_admin != null) uninterruptibleThreadRunner.runUninterruptibly { stopAdmin() }
     uninterruptibleThreadRunner.shutdown()
   }
 
-  /**
-   * @return The Set of TopicPartitions for a given topic
-   */
-  def fetchTopicPartitions(): Set[TopicPartition] = uninterruptibleThreadRunner.runUninterruptibly {
-    assert(Thread.currentThread().isInstanceOf[UninterruptibleThread])

Review comment:
       I've just finished the testing of `UninterruptibleThread` removal from `KafkaOffsetReader`. We can put it here if we agree. Both side we can come up w/ pros and cons.
   
   > get rid of using UninterruptibleThread altogether.
   
   Not sure it's possible or at least not super easy. In executor side it's used because of multiple reasons. Please see: https://github.com/apache/spark/blob/2e3aa2f0232a539346da3df8a20cd8e7c2b7dd4f/core/src/main/scala/org/apache/spark/executor/Executor.scala#L101-L104
   




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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-691106987






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



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


[GitHub] [spark] SparkQA commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-701426624


   **[Test build #129278 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129278/testReport)** for PR 29729 at commit [`bc6fbdc`](https://github.com/apache/spark/commit/bc6fbdc8ad08ed1f4e67eef2607b61cbb46c9313).


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



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


[GitHub] [spark] SparkQA removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-701471444


   **[Test build #129280 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129280/testReport)** for PR 29729 at commit [`bc6fbdc`](https://github.com/apache/spark/commit/bc6fbdc8ad08ed1f4e67eef2607b61cbb46c9313).


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



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


[GitHub] [spark] SparkQA removed a comment on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-703610985


   **[Test build #129413 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129413/testReport)** for PR 29729 at commit [`34333b9`](https://github.com/apache/spark/commit/34333b915981e115999a9c0649c6fab607aa4e29).


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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29729: [SPARK-32032][SS] Avoid infinite wait in driver because of KafkaConsumer.poll(long) API

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29729:
URL: https://github.com/apache/spark/pull/29729#issuecomment-691888256






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



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