Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
5 changes: 3 additions & 2 deletions docs/structured-streaming-kafka-integration.md
Original file line number Diff line number Diff line change
Expand Up @@ -440,9 +440,10 @@ The following configurations are optional:
<tr>
<td>kafkaConsumer.pollTimeoutMs</td>
<td>long</td>
<td>512</td>
<td>120000</td>
<td>streaming and batch</td>
<td>The timeout in milliseconds to poll data from Kafka in executors.</td>
<td>The timeout in milliseconds to poll data from Kafka in executors. When not defined it falls
back to <code>spark.network.timeout</code>.</td>
</tr>
<tr>
<td>fetchOffset.numRetries</td>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ package org.apache.spark.sql.kafka010.consumer

import java.{util => ju}
import java.io.Closeable
import java.time.Duration
import java.util.concurrent.TimeoutException

import scala.collection.JavaConverters._
Expand Down Expand Up @@ -73,7 +74,7 @@ private[kafka010] class InternalKafkaConsumer(

// Seek to the offset because we may call seekToBeginning or seekToEnd before this.
seek(offset)
val p = consumer.poll(pollTimeoutMs)
val p = consumer.poll(Duration.ofMillis(pollTimeoutMs))
val r = p.records(topicPartition)
logDebug(s"Polled $groupId ${p.partitions()} ${r.size}")
val offsetAfterPoll = consumer.position(topicPartition)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
package org.apache.spark.streaming.kafka010

import java.{util => ju}
import java.time.Duration

import scala.collection.JavaConverters._

Expand Down Expand Up @@ -203,7 +204,7 @@ private[kafka010] class InternalKafkaConsumer[K, V](
}

private def poll(timeout: Long): Unit = {
val p = consumer.poll(timeout)
val p = consumer.poll(Duration.ofMillis(timeout))
val r = p.records(topicPartition)
logDebug(s"Polled ${p.partitions()} ${r.size}")
buffer = r.listIterator
Expand Down