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 2019/05/22 10:18:18 UTC

[GitHub] [spark] skonto 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.

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

 ##########
 File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaProducer.scala
 ##########
 @@ -18,103 +18,183 @@
 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
 
-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)
+    logDebug(s"Created a new instance of KafkaProducer for " +
+      s"$kafkaParams with Id: $id")
+    closed = false
+    producer
+  }
+  @GuardedBy("this")
+  private var closed: Boolean = true
+  private def close(): Unit = {
+    try {
+      this.synchronized {
+        if (!closed) {
+          closed = true
+          kafkaProducer.close()
+          logDebug(s"Closed kafka producer: $this")
+        }
+      }
+    } 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 = {
-      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 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
-      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
+      logDebug(s"Evicting kafka producer $producer, due to ${notification.getCause}.")
+      if (producer.inUse()) {
+        // When `inuse` producer is evicted we wait for it to be released by all the tasks,
+        // before finally closing it.
 
 Review comment:
   Is this logic safe when multiple-queries will use the same producer cache at 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


With regards,
Apache Git Services

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