vanzin commented on a change in pull request #26845: [SPARK-21869][SS] Revise 
Kafka producer pool to implement 'expire' correctly
URL: https://github.com/apache/spark/pull/26845#discussion_r358419105
 
 

 ##########
 File path: 
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/producer/InternalKafkaProducerPool.scala
 ##########
 @@ -0,0 +1,205 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.kafka010.producer
+
+import java.{util => ju}
+import java.util.concurrent.{ScheduledExecutorService, ScheduledFuture, 
TimeUnit}
+import javax.annotation.concurrent.GuardedBy
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.kafka.clients.producer.KafkaProducer
+
+import org.apache.spark.{SparkConf, SparkEnv}
+import org.apache.spark.internal.Logging
+import org.apache.spark.kafka010.{KafkaConfigUpdater, KafkaRedactionUtil}
+import 
org.apache.spark.sql.kafka010.{PRODUCER_CACHE_EVICTOR_THREAD_RUN_INTERVAL, 
PRODUCER_CACHE_TIMEOUT}
+import org.apache.spark.util.{Clock, ShutdownHookManager, SystemClock, 
ThreadUtils, Utils}
+
+/**
+ * Provides object pool for [[CachedKafkaProducer]] which is grouped by
+ * 
[[org.apache.spark.sql.kafka010.producer.InternalKafkaProducerPool.CacheKey]].
+ */
+private[producer] class InternalKafkaProducerPool(
+    executorService: ScheduledExecutorService,
+    val clock: Clock,
+    conf: SparkConf) extends Logging {
+  import InternalKafkaProducerPool._
+
+  def this(sparkConf: SparkConf) = {
+    this(ThreadUtils.newDaemonSingleThreadScheduledExecutor(
+      "kafka-producer-cache-evictor"), new SystemClock, sparkConf)
+  }
+
+  /** exposed for testing */
+  private[producer] val cacheExpireTimeout: Long = 
conf.get(PRODUCER_CACHE_TIMEOUT)
+
+  private val evictorThreadRunIntervalMillis = 
conf.get(PRODUCER_CACHE_EVICTOR_THREAD_RUN_INTERVAL)
+
+  @GuardedBy("this")
+  private val cache = new mutable.HashMap[CacheKey, CachedProducerEntry]
+
+  private def startEvictorThread(): Option[ScheduledFuture[_]] = {
+    if (evictorThreadRunIntervalMillis > 0) {
+      val future = executorService.scheduleAtFixedRate(() => {
+        Utils.tryLogNonFatalError(evictExpired())
+      }, 0, evictorThreadRunIntervalMillis, TimeUnit.MILLISECONDS)
+      Some(future)
+    } else {
+      None
+    }
+  }
+
+  private var scheduled = startEvictorThread()
+
+  /**
+   * 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[producer] def acquire(kafkaParams: ju.Map[String, Object]): 
CachedKafkaProducer = {
+    val updatedKafkaProducerConfiguration =
+      KafkaConfigUpdater("executor", kafkaParams.asScala.toMap)
+        .setAuthenticationConfigIfNeeded()
+        .build()
+    val paramsSeq: Seq[(String, Object)] = 
paramsToSeq(updatedKafkaProducerConfiguration)
+    synchronized {
+      val entry = cache.getOrElseUpdate(paramsSeq, {
+        val producer = createKafkaProducer(paramsSeq)
+        val cachedProducer = new CachedKafkaProducer(paramsSeq, producer)
+        new CachedProducerEntry(cachedProducer, clock, cacheExpireTimeout)
+      })
+      entry.handleBorrowed()
+      entry.producer
+    }
+  }
+
+  private[producer] def release(producer: CachedKafkaProducer): Unit = {
+    def closeProducerNotInCache(producer: CachedKafkaProducer): Unit = {
+      logWarning(s"Released producer ${producer.id} is not a member of the 
cache. Closing.")
+      producer.close()
+    }
+
+    synchronized {
+      cache.get(producer.cacheKey) match {
+        case Some(entry) if entry.producer.id == producer.id => 
entry.handleReturned()
+        case _ => closeProducerNotInCache(producer)
+      }
+    }
+  }
+
+  private[producer] def shutdown(): Unit = {
+    ThreadUtils.shutdown(executorService)
 
 Review comment:
   You need to cancel the scheduler task before calling `shutdown`.

----------------------------------------------------------------
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:
us...@infra.apache.org


With regards,
Apache Git Services

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

Reply via email to