-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-19968][SS] Use a cached instance of KafkaProducer instead of creating one every batch.
#17308
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Closed
Closed
[SPARK-19968][SS] Use a cached instance of KafkaProducer instead of creating one every batch.
#17308
Changes from all commits
Commits
Show all changes
13 commits
Select commit
Hold shift + click to select a range
8224596
[SPARK-19968][SS] Use a cached instance of KafkaProducer instead of c…
ScrapCodes e07e77e
Self review and code style improvements.
ScrapCodes d6e4088
[SPARK-20737] Mechanism for cleanup hooks, for structured-streaming s…
ScrapCodes 3ec9981
Synchronized access to local hash map.
ScrapCodes c614bc0
Style feedback
ScrapCodes e5cd1e6
Implementation using guava cache.
ScrapCodes ef2d6cd
Added shutdown hook as stop at driver is not very reliable.
ScrapCodes d2b3ecd
Corrected some style issues.
ScrapCodes 15dfc80
removed a stray change.
ScrapCodes 039d063
Code review, feedback.
ScrapCodes 1c9f892
Using spark conf instead of System.getProperties.
ScrapCodes 588fa03
Michael's feedback!
ScrapCodes a10276a
Using Loading cache, and few style related fixes.
ScrapCodes File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
112 changes: 112 additions & 0 deletions
112
...nal/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaProducer.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,112 @@ | ||
| /* | ||
| * 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 | ||
|
|
||
| import java.{util => ju} | ||
| import java.util.concurrent.{ConcurrentMap, ExecutionException, TimeUnit} | ||
|
|
||
| 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 | ||
|
|
||
| private[kafka010] object CachedKafkaProducer extends Logging { | ||
|
|
||
| private type Producer = KafkaProducer[Array[Byte], Array[Byte]] | ||
|
|
||
| private lazy val cacheExpireTimeout: Long = | ||
| SparkEnv.get.conf.getTimeAsMs("spark.kafka.producer.cache.timeout", "10m") | ||
|
|
||
| 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 removalListener = new RemovalListener[Seq[(String, Object)], Producer]() { | ||
| 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) | ||
| } | ||
| } | ||
|
|
||
| private lazy val guavaCache: LoadingCache[Seq[(String, Object)], Producer] = | ||
| CacheBuilder.newBuilder().expireAfterAccess(cacheExpireTimeout, TimeUnit.MILLISECONDS) | ||
| .removalListener(removalListener) | ||
| .build[Seq[(String, Object)], Producer](cacheLoader) | ||
|
|
||
| private def createKafkaProducer(producerConfiguration: ju.Map[String, Object]): Producer = { | ||
| val kafkaProducer: Producer = new Producer(producerConfiguration) | ||
| logDebug(s"Created a new instance of KafkaProducer for $producerConfiguration.") | ||
| kafkaProducer | ||
| } | ||
|
|
||
| /** | ||
| * 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 def paramsToSeq(kafkaParams: ju.Map[String, Object]): Seq[(String, Object)] = { | ||
| val paramsSeq: Seq[(String, Object)] = kafkaParams.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) | ||
| } | ||
|
|
||
| /** 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) | ||
| } | ||
| } | ||
|
|
||
| private def clear(): Unit = { | ||
| logInfo("Cleaning up guava cache.") | ||
| guavaCache.invalidateAll() | ||
| } | ||
|
|
||
| // Intended for testing purpose only. | ||
| private def getAsMap: ConcurrentMap[Seq[(String, Object)], Producer] = guavaCache.asMap() | ||
| } |
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
78 changes: 78 additions & 0 deletions
78
...afka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/CachedKafkaProducerSuite.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,78 @@ | ||
| /* | ||
| * 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 | ||
|
|
||
| import java.{util => ju} | ||
| import java.util.concurrent.ConcurrentMap | ||
|
|
||
| import org.apache.kafka.clients.producer.KafkaProducer | ||
| import org.apache.kafka.common.serialization.ByteArraySerializer | ||
| import org.scalatest.PrivateMethodTester | ||
|
|
||
| import org.apache.spark.sql.test.SharedSQLContext | ||
|
|
||
| class CachedKafkaProducerSuite extends SharedSQLContext with PrivateMethodTester { | ||
|
|
||
| type KP = KafkaProducer[Array[Byte], Array[Byte]] | ||
|
|
||
| protected override def beforeEach(): Unit = { | ||
| super.beforeEach() | ||
| val clear = PrivateMethod[Unit]('clear) | ||
| CachedKafkaProducer.invokePrivate(clear()) | ||
| } | ||
|
|
||
| test("Should return the cached instance on calling getOrCreate with same params.") { | ||
| val kafkaParams = new ju.HashMap[String, Object]() | ||
| kafkaParams.put("acks", "0") | ||
| // Here only host should be resolvable, it does not need a running instance of kafka server. | ||
| kafkaParams.put("bootstrap.servers", "127.0.0.1:9022") | ||
| kafkaParams.put("key.serializer", classOf[ByteArraySerializer].getName) | ||
| kafkaParams.put("value.serializer", classOf[ByteArraySerializer].getName) | ||
| val producer = CachedKafkaProducer.getOrCreate(kafkaParams) | ||
| val producer2 = CachedKafkaProducer.getOrCreate(kafkaParams) | ||
| assert(producer == producer2) | ||
|
|
||
| val cacheMap = PrivateMethod[ConcurrentMap[Seq[(String, Object)], KP]]('getAsMap) | ||
| val map = CachedKafkaProducer.invokePrivate(cacheMap()) | ||
| assert(map.size == 1) | ||
| } | ||
|
|
||
| test("Should close the correct kafka producer for the given kafkaPrams.") { | ||
| val kafkaParams = new ju.HashMap[String, Object]() | ||
| kafkaParams.put("acks", "0") | ||
| kafkaParams.put("bootstrap.servers", "127.0.0.1:9022") | ||
| kafkaParams.put("key.serializer", classOf[ByteArraySerializer].getName) | ||
| kafkaParams.put("value.serializer", classOf[ByteArraySerializer].getName) | ||
| val producer: KP = CachedKafkaProducer.getOrCreate(kafkaParams) | ||
| kafkaParams.put("acks", "1") | ||
| val producer2: KP = CachedKafkaProducer.getOrCreate(kafkaParams) | ||
| // With updated conf, a new producer instance should be created. | ||
| assert(producer != producer2) | ||
|
|
||
| val cacheMap = PrivateMethod[ConcurrentMap[Seq[(String, Object)], KP]]('getAsMap) | ||
| val map = CachedKafkaProducer.invokePrivate(cacheMap()) | ||
| assert(map.size == 2) | ||
|
|
||
| CachedKafkaProducer.close(kafkaParams) | ||
| val map2 = CachedKafkaProducer.invokePrivate(cacheMap()) | ||
| assert(map2.size == 1) | ||
| import scala.collection.JavaConverters._ | ||
| val (seq: Seq[(String, Object)], _producer: KP) = map2.asScala.toArray.apply(0) | ||
| assert(_producer == producer) | ||
| } | ||
| } | ||
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We just know there is one KP by this assert. Seems we should also verify if we close the correct KP?