gaborgsomogyi commented on a change in pull request #19096: [SPARK-21869][SS] A
cached Kafka producer should not be closed if any task is using it - adds inuse
tracking.
URL: https://github.com/apache/spark/pull/19096#discussion_r266927130
##########
File path:
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaProducer.scala
##########
@@ -18,103 +18,180 @@
package org.apache.spark.sql.kafka010
import java.{util => ju}
-import java.util.concurrent.{ConcurrentMap, ExecutionException, TimeUnit}
+import java.util.concurrent.{ConcurrentLinkedQueue, ConcurrentMap,
ExecutionException, TimeUnit}
+import java.util.concurrent.atomic.AtomicInteger
+
+import scala.collection.JavaConverters._
+import scala.util.control.NonFatal
import com.google.common.cache._
import com.google.common.util.concurrent.{ExecutionError,
UncheckedExecutionException}
import org.apache.kafka.clients.producer.KafkaProducer
-import scala.collection.JavaConverters._
-import scala.util.control.NonFatal
import org.apache.spark.SparkEnv
import org.apache.spark.internal.Logging
-private[kafka010] object CachedKafkaProducer extends Logging {
+private[kafka010] case class CachedKafkaProducer(
+ private val id: String = ju.UUID.randomUUID().toString,
+ private val inUseCount: AtomicInteger = new AtomicInteger(0),
+ private val kafkaParams: Seq[(String, Object)]) extends Logging {
- private type Producer = KafkaProducer[Array[Byte], Array[Byte]]
+ private val configMap = kafkaParams.map(x => x._1 -> x._2).toMap.asJava
+
+ lazy val kafkaProducer: KafkaProducer[Array[Byte], Array[Byte]] = {
+ val producer = new KafkaProducer[Array[Byte], Array[Byte]](configMap)
+ logDebug(s"Created a new instance of KafkaProducer for " +
+ s"$kafkaParams with Id: $id")
+ closed = false
+ producer
+ }
+ private var isCached: Boolean = true
+ private var closed: Boolean = true
+ private def close(): Unit = {
+ try {
+ this.synchronized {
+ if (!closed) {
+ closed = true
+ kafkaProducer.close()
+ logInfo(s"Closed kafka producer: $kafkaProducer")
+ }
+ }
+ } catch {
+ case NonFatal(e) =>
+ logWarning(s"Error while closing kafka producer with params:
$kafkaParams", e)
+ }
+ }
+
+ private def inUse(): Boolean = {
+ inUseCount.get() > 0
+ }
+ private def unCache(): Unit = {
+ isCached = false
+ }
+ private[kafka010] def getInUseCount: Int = inUseCount.get()
+
+ private[kafka010] def getKafkaParams: Seq[(String, Object)] = kafkaParams
+
+ private[kafka010] def flush(): Unit = {
+ kafkaProducer.flush()
+ }
+
+ private[kafka010] def isClosed: Boolean = closed
+}
+
+private[kafka010] object CachedKafkaProducer extends Logging {
private val defaultCacheExpireTimeout = TimeUnit.MINUTES.toMillis(10)
private lazy val cacheExpireTimeout: Long =
Option(SparkEnv.get).map(_.conf.getTimeAsMs(
- "spark.kafka.producer.cache.timeout",
- s"${defaultCacheExpireTimeout}ms")).getOrElse(defaultCacheExpireTimeout)
+ key = "spark.kafka.producer.cache.timeout",
+ defaultValue =
s"${defaultCacheExpireTimeout}ms")).getOrElse(defaultCacheExpireTimeout)
- private val cacheLoader = new CacheLoader[Seq[(String, Object)], Producer] {
- override def load(config: Seq[(String, Object)]): Producer = {
- val configMap = config.map(x => x._1 -> x._2).toMap.asJava
- createKafkaProducer(configMap)
+ private val cacheLoader = new CacheLoader[Seq[(String, Object)],
CachedKafkaProducer] {
+ override def load(params: Seq[(String, Object)]): CachedKafkaProducer = {
+ CachedKafkaProducer(kafkaParams = params)
}
}
- private val removalListener = new RemovalListener[Seq[(String, Object)],
Producer]() {
+ private val closeQueue = new ConcurrentLinkedQueue[CachedKafkaProducer]()
+
+ private val removalListener = new RemovalListener[Seq[(String, Object)],
CachedKafkaProducer]() {
override def onRemoval(
- notification: RemovalNotification[Seq[(String, Object)], Producer]):
Unit = {
- val paramsSeq: Seq[(String, Object)] = notification.getKey
- val producer: Producer = notification.getValue
- logDebug(
- s"Evicting kafka producer $producer params: $paramsSeq, due to
${notification.getCause}")
- close(paramsSeq, producer)
+ notification: RemovalNotification[Seq[(String, Object)],
CachedKafkaProducer]): Unit = {
+ val producer: CachedKafkaProducer = notification.getValue
+ if (producer.inUse()) {
+ logDebug(s"Evicting kafka producer $producer, due to
${notification.getCause}")
+ // When `inuse` producer is evicted we wait for it to be released
before finally closing it.
+ closeQueue.add(producer)
+ producer.unCache()
+ } else {
+ close(producer)
+ }
}
}
- private lazy val guavaCache: LoadingCache[Seq[(String, Object)], Producer] =
+ private lazy val guavaCache: LoadingCache[Seq[(String, Object)],
CachedKafkaProducer] =
CacheBuilder.newBuilder().expireAfterAccess(cacheExpireTimeout,
TimeUnit.MILLISECONDS)
.removalListener(removalListener)
- .build[Seq[(String, Object)], Producer](cacheLoader)
-
- private def createKafkaProducer(producerConfiguration: ju.Map[String,
Object]): Producer = {
- val updatedKafkaProducerConfiguration =
- KafkaConfigUpdater("executor", producerConfiguration.asScala.toMap)
- .setAuthenticationConfigIfNeeded()
- .build()
- val kafkaProducer: Producer = new
Producer(updatedKafkaProducerConfiguration)
- logDebug(s"Created a new instance of KafkaProducer for
$updatedKafkaProducerConfiguration.")
- kafkaProducer
- }
+ .build[Seq[(String, Object)], CachedKafkaProducer](cacheLoader)
+
+ private def updatedAuthConfigIfNeeded(kafkaParamsMap: ju.Map[String,
Object]) =
+ KafkaConfigUpdater("executor", kafkaParamsMap.asScala.toMap)
+ .setAuthenticationConfigIfNeeded()
+ .build()
/**
* Get a cached KafkaProducer for a given configuration. If matching
KafkaProducer doesn't
* exist, a new KafkaProducer will be created. KafkaProducer is thread safe,
it is best to keep
* one instance per specified kafkaParams.
*/
- private[kafka010] def getOrCreate(kafkaParams: ju.Map[String, Object]):
Producer = {
- val paramsSeq: Seq[(String, Object)] = paramsToSeq(kafkaParams)
+ private[kafka010] def acquire(kafkaParamsMap: ju.Map[String, Object]):
CachedKafkaProducer = {
+ val paramsSeq: Seq[(String, Object)] =
paramsToSeq(updatedAuthConfigIfNeeded(kafkaParamsMap))
Review comment:
If delegation token expired then new instance has to be created. I've just
created a jira to add dynamic configuration possibility
[here](https://issues.apache.org/jira/browse/KAFKA-8128) but that's another
question.
Thinking about this a bit more the approach what you've added here looks
good and would end-up in less task retry (having token in key). The cache
eviction would close the not used and old producer instances. As I've just
checked this change is not in the latest change set.
----------------------------------------------------------------
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:
[email protected]
With regards,
Apache Git Services
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]