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
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ package org.apache.spark.sql.kafka010

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 @@ -71,7 +72,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.sql.kafka010

import java.{util => ju}
import java.time.{Duration => jDuration}
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This will avoid conflicts with import scala.concurrent.duration.Duration at line 27.

import java.util.concurrent.Executors

import scala.collection.JavaConverters._
Expand Down Expand Up @@ -116,7 +117,7 @@ private[kafka010] class KafkaOffsetReader(
def fetchTopicPartitions(): Set[TopicPartition] = runUninterruptibly {
assert(Thread.currentThread().isInstanceOf[UninterruptibleThread])
// Poll to get the latest assigned partitions
consumer.poll(0)
consumer.poll(jDuration.ZERO)
val partitions = consumer.assignment()
consumer.pause(partitions)
partitions.asScala.toSet
Expand Down Expand Up @@ -379,7 +380,7 @@ private[kafka010] class KafkaOffsetReader(

withRetriesWithoutInterrupt {
// Poll to get the latest assigned partitions
consumer.poll(0)
consumer.poll(jDuration.ZERO)
val partitions = consumer.assignment()

if (!fetchingEarliestOffset) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ package org.apache.spark.sql.kafka010
import java.io.{File, IOException}
import java.net.{InetAddress, InetSocketAddress}
import java.nio.charset.StandardCharsets
import java.time.Duration
import java.util.{Collections, Properties, UUID}
import java.util.concurrent.TimeUnit
import javax.security.auth.login.Configuration
Expand Down Expand Up @@ -455,7 +456,7 @@ class KafkaTestUtils(
val kc = new KafkaConsumer[String, String](consumerConfiguration)
logInfo("Created consumer to get earliest offsets")
kc.subscribe(topics.asJavaCollection)
kc.poll(0)
kc.poll(Duration.ZERO)
val partitions = kc.assignment()
kc.pause(partitions)
kc.seekToBeginning(partitions)
Expand All @@ -469,7 +470,7 @@ class KafkaTestUtils(
val kc = new KafkaConsumer[String, String](consumerConfiguration)
logInfo("Created consumer to get latest offsets")
kc.subscribe(topics.asJavaCollection)
kc.poll(0)
kc.poll(Duration.ZERO)
val partitions = kc.assignment()
kc.pause(partitions)
kc.seekToEnd(partitions)
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.{lang => jl, util => ju}
import java.time.Duration
import java.util.Locale

import scala.collection.JavaConverters._
Expand Down Expand Up @@ -105,7 +106,7 @@ private case class Subscribe[K, V](
val shouldSuppress =
aor != null && aor.asInstanceOf[String].toUpperCase(Locale.ROOT) == "NONE"
try {
consumer.poll(0)
consumer.poll(Duration.ZERO)
} catch {
case x: NoOffsetForPartitionException if shouldSuppress =>
logWarning("Catching NoOffsetForPartitionException since " +
Expand Down Expand Up @@ -159,7 +160,7 @@ private case class SubscribePattern[K, V](
val shouldSuppress =
aor != null && aor.asInstanceOf[String].toUpperCase(Locale.ROOT) == "NONE"
try {
consumer.poll(0)
consumer.poll(Duration.ZERO)
} catch {
case x: NoOffsetForPartitionException if shouldSuppress =>
logWarning("Catching NoOffsetForPartitionException since " +
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 java.util.concurrent.ConcurrentLinkedQueue
import java.util.concurrent.atomic.AtomicReference

Expand Down Expand Up @@ -168,7 +169,7 @@ private[spark] class DirectKafkaInputDStream[K, V](
private def paranoidPoll(c: Consumer[K, V]): Unit = {
// don't actually want to consume any messages, so pause all partitions
c.pause(c.assignment())
val msgs = c.poll(0)
val msgs = c.poll(Duration.ZERO)
if (!msgs.isEmpty) {
// position should be minimum offset per topicpartition
msgs.asScala.foldLeft(Map[TopicPartition, Long]()) { (acc, m) =>
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
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ package org.apache.spark.streaming.kafka010

import java.io.File
import java.lang.{Long => JLong}
import java.time.{Duration => jDuration}
import java.util.{Arrays, HashMap => JHashMap, Map => JMap, UUID}
import java.util.concurrent.ConcurrentHashMap
import java.util.concurrent.ConcurrentLinkedQueue
Expand Down Expand Up @@ -255,7 +256,7 @@ class DirectKafkaStreamSuite
preferredHosts,
ConsumerStrategies.Subscribe[String, String](List(topic), kafkaParams.asScala),
new DefaultPerPartitionConfig(sparkConf))
s.consumer.poll(0)
s.consumer.poll(jDuration.ZERO)
assert(
s.consumer.position(topicPartition) >= offsetBeforeStart,
"Start offset not from latest"
Expand Down Expand Up @@ -311,7 +312,7 @@ class DirectKafkaStreamSuite
kafkaParams.asScala,
Map(topicPartition -> 11L)),
new DefaultPerPartitionConfig(sparkConf))
s.consumer.poll(0)
s.consumer.poll(jDuration.ZERO)
assert(
s.consumer.position(topicPartition) >= offsetBeforeStart,
"Start offset not from latest"
Expand Down Expand Up @@ -473,7 +474,7 @@ class DirectKafkaStreamSuite
ssc.stop()
val consumer = new KafkaConsumer[String, String](kafkaParams)
consumer.subscribe(Arrays.asList(topic))
consumer.poll(0)
consumer.poll(jDuration.ZERO)
committed.asScala.foreach {
case (k, v) =>
// commits are async, not exactly once
Expand Down