HeartSaVioR 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_r272468707
##########
File path:
external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaProducer.scala
##########
@@ -40,81 +90,114 @@ 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)
+ private[kafka010] def acquire(kafkaParamsMap: ju.Map[String, Object]):
CachedKafkaProducer = {
+ val paramsSeq: Seq[(String, Object)] = paramsToSeq(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.
+ if (producer.inUse()) {
+ // So that we do not end up with -ve in-use counts.
+ producer.inUseCount.decrementAndGet()
+ logDebug(s"Released producer $producer.")
+ } else {
+ logWarning(s"Tried to release a not in use producer, $producer.")
+ }
+ if (failing) {
+ // If this producer is failing to write, we remove it from cache.
+ // So that it is re-created, eventually.
+ val cachedProducer = guavaCache.getIfPresent(producer.kafkaParams)
+ if (cachedProducer != null && cachedProducer.id == producer.id) {
+ logDebug(s"Invalidating a failing producer: $producer.")
+ guavaCache.invalidate(producer.kafkaParams)
+ }
+ }
+ }
+ if (!producer.inUse() && !producer.isCached) {
Review comment:
Could you point out which you don't understand my explanation of race
condition?
In short, `Adding producer to closeQueue` and `marking producer as uncached`
are not being done atomically given there's no synchronization in
`removalListener.onRemoval`. There's a case producer is added to closeQueue but
isCached is still `true` - race condition happens here.
In other words, suppose `inCount` in producer is exactly correct and
thread-safe: then why we need close queue? There should be "exactly only once"
when inUse = false, either `onRemoval` or `release`, then closeQueue doesn't be
needed and just closing the producer should work. `closeQueue` is just for
mitigation when inCount is off, and looping closeQueue every time when other
producer is closing is not free.
----------------------------------------------------------------
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]