jose-torres 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_r315305934
##########
File path:
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaProducer.scala
##########
@@ -18,111 +18,191 @@
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 javax.annotation.concurrent.GuardedBy
+
+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
import org.apache.spark.kafka010.{KafkaConfigUpdater, KafkaRedactionUtil}
-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 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)
+ if (log.isDebugEnabled()) {
+ val redactedParamsSeq = KafkaRedactionUtil.redactParams(kafkaParams)
+ logDebug(s"Created a new instance of KafkaProducer for
$redactedParamsSeq, with Id: $id.")
+ }
+ closed = false
+ producer
+ }
+ @GuardedBy("this")
+ private var closed: Boolean = true
+ private def close(): Unit = {
+ try {
+ this.synchronized {
+ if (!closed) {
+ closed = true
+ if (log.isInfoEnabled()) {
+ val redactedParamsSeq =
KafkaRedactionUtil.redactParams(kafkaParams)
+ logInfo(s"Closing the KafkaProducer with params:
${redactedParamsSeq.mkString("\n")}.")
+ }
+ kafkaProducer.close()
+ }
+ }
+ } catch {
+ case NonFatal(e) =>
+ logWarning(s"Error while closing kafka producer with params:
$kafkaParams", e)
+ }
+ }
+
+ private def inUse(): Boolean = inUseCount.get() > 0
+
+ private[kafka010] def getInUseCount: Int = inUseCount.get()
- private type Producer = KafkaProducer[Array[Byte], Array[Byte]]
+ 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.get(PRODUCER_CACHE_TIMEOUT))
.getOrElse(defaultCacheExpireTimeout)
- private val cacheLoader = new CacheLoader[Seq[(String, Object)], Producer] {
- override def load(config: Seq[(String, Object)]): Producer = {
- createKafkaProducer(config)
+ 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 def updatedAuthConfigIfNeeded(kafkaParamsMap: ju.Map[String,
Object]) =
+ KafkaConfigUpdater("executor", kafkaParamsMap.asScala.toMap)
+ .setAuthenticationConfigIfNeeded()
+ .build()
+
+ 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
+ notification: RemovalNotification[Seq[(String, Object)],
CachedKafkaProducer]): Unit = {
+ val producer: CachedKafkaProducer = notification.getValue
if (log.isDebugEnabled()) {
- val redactedParamsSeq = KafkaRedactionUtil.redactParams(paramsSeq)
+ val redactedParamsSeq =
KafkaRedactionUtil.redactParams(producer.kafkaParams)
logDebug(s"Evicting kafka producer $producer params:
$redactedParamsSeq, " +
s"due to ${notification.getCause}")
}
- close(paramsSeq, producer)
+ if (producer.inUse()) {
+ // When `inuse` producer is evicted we wait for it to be released by
all the tasks,
+ // before finally closing it.
+ closeQueue.add(producer)
+ } else {
+ producer.close()
+ }
}
}
- 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(paramsSeq: Seq[(String, Object)]): Producer
= {
- val kafkaProducer: Producer = new Producer(paramsSeq.toMap.asJava)
- if (log.isDebugEnabled()) {
- val redactedParamsSeq = KafkaRedactionUtil.redactParams(paramsSeq)
- logDebug(s"Created a new instance of KafkaProducer for
$redactedParamsSeq.")
- }
- kafkaProducer
- }
+ .build[Seq[(String, Object)], CachedKafkaProducer](cacheLoader)
/**
* 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 updatedKafkaProducerConfiguration =
- KafkaConfigUpdater("executor", kafkaParams.asScala.toMap)
- .setAuthenticationConfigIfNeeded()
- .build()
- val paramsSeq: Seq[(String, Object)] =
paramsToSeq(updatedKafkaProducerConfiguration)
+ private[kafka010] def acquire(kafkaParamsMap: ju.Map[String, Object]):
CachedKafkaProducer = {
+ val paramsSeq: Seq[(String, Object)] =
paramsToSeq(updatedAuthConfigIfNeeded(kafkaParamsMap))
try {
- guavaCache.get(paramsSeq)
+ val producer = this.synchronized {
+ val cachedKafkaProducer: CachedKafkaProducer =
guavaCache.get(paramsSeq)
+ cachedKafkaProducer.inUseCount.incrementAndGet()
+ logDebug(s"Granted producer $cachedKafkaProducer")
+ cachedKafkaProducer
+ }
+ producer
} catch {
- case e @ (_: ExecutionException | _: UncheckedExecutionException | _:
ExecutionError)
+ case e@(_: ExecutionException | _: UncheckedExecutionException | _:
ExecutionError)
if e.getCause != null =>
throw e.getCause
}
}
- private def paramsToSeq(kafkaParams: ju.Map[String, Object]): Seq[(String,
Object)] = {
- val paramsSeq: Seq[(String, Object)] = kafkaParams.asScala.toSeq.sortBy(x
=> x._1)
+ private def paramsToSeq(kafkaParamsMap: ju.Map[String, Object]):
Seq[(String, Object)] = {
+ val paramsSeq: Seq[(String, Object)] =
kafkaParamsMap.asScala.toSeq.sortBy(x => x._1)
paramsSeq
}
- /** For explicitly closing kafka producer */
- private[kafka010] def close(kafkaParams: ju.Map[String, Object]): Unit = {
- val paramsSeq = paramsToSeq(kafkaParams)
- guavaCache.invalidate(paramsSeq)
+ /* Release a kafka producer back to the kafka cache. We simply decrement
it's inuse count. */
+ private[kafka010] def release(producer: CachedKafkaProducer, failing:
Boolean): Unit = {
+ this.synchronized {
+ // It should be ok to call release multiple times on the same producer
object.
Review comment:
But it's not really okay, right? If task A calls release multiple times, the
producer might have its inUseCount decremented to 0 even though task B is using
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:
[email protected]
With regards,
Apache Git Services
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]