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/03/19 23:27:35 UTC

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

zsxwing 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_r267132711
 
 

 ##########
 File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaProducer.scala
 ##########
 @@ -40,81 +94,103 @@ private[kafka010] object CachedKafkaProducer extends Logging {
       "spark.kafka.producer.cache.timeout",
       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)
 
   /**
    * 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)
-    try {
-      guavaCache.get(paramsSeq)
-    } catch {
-      case e @ (_: ExecutionException | _: UncheckedExecutionException | _: ExecutionError)
-        if e.getCause != null =>
-        throw e.getCause
+  private[kafka010] def acquire(kafkaParamsMap: ju.Map[String, Object]): CachedKafkaProducer =
+    this.synchronized {
+      val paramsSeq: Seq[(String, Object)] = paramsToSeq(kafkaParamsMap)
+      try {
+        val cachedKafkaProducer: CachedKafkaProducer = guavaCache.get(paramsSeq)
+        val useCount = cachedKafkaProducer.inUseCount.incrementAndGet()
+        logDebug(s"Granted producer $cachedKafkaProducer, inuse-count: $useCount")
+        cachedKafkaProducer
+      } catch {
+        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 simple decrement it's inuse count. */
+  private[kafka010] def release(producer: CachedKafkaProducer, failing: Boolean): Unit = {
+    this.synchronized {
+      val inUseCount = producer.inUseCount.decrementAndGet()
+      logDebug(s"Released producer $producer, updated inuse count: $inUseCount")
+      if (failing) {
+        // If this producer is failing to write, we remove it from cache.
+        // So that it is re-created, eventually.
+        logDebug(s"Invalidated a failing producer: $producer.")
+        guavaCache.invalidate(producer.kafkaParams)
+      }
+      if (!producer.inUse() && !producer.isCached) {
+        // it will take care of removing it from close queue as well.
+        close(producer)
+      }
+    }
   }
 
-  /** Auto close on cache evict */
-  private def close(paramsSeq: Seq[(String, Object)], producer: Producer): Unit = {
-    try {
-      logInfo(s"Closing the KafkaProducer with params: ${paramsSeq.mkString("\n")}.")
-      producer.close()
-    } catch {
-      case NonFatal(e) => logWarning("Error while closing kafka producer.", e)
+  /** Close this producer and process pending closes. */
+  private def close(producer: CachedKafkaProducer): Unit = {
+    producer.close()
+    // Check and close any other producers previously evicted, but pending to be closed.
+    for (p <- closeQueue.iterator().asScala) {
 
 Review comment:
   Could you move this codes out of the lock? Closing producers takes time as it will block until all pending requests are sent. We can avoid blocking other threads that are acquiring producers.

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