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/10/30 13:44:16 UTC

[GitHub] [spark] gaborgsomogyi commented on a change in pull request #25853: [SPARK-21869][SS] Apply Apache Commons Pool to Kafka producer

gaborgsomogyi commented on a change in pull request #25853: [SPARK-21869][SS] Apply Apache Commons Pool to Kafka producer
URL: https://github.com/apache/spark/pull/25853#discussion_r340620619
 
 

 ##########
 File path: external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/InternalKafkaProducerPool.scala
 ##########
 @@ -0,0 +1,68 @@
+/*
+ * 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 scala.collection.JavaConverters._
+
+import org.apache.commons.pool2.PooledObject
+
+import org.apache.spark.SparkConf
+import org.apache.spark.sql.kafka010.InternalKafkaProducerPool.CacheKey
+
+private[kafka010] class InternalKafkaProducerPool(
+    objectFactory: ProducerObjectFactory,
+    poolConfig: ProducerPoolConfig)
+  extends InternalKafkaConnectorPool[CacheKey, CachedKafkaProducer](
+      objectFactory,
+      poolConfig,
+      new CustomSwallowedExceptionListener("producer")) {
+
+  def this(conf: SparkConf) = {
+    this(new ProducerObjectFactory, new ProducerPoolConfig(conf))
+  }
+
+  protected def createKey(producer: CachedKafkaProducer): CacheKey = {
+    InternalKafkaProducerPool.toCacheKey(producer.kafkaParams)
+  }
+}
+
+private class ProducerPoolConfig(conf: SparkConf) extends PoolConfig[CachedKafkaProducer] {
+  def softMaxSize: Int = conf.get(PRODUCER_CACHE_CAPACITY)
+  def jmxEnabled: Boolean = conf.get(PRODUCER_CACHE_JMX_ENABLED)
+  def minEvictableIdleTimeMillis: Long = conf.get(PRODUCER_CACHE_TIMEOUT)
+  def evictorThreadRunIntervalMillis: Long = conf.get(PRODUCER_CACHE_EVICTOR_THREAD_RUN_INTERVAL)
+  def jmxNamePrefix: String = "kafka010-cached-simple-kafka-producer-pool"
+}
+
+private class ProducerObjectFactory extends ObjectFactory[CacheKey, CachedKafkaProducer] {
+  protected def createValue(
+      key: CacheKey,
+      kafkaParams: ju.Map[String, Object]): CachedKafkaProducer = {
+    new CachedKafkaProducer(kafkaParams)
+  }
+}
+
+private[kafka010] object InternalKafkaProducerPool {
+  type CacheKey = Seq[(String, Object)]
+
+  def toCacheKey(params: ju.Map[String, Object]): CacheKey = {
 
 Review comment:
   Sorted `Seq` as key is coming from the original implementation. I understand the same way, namely different `Map` implementations or different number of buckets (this is dynamically changing when `loadFactor` reached) may end-up in different order which trigger a cache miss. The sorted `Seq` approach is one way to overcame this situation.
   
   I was thinking about to use `Map` as key but didn't change because:
   * To use the same `Map` implementation is not enforced on API level and I think it would be an overkill to introduce it (depending on implementation detail on class API would make the code less flexible)
   * Depending on implementations which provide insertion order is just error prone because multiple code places must match in terms of order and I can hardly believe devs will aware of this constraint
   * The only option what I see is a `Map` implementation which makes ordering in insertion time for example `TreeMap` (but considering the first point I'm not really a big fan of that).
   

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